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

HashAggBatch/HashAggTemplate call incoming.cleanup() twice resulting in warnings

    XMLWordPrintableJSON

Details

    Description

      In this case j1,j2 are views created on top of parquet files, BOTH views have order by on multiple columns in different order with nulls first/last.
      Also, table in in view j1, consists of 99 parquet files. See attached views.txt file on how to create views (make sure to create views in a different workspace, views have the same names as tables)

      select DISTINCT
              COALESCE(j1.c_varchar || j2.c_varchar || 'EMPTY') as concatentated_string
      from
              j1  INNER JOIN j2 ON
              (j1.d18 = j2.d18)
      ;
      

      The same can be reproduced with parquet files and subqueries:
      (pay attention parquet files are named the same as views: j1, j2)

      select DISTINCT
              COALESCE(sq1.c_varchar || sq2.c_varchar || 'EMPTY') as concatentated_string
      from
              (select c_varchar, c_integer from j1 order by j1.c_varchar desc nulls first ) as sq1(c_varchar, c_integer)
              INNER JOIN
              (select c_varchar, c_integer from j2 order by j2.c_varchar nulls last) as sq2(c_varchar, c_integer)
              ON (sq1.c_integer = sq2.c_integer)
      

      You do need to have sort in order to reproduce the problem.
      This query works:

      select DISTINCT
              COALESCE(j1.c_varchar || j2.c_varchar || 'EMPTY') as concatentated_string
      from j1,j2
      where j1.c_integer = j2.c_integer;
      
      2015-04-01 00:43:42,455 [2ae4c0c0-c408-3e66-4fb3-e7bf80a42bad:foreman] INFO  o.a.d.e.s.parquet.ParquetGroupScan - Load Parquet RowGroup block maps: Executed 99 out of 99 using 16 threads. Time: 20ms total, 2.877318ms avg, 3ms max.
      2015-04-01 00:43:42,458 [2ae4c0c0-c408-3e66-4fb3-e7bf80a42bad:foreman] INFO  o.a.d.e.s.schedule.BlockMapBuilder - Failure finding Drillbit running on host atsqa4-136.qa.lab.  Skipping affinity to that host.
      2015-04-01 00:43:42,458 [2ae4c0c0-c408-3e66-4fb3-e7bf80a42bad:foreman] INFO  o.a.d.e.s.parquet.ParquetGroupScan - Load Parquet RowGroup block maps: Executed 1 out of 1 using 1 threads. Time: 1ms total, 1.562620ms avg, 1ms max.
      2015-04-01 00:43:42,485 [2ae4c0c0-c408-3e66-4fb3-e7bf80a42bad:foreman] INFO  o.a.drill.exec.work.foreman.Foreman - State change requested.  PENDING --> RUNNING
      2015-04-01 00:43:45,613 [2ae4c0c0-c408-3e66-4fb3-e7bf80a42bad:frag:0:0] WARN  o.a.d.e.p.i.xsort.ExternalSortBatch - Starting to merge. 32 batch groups. Current allocated memory: 16642330
      2015-04-01 00:43:45,676 [2ae4c0c0-c408-3e66-4fb3-e7bf80a42bad:frag:0:0] INFO  o.a.d.exec.vector.BaseValueVector - Realloc vector null. [16384] -> [32768]
      2015-04-01 00:43:45,676 [2ae4c0c0-c408-3e66-4fb3-e7bf80a42bad:frag:0:0] INFO  o.a.d.exec.vector.BaseValueVector - Realloc vector ``c_varchar`(VARCHAR:OPTIONAL)_bits`(UINT1:REQUIRED). [4096] -> [8192]
      2015-04-01 00:43:45,679 [2ae4c0c0-c408-3e66-4fb3-e7bf80a42bad:frag:0:0] INFO  o.a.d.exec.vector.BaseValueVector - Realloc vector null. [32768] -> [65536]
      2015-04-01 00:43:45,680 [2ae4c0c0-c408-3e66-4fb3-e7bf80a42bad:frag:0:0] INFO  o.a.d.exec.vector.BaseValueVector - Realloc vector ``c_varchar`(VARCHAR:OPTIONAL)_bits`(UINT1:REQUIRED). [8192] -> [16384]
      2015-04-01 00:43:45,709 [2ae4c0c0-c408-3e66-4fb3-e7bf80a42bad:frag:0:0] WARN  o.a.d.exec.memory.AtomicRemainder - Tried to close remainder, but it has already been closed
      java.lang.Exception: null
              at org.apache.drill.exec.memory.AtomicRemainder.close(AtomicRemainder.java:196) [drill-java-exec-0.8.0-SNAPSHOT-rebuffed.jar:0.8.0-SNAPSHOT]
              at org.apache.drill.exec.memory.Accountor.close(Accountor.java:386) [drill-java-exec-0.8.0-SNAPSHOT-rebuffed.jar:0.8.0-SNAPSHOT]
              at org.apache.drill.exec.memory.TopLevelAllocator$ChildAllocator.close(TopLevelAllocator.java:298) [drill-java-exec-0.8.0-SNAPSHOT-rebuffed.jar:0.8.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.xsort.ExternalSortBatch.cleanup(ExternalSortBatch.java:162) [drill-java-exec-0.8.0-SNAPSHOT-rebuffed.jar:0.8.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.cleanup(IteratorValidatorBatchIterator.java:148) [drill-java-exec-0.8.0-SNAPSHOT-rebuffed.jar:0.8.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractSingleRecordBatch.cleanup(AbstractSingleRecordBatch.java:121) [drill-java-exec-0.8.0-SNAPSHOT-rebuffed.jar:0.8.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.svremover.RemovingRecordBatch.cleanup(RemovingRecordBatch.java:191) [drill-java-exec-0.8.0-SNAPSHOT-rebuffed.jar:0.8.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.cleanup(IteratorValidatorBatchIterator.java:148) [drill-java-exec-0.8.0-SNAPSHOT-rebuffed.jar:0.8.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.join.MergeJoinBatch.cleanup(MergeJoinBatch.java:260) [drill-java-exec-0.8.0-SNAPSHOT-rebuffed.jar:0.8.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.cleanup(IteratorValidatorBatchIterator.java:148) [drill-java-exec-0.8.0-SNAPSHOT-rebuffed.jar:0.8.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractSingleRecordBatch.cleanup(AbstractSingleRecordBatch.java:121) [drill-java-exec-0.8.0-SNAPSHOT-rebuffed.jar:0.8.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.cleanup(IteratorValidatorBatchIterator.java:148) [drill-java-exec-0.8.0-SNAPSHOT-rebuffed.jar:0.8.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.aggregate.HashAggBatch.cleanup(HashAggBatch.java:281) [drill-java-exec-0.8.0-SNAPSHOT-rebuffed.jar:0.8.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.cleanup(IteratorValidatorBatchIterator.java:148) [drill-java-exec-0.8.0-SNAPSHOT-rebuffed.jar:0.8.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.ScreenCreator$ScreenRoot.internalStop(ScreenCreator.java:176) [drill-java-exec-0.8.0-SNAPSHOT-rebuffed.jar:0.8.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.ScreenCreator$ScreenRoot.innerNext(ScreenCreator.java:123) [drill-java-exec-0.8.0-SNAPSHOT-rebuffed.jar:0.8.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.BaseRootExec.next(BaseRootExec.java:58) [drill-java-exec-0.8.0-SNAPSHOT-rebuffed.jar:0.8.0-SNAPSHOT]
              at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:163) [drill-java-exec-0.8.0-SNAPSHOT-rebuffed.jar:0.8.0-SNAPSHOT]
              at org.apache.drill.common.SelfCleaningRunnable.run(SelfCleaningRunnable.java:38) [drill-common-0.8.0-SNAPSHOT-rebuffed.jar:0.8.0-SNAPSHOT]
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) [na:1.7.0_71]
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_71]
              at java.lang.Thread.run(Thread.java:745) [na:1.7.0_71]
      

      Explain plan:

      00-01      HashAgg(group=[{0}])
      00-02        Project(concatentated_string=[||(||($0, $2), 'EMPTY')])
      00-03          MergeJoin(condition=[=($1, $3)], joinType=[inner])
      00-05            SelectionVectorRemover
      00-07              Sort(sort0=[$1], dir0=[ASC])
      00-09                SelectionVectorRemover
      00-11                  Sort(sort0=[$0], dir0=[DESC-nulls-first])
      00-13                    Project(c_varchar=[$1], c_integer=[$0])
      00-15                      Scan(groupscan=[ParquetGroupScan [entries=[ReadEntryWithPath [path=maprfs:/drill/testdata/joins/j1]], selectionRoot=/drill/testdata/joins/j1, numFiles=1, columns=[`c_varchar`, `c_integer`]]])
      00-04            Project(c_varchar0=[$0], c_integer0=[$1])
      00-06              SelectionVectorRemover
      00-08                Sort(sort0=[$1], dir0=[ASC])
      00-10                  SelectionVectorRemover
      00-12                    Sort(sort0=[$0], dir0=[ASC-nulls-last])
      00-14                      Project(c_varchar=[$1], c_integer=[$0])
      00-16                        Scan(groupscan=[ParquetGroupScan [entries=[ReadEntryWithPath [path=maprfs:/drill/testdata/joins/j2]], selectionRoot=/drill/testdata/joins/j2, numFiles=1, columns=[`c_va
      

      I'm sure there is an easier way to reproduce this failure, can't think about it right now. Please talk to me if you have trouble reproducing this failure. Issue with the "maybe" redundant sort will be addressed in a different bug.

      Attachments

        1. DRILL-2643.patch
          1 kB
          Venki Korukanti
        2. t2.parquet
          0.5 kB
          Victoria Markman
        3. t1.parquet
          0.5 kB
          Victoria Markman

        Activity

          People

            vkorukanti Venki Korukanti
            vicky Victoria Markman
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: