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

TPC-H tests fail with OOM

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Critical
    • Resolution: Duplicate
    • 1.14.0
    • None
    • Execution - Flow
    • None

    Description

      On latest Apache master, we are observing that there are multiple test failures. It looks like Drill runs out of Direct memory and queries fail with OOM. Few other queries fail probably fail because they are unable to connect to Drillbits.

      It looks like one of the recent commits caused this.

      Commit ID Status
      24193b1b038a6315681a65c76a67034b64f71fc5 FAIL
      883c8d94b0021a83059fa79563dd516c4299b70a FAIL
      2601cdd33e0685f59a7bf2ac72541bd9dcaaa18f FAIL
      9173308710c3decf8ff745493ad3e85ccdaf7c37 PASS
      c6549e58859397c88cb1de61b4f6eee52a07ed0c PASS

      Two example queries + exceptions below. Also query log attached.

      Query 1: Advanced/tpch/tpch_sf100/parquet/10.q

       select
       c.c_custkey,
       c.c_name,
       sum(l.l_extendedprice * (1 - l.l_discount)) as revenue,
       c.c_acctbal,
       n.n_name,
       c.c_address,
       c.c_phone,
       c.c_comment
       from
       customer c,
       orders o,
       lineitem l,
       nation n
       where
       c.c_custkey = o.o_custkey
       and l.l_orderkey = o.o_orderkey
       and o.o_orderdate >= date '1994-03-01'
       and o.o_orderdate < date '1994-03-01' + interval '3' month
       and l.l_returnflag = 'R'
       and c.c_nationkey = n.n_nationkey
       group by
       c.c_custkey,
       c.c_name,
       c.c_acctbal,
       c.c_phone,
       n.n_name,
       c.c_address,
       c.c_comment
       order by
       revenue desc
       limit 20
      

      Exception:

      java.sql.SQLException: RESOURCE ERROR: One or more nodes ran out of memory while executing the query.
      
      AGGR OOM at First Phase. Partitions: 8. Estimated batch size: 18481152. values size: 1048576. Output alloc size: 1048576. Planned batches: 8 Memory limit: 313709266 so far allocated: 2097152. 
       Fragment 4:88
      
      [Error Id: 81017b59-dfa3-4db9-8673-bee7b80f8acd on atsqa6c82.qa.lab:31010]
      
      (org.apache.drill.exec.exception.OutOfMemoryException) AGGR OOM at First Phase. Partitions: 8. Estimated batch size: 18481152. values size: 1048576. Output alloc size: 1048576. Planned batches: 8 Memory limit: 313709266 so far allocated: 2097152. 
       org.apache.drill.exec.test.generated.HashAggregatorGen3296.spillIfNeeded():1419
       org.apache.drill.exec.test.generated.HashAggregatorGen3296.doSpill():1381
       org.apache.drill.exec.test.generated.HashAggregatorGen3296.checkGroupAndAggrValues():1304
       org.apache.drill.exec.test.generated.HashAggregatorGen3296.doWork():592
       org.apache.drill.exec.physical.impl.aggregate.HashAggBatch.innerNext():176
       org.apache.drill.exec.record.AbstractRecordBatch.next():164
       org.apache.drill.exec.physical.impl.BaseRootExec.next():105
       org.apache.drill.exec.physical.impl.partitionsender.PartitionSenderRootExec.innerNext():152
       org.apache.drill.exec.physical.impl.BaseRootExec.next():95
       org.apache.drill.exec.work.fragment.FragmentExecutor$1.run():292
       org.apache.drill.exec.work.fragment.FragmentExecutor$1.run():279
       java.security.AccessController.doPrivileged():-2
       javax.security.auth.Subject.doAs():422
       org.apache.hadoop.security.UserGroupInformation.doAs():1595
       org.apache.drill.exec.work.fragment.FragmentExecutor.run():279
       org.apache.drill.common.SelfCleaningRunnable.run():38
       java.util.concurrent.ThreadPoolExecutor.runWorker():1149
       java.util.concurrent.ThreadPoolExecutor$Worker.run():624
       java.lang.Thread.run():748
      
      at org.apache.drill.jdbc.impl.DrillCursor.nextRowInternally(DrillCursor.java:530)
       at org.apache.drill.jdbc.impl.DrillCursor.next(DrillCursor.java:634)
       at oadd.org.apache.calcite.avatica.AvaticaResultSet.next(AvaticaResultSet.java:207)
       at org.apache.drill.jdbc.impl.DrillResultSetImpl.next(DrillResultSetImpl.java:155)
       at org.apache.drill.test.framework.DrillTestJdbc.executeQuery(DrillTestJdbc.java:253)
       at org.apache.drill.test.framework.DrillTestJdbc.run(DrillTestJdbc.java:115)
       at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
       at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
       at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
       at java.lang.Thread.run(Thread.java:748)
       Caused by: oadd.org.apache.drill.common.exceptions.UserRemoteException: RESOURCE ERROR: One or more nodes ran out of memory while executing the query.
      
      AGGR OOM at First Phase. Partitions: 8. Estimated batch size: 18481152. values size: 1048576. Output alloc size: 1048576. Planned batches: 8 Memory limit: 313709266 so far allocated: 2097152. 
       Fragment 4:88
      
      [Error Id: 81017b59-dfa3-4db9-8673-bee7b80f8acd on atsqa6c82.qa.lab:31010]
      
      (org.apache.drill.exec.exception.OutOfMemoryException) AGGR OOM at First Phase. Partitions: 8. Estimated batch size: 18481152. values size: 1048576. Output alloc size: 1048576. Planned batches: 8 Memory limit: 313709266 so far allocated: 2097152. 
       org.apache.drill.exec.test.generated.HashAggregatorGen3296.spillIfNeeded():1419
       org.apache.drill.exec.test.generated.HashAggregatorGen3296.doSpill():1381
       org.apache.drill.exec.test.generated.HashAggregatorGen3296.checkGroupAndAggrValues():1304
       org.apache.drill.exec.test.generated.HashAggregatorGen3296.doWork():592
       org.apache.drill.exec.physical.impl.aggregate.HashAggBatch.innerNext():176
       org.apache.drill.exec.record.AbstractRecordBatch.next():164
       org.apache.drill.exec.physical.impl.BaseRootExec.next():105
       org.apache.drill.exec.physical.impl.partitionsender.PartitionSenderRootExec.innerNext():152
       org.apache.drill.exec.physical.impl.BaseRootExec.next():95
       org.apache.drill.exec.work.fragment.FragmentExecutor$1.run():292
       org.apache.drill.exec.work.fragment.FragmentExecutor$1.run():279
       java.security.AccessController.doPrivileged():-2
       javax.security.auth.Subject.doAs():422
       org.apache.hadoop.security.UserGroupInformation.doAs():1595
       org.apache.drill.exec.work.fragment.FragmentExecutor.run():279
       org.apache.drill.common.SelfCleaningRunnable.run():38
       java.util.concurrent.ThreadPoolExecutor.runWorker():1149
       java.util.concurrent.ThreadPoolExecutor$Worker.run():624
       java.lang.Thread.run():748
      

      Query 2: 
      Advanced/tpch/tpch_sf100/parquet/08.q

      select
      o_year,
      sum(case
      when nation = 'EGYPT' then volume
      else 0
      end) / sum(volume) as mkt_share
      from
      (
      select
      extract(year from o.o_orderdate) as o_year,
      l.l_extendedprice * (1 - l.l_discount) as volume,
      n2.n_name as nation
      from
      part p,
      supplier s,
      lineitem l,
      orders o,
      customer c,
      nation n1,
      nation n2,
      region r
      where
      p.p_partkey = l.l_partkey
      and s.s_suppkey = l.l_suppkey
      and l.l_orderkey = o.o_orderkey
      and o.o_custkey = c.c_custkey
      and c.c_nationkey = n1.n_nationkey
      and n1.n_regionkey = r.r_regionkey
      and r.r_name = 'MIDDLE EAST'
      and s.s_nationkey = n2.n_nationkey
      and o.o_orderdate between date '1995-01-01' and date '1996-12-31'
      and p.p_type = 'PROMO BRUSHED COPPER'
      ) as all_nations
      group by
      o_year
      order by
      o_year
      

      Exception:

      java.sql.SQLException: RESOURCE ERROR: One or more nodes ran out of memory while executing the query.
      
      Failure allocating buffer.
      Fragment 4:57
      
      [Error Id: a5eeae54-ac8f-42fa-9af1-03247e6bc316 on atsqa6c82.qa.lab:31010]
      
        (org.apache.drill.exec.exception.OutOfMemoryException) Failure allocating buffer.
          io.netty.buffer.PooledByteBufAllocatorL.allocate():67
          org.apache.drill.exec.memory.AllocationManager.<init>():84
          org.apache.drill.exec.memory.BaseAllocator.bufferWithoutReservation():258
          org.apache.drill.exec.memory.BaseAllocator.buffer():241
          org.apache.drill.exec.memory.BaseAllocator.buffer():211
          org.apache.drill.exec.vector.VarCharVector.allocateNew():389
          org.apache.drill.exec.vector.NullableVarCharVector.allocateNew():236
          org.apache.drill.exec.vector.AllocationHelper.allocatePrecomputedChildCount():41
          org.apache.drill.exec.vector.AllocationHelper.allocate():54
          org.apache.drill.exec.vector.AllocationHelper.allocate():28
          org.apache.drill.exec.physical.impl.ScanBatch$Mutator.populateImplicitVectors():446
          org.apache.drill.exec.physical.impl.ScanBatch$Mutator.access$200():304
          org.apache.drill.exec.physical.impl.ScanBatch.populateImplicitVectorsAndSetCount():267
          org.apache.drill.exec.physical.impl.ScanBatch.next():175
          org.apache.drill.exec.record.AbstractRecordBatch.next():118
          org.apache.drill.exec.record.AbstractRecordBatch.next():108
          org.apache.drill.exec.record.AbstractUnaryRecordBatch.innerNext():63
          org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.innerNext():137
          org.apache.drill.exec.record.AbstractRecordBatch.next():164
          org.apache.drill.exec.record.AbstractRecordBatch.next():118
          org.apache.drill.exec.test.generated.HashJoinProbeGen4786.executeProbePhase():127
          org.apache.drill.exec.test.generated.HashJoinProbeGen4786.probeAndProject():235
          org.apache.drill.exec.physical.impl.join.HashJoinBatch.innerNext():220
          org.apache.drill.exec.record.AbstractRecordBatch.next():164
          org.apache.drill.exec.record.AbstractRecordBatch.next():118
          org.apache.drill.exec.test.generated.HashJoinProbeGen4788.executeProbePhase():127
          org.apache.drill.exec.test.generated.HashJoinProbeGen4788.probeAndProject():235
          org.apache.drill.exec.physical.impl.join.HashJoinBatch.innerNext():220
          org.apache.drill.exec.record.AbstractRecordBatch.next():164
          org.apache.drill.exec.physical.impl.BaseRootExec.next():105
          org.apache.drill.exec.physical.impl.partitionsender.PartitionSenderRootExec.innerNext():152
          org.apache.drill.exec.physical.impl.BaseRootExec.next():95
          org.apache.drill.exec.work.fragment.FragmentExecutor$1.run():292
          org.apache.drill.exec.work.fragment.FragmentExecutor$1.run():279
          java.security.AccessController.doPrivileged():-2
          javax.security.auth.Subject.doAs():422
          org.apache.hadoop.security.UserGroupInformation.doAs():1595
          org.apache.drill.exec.work.fragment.FragmentExecutor.run():279
          org.apache.drill.common.SelfCleaningRunnable.run():38
          java.util.concurrent.ThreadPoolExecutor.runWorker():1149
          java.util.concurrent.ThreadPoolExecutor$Worker.run():624
          java.lang.Thread.run():748
        Caused By (io.netty.util.internal.OutOfDirectMemoryError) failed to allocate 16777216 byte(s) of direct memory (used: 34359738368, max: 34359738368)
          io.netty.util.internal.PlatformDependent.incrementMemoryCounter():510
          io.netty.util.internal.PlatformDependent.allocateDirectNoCleaner():464
          io.netty.buffer.PoolArena$DirectArena.allocateDirect():766
          io.netty.buffer.PoolArena$DirectArena.newChunk():742
          io.netty.buffer.PoolArena.allocateNormal():244
          io.netty.buffer.PoolArena.allocate():226
          io.netty.buffer.PoolArena.allocate():146
          io.netty.buffer.PooledByteBufAllocatorL$InnerAllocator.newDirectBufferL():169
          io.netty.buffer.PooledByteBufAllocatorL$InnerAllocator.directBuffer():201
          io.netty.buffer.PooledByteBufAllocatorL.allocate():65
          org.apache.drill.exec.memory.AllocationManager.<init>():84
          org.apache.drill.exec.memory.BaseAllocator.bufferWithoutReservation():258
          org.apache.drill.exec.memory.BaseAllocator.buffer():241
          org.apache.drill.exec.memory.BaseAllocator.buffer():211
          org.apache.drill.exec.vector.VarCharVector.allocateNew():389
          org.apache.drill.exec.vector.NullableVarCharVector.allocateNew():236
          org.apache.drill.exec.vector.AllocationHelper.allocatePrecomputedChildCount():41
          org.apache.drill.exec.vector.AllocationHelper.allocate():54
          org.apache.drill.exec.vector.AllocationHelper.allocate():28
          org.apache.drill.exec.physical.impl.ScanBatch$Mutator.populateImplicitVectors():446
          org.apache.drill.exec.physical.impl.ScanBatch$Mutator.access$200():304
          org.apache.drill.exec.physical.impl.ScanBatch.populateImplicitVectorsAndSetCount():267
          org.apache.drill.exec.physical.impl.ScanBatch.next():175
          org.apache.drill.exec.record.AbstractRecordBatch.next():118
          org.apache.drill.exec.record.AbstractRecordBatch.next():108
          org.apache.drill.exec.record.AbstractUnaryRecordBatch.innerNext():63
          org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.innerNext():137
          org.apache.drill.exec.record.AbstractRecordBatch.next():164
          org.apache.drill.exec.record.AbstractRecordBatch.next():118
          org.apache.drill.exec.test.generated.HashJoinProbeGen4786.executeProbePhase():127
          org.apache.drill.exec.test.generated.HashJoinProbeGen4786.probeAndProject():235
          org.apache.drill.exec.physical.impl.join.HashJoinBatch.innerNext():220
          org.apache.drill.exec.record.AbstractRecordBatch.next():164
          org.apache.drill.exec.record.AbstractRecordBatch.next():118
          org.apache.drill.exec.test.generated.HashJoinProbeGen4788.executeProbePhase():127
          org.apache.drill.exec.test.generated.HashJoinProbeGen4788.probeAndProject():235
          org.apache.drill.exec.physical.impl.join.HashJoinBatch.innerNext():220
          org.apache.drill.exec.record.AbstractRecordBatch.next():164
          org.apache.drill.exec.physical.impl.BaseRootExec.next():105
          org.apache.drill.exec.physical.impl.partitionsender.PartitionSenderRootExec.innerNext():152
          org.apache.drill.exec.physical.impl.BaseRootExec.next():95
          org.apache.drill.exec.work.fragment.FragmentExecutor$1.run():292
          org.apache.drill.exec.work.fragment.FragmentExecutor$1.run():279
          java.security.AccessController.doPrivileged():-2
          javax.security.auth.Subject.doAs():422
          org.apache.hadoop.security.UserGroupInformation.doAs():1595
          org.apache.drill.exec.work.fragment.FragmentExecutor.run():279
          org.apache.drill.common.SelfCleaningRunnable.run():38
          java.util.concurrent.ThreadPoolExecutor.runWorker():1149
          java.util.concurrent.ThreadPoolExecutor$Worker.run():624
          java.lang.Thread.run():748
      

      Attachments

        1. drillbit.log.txt
          37 kB
          Abhishek Girish

        Issue Links

          Activity

            People

              vitalii Vitalii Diravka
              agirish Abhishek Girish
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: