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

Assertion Error in Managed External Sort when dealing with repeated maps

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Fixed
    • 1.10.0
    • 1.12.0
    • None
    • None

    Description

      git.commit.id.abbrev=3e8b01d

      The below query fails with an Assertion Error (I am running with assertions enabled)

      ALTER SESSION SET `exec.sort.disable_managed` = false;
      alter session set `planner.width.max_per_node` = 1;
      alter session set `planner.disable_exchanges` = true;
      alter session set `planner.width.max_per_query` = 1;
      alter session set `planner.memory.max_query_memory_per_node` = 152428800;
      select count(*) from (
      select * from (
      select event_info.uid, transaction_info.trans_id, event_info.event.evnt_id
      from (
       select userinfo.transaction.trans_id trans_id, max(userinfo.event.event_time) max_event_time
       from (
           select uid, flatten(events) event, flatten(transactions) transaction from dfs.`/drill/testdata/resource-manager/nested-large.json`
       ) userinfo
       where userinfo.transaction.trans_time >= userinfo.event.event_time
       group by userinfo.transaction.trans_id
      ) transaction_info
      inner join
      (
       select uid, flatten(events) event
       from dfs.`/drill/testdata/resource-manager/nested-large.json`
      ) event_info
      on transaction_info.max_event_time = event_info.event.event_time) d order by features[0].type) d1 where d1.uid < -1;
      

      Below is the error from the logs

      [Error Id: 26983344-dee3-4a33-8508-ad125f01fee6 on qa-node190.qa.lab:31010]
              at org.apache.drill.common.exceptions.UserException$Builder.build(UserException.java:544) ~[drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.work.fragment.FragmentExecutor.sendFinalState(FragmentExecutor.java:295) [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.work.fragment.FragmentExecutor.cleanup(FragmentExecutor.java:160) [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:264) [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.common.SelfCleaningRunnable.run(SelfCleaningRunnable.java:38) [drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) [na:1.7.0_111]
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_111]
              at java.lang.Thread.run(Thread.java:745) [na:1.7.0_111]
      Caused by: java.lang.RuntimeException: java.lang.AssertionError
              at org.apache.drill.common.DeferredException.addThrowable(DeferredException.java:101) ~[drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.work.fragment.FragmentExecutor.fail(FragmentExecutor.java:409) [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:250) [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              ... 4 common frames omitted
      Caused by: java.lang.AssertionError: null
              at org.apache.drill.exec.vector.complex.RepeatedMapVector.load(RepeatedMapVector.java:444) ~[vector-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.cache.VectorAccessibleSerializable.readFromStream(VectorAccessibleSerializable.java:118) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.xsort.managed.BatchGroup$SpilledRun.getBatch(BatchGroup.java:222) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.xsort.managed.BatchGroup$SpilledRun.getNextIndex(BatchGroup.java:196) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.test.generated.PriorityQueueCopierGen23.setup(PriorityQueueCopierTemplate.java:60) ~[na:na]
              at org.apache.drill.exec.physical.impl.xsort.managed.CopierHolder.createCopier(CopierHolder.java:116) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.xsort.managed.CopierHolder.access$200(CopierHolder.java:45) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.xsort.managed.CopierHolder$BatchMerger.<init>(CopierHolder.java:210) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.xsort.managed.CopierHolder$BatchMerger.<init>(CopierHolder.java:171) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.xsort.managed.CopierHolder.startFinalMerge(CopierHolder.java:85) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.mergeSpilledRuns(ExternalSortBatch.java:1213) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.load(ExternalSortBatch.java:689) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.innerNext(ExternalSortBatch.java:559) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:162) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.next(IteratorValidatorBatchIterator.java:215) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:109) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractSingleRecordBatch.innerNext(AbstractSingleRecordBatch.java:51) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.svremover.RemovingRecordBatch.innerNext(RemovingRecordBatch.java:93) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:162) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.next(IteratorValidatorBatchIterator.java:215) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:109) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractSingleRecordBatch.innerNext(AbstractSingleRecordBatch.java:51) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:162) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.next(IteratorValidatorBatchIterator.java:215) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:109) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractSingleRecordBatch.innerNext(AbstractSingleRecordBatch.java:51) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.svremover.RemovingRecordBatch.innerNext(RemovingRecordBatch.java:93) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:162) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.next(IteratorValidatorBatchIterator.java:215) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:109) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractSingleRecordBatch.innerNext(AbstractSingleRecordBatch.java:51) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.innerNext(ProjectRecordBatch.java:135) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:162) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.next(IteratorValidatorBatchIterator.java:215) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:109) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch.innerNext(StreamingAggBatch.java:140) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:162) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.next(IteratorValidatorBatchIterator.java:215) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:109) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractSingleRecordBatch.innerNext(AbstractSingleRecordBatch.java:51) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.innerNext(ProjectRecordBatch.java:135) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:162) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.next(IteratorValidatorBatchIterator.java:215) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.BaseRootExec.next(BaseRootExec.java:104) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.ScreenCreator$ScreenRoot.innerNext(ScreenCreator.java:81) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.BaseRootExec.next(BaseRootExec.java:94) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.work.fragment.FragmentExecutor$1.run(FragmentExecutor.java:234) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at org.apache.drill.exec.work.fragment.FragmentExecutor$1.run(FragmentExecutor.java:227) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              at java.security.AccessController.doPrivileged(Native Method) ~[na:1.7.0_111]
              at javax.security.auth.Subject.doAs(Subject.java:415) ~[na:1.7.0_111]
              at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1595) ~[hadoop-common-2.7.0-mapr-1607.jar:na]
              at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:227) [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
              ... 4 common frames omitted
      

      Attachments

        1. 27004a3c-c53d-52d1-c7ed-4beb563447f9.sys.drill
          14 kB
          Rahul Kumar Challapalli
        2. drillbit.log
          4.04 MB
          Rahul Kumar Challapalli

        Issue Links

          Activity

            People

              paul-rogers Paul Rogers
              rkins Rahul Kumar Challapalli
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: