[jira] [Commented] (KAFKA-1429) Yet another deadlock in controller shutdown
[ https://issues.apache.org/jira/browse/KAFKA-1429?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15390887#comment-15390887 ] Pengwei commented on KAFKA-1429: hi Jun, I have open a PR for this issue. Can you review it? thanks~ > Yet another deadlock in controller shutdown > --- > > Key: KAFKA-1429 > URL: https://issues.apache.org/jira/browse/KAFKA-1429 > Project: Kafka > Issue Type: Bug > Components: controller >Affects Versions: 0.8.1 >Reporter: Dmitry Bugaychenko >Assignee: Neha Narkhede > Attachments: kafka_0.9.0.0_controller_dead_lock.patch > > > Found one more case of deadlock in controller during shutdown: > {code} > ZkClient-EventThread-57-192.168.41.148:2181,192.168.36.250:2181,192.168.41.207:2181 > id=57 state=TIMED_WAITING > - waiting on <0x288a66ec> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > - locked <0x288a66ec> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > at sun.misc.Unsafe.park(Native Method) > at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:226) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2082) > at > java.util.concurrent.ThreadPoolExecutor.awaitTermination(ThreadPoolExecutor.java:1468) > at kafka.utils.KafkaScheduler.shutdown(KafkaScheduler.scala:88) > at > kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply$mcV$sp(KafkaController.scala:339) > at > kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337) > at > kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337) > at kafka.utils.Utils$.inLock(Utils.scala:538) > at > kafka.controller.KafkaController.onControllerResignation(KafkaController.scala:337) > at > kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply$mcZ$sp(KafkaController.scala:1068) > at > kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067) > at > kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067) > at kafka.utils.Utils$.inLock(Utils.scala:538) > at > kafka.controller.KafkaController$SessionExpirationListener.handleNewSession(KafkaController.scala:1067) > at org.I0Itec.zkclient.ZkClient$4.run(ZkClient.java:472) > at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71) > Locked synchronizers: count = 1 > - java.util.concurrent.locks.ReentrantLock$NonfairSync@22b9b31a > kafka-scheduler-0 id=172 state=WAITING > - waiting on <0x22b9b31a> (a > java.util.concurrent.locks.ReentrantLock$NonfairSync) > - locked <0x22b9b31a> (a > java.util.concurrent.locks.ReentrantLock$NonfairSync) > owned by > ZkClient-EventThread-57-192.168.41.148:2181,192.168.36.250:2181,192.168.41.207:2181 > id=57 > at sun.misc.Unsafe.park(Native Method) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:834) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:867) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1197) > at > java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:214) > at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:290) > at kafka.utils.Utils$.inLock(Utils.scala:536) > at > kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1110) > at > kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1108) > at > scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98) > at > scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98) > at > scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:226) > at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:39) > at scala.collection.mutable.HashMap.foreach(HashMap.scala:98) > at > kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1108) > at >
[jira] [Commented] (KAFKA-1429) Yet another deadlock in controller shutdown
[ https://issues.apache.org/jira/browse/KAFKA-1429?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15369380#comment-15369380 ] ASF GitHub Bot commented on KAFKA-1429: --- GitHub user pengwei-li opened a pull request: https://github.com/apache/kafka/pull/1603 KAFKA-1429: Yet another deadlock in controller shutdown Author: pengweiReviewers: NA You can merge this pull request into a Git repository by running: $ git pull https://github.com/pengwei-li/kafka trunk Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/1603.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #1603 commit a920d4e9807add634cc44e4b7cf9e156edd515cf Author: pengwei-li Date: 2016-07-10T00:31:56Z KAFKA-1429: Yet another deadlock in controller shutdown Author: pengwei Reviewers: NA > Yet another deadlock in controller shutdown > --- > > Key: KAFKA-1429 > URL: https://issues.apache.org/jira/browse/KAFKA-1429 > Project: Kafka > Issue Type: Bug > Components: controller >Affects Versions: 0.8.1 >Reporter: Dmitry Bugaychenko >Assignee: Neha Narkhede > Attachments: kafka_0.9.0.0_controller_dead_lock.patch > > > Found one more case of deadlock in controller during shutdown: > {code} > ZkClient-EventThread-57-192.168.41.148:2181,192.168.36.250:2181,192.168.41.207:2181 > id=57 state=TIMED_WAITING > - waiting on <0x288a66ec> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > - locked <0x288a66ec> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > at sun.misc.Unsafe.park(Native Method) > at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:226) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2082) > at > java.util.concurrent.ThreadPoolExecutor.awaitTermination(ThreadPoolExecutor.java:1468) > at kafka.utils.KafkaScheduler.shutdown(KafkaScheduler.scala:88) > at > kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply$mcV$sp(KafkaController.scala:339) > at > kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337) > at > kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337) > at kafka.utils.Utils$.inLock(Utils.scala:538) > at > kafka.controller.KafkaController.onControllerResignation(KafkaController.scala:337) > at > kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply$mcZ$sp(KafkaController.scala:1068) > at > kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067) > at > kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067) > at kafka.utils.Utils$.inLock(Utils.scala:538) > at > kafka.controller.KafkaController$SessionExpirationListener.handleNewSession(KafkaController.scala:1067) > at org.I0Itec.zkclient.ZkClient$4.run(ZkClient.java:472) > at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71) > Locked synchronizers: count = 1 > - java.util.concurrent.locks.ReentrantLock$NonfairSync@22b9b31a > kafka-scheduler-0 id=172 state=WAITING > - waiting on <0x22b9b31a> (a > java.util.concurrent.locks.ReentrantLock$NonfairSync) > - locked <0x22b9b31a> (a > java.util.concurrent.locks.ReentrantLock$NonfairSync) > owned by > ZkClient-EventThread-57-192.168.41.148:2181,192.168.36.250:2181,192.168.41.207:2181 > id=57 > at sun.misc.Unsafe.park(Native Method) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:834) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:867) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1197) > at > java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:214) > at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:290) > at kafka.utils.Utils$.inLock(Utils.scala:536) > at > kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1110) > at >
[jira] [Commented] (KAFKA-1429) Yet another deadlock in controller shutdown
[ https://issues.apache.org/jira/browse/KAFKA-1429?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15343716#comment-15343716 ] Jun Rao commented on KAFKA-1429: [~pengwei], yes, it's a real bug. Thanks for finding and reporting this. It seems that we will also need to fix SessionExpirationListener in KafkaController. For easy review, is it convenient for you to submit the patch through a pull request (see https://cwiki.apache.org/confluence/display/KAFKA/Contributing+Code+Changes)? > Yet another deadlock in controller shutdown > --- > > Key: KAFKA-1429 > URL: https://issues.apache.org/jira/browse/KAFKA-1429 > Project: Kafka > Issue Type: Bug > Components: controller >Affects Versions: 0.8.1 >Reporter: Dmitry Bugaychenko >Assignee: Neha Narkhede > Attachments: kafka_0.9.0.0_controller_dead_lock.patch > > > Found one more case of deadlock in controller during shutdown: > {code} > ZkClient-EventThread-57-192.168.41.148:2181,192.168.36.250:2181,192.168.41.207:2181 > id=57 state=TIMED_WAITING > - waiting on <0x288a66ec> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > - locked <0x288a66ec> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > at sun.misc.Unsafe.park(Native Method) > at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:226) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2082) > at > java.util.concurrent.ThreadPoolExecutor.awaitTermination(ThreadPoolExecutor.java:1468) > at kafka.utils.KafkaScheduler.shutdown(KafkaScheduler.scala:88) > at > kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply$mcV$sp(KafkaController.scala:339) > at > kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337) > at > kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337) > at kafka.utils.Utils$.inLock(Utils.scala:538) > at > kafka.controller.KafkaController.onControllerResignation(KafkaController.scala:337) > at > kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply$mcZ$sp(KafkaController.scala:1068) > at > kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067) > at > kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067) > at kafka.utils.Utils$.inLock(Utils.scala:538) > at > kafka.controller.KafkaController$SessionExpirationListener.handleNewSession(KafkaController.scala:1067) > at org.I0Itec.zkclient.ZkClient$4.run(ZkClient.java:472) > at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71) > Locked synchronizers: count = 1 > - java.util.concurrent.locks.ReentrantLock$NonfairSync@22b9b31a > kafka-scheduler-0 id=172 state=WAITING > - waiting on <0x22b9b31a> (a > java.util.concurrent.locks.ReentrantLock$NonfairSync) > - locked <0x22b9b31a> (a > java.util.concurrent.locks.ReentrantLock$NonfairSync) > owned by > ZkClient-EventThread-57-192.168.41.148:2181,192.168.36.250:2181,192.168.41.207:2181 > id=57 > at sun.misc.Unsafe.park(Native Method) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:834) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:867) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1197) > at > java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:214) > at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:290) > at kafka.utils.Utils$.inLock(Utils.scala:536) > at > kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1110) > at > kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1108) > at > scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98) > at > scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98) > at > scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:226) > at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:39) > at scala.collection.mutable.HashMap.foreach(HashMap.scala:98) > at >
[jira] [Commented] (KAFKA-1429) Yet another deadlock in controller shutdown
[ https://issues.apache.org/jira/browse/KAFKA-1429?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15331523#comment-15331523 ] Pengwei commented on KAFKA-1429: I can still reproduce this issue in 0.9.0.0 or 0.9.0.1, Controller session expire and holding the controllerLock lock and try to wait the autoRebalanceScheduler thread to shutdown, but the autoRebalanceScheduler is running for the parttion rebalance and this thread want the controllerLock : "ZkClient-EventThread-18-9.94.1.21:2181,9.94.1.22:2181,9.94.1.23:2181/oms-cluster-1" #18 daemon prio=5 os_prio=0 tid=0x00c3f800 nid=0x49be waiting on condition [0x7f466603d000] java.lang.Thread.State: TIMED_WAITING (parking) at sun.misc.Unsafe.park(Native Method) - parking to wait for <0xd9eaace0> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215) at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078) at java.util.concurrent.ThreadPoolExecutor.awaitTermination(ThreadPoolExecutor.java:1465) at kafka.utils.KafkaScheduler.shutdown(KafkaScheduler.scala:98) at kafka.controller.KafkaController.onControllerResignation(KafkaController.scala:370) at kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply$mcZ$sp(KafkaController.scala:1171) at kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1170) at kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1170) at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:262) at kafka.controller.KafkaController$SessionExpirationListener.handleNewSession(KafkaController.scala:1170) at org.I0Itec.zkclient.ZkClient$6.run(ZkClient.java:734) at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71) "kafka-scheduler-12" #143 daemon prio=5 os_prio=0 tid=0x7f465c4cc000 nid=0x566f waiting on condition [0x7f466260e000] java.lang.Thread.State: WAITING (parking) at sun.misc.Unsafe.park(Native Method) - parking to wait for <0xc0a9a480> (a java.util.concurrent.locks.ReentrantLock$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.acquireQueued(AbstractQueuedSynchronizer.java:870) at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199) at java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:209) at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:285) at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:260) at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1198) at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1194) at scala.collection.immutable.Map$Map3.foreach(Map.scala:161) at kafka.controller.KafkaController.kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance(KafkaController.scala:1194) at kafka.controller.KafkaController$$anonfun$onControllerFailover$1.apply$mcV$sp(KafkaController.scala:344) at kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(KafkaScheduler.scala:110) at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:60) 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:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) > Yet another deadlock in controller shutdown > --- > > Key: KAFKA-1429 > URL: https://issues.apache.org/jira/browse/KAFKA-1429 > Project: Kafka > Issue Type: Bug > Components: controller >Affects Versions: 0.8.1 >Reporter: Dmitry Bugaychenko >Assignee: Neha Narkhede > > Found one
[jira] [Commented] (KAFKA-1429) Yet another deadlock in controller shutdown
[ https://issues.apache.org/jira/browse/KAFKA-1429?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13985300#comment-13985300 ] Dmitry Bugaychenko commented on KAFKA-1429: --- Yes, we have auto.leader.rebalance.enable set to true. What is the estimated release date for 0.8.2 with the fixes? Yet another deadlock in controller shutdown --- Key: KAFKA-1429 URL: https://issues.apache.org/jira/browse/KAFKA-1429 Project: Kafka Issue Type: Bug Components: controller Affects Versions: 0.8.1 Reporter: Dmitry Bugaychenko Assignee: Neha Narkhede Found one more case of deadlock in controller during shutdown: {code} ZkClient-EventThread-57-192.168.41.148:2181,192.168.36.250:2181,192.168.41.207:2181 id=57 state=TIMED_WAITING - waiting on 0x288a66ec (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) - locked 0x288a66ec (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) at sun.misc.Unsafe.park(Native Method) at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:226) at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2082) at java.util.concurrent.ThreadPoolExecutor.awaitTermination(ThreadPoolExecutor.java:1468) at kafka.utils.KafkaScheduler.shutdown(KafkaScheduler.scala:88) at kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply$mcV$sp(KafkaController.scala:339) at kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337) at kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337) at kafka.utils.Utils$.inLock(Utils.scala:538) at kafka.controller.KafkaController.onControllerResignation(KafkaController.scala:337) at kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply$mcZ$sp(KafkaController.scala:1068) at kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067) at kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067) at kafka.utils.Utils$.inLock(Utils.scala:538) at kafka.controller.KafkaController$SessionExpirationListener.handleNewSession(KafkaController.scala:1067) at org.I0Itec.zkclient.ZkClient$4.run(ZkClient.java:472) at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71) Locked synchronizers: count = 1 - java.util.concurrent.locks.ReentrantLock$NonfairSync@22b9b31a kafka-scheduler-0 id=172 state=WAITING - waiting on 0x22b9b31a (a java.util.concurrent.locks.ReentrantLock$NonfairSync) - locked 0x22b9b31a (a java.util.concurrent.locks.ReentrantLock$NonfairSync) owned by ZkClient-EventThread-57-192.168.41.148:2181,192.168.36.250:2181,192.168.41.207:2181 id=57 at sun.misc.Unsafe.park(Native Method) at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186) at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:834) at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:867) at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1197) at java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:214) at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:290) at kafka.utils.Utils$.inLock(Utils.scala:536) at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1110) at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1108) at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98) at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98) at scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:226) at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:39) at scala.collection.mutable.HashMap.foreach(HashMap.scala:98) at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1108) at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1087) at scala.collection.immutable.Map$Map4.foreach(Map.scala:181)
[jira] [Commented] (KAFKA-1429) Yet another deadlock in controller shutdown
[ https://issues.apache.org/jira/browse/KAFKA-1429?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13985558#comment-13985558 ] Jun Rao commented on KAFKA-1429: Probably in a month or two. Yet another deadlock in controller shutdown --- Key: KAFKA-1429 URL: https://issues.apache.org/jira/browse/KAFKA-1429 Project: Kafka Issue Type: Bug Components: controller Affects Versions: 0.8.1 Reporter: Dmitry Bugaychenko Assignee: Neha Narkhede Found one more case of deadlock in controller during shutdown: {code} ZkClient-EventThread-57-192.168.41.148:2181,192.168.36.250:2181,192.168.41.207:2181 id=57 state=TIMED_WAITING - waiting on 0x288a66ec (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) - locked 0x288a66ec (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) at sun.misc.Unsafe.park(Native Method) at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:226) at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2082) at java.util.concurrent.ThreadPoolExecutor.awaitTermination(ThreadPoolExecutor.java:1468) at kafka.utils.KafkaScheduler.shutdown(KafkaScheduler.scala:88) at kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply$mcV$sp(KafkaController.scala:339) at kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337) at kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337) at kafka.utils.Utils$.inLock(Utils.scala:538) at kafka.controller.KafkaController.onControllerResignation(KafkaController.scala:337) at kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply$mcZ$sp(KafkaController.scala:1068) at kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067) at kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067) at kafka.utils.Utils$.inLock(Utils.scala:538) at kafka.controller.KafkaController$SessionExpirationListener.handleNewSession(KafkaController.scala:1067) at org.I0Itec.zkclient.ZkClient$4.run(ZkClient.java:472) at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71) Locked synchronizers: count = 1 - java.util.concurrent.locks.ReentrantLock$NonfairSync@22b9b31a kafka-scheduler-0 id=172 state=WAITING - waiting on 0x22b9b31a (a java.util.concurrent.locks.ReentrantLock$NonfairSync) - locked 0x22b9b31a (a java.util.concurrent.locks.ReentrantLock$NonfairSync) owned by ZkClient-EventThread-57-192.168.41.148:2181,192.168.36.250:2181,192.168.41.207:2181 id=57 at sun.misc.Unsafe.park(Native Method) at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186) at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:834) at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:867) at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1197) at java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:214) at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:290) at kafka.utils.Utils$.inLock(Utils.scala:536) at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1110) at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1108) at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98) at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98) at scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:226) at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:39) at scala.collection.mutable.HashMap.foreach(HashMap.scala:98) at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1108) at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1087) at scala.collection.immutable.Map$Map4.foreach(Map.scala:181) at
[jira] [Commented] (KAFKA-1429) Yet another deadlock in controller shutdown
[ https://issues.apache.org/jira/browse/KAFKA-1429?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13984295#comment-13984295 ] Dmitry Bugaychenko commented on KAFKA-1429: --- Updated libs to 0.8.1.1, will see how it goes. Yet another deadlock in controller shutdown --- Key: KAFKA-1429 URL: https://issues.apache.org/jira/browse/KAFKA-1429 Project: Kafka Issue Type: Bug Components: controller Affects Versions: 0.8.1 Reporter: Dmitry Bugaychenko Assignee: Neha Narkhede Found one more case of deadlock in controller during shutdown: {code} ZkClient-EventThread-57-192.168.41.148:2181,192.168.36.250:2181,192.168.41.207:2181 id=57 state=TIMED_WAITING - waiting on 0x288a66ec (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) - locked 0x288a66ec (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) at sun.misc.Unsafe.park(Native Method) at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:226) at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2082) at java.util.concurrent.ThreadPoolExecutor.awaitTermination(ThreadPoolExecutor.java:1468) at kafka.utils.KafkaScheduler.shutdown(KafkaScheduler.scala:88) at kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply$mcV$sp(KafkaController.scala:339) at kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337) at kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337) at kafka.utils.Utils$.inLock(Utils.scala:538) at kafka.controller.KafkaController.onControllerResignation(KafkaController.scala:337) at kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply$mcZ$sp(KafkaController.scala:1068) at kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067) at kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067) at kafka.utils.Utils$.inLock(Utils.scala:538) at kafka.controller.KafkaController$SessionExpirationListener.handleNewSession(KafkaController.scala:1067) at org.I0Itec.zkclient.ZkClient$4.run(ZkClient.java:472) at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71) Locked synchronizers: count = 1 - java.util.concurrent.locks.ReentrantLock$NonfairSync@22b9b31a kafka-scheduler-0 id=172 state=WAITING - waiting on 0x22b9b31a (a java.util.concurrent.locks.ReentrantLock$NonfairSync) - locked 0x22b9b31a (a java.util.concurrent.locks.ReentrantLock$NonfairSync) owned by ZkClient-EventThread-57-192.168.41.148:2181,192.168.36.250:2181,192.168.41.207:2181 id=57 at sun.misc.Unsafe.park(Native Method) at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186) at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:834) at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:867) at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1197) at java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:214) at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:290) at kafka.utils.Utils$.inLock(Utils.scala:536) at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1110) at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1108) at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98) at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98) at scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:226) at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:39) at scala.collection.mutable.HashMap.foreach(HashMap.scala:98) at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1108) at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1087) at scala.collection.immutable.Map$Map4.foreach(Map.scala:181) at
[jira] [Commented] (KAFKA-1429) Yet another deadlock in controller shutdown
[ https://issues.apache.org/jira/browse/KAFKA-1429?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13983655#comment-13983655 ] Neha Narkhede commented on KAFKA-1429: -- Could you try this on 0.8.1.1? We fixed a bunch of deadlock related issues in 0.8.1.1 Yet another deadlock in controller shutdown --- Key: KAFKA-1429 URL: https://issues.apache.org/jira/browse/KAFKA-1429 Project: Kafka Issue Type: Bug Components: controller Affects Versions: 0.8.1 Reporter: Dmitry Bugaychenko Assignee: Neha Narkhede Found one more case of deadlock in controller during shutdown: {code} ZkClient-EventThread-57-192.168.41.148:2181,192.168.36.250:2181,192.168.41.207:2181 id=57 state=TIMED_WAITING - waiting on 0x288a66ec (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) - locked 0x288a66ec (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) at sun.misc.Unsafe.park(Native Method) at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:226) at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2082) at java.util.concurrent.ThreadPoolExecutor.awaitTermination(ThreadPoolExecutor.java:1468) at kafka.utils.KafkaScheduler.shutdown(KafkaScheduler.scala:88) at kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply$mcV$sp(KafkaController.scala:339) at kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337) at kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337) at kafka.utils.Utils$.inLock(Utils.scala:538) at kafka.controller.KafkaController.onControllerResignation(KafkaController.scala:337) at kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply$mcZ$sp(KafkaController.scala:1068) at kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067) at kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067) at kafka.utils.Utils$.inLock(Utils.scala:538) at kafka.controller.KafkaController$SessionExpirationListener.handleNewSession(KafkaController.scala:1067) at org.I0Itec.zkclient.ZkClient$4.run(ZkClient.java:472) at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71) Locked synchronizers: count = 1 - java.util.concurrent.locks.ReentrantLock$NonfairSync@22b9b31a kafka-scheduler-0 id=172 state=WAITING - waiting on 0x22b9b31a (a java.util.concurrent.locks.ReentrantLock$NonfairSync) - locked 0x22b9b31a (a java.util.concurrent.locks.ReentrantLock$NonfairSync) owned by ZkClient-EventThread-57-192.168.41.148:2181,192.168.36.250:2181,192.168.41.207:2181 id=57 at sun.misc.Unsafe.park(Native Method) at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186) at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:834) at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:867) at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1197) at java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:214) at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:290) at kafka.utils.Utils$.inLock(Utils.scala:536) at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1110) at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1108) at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98) at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98) at scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:226) at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:39) at scala.collection.mutable.HashMap.foreach(HashMap.scala:98) at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1108) at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1087) at scala.collection.immutable.Map$Map4.foreach(Map.scala:181) at