You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@drill.apache.org by "Vlad Rozov (JIRA)" <ji...@apache.org> on 2018/08/31 18:34:00 UTC

[jira] [Assigned] (DRILL-6247) Minor fragments remain in CANCELLATION_REQUESTED state after query failure

     [ https://issues.apache.org/jira/browse/DRILL-6247?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Vlad Rozov reassigned DRILL-6247:
---------------------------------

    Assignee:     (was: Vlad Rozov)

> Minor fragments remain in CANCELLATION_REQUESTED state after query failure
> --------------------------------------------------------------------------
>
>                 Key: DRILL-6247
>                 URL: https://issues.apache.org/jira/browse/DRILL-6247
>             Project: Apache Drill
>          Issue Type: Bug
>          Components: Execution - Flow
>    Affects Versions: 1.13.0
>            Reporter: Khurram Faraaz
>            Priority: Major
>         Attachments: 25569e98-10f9-2fe2-9dec-0a42f3ad45fa.sys.drill, cancellation_requested_march_14.png, drillbit_snippet.log, jstack_cancellation_requested.txt
>
>
> Once a query fails, in this case due to an OOM in RPC we see many minor fragments are reported to be in CANCELLATION_REQUESTED state on Web UI after query has failed. The problem is reproducible. drillbit.log for this failure and jstack output are attached here.
> To reproduce the problem on a 4 node cluster.
> alter system reset all;
> alter system set `planner.slice_target`=1;
> Failing query => SELECT * , FLATTEN(arr) FROM many_json_files;
> Drill 1.13.0-SNAPSHOT, commit id: 766315ea17377199897d685ab801edd38394fe01
> Stack trace from output of jstack, fragment 0:0 is reported to be in CANCELLATION_REQUESTED state on Drill Web UI
> jstack -l 13488 > jstack_DRILL_6235.txt
> {noformat}
> "25569e98-10f9-2fe2-9dec-0a42f3ad45fa:frag:0:0" #87 daemon prio=10 os_prio=0 tid=0x00007f9d01374360 nid=0x2ff5 waiting on condition [0x00007f9cd5536000]
>  java.lang.Thread.State: WAITING (parking)
>  at sun.misc.Unsafe.park(Native Method)
>  - parking to wait for <0x00000007a388b300> (a org.apache.drill.exec.rpc.ResettableBarrier$InternalSynchronizer)
>  at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>  at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
>  at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
>  at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
>  at org.apache.drill.exec.rpc.ResettableBarrier.await(ResettableBarrier.java:70)
>  at org.apache.drill.exec.rpc.AbstractRemoteConnection$WriteManager.waitForWritable(AbstractRemoteConnection.java:114)
>  at org.apache.drill.exec.rpc.AbstractRemoteConnection.blockOnNotWritable(AbstractRemoteConnection.java:76)
>  at org.apache.drill.exec.rpc.RpcBus.send(RpcBus.java:108)
>  at org.apache.drill.exec.rpc.user.UserServer$BitToUserConnection.sendData(UserServer.java:275)
>  at org.apache.drill.exec.ops.AccountingUserConnection.sendData(AccountingUserConnection.java:42)
>  at org.apache.drill.exec.physical.impl.ScreenCreator$ScreenRoot.innerNext(ScreenCreator.java:120)
>  at org.apache.drill.exec.physical.impl.BaseRootExec.next(BaseRootExec.java:95)
>  at org.apache.drill.exec.work.fragment.FragmentExecutor$1.run(FragmentExecutor.java:233)
>  at org.apache.drill.exec.work.fragment.FragmentExecutor$1.run(FragmentExecutor.java:226)
>  at java.security.AccessController.doPrivileged(Native Method)
>  at javax.security.auth.Subject.doAs(Subject.java:422)
>  at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1595)
>  at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:226)
>  at org.apache.drill.common.SelfCleaningRunnable.run(SelfCleaningRunnable.java:38)
>  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)
> {noformat}
> Stack trace from drillbit.log
> {noformat}
> 2018-03-14 10:52:44,545 [25569e98-10f9-2fe2-9dec-0a42f3ad45fa:frag:1:49] INFO o.a.d.e.w.fragment.FragmentExecutor - User Error Occurred: One or more nodes ran out of memory while executing the query. (Failure allocating buffer.)
> org.apache.drill.common.exceptions.UserException: RESOURCE ERROR: One or more nodes ran out of memory while executing the query.
> Failure allocating buffer.
> [Error Id: b83884df-af31-411a-9b28-554c294a7357 ]
>  at org.apache.drill.common.exceptions.UserException$Builder.build(UserException.java:633) ~[drill-common-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:243) [drill-java-exec-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at org.apache.drill.common.SelfCleaningRunnable.run(SelfCleaningRunnable.java:38) [drill-common-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [na:1.8.0_161]
>  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [na:1.8.0_161]
>  at java.lang.Thread.run(Thread.java:748) [na:1.8.0_161]
> Caused by: org.apache.drill.exec.exception.OutOfMemoryException: Failure allocating buffer.
>  at io.netty.buffer.PooledByteBufAllocatorL.allocate(PooledByteBufAllocatorL.java:67) ~[drill-memory-base-1.13.0-SNAPSHOT.jar:4.0.48.Final]
>  at org.apache.drill.exec.memory.AllocationManager.<init>(AllocationManager.java:83) ~[drill-memory-base-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at org.apache.drill.exec.memory.BaseAllocator.bufferWithoutReservation(BaseAllocator.java:258) ~[drill-memory-base-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at org.apache.drill.exec.memory.BaseAllocator.buffer(BaseAllocator.java:241) ~[drill-memory-base-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at org.apache.drill.exec.memory.BaseAllocator.buffer(BaseAllocator.java:211) ~[drill-memory-base-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at org.apache.drill.exec.vector.BigIntVector.allocateBytes(BigIntVector.java:239) ~[vector-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at org.apache.drill.exec.vector.BigIntVector.allocateNew(BigIntVector.java:219) ~[vector-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at org.apache.drill.exec.vector.RepeatedBigIntVector.allocateNew(RepeatedBigIntVector.java:272) ~[vector-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at org.apache.drill.exec.vector.AllocationHelper.allocatePrecomputedChildCount(AllocationHelper.java:41) ~[vector-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at org.apache.drill.exec.vector.AllocationHelper.allocate(AllocationHelper.java:66) ~[vector-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at org.apache.drill.exec.record.RecordBatchSizer$ColumnSize.allocateVector(RecordBatchSizer.java:403) ~[drill-java-exec-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at org.apache.drill.exec.physical.impl.flatten.FlattenRecordBatch.doAlloc(FlattenRecordBatch.java:276) ~[drill-java-exec-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at org.apache.drill.exec.physical.impl.flatten.FlattenRecordBatch.handleRemainder(FlattenRecordBatch.java:240) ~[drill-java-exec-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at org.apache.drill.exec.physical.impl.flatten.FlattenRecordBatch.innerNext(FlattenRecordBatch.java:165) ~[drill-java-exec-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:164) ~[drill-java-exec-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at org.apache.drill.exec.physical.impl.BaseRootExec.next(BaseRootExec.java:105) ~[drill-java-exec-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at org.apache.drill.exec.physical.impl.SingleSenderCreator$SingleSenderRootExec.innerNext(SingleSenderCreator.java:93) ~[drill-java-exec-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at org.apache.drill.exec.physical.impl.BaseRootExec.next(BaseRootExec.java:95) ~[drill-java-exec-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at org.apache.drill.exec.work.fragment.FragmentExecutor$1.run(FragmentExecutor.java:233) ~[drill-java-exec-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at org.apache.drill.exec.work.fragment.FragmentExecutor$1.run(FragmentExecutor.java:226) ~[drill-java-exec-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  at java.security.AccessController.doPrivileged(Native Method) ~[na:1.8.0_161]
>  at javax.security.auth.Subject.doAs(Subject.java:422) ~[na:1.8.0_161]
>  at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1595) ~[hadoop-common-2.7.0-mapr-1707.jar:na]
>  at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:226) [drill-java-exec-1.13.0-SNAPSHOT.jar:1.13.0-SNAPSHOT]
>  ... 4 common frames omitted
> Caused by: io.netty.util.internal.OutOfDirectMemoryError: failed to allocate 33595392 byte(s) of direct memory (used: 8574050304, max: 8589934592)
>  at io.netty.util.internal.PlatformDependent.incrementMemoryCounter(PlatformDependent.java:510) ~[netty-common-4.0.48.Final.jar:4.0.48.Final]
>  at io.netty.util.internal.PlatformDependent.allocateDirectNoCleaner(PlatformDependent.java:464) ~[netty-common-4.0.48.Final.jar:4.0.48.Final]
>  at io.netty.buffer.UnpooledUnsafeNoCleanerDirectByteBuf.allocateDirect(UnpooledUnsafeNoCleanerDirectByteBuf.java:30) ~[netty-buffer-4.0.48.Final.jar:4.0.48.Final]
>  at io.netty.buffer.UnpooledByteBufAllocator$InstrumentedUnpooledUnsafeNoCleanerDirectByteBuf.allocateDirect(UnpooledByteBufAllocator.java:169) ~[netty-buffer-4.0.48.Final.jar:4.0.48.Final]
>  at io.netty.buffer.UnpooledUnsafeDirectByteBuf.<init>(UnpooledUnsafeDirectByteBuf.java:67) ~[netty-buffer-4.0.48.Final.jar:4.0.48.Final]
>  at io.netty.buffer.UnpooledUnsafeNoCleanerDirectByteBuf.<init>(UnpooledUnsafeNoCleanerDirectByteBuf.java:25) ~[netty-buffer-4.0.48.Final.jar:4.0.48.Final]
>  at io.netty.buffer.UnpooledByteBufAllocator$InstrumentedUnpooledUnsafeNoCleanerDirectByteBuf.<init>(UnpooledByteBufAllocator.java:164) ~[netty-buffer-4.0.48.Final.jar:4.0.48.Final]
>  at io.netty.buffer.UnpooledByteBufAllocator.newDirectBuffer(UnpooledByteBufAllocator.java:73) ~[netty-buffer-4.0.48.Final.jar:4.0.48.Final]
>  at io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:179) ~[netty-buffer-4.0.48.Final.jar:4.0.48.Final]
>  at io.netty.buffer.PooledByteBufAllocatorL$InnerAllocator.newDirectBufferL(PooledByteBufAllocatorL.java:157) ~[drill-memory-base-1.13.0-SNAPSHOT.jar:4.0.48.Final]
>  at io.netty.buffer.PooledByteBufAllocatorL$InnerAllocator.directBuffer(PooledByteBufAllocatorL.java:201) ~[drill-memory-base-1.13.0-SNAPSHOT.jar:4.0.48.Final]
>  at io.netty.buffer.PooledByteBufAllocatorL.allocate(PooledByteBufAllocatorL.java:65) ~[drill-memory-base-1.13.0-SNAPSHOT.jar:4.0.48.Final]
>  ... 27 common frames omitted
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)