Uploaded image for project: 'Apache Drill'
  1. Apache Drill
  2. DRILL-1760

Count on a map fails with SchemaChangeException

    XMLWordPrintableJSON

Details

    • Improvement
    • Status: In Progress
    • Major
    • Resolution: Unresolved
    • 1.0.0
    • None
    • Functions - Drill
    • None

    Description

      Take yelp business dataset and run

      select count(attributes) from dfs.`/path/to/yelp-business.json`
      

      you should read

      org.apache.drill.exec.exception.SchemaChangeException: Failure while materializing expression. 
      Error in expression at index -1.  Error: Missing function implementation: [count(MAP-REQUIRED)].  Full expression: --UNKNOWN EXPRESSION--.
      	at org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch.createAggregatorInternal(StreamingAggBatch.java:221) [classes/:na]
      	at org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch.createAggregator(StreamingAggBatch.java:173) [classes/:na]
      	at org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch.buildSchema(StreamingAggBatch.java:89) [classes/:na]
      	at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.buildSchema(IteratorValidatorBatchIterator.java:80) [classes/:na]
      	at org.apache.drill.exec.record.AbstractSingleRecordBatch.buildSchema(AbstractSingleRecordBatch.java:109) [classes/:na]
      	at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.buildSchema(IteratorValidatorBatchIterator.java:80) [classes/:na]
      	at org.apache.drill.exec.physical.impl.svremover.RemovingRecordBatch.buildSchema(RemovingRecordBatch.java:64) [classes/:na]
      	at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.buildSchema(IteratorValidatorBatchIterator.java:80) [classes/:na]
      	at org.apache.drill.exec.physical.impl.ScreenCreator$ScreenRoot.buildSchema(ScreenCreator.java:95) [classes/:na]
      	at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:111) [classes/:na]
      	at org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:249) [classes/:na]
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) [na:1.7.0_65]
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_65]
      	at java.lang.Thread.run(Thread.java:745) [na:1.7.0_65]
      

      I would expect to be able run count query on `attributes` field given that I can run a select on the same field.

      Attachments

        Issue Links

          Activity

            People

              hgunes Hanifi Gunes
              hgunes Hanifi Gunes
              Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

                Created:
                Updated: