You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@drill.apache.org by "Khurram Faraaz (JIRA)" <ji...@apache.org> on 2017/06/06 07:57:18 UTC

[jira] [Commented] (DRILL-5563) Stop non foreman Drillbit results in IllegalStateException: Allocator[ROOT] closed with outstanding child allocators.

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

Khurram Faraaz commented on DRILL-5563:
---------------------------------------

A similar failure (AssertionError and IllegalStateException) is seen when we follow below steps
{noformat}
Run a concurrent Java program that executes TPCDS Query11
while the above concurrent Java program is under execution
    stop any one non-foreman Drillbit (from another shell, using below command)
       (say on node 2) ./bin/drillbit.sh stop
{noformat}

Details from drillbit.log from the non-foreman node (node 2)
{noformat}
2017-06-06 07:46:45,127 [26c9a242-dfa1-35be-b5f1-ff6b4fa66086:frag:19:0] INFO  o.a.d.e.w.fragment.FragmentExecutor - 26c9a242-dfa1-35be-b5f1-ff6b4fa66086:19:0: State change requested RUNNING --> FAILED
2017-06-06 07:46:45,137 [26c9a242-dfa1-35be-b5f1-ff6b4fa66086:frag:19:0] INFO  o.a.d.e.w.fragment.FragmentExecutor - 26c9a242-dfa1-35be-b5f1-ff6b4fa66086:19:0: State change requested FAILED --> FINISHED
2017-06-06 07:46:45,139 [26c9a242-dfa1-35be-b5f1-ff6b4fa66086:frag:19:0] ERROR o.a.d.e.w.fragment.FragmentExecutor - SYSTEM ERROR: AssertionError: Failure while stopping processing for operator id 10. Currently have states of processing:false, setup:false, waiting:true.

Fragment 19:0

[Error Id: d2cd4cd1-0861-4c51-a8ae-22fb0f85bb29 on centos-02.qa.lab:31010]
org.apache.drill.common.exceptions.UserException: SYSTEM ERROR: AssertionError: Failure while stopping processing for operator id 10. Currently have states of processing:false, setup:false, waiting:true.

Fragment 19:0

