You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@drill.apache.org by "Roman (JIRA)" <ji...@apache.org> on 2017/08/04 17:06:00 UTC

[jira] [Commented] (DRILL-3119) Query stays in "CANCELLATION_REQUESTED" status in UI after OOM of Direct buffer memory

    [ https://issues.apache.org/jira/browse/DRILL-3119?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16114634#comment-16114634 ] 

Roman commented on DRILL-3119:
------------------------------

I tried to reproduce this issue on 1 drillbit cluster using mapr-drill-1.2.0.201510071035 with query:

{code:title=Query|borderStyle=solid}
select count(*) from
(
select a.* from dfs.tmp.`lineitembig.dat` a, dfs.tmp.`lineitembig.dat` b
where a.columns[0]=b.columns[0]
);
{code}

And I can get similar error:
{code:title=Error|borderStyle=solid}
Exception in thread "267b60a0-d710-d8b9-1ff0-62761fcf4c1e:frag:1:0" java.lang.OutOfMemoryError: Direct buffer memory
	at java.nio.Bits.reserveMemory(Bits.java:694)
	at java.nio.DirectByteBuffer.<init>(DirectByteBuffer.java:123)
	at java.nio.ByteBuffer.allocateDirect(ByteBuffer.java:311)
	at io.netty.util.internal.Cleaner0.<clinit>(Cleaner0.java:37)
	at io.netty.util.internal.PlatformDependent0.freeDirectBuffer(PlatformDependent0.java:147)
	at io.netty.util.internal.PlatformDependent.freeDirectBuffer(PlatformDependent.java:281)
	at io.netty.buffer.PoolArena$DirectArena.destroyChunk(PoolArena.java:448)
	at io.netty.buffer.PoolChunkList.free(PoolChunkList.java:70)
	at io.netty.buffer.PoolArena.free(PoolArena.java:203)
	at io.netty.buffer.PooledByteBuf.deallocate(PooledByteBuf.java:147)
	at io.netty.buffer.AbstractReferenceCountedByteBuf.release(AbstractReferenceCountedByteBuf.java:128)
	at io.netty.buffer.WrappedByteBuf.release(WrappedByteBuf.java:825)
	at io.netty.buffer.UnsafeDirectLittleEndian.release(UnsafeDirectLittleEndian.java:238)
	at io.netty.buffer.AbstractDerivedByteBuf.release(AbstractDerivedByteBuf.java:55)
	at io.netty.buffer.DrillBuf.release(DrillBuf.java:252)
	at io.netty.buffer.DrillBuf.release(DrillBuf.java:259)
	at io.netty.buffer.DrillBuf.release(DrillBuf.java:259)
	at io.netty.buffer.DrillBuf.release(DrillBuf.java:259)
	at io.netty.buffer.DrillBuf.release(DrillBuf.java:239)
	at org.apache.drill.exec.vector.BaseDataValueVector.clear(BaseDataValueVector.java:39)
	at org.apache.drill.exec.vector.VarCharVector.clear(VarCharVector.java:206)
	at org.apache.drill.exec.vector.NullableVarCharVector.clear(NullableVarCharVector.java:151)
	at org.apache.drill.exec.record.HyperVectorWrapper.clear(HyperVectorWrapper.java:82)
	at org.apache.drill.exec.record.VectorContainer.zeroVectors(VectorContainer.java:312)
	at org.apache.drill.exec.record.VectorContainer.clear(VectorContainer.java:296)
	at org.apache.drill.exec.physical.impl.join.HashJoinBatch.close(HashJoinBatch.java:529)
	at org.apache.drill.exec.physical.impl.BaseRootExec.close(BaseRootExec.java:122)
	at org.apache.drill.exec.work.fragment.FragmentExecutor.closeOutResources(FragmentExecutor.java:341)
	at org.apache.drill.exec.work.fragment.FragmentExecutor.cleanup(FragmentExecutor.java:173)
	at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:292)
	at org.apache.drill.common.SelfCleaningRunnable.run(SelfCleaningRunnable.java:38)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:748)
{code}

And after this error Drill hangs in CANCELLATION_REQUESTED state with jstack:
{code:title=Jstack|borderStyle=solid}
"267b60a0-d710-d8b9-1ff0-62761fcf4c1e:frag:2:0" #63 daemon prio=10 os_prio=0 tid=0x00007f3dfc694000 nid=0x492d waiting on condition [0x00007f3de2d7f000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000f5fb1278> (a java.util.concurrent.Semaphore$NonfairSync)
        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 java.util.concurrent.Semaphore.acquire(Semaphore.java:467)
        at org.apache.drill.exec.ops.SendingAccountor.waitForSendComplete(SendingAccountor.java:48)
        - locked <0x00000000f5fb1240> (a org.apache.drill.exec.ops.SendingAccountor)
        at org.apache.drill.exec.ops.FragmentContext.waitForSendComplete(FragmentContext.java:436)
        at org.apache.drill.exec.physical.impl.BaseRootExec.close(BaseRootExec.java:112)
        at org.apache.drill.exec.physical.impl.partitionsender.PartitionSenderRootExec.close(PartitionSenderRootExec.java:336)
        at org.apache.drill.exec.work.fragment.FragmentExecutor.closeOutResources(FragmentExecutor.java:341)
        at org.apache.drill.exec.work.fragment.FragmentExecutor.cleanup(FragmentExecutor.java:173)
        at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:292)
        at org.apache.drill.common.SelfCleaningRunnable.run(SelfCleaningRunnable.java:38)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:748)
{code}

Under the same conditions in Drill 1.11 (after DRILL-5599), I got 2 types of error (see below) Drillbit changes state to FAILED and does not hang.

{code:title=Error 1|borderStyle=solid}
Error: RESOURCE ERROR: One or more nodes ran out of memory while executing the query.

Failure while allocating buffer.
Fragment 1:1

[Error Id: 3e789503-aa54-44a6-ba22-03b40d4b3549 on node1:31010] (state=,code=0)
{code}

{code:title=Error 2|borderStyle=solid}
Error: SYSTEM ERROR: ChannelClosedException: Channel closed /192.168.121.7:35708 <--> node1/192.168.121.7:31012.

Fragment 5:0

[Error Id: f3c19f7d-f762-44c7-ab9e-f16162d39821 on node1:31010] (state=,code=0)
{code}

I tried to change memory setting, a size of tables but I did not get reproduce of a hang in Drill 1.11. Also, according to the similar steps for reproducing and same error trace, this bug is duplicate DRILL-3241 (which also duplicate DRILL-5599) so I close this ticket as a duplicate.

> Query stays in "CANCELLATION_REQUESTED" status in UI after OOM of Direct buffer memory
> --------------------------------------------------------------------------------------
>
>                 Key: DRILL-3119
>                 URL: https://issues.apache.org/jira/browse/DRILL-3119
>             Project: Apache Drill
>          Issue Type: Bug
>          Components: Execution - Flow
>    Affects Versions: 1.0.0
>            Reporter: Hao Zhu
>            Assignee: Roman
>
> Tested in 1.0.0 with below commit id:
> {code}
> > select * from sys.version;
> +-------------------------------------------+--------------------------------------------------------------------+----------------------------+--------------+----------------------------+
> |                 commit_id                 |                           commit_message                           |        commit_time         | build_email  |         build_time         |
> +-------------------------------------------+--------------------------------------------------------------------+----------------------------+--------------+----------------------------+
> | d8b19759657698581cc0d01d7038797952888123  | DRILL-3100: TestImpersonationDisabledWithMiniDFS fails on Windows  | 15.05.2015 @ 01:18:03 EDT  | Unknown      | 15.05.2015 @ 03:07:10 EDT  |
> +-------------------------------------------+--------------------------------------------------------------------+----------------------------+--------------+----------------------------+
> 1 row selected (0.26 seconds)
> {code}
> How to reproduce:
> 1. Single node cluster.
> 2.  Reduce DRILL_MAX_DIRECT_MEMORY="2G".
> 3. Run a hash join which is big enough to trigger OOM.
> eg:
> {code}
> select count(*) from
> (
> select a.* from dfs.root.`user/hive/warehouse/passwords_csv_big` a, dfs.root.`user/hive/warehouse/passwords_csv_big` b
> where a.columns[1]=b.columns[1]
> );
> {code}
> After that, drillbit.log shows OOM:
> {code}
> 2015-05-16 19:24:34,391 [2aa866ba-8939-b184-0ba2-291734329f88:frag:4:4] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:4:4: State change requested from RUNNING --> FINISHED for
> 2015-05-16 19:24:34,391 [2aa866ba-8939-b184-0ba2-291734329f88:frag:4:4] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:4:4. New state: FINISHED
> 2015-05-16 19:24:38,561 [BitServer-5] ERROR o.a.d.exec.rpc.RpcExceptionHandler - Exception in RPC communication.  Connection: /10.0.0.31:31012 <--> /10.0.0.31:41923 (data server).  Closing connection.
> io.netty.handler.codec.DecoderException: java.lang.OutOfMemoryError: Direct buffer memory
> 	at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:233) ~[netty-codec-4.0.27.Final.jar:4.0.27.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339) [netty-transport-4.0.27.Final.jar:4.0.27.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324) [netty-transport-4.0.27.Final.jar:4.0.27.Final]
> 	at io.netty.channel.ChannelInboundHandlerAdapter.channelRead(ChannelInboundHandlerAdapter.java:86) [netty-transport-4.0.27.Final.jar:4.0.27.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339) [netty-transport-4.0.27.Final.jar:4.0.27.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324) [netty-transport-4.0.27.Final.jar:4.0.27.Final]
> 	at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:847) [netty-transport-4.0.27.Final.jar:4.0.27.Final]
> 	at io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:618) [netty-transport-native-epoll-4.0.27.Final-linux-x86_64.jar:na]
> 	at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:329) [netty-transport-native-epoll-4.0.27.Final-linux-x86_64.jar:na]
> 	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:250) [netty-transport-native-epoll-4.0.27.Final-linux-x86_64.jar:na]
> 	at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111) [netty-common-4.0.27.Final.jar:4.0.27.Final]
> 	at java.lang.Thread.run(Thread.java:745) [na:1.8.0_45]
> Caused by: java.lang.OutOfMemoryError: Direct buffer memory
> 	at java.nio.Bits.reserveMemory(Bits.java:658) ~[na:1.8.0_45]
> 	at java.nio.DirectByteBuffer.<init>(DirectByteBuffer.java:123) ~[na:1.8.0_45]
> 	at java.nio.ByteBuffer.allocateDirect(ByteBuffer.java:311) ~[na:1.8.0_45]
> 	at io.netty.buffer.PoolArena$DirectArena.newChunk(PoolArena.java:437) ~[netty-buffer-4.0.27.Final.jar:4.0.27.Final]
> 	at io.netty.buffer.PoolArena.allocateNormal(PoolArena.java:179) ~[netty-buffer-4.0.27.Final.jar:4.0.27.Final]
> 	at io.netty.buffer.PoolArena.allocate(PoolArena.java:168) ~[netty-buffer-4.0.27.Final.jar:4.0.27.Final]
> 	at io.netty.buffer.PoolArena.reallocate(PoolArena.java:280) ~[netty-buffer-4.0.27.Final.jar:4.0.27.Final]
> 	at io.netty.buffer.PooledByteBuf.capacity(PooledByteBuf.java:110) ~[netty-buffer-4.0.27.Final.jar:4.0.27.Final]
> 	at io.netty.buffer.AbstractByteBuf.ensureWritable(AbstractByteBuf.java:251) ~[netty-buffer-4.0.27.Final.jar:4.0.27.Final]
> 	at io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:849) ~[netty-buffer-4.0.27.Final.jar:4.0.27.Final]
> 	at io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:841) ~[netty-buffer-4.0.27.Final.jar:4.0.27.Final]
> 	at io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:831) ~[netty-buffer-4.0.27.Final.jar:4.0.27.Final]
> 	at io.netty.buffer.WrappedByteBuf.writeBytes(WrappedByteBuf.java:600) ~[netty-buffer-4.0.27.Final.jar:4.0.27.Final]
> 	at io.netty.buffer.UnsafeDirectLittleEndian.writeBytes(UnsafeDirectLittleEndian.java:28) ~[drill-java-exec-1.0.0-SNAPSHOT-rebuffed.jar:4.0.27.Final]
> 	at io.netty.handler.codec.ByteToMessageDecoder$1.cumulate(ByteToMessageDecoder.java:92) ~[netty-codec-4.0.27.Final.jar:4.0.27.Final]
> 	at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:227) ~[netty-codec-4.0.27.Final.jar:4.0.27.Final]
> 	... 11 common frames omitted
> 2015-05-16 19:24:38,562 [BitClient-1] ERROR o.a.d.exec.rpc.RpcExceptionHandler - Exception in RPC communication.  Connection: /10.0.0.31:41923 <--> /10.0.0.31:31012 (data client).  Closing connection.
> java.io.IOException: syscall:read(...)() failed: Connection reset by peer
> 2015-05-16 19:24:38,562 [BitClient-1] INFO  o.a.drill.exec.rpc.data.DataClient - Channel closed /10.0.0.31:41923 <--> /10.0.0.31:31012.
> 2015-05-16 19:24:38,563 [BitServer-5] WARN  o.a.d.exec.rpc.RpcExceptionHandler - Exception occurred with closed channel.  Connection: /10.0.0.31:31012 <--> /10.0.0.31:41923 (data server)
> java.nio.channels.ClosedChannelException: null
> 2015-05-16 19:24:38,564 [BitServer-5] INFO  o.a.d.exec.rpc.ProtobufLengthDecoder - Channel is closed, discarding remaining 100358 byte(s) in buffer.
> 2015-05-16 19:25:36,820 [UserServer-1] INFO  o.a.drill.exec.work.foreman.Foreman - State change requested.  RUNNING --> CANCELLATION_REQUESTED
> 2015-05-16 19:25:36,825 [BitServer-4] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:0:0: State change requested from RUNNING --> CANCELLATION_REQUESTED for
> 2015-05-16 19:25:36,825 [BitServer-4] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:0:0. New state: CANCELLATION_REQUESTED
> 2015-05-16 19:25:36,827 [2aa866ba-8939-b184-0ba2-291734329f88:frag:0:0] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:0:0: State change requested from CANCELLATION_REQUESTED --> FINISHED for
> 2015-05-16 19:25:36,827 [2aa866ba-8939-b184-0ba2-291734329f88:frag:0:0] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:0:0. New state: CANCELLED
> 2015-05-16 19:25:36,827 [BitServer-4] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:5:0: State change requested from RUNNING --> CANCELLATION_REQUESTED for
> 2015-05-16 19:25:36,827 [BitServer-4] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:5:0. New state: CANCELLATION_REQUESTED
> 2015-05-16 19:25:36,827 [WorkManager-28] WARN  o.a.d.exec.rpc.control.WorkEventBus - Fragment 2aa866ba-8939-b184-0ba2-291734329f88:0:0 not found in the work bus.
> 2015-05-16 19:25:36,831 [BitServer-4] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:5:1: State change requested from RUNNING --> CANCELLATION_REQUESTED for
> 2015-05-16 19:25:36,832 [BitServer-4] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:5:1. New state: CANCELLATION_REQUESTED
> 2015-05-16 19:25:36,832 [BitServer-4] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:5:2: State change requested from RUNNING --> CANCELLATION_REQUESTED for
> 2015-05-16 19:25:36,832 [BitServer-4] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:5:2. New state: CANCELLATION_REQUESTED
> 2015-05-16 19:25:36,833 [BitServer-4] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:5:3: State change requested from RUNNING --> CANCELLATION_REQUESTED for
> 2015-05-16 19:25:36,833 [BitServer-4] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:5:3. New state: CANCELLATION_REQUESTED
> 2015-05-16 19:25:36,834 [BitServer-4] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:5:4: State change requested from RUNNING --> CANCELLATION_REQUESTED for
> 2015-05-16 19:25:36,834 [BitServer-4] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:5:4. New state: CANCELLATION_REQUESTED
> 2015-05-16 19:25:36,835 [BitServer-4] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:5:5: State change requested from RUNNING --> CANCELLATION_REQUESTED for
> 2015-05-16 19:25:36,835 [BitServer-4] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:5:5. New state: CANCELLATION_REQUESTED
> 2015-05-16 19:25:36,836 [BitServer-4] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:2:0: State change requested from RUNNING --> CANCELLATION_REQUESTED for
> 2015-05-16 19:25:36,836 [BitServer-4] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:2:0. New state: CANCELLATION_REQUESTED
> 2015-05-16 19:25:36,837 [BitServer-4] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:1:0: State change requested from RUNNING --> CANCELLATION_REQUESTED for
> 2015-05-16 19:25:36,838 [BitServer-4] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:1:0. New state: CANCELLATION_REQUESTED
> 2015-05-16 19:25:36,839 [BitServer-4] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:1:1: State change requested from RUNNING --> CANCELLATION_REQUESTED for
> 2015-05-16 19:25:36,839 [BitServer-4] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:1:1. New state: CANCELLATION_REQUESTED
> 2015-05-16 19:25:36,839 [BitServer-4] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:1:2: State change requested from RUNNING --> CANCELLATION_REQUESTED for
> 2015-05-16 19:25:36,839 [BitServer-4] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:1:2. New state: CANCELLATION_REQUESTED
> 2015-05-16 19:25:36,843 [BitServer-4] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:1:3: State change requested from RUNNING --> CANCELLATION_REQUESTED for
> 2015-05-16 19:25:36,848 [BitServer-4] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:1:3. New state: CANCELLATION_REQUESTED
> 2015-05-16 19:25:36,849 [BitServer-4] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:1:4: State change requested from RUNNING --> CANCELLATION_REQUESTED for
> 2015-05-16 19:25:36,849 [BitServer-4] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:1:4. New state: CANCELLATION_REQUESTED
> 2015-05-16 19:25:36,849 [BitServer-4] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:1:5: State change requested from RUNNING --> CANCELLATION_REQUESTED for
> 2015-05-16 19:25:36,850 [BitServer-4] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:1:5. New state: CANCELLATION_REQUESTED
> 2015-05-16 19:25:36,850 [BitServer-4] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:3:0: State change requested from RUNNING --> CANCELLATION_REQUESTED for
> 2015-05-16 19:25:36,850 [BitServer-4] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:3:0. New state: CANCELLATION_REQUESTED
> 2015-05-16 19:25:36,851 [BitServer-4] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:4:1: State change requested from RUNNING --> CANCELLATION_REQUESTED for
> 2015-05-16 19:25:36,851 [BitServer-4] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:4:1. New state: CANCELLATION_REQUESTED
> 2015-05-16 19:25:36,854 [BitServer-4] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:4:3: State change requested from RUNNING --> CANCELLATION_REQUESTED for
> 2015-05-16 19:25:36,854 [BitServer-4] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:4:3. New state: CANCELLATION_REQUESTED
> 2015-05-16 19:25:36,854 [BitServer-4] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:4:5: State change requested from RUNNING --> CANCELLATION_REQUESTED for
> 2015-05-16 19:25:36,854 [BitServer-4] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:4:5. New state: CANCELLATION_REQUESTED
> 2015-05-16 19:25:36,964 [2aa866ba-8939-b184-0ba2-291734329f88:frag:1:0] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:1:0: State change requested from CANCELLATION_REQUESTED --> FINISHED for
> 2015-05-16 19:25:36,964 [2aa866ba-8939-b184-0ba2-291734329f88:frag:1:0] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:1:0. New state: CANCELLED
> 2015-05-16 19:25:36,965 [WorkManager-21] WARN  o.a.d.exec.rpc.control.WorkEventBus - Fragment 2aa866ba-8939-b184-0ba2-291734329f88:1:0 not found in the work bus.
> 2015-05-16 19:25:36,997 [2aa866ba-8939-b184-0ba2-291734329f88:frag:1:4] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:1:4: State change requested from CANCELLATION_REQUESTED --> FINISHED for
> 2015-05-16 19:25:36,997 [2aa866ba-8939-b184-0ba2-291734329f88:frag:1:4] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:1:4. New state: CANCELLED
> 2015-05-16 19:25:36,997 [WorkManager-26] WARN  o.a.d.exec.rpc.control.WorkEventBus - Fragment 2aa866ba-8939-b184-0ba2-291734329f88:1:4 not found in the work bus.
> 2015-05-16 19:25:37,032 [2aa866ba-8939-b184-0ba2-291734329f88:frag:1:2] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:1:2: State change requested from CANCELLATION_REQUESTED --> FINISHED for
> 2015-05-16 19:25:37,032 [2aa866ba-8939-b184-0ba2-291734329f88:frag:1:2] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:1:2. New state: CANCELLED
> 2015-05-16 19:25:37,033 [WorkManager-23] WARN  o.a.d.exec.rpc.control.WorkEventBus - Fragment 2aa866ba-8939-b184-0ba2-291734329f88:1:2 not found in the work bus.
> 2015-05-16 19:25:37,079 [2aa866ba-8939-b184-0ba2-291734329f88:frag:1:5] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:1:5: State change requested from CANCELLATION_REQUESTED --> FINISHED for
> 2015-05-16 19:25:37,079 [2aa866ba-8939-b184-0ba2-291734329f88:frag:1:5] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:1:5. New state: CANCELLED
> 2015-05-16 19:25:37,079 [WorkManager-17] WARN  o.a.d.exec.rpc.control.WorkEventBus - Fragment 2aa866ba-8939-b184-0ba2-291734329f88:1:5 not found in the work bus.
> 2015-05-16 19:25:37,269 [2aa866ba-8939-b184-0ba2-291734329f88:frag:1:3] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:1:3: State change requested from CANCELLATION_REQUESTED --> FINISHED for
> 2015-05-16 19:25:37,269 [2aa866ba-8939-b184-0ba2-291734329f88:frag:1:3] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:1:3. New state: CANCELLED
> 2015-05-16 19:25:37,274 [WorkManager-24] WARN  o.a.d.exec.rpc.control.WorkEventBus - Fragment 2aa866ba-8939-b184-0ba2-291734329f88:1:3 not found in the work bus.
> 2015-05-16 19:25:37,379 [2aa866ba-8939-b184-0ba2-291734329f88:frag:1:1] INFO  o.a.d.e.w.fragment.FragmentExecutor - 2aa866ba-8939-b184-0ba2-291734329f88:1:1: State change requested from CANCELLATION_REQUESTED --> FINISHED for
> 2015-05-16 19:25:37,379 [2aa866ba-8939-b184-0ba2-291734329f88:frag:1:1] INFO  o.a.d.e.w.f.AbstractStatusReporter - State changed for 2aa866ba-8939-b184-0ba2-291734329f88:1:1. New state: CANCELLED
> 2015-05-16 19:25:37,380 [WorkManager-22] WARN  o.a.d.exec.rpc.control.WorkEventBus - Fragment 2aa866ba-8939-b184-0ba2-291734329f88:1:1 not found in the work bus.
> 2015-05-16 19:25:40,209 [UserServer-1] INFO  o.a.drill.exec.work.foreman.Foreman - State change requested.  CANCELLATION_REQUESTED --> CANCELLATION_REQUESTED
> {code}
> Since then, the query will stay as "CANCELLATION_REQUESTED" status and can not be cleaned without drillbit restart.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)