jon-wei opened a new issue #6201: Deadlock on overlord with HttpRemoteTaskRunner URL: https://github.com/apache/incubator-druid/issues/6201 We noticed deadlocks on the overlord in our test cluster, which was using HttpRemoteTaskRunner: ``` "hrtr-pending-tasks-runner-0" #193 daemon prio=5 os_prio=0 tid=0x00007fc5a8020000 nid=0x72ec waiting for monitor entry [0x00007fc5644b4000] java.lang.Thread.State: BLOCKED (on object monitor) at io.druid.indexing.overlord.hrtr.HttpRemoteTaskRunner.runTaskOnWorker(HttpRemoteTaskRunner.java:379) - waiting to lock <0x00000000c1544c10> (a java.lang.Object) at io.druid.indexing.overlord.hrtr.HttpRemoteTaskRunner.lambda$addPendingTaskToExecutor$7(HttpRemoteTaskRunner.java:1005) at io.druid.indexing.overlord.hrtr.HttpRemoteTaskRunner$$Lambda$133/748833484.run(Unknown Source) 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) ``` ``` java.lang.Thread.State: BLOCKED (on object monitor) at io.druid.indexing.overlord.hrtr.HttpRemoteTaskRunner.getKnownTasks(HttpRemoteTaskRunner.java:1111) - waiting to lock <0x00000000c1544c10> (a java.lang.Object) at io.druid.indexing.overlord.TaskQueue.manage(TaskQueue.java:232) at io.druid.indexing.overlord.TaskQueue.access$000(TaskQueue.java:69) at io.druid.indexing.overlord.TaskQueue$1.run(TaskQueue.java:136) 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) ``` ``` "HttpRemoteTaskRunner-worker-sync-4" #116 daemon prio=5 os_prio=0 tid=0x00007fc588018800 nid=0x724c waiting for monitor entry [0x00007fc569801000] java.lang.Thread.State: BLOCKED (on object monitor) at io.druid.server.coordination.ChangeRequestHttpSyncer.stop(ChangeRequestHttpSyncer.java:138) - waiting to lock <0x00000000c1544980> (a io.druid.concurrent.LifecycleLock) at io.druid.indexing.overlord.hrtr.WorkerHolder.stop(WorkerHolder.java:333) at io.druid.indexing.overlord.hrtr.HttpRemoteTaskRunner.removeWorker(HttpRemoteTaskRunner.java:532) - locked <0x00000000c1540300> (a java.util.concurrent.ConcurrentHashMap) at io.druid.indexing.overlord.hrtr.HttpRemoteTaskRunner.lambda$scheduleSyncMonitoring$2(HttpRemoteTaskRunner.java:642) - locked <0x00000000c1540300> (a java.util.concurrent.ConcurrentHashMap) at io.druid.indexing.overlord.hrtr.HttpRemoteTaskRunner$$Lambda$90/1851975649.run(Unknown Source) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) 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) ``` ``` java.lang.Thread.State: BLOCKED (on object monitor) at io.druid.indexing.overlord.hrtr.HttpRemoteTaskRunner.taskAddedOrUpdated(HttpRemoteTaskRunner.java:1174) - waiting to lock <0x00000000c1544c10> (a java.lang.Object) at io.druid.indexing.overlord.hrtr.HttpRemoteTaskRunner$$Lambda$87/935105358.taskAddedOrUpdated(Unknown Source) at io.druid.indexing.overlord.hrtr.WorkerHolder$2.notifyListener(WorkerHolder.java:449) at io.druid.indexing.overlord.hrtr.WorkerHolder$2.deltaSync(WorkerHolder.java:442) at io.druid.server.coordination.ChangeRequestHttpSyncer$1.onSuccess(ChangeRequestHttpSyncer.java:269) - locked <0x00000000c1543b50> (a io.druid.concurrent.LifecycleLock) at io.druid.server.coordination.ChangeRequestHttpSyncer$1.onSuccess(ChangeRequestHttpSyncer.java:225) at com.google.common.util.concurrent.Futures$4.run(Futures.java:1181) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) 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) ```
---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