[Error Id: d2cd4cd1-0861-4c51-a8ae-22fb0f85bb29 on centos-02.qa.lab:31010]
        at org.apache.drill.common.exceptions.UserException$Builder.build(UserException.java:544) ~[drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.FragmentExecutor.sendFinalState(FragmentExecutor.java:295) [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.FragmentExecutor.cleanup(FragmentExecutor.java:160) [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:264) [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.common.SelfCleaningRunnable.run(SelfCleaningRunnable.java:38) [drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_65]
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_65]
        at java.lang.Thread.run(Thread.java:745) [na:1.8.0_65]
Caused by: java.lang.RuntimeException: java.lang.AssertionError: Failure while stopping processing for operator id 10. Currently have states of processing:false, setup:false, waiting:true.
        at org.apache.drill.common.DeferredException.addThrowable(DeferredException.java:101) ~[drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.FragmentExecutor.fail(FragmentExecutor.java:409) [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:250) [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        ... 4 common frames omitted
Caused by: java.lang.AssertionError: Failure while stopping processing for operator id 10. Currently have states of processing:false, setup:false, waiting:true.
        at org.apache.drill.exec.ops.OperatorStats.stopProcessing(OperatorStats.java:167) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.ScanBatch.next(ScanBatch.java:255) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.next(IteratorValidatorBatchIterator.java:215) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:109) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractSingleRecordBatch.innerNext(AbstractSingleRecordBatch.java:51) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.innerNext(ProjectRecordBatch.java:133) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:162) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.next(IteratorValidatorBatchIterator.java:215) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.test.generated.HashJoinProbeGen160.executeProbePhase(HashJoinProbeTemplate.java:119) ~[na:na]
        at org.apache.drill.exec.test.generated.HashJoinProbeGen160.probeAndProject(HashJoinProbeTemplate.java:225) ~[na:na]
        at org.apache.drill.exec.physical.impl.join.HashJoinBatch.innerNext(HashJoinBatch.java:237) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:162) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.next(IteratorValidatorBatchIterator.java:215) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.test.generated.HashJoinProbeGen280.executeProbePhase(HashJoinProbeTemplate.java:119) ~[na:na]
        at org.apache.drill.exec.test.generated.HashJoinProbeGen280.probeAndProject(HashJoinProbeTemplate.java:225) ~[na:na]
        at org.apache.drill.exec.physical.impl.join.HashJoinBatch.innerNext(HashJoinBatch.java:237) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:162) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.next(IteratorValidatorBatchIterator.java:215) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:109) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractSingleRecordBatch.innerNext(AbstractSingleRecordBatch.java:51) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.innerNext(ProjectRecordBatch.java:133) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:162) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.next(IteratorValidatorBatchIterator.java:215) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:109) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractSingleRecordBatch.innerNext(AbstractSingleRecordBatch.java:51) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.innerNext(ProjectRecordBatch.java:133) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:162) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.next(IteratorValidatorBatchIterator.java:215) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.test.generated.HashAggregatorGen464.doWork(HashAggTemplate.java:312) ~[na:na]
        at org.apache.drill.exec.physical.impl.aggregate.HashAggBatch.innerNext(HashAggBatch.java:146) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:162) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.next(IteratorValidatorBatchIterator.java:215) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:109) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractSingleRecordBatch.innerNext(AbstractSingleRecordBatch.java:51) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.innerNext(ProjectRecordBatch.java:133) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:162) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.next(IteratorValidatorBatchIterator.java:215) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.BaseRootExec.next(BaseRootExec.java:105) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.SingleSenderCreator$SingleSenderRootExec.innerNext(SingleSenderCreator.java:92) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.physical.impl.BaseRootExec.next(BaseRootExec.java:95) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.FragmentExecutor$1.run(FragmentExecutor.java:234) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.work.fragment.FragmentExecutor$1.run(FragmentExecutor.java:227) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
       at java.security.AccessController.doPrivileged(Native Method) ~[na:1.8.0_65]
        at javax.security.auth.Subject.doAs(Subject.java:422) ~[na:1.8.0_65]
        at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1595) ~[hadoop-common-2.7.0-mapr-1607.jar:na]
        at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:227) [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        ... 4 common frames omitted
2017-06-06 07:46:45,263 [Drillbit-ShutdownHook#0] ERROR o.a.d.exec.server.BootStrapContext - Pool did not terminate
2017-06-06 07:46:45,265 [Drillbit-ShutdownHook#0] ERROR o.a.d.exec.server.BootStrapContext - Error while closing
java.lang.IllegalStateException: allocator[op:24:0:7:HashJoinPOP]: allocated t1 (4440064) + allocated t2 (4472832). Someone released memory while in verification.
        at org.apache.drill.exec.memory.BaseAllocator.verifyAllocator(BaseAllocator.java:714) ~[drill-memory-base-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.memory.BaseAllocator.verifyAllocator(BaseAllocator.java:614) ~[drill-memory-base-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.memory.BaseAllocator.verifyAllocator(BaseAllocator.java:614) ~[drill-memory-base-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.memory.BaseAllocator.verifyAllocator(BaseAllocator.java:585) ~[drill-memory-base-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.memory.BaseAllocator.close(BaseAllocator.java:480) ~[drill-memory-base-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.common.AutoCloseables.close(AutoCloseables.java:76) [drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.common.AutoCloseables.close(AutoCloseables.java:64) [drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.server.BootStrapContext.close(BootStrapContext.java:247) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.common.AutoCloseables.close(AutoCloseables.java:76) [drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.common.AutoCloseables.close(AutoCloseables.java:64) [drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.server.Drillbit.close(Drillbit.java:159) [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.server.Drillbit$ShutdownThread.run(Drillbit.java:253) [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
2017-06-06 07:46:45,266 [Drillbit-ShutdownHook#0] WARN  o.apache.drill.exec.server.Drillbit - Failure on close()
java.lang.IllegalStateException: Allocator[rpc:bit-control] closed with outstanding buffers allocated (14).
Allocator(rpc:bit-control) 0/896/6356992/9223372036854775807 (res/actual/peak/limit)
  child allocators: 0
  ledgers: 14
ledger[301193] allocator: rpc:bit-control), isOwning: true, size: 64, references: 1, life: 9701293449908983..0, allocatorManager: [160007, life: 9701293449905721..0] holds 1 buffers.
        DrillBuf[319027], udle: [160008 0..64]
    ledger[301239] allocator: rpc:bit-control), isOwning: true, size: 64, references: 1, life: 9701293451717426..0, allocatorManager: [160032, life: 9701293451713751..0] holds 1 buffers.
        DrillBuf[319073], udle: [160033 0..64]
    ledger[301195] allocator: rpc:bit-control), isOwning: true, size: 64, references: 1, life: 9701293450087203..0, allocatorManager: [160009, life: 9701293450083915..0] holds 1 buffers.
        DrillBuf[319029], udle: [160010 0..64]
    ledger[301202] allocator: rpc:bit-control), isOwning: true, size: 64, references: 1, life: 9701293450269806..0, allocatorManager: [160012, life: 9701293450266309..0] holds 1 buffers.
        DrillBuf[319036], udle: [160013 0..64]
    ledger[301175] allocator: rpc:bit-control), isOwning: true, size: 64, references: 2, life: 9701293449386234..0, allocatorManager: [159998, life: 9701293449382878..0] holds 2 buffers.
        DrillBuf[319009], udle: [159999 0..64]
        DrillBuf[321401], udle: [159999 10..34]
    ledger[301180] allocator: rpc:bit-control), isOwning: true, size: 64, references: 1, life: 9701293449557605..0, allocatorManager: [160002, life: 9701293449554217..0] holds 1 buffers.
        DrillBuf[319014], udle: [160003 0..64]
    ledger[301232] allocator: rpc:bit-control), isOwning: true, size: 64, references: 1, life: 9701293451365570..0, allocatorManager: [160028, life: 9701293451361942..0] holds 1 buffers.
        DrillBuf[319066], udle: [160029 0..64]
    ledger[301217] allocator: rpc:bit-control), isOwning: true, size: 64, references: 1, life: 9701293450817227..0, allocatorManager: [160020, life: 9701293450813550..0] holds 1 buffers.
        DrillBuf[319051], udle: [160021 0..64]
    ledger[301225] allocator: rpc:bit-control), isOwning: true, size: 64, references: 1, life: 9701293451166718..0, allocatorManager: [160025, life: 9701293451163490..0] holds 1 buffers.
        DrillBuf[319059], udle: [160026 0..64]
    ledger[301212] allocator: rpc:bit-control), isOwning: true, size: 64, references: 1, life: 9701293450631495..0, allocatorManager: [160017, life: 9701293450628035..0] holds 1 buffers.
        DrillBuf[319046], udle: [160018 0..64]
    ledger[301221] allocator: rpc:bit-control), isOwning: true, size: 64, references: 1, life: 9701293450995205..0, allocatorManager: [160022, life: 9701293450991972..0] holds 1 buffers.
        DrillBuf[319055], udle: [160023 0..64]
    ledger[301186] allocator: rpc:bit-control), isOwning: true, size: 64, references: 1, life: 9701293449732947..0, allocatorManager: [160003, life: 9701293449729739..0] holds 1 buffers.
        DrillBuf[319020], udle: [160004 0..64]
    ledger[301236] allocator: rpc:bit-control), isOwning: true, size: 64, references: 1, life: 9701293451540460..0, allocatorManager: [160030, life: 9701293451537146..0] holds 1 buffers.
        DrillBuf[319070], udle: [160031 0..64]
    ledger[301207] allocator: rpc:bit-control), isOwning: true, size: 64, references: 1, life: 9701293450445230..0, allocatorManager: [160015, life: 9701293450441966..0] holds 1 buffers.
        DrillBuf[319041], udle: [160016 0..64]
  reservations: 0

        at org.apache.drill.exec.memory.BaseAllocator.close(BaseAllocator.java:499) ~[drill-memory-base-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.common.AutoCloseables.close(AutoCloseables.java:76) ~[drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.common.AutoCloseables.close(AutoCloseables.java:64) ~[drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.service.ServiceEngine.close(ServiceEngine.java:149) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.common.AutoCloseables.close(AutoCloseables.java:76) ~[drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.common.AutoCloseables.close(AutoCloseables.java:64) ~[drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.server.Drillbit.close(Drillbit.java:159) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
        at org.apache.drill.exec.server.Drillbit$ShutdownThread.run(Drillbit.java:253) [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
2017-06-06 07:46:45,266 [Drillbit-ShutdownHook#0] INFO  o.apache.drill.exec.server.Drillbit - Shutdown completed (11595 ms).
{noformat}

> Stop non foreman Drillbit results in IllegalStateException: Allocator[ROOT] closed with outstanding child allocators.
> ---------------------------------------------------------------------------------------------------------------------
>
>                 Key: DRILL-5563
>                 URL: https://issues.apache.org/jira/browse/DRILL-5563
>             Project: Apache Drill
>          Issue Type: Bug
>          Components: Execution - Flow
>    Affects Versions: 1.11.0
>         Environment: 3 node CentOS cluster
>            Reporter: Khurram Faraaz
>
> Stopping the non-foreman Drillbit normally (as shown below) results in IllegalStateException: Allocator[ROOT] closed with outstanding child allocators.
> /opt/mapr/drill/drill-1.11.0/bin/drillbit.sh stop
> Drill 1.11.0 commit ID: d11aba2
> Details from drillbit.log
> {noformat}
> Mon Jun  5 09:29:09 UTC 2017 Terminating drillbit pid 28182
> 2017-06-05 09:29:09,651 [Drillbit-ShutdownHook#0] INFO  o.apache.drill.exec.server.Drillbit - Received shutdown request.
> 2017-06-05 09:29:11,691 [pool-6-thread-1] INFO  o.a.drill.exec.rpc.user.UserServer - closed eventLoopGroup io.netty.channel.nio.NioEventLoopGroup@55511dc2 in 1004 ms
> 2017-06-05 09:29:11,691 [pool-6-thread-2] INFO  o.a.drill.exec.rpc.data.DataServer - closed eventLoopGroup io.netty.channel.nio.NioEventLoopGroup@4078d750 in 1004 ms
> 2017-06-05 09:29:11,692 [pool-6-thread-1] INFO  o.a.drill.exec.service.ServiceEngine - closed userServer in 1005 ms
> 2017-06-05 09:29:11,692 [pool-6-thread-2] INFO  o.a.drill.exec.service.ServiceEngine - closed dataPool in 1005 ms
> 2017-06-05 09:29:11,701 [Drillbit-ShutdownHook#0] INFO  o.a.drill.exec.compile.CodeCompiler - Stats: code gen count: 21, cache miss count: 7, hit rate: 67%
> 2017-06-05 09:29:11,709 [Drillbit-ShutdownHook#0] ERROR o.a.d.exec.server.BootStrapContext - Error while closing
> java.lang.IllegalStateException: Allocator[ROOT] closed with outstanding child allocators.
> Allocator(ROOT) 0/8000000/201359872/17179869184 (res/actual/peak/limit)
>   child allocators: 4
>     Allocator(frag:3:2) 2000000/0/0/20000000000 (res/actual/peak/limit)
>       child allocators: 0
>       ledgers: 0
>       reservations: 0
>     Allocator(frag:4:2) 2000000/0/0/20000000000 (res/actual/peak/limit)
>       child allocators: 0
>       ledgers: 0
>       reservations: 0
>     Allocator(frag:1:2) 2000000/0/0/20000000000 (res/actual/peak/limit)
>       child allocators: 0
>       ledgers: 0
>       reservations: 0
>     Allocator(frag:2:2) 2000000/0/0/20000000000 (res/actual/peak/limit)
>       child allocators: 0
>       ledgers: 0
>       reservations: 0
>   ledgers: 0
>   reservations: 0
>         at org.apache.drill.exec.memory.BaseAllocator.close(BaseAllocator.java:492) ~[drill-memory-base-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
>         at org.apache.drill.common.AutoCloseables.close(AutoCloseables.java:76) [drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
>         at org.apache.drill.common.AutoCloseables.close(AutoCloseables.java:64) [drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
>         at org.apache.drill.exec.server.BootStrapContext.close(BootStrapContext.java:247) ~[drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
>         at org.apache.drill.common.AutoCloseables.close(AutoCloseables.java:76) [drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
>         at org.apache.drill.common.AutoCloseables.close(AutoCloseables.java:64) [drill-common-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
>         at org.apache.drill.exec.server.Drillbit.close(Drillbit.java:159) [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
>         at org.apache.drill.exec.server.Drillbit$ShutdownThread.run(Drillbit.java:253) [drill-java-exec-1.11.0-SNAPSHOT.jar:1.11.0-SNAPSHOT]
> 2017-06-05 09:29:11,709 [Drillbit-ShutdownHook#0] INFO  o.apache.drill.exec.server.Drillbit - Shutdown completed (2057 ms).
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)