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

Root allocator should prevent allocating more than the available direct memory

    XMLWordPrintableJSON

Details

    Description

      git commit # : 09b262776e965ea17a6a863801f7e1ee3e5b3d5a

      I ran the below 2 queries (each query duplicated 10 times.....so total 20 queries) using 10 different clients on an 8 node cluster. The drillbit on one of the nodes hits an OOM error. The allocator should have caught this earlier.

      Query 1:

      select count(*) 
      from (
          select l_orderkey, l_partkey, l_suppkey 
          from lineitem_nocompression_256
          group by l_orderkey, l_partkey, l_suppkey
      ) s
      

      Query 2 :

      select count(*) from
              dfs.concurrency.customer_nocompression_256_filtered c,
              dfs.concurrency.orders_nocompression_256 o,
              dfs.concurrency.lineitem_nocompression_256 l
          where
              c.c_custkey = o.o_custkey
              and l.l_orderkey = o.o_orderkey
      

      Exception from the logs

      Failure allocating buffer.
      
      [Error Id: cd71a6a0-7f41-4fe4-8bbb-294119adfebf ]
              at org.apache.drill.common.exceptions.UserException$Builder.build(UserException.java:543) ~[drill-common-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:267) [drill-java-exec-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.common.SelfCleaningRunnable.run(SelfCleaningRunnable.java:38) [drill-common-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) [na:1.7.0_51]
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_51]
              at java.lang.Thread.run(Thread.java:744) [na:1.7.0_51]
      Caused by: org.apache.drill.exec.exception.OutOfMemoryException: Failure allocating buffer.
              at io.netty.buffer.PooledByteBufAllocatorL.allocate(PooledByteBufAllocatorL.java:64) ~[drill-memory-base-1.7.0-SNAPSHOT.jar:4.0.27.Final]
              at org.apache.drill.exec.memory.AllocationManager.<init>(AllocationManager.java:80) ~[drill-memory-base-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.memory.BaseAllocator.bufferWithoutReservation(BaseAllocator.java:239) ~[drill-memory-base-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.memory.BaseAllocator.buffer(BaseAllocator.java:221) ~[drill-memory-base-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.memory.BaseAllocator.buffer(BaseAllocator.java:191) ~[drill-memory-base-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.vector.IntVector.allocateBytes(IntVector.java:200) ~[vector-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.vector.IntVector.allocateNew(IntVector.java:182) ~[vector-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.test.generated.HashTableGen54.allocMetadataVector(HashTableTemplate.java:757) ~[na:na]
              at org.apache.drill.exec.test.generated.HashTableGen54.resizeAndRehashIfNeeded(HashTableTemplate.java:722) ~[na:na]
              at org.apache.drill.exec.test.generated.HashTableGen54.insertEntry(HashTableTemplate.java:631) ~[na:na]
              at org.apache.drill.exec.test.generated.HashTableGen54.put(HashTableTemplate.java:609) ~[na:na]
              at org.apache.drill.exec.test.generated.HashTableGen54.put(HashTableTemplate.java:542) ~[na:na]
              at org.apache.drill.exec.test.generated.HashAggregatorGen52.checkGroupAndAggrValues(HashAggTemplate.java:542) ~[na:na]
              at org.apache.drill.exec.test.generated.HashAggregatorGen52.doWork(HashAggTemplate.java:300) ~[na:na]
              at org.apache.drill.exec.physical.impl.aggregate.HashAggBatch.innerNext(HashAggBatch.java:133) ~[drill-java-exec-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:162) ~[drill-java-exec-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119) ~[drill-java-exec-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:109) ~[drill-java-exec-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractSingleRecordBatch.innerNext(AbstractSingleRecordBatch.java:51) ~[drill-java-exec-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.innerNext(ProjectRecordBatch.java:129) ~[drill-java-exec-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:162) ~[drill-java-exec-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119) ~[drill-java-exec-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:109) ~[drill-java-exec-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch.innerNext(StreamingAggBatch.java:137) ~[drill-java-exec-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:162) ~[drill-java-exec-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.BaseRootExec.next(BaseRootExec.java:104) ~[drill-java-exec-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.SingleSenderCreator$SingleSenderRootExec.innerNext(SingleSenderCreator.java:92) ~[drill-java-exec-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.physical.impl.BaseRootExec.next(BaseRootExec.java:94) ~[drill-java-exec-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.work.fragment.FragmentExecutor$1.run(FragmentExecutor.java:257) ~[drill-java-exec-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at org.apache.drill.exec.work.fragment.FragmentExecutor$1.run(FragmentExecutor.java:251) ~[drill-java-exec-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              at java.security.AccessController.doPrivileged(Native Method) ~[na:1.7.0_51]
              at javax.security.auth.Subject.doAs(Subject.java:415) ~[na:1.7.0_51]
              at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1595) ~[hadoop-common-2.7.0-mapr-1602.jar:na]
              at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:251) [drill-java-exec-1.7.0-SNAPSHOT.jar:1.7.0-SNAPSHOT]
              ... 4 common frames omitted
      Caused by: java.lang.OutOfMemoryError: Direct buffer memory
              at java.nio.Bits.reserveMemory(Bits.java:658) ~[na:1.7.0_51]
              at java.nio.DirectByteBuffer.<init>(DirectByteBuffer.java:123) ~[na:1.7.0_51]
              at java.nio.ByteBuffer.allocateDirect(ByteBuffer.java:306) ~[na:1.7.0_51]
              at io.netty.buffer.UnpooledUnsafeDirectByteBuf.allocateDirect(UnpooledUnsafeDirectByteBuf.java:108) ~[netty-buffer-4.0.27.Final.jar:4.0.27.Final]
              at io.netty.buffer.UnpooledUnsafeDirectByteBuf.<init>(UnpooledUnsafeDirectByteBuf.java:69) ~[netty-buffer-4.0.27.Final.jar:4.0.27.Final]
              at io.netty.buffer.UnpooledByteBufAllocator.newDirectBuffer(UnpooledByteBufAllocator.java:50) ~[netty-buffer-4.0.27.Final.jar:4.0.27.Final]
              at io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:155) ~[netty-buffer-4.0.27.Final.jar:4.0.27.Final]
              at io.netty.buffer.PooledByteBufAllocatorL$InnerAllocator.newDirectBufferL(PooledByteBufAllocatorL.java:155) ~[drill-memory-base-1.7.0-SNAPSHOT.jar:4.0.27.Final]
              at io.netty.buffer.PooledByteBufAllocatorL$InnerAllocator.directBuffer(PooledByteBufAllocatorL.java:195) ~[drill-memory-base-1.7.0-SNAPSHOT.jar:4.0.27.Final]
              at io.netty.buffer.PooledByteBufAllocatorL.allocate(PooledByteBufAllocatorL.java:62) ~[drill-memory-base-1.7.0-SNAPSHOT.jar:4.0.27.Final]
              ... 37 common frames omitted
      

      Attachments

        1. error.log
          2.19 MB
          Rahul Kumar Challapalli

        Activity

          People

            ben-zvi Boaz Ben-Zvi
            rkins Rahul Kumar Challapalli
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated: