[ https://issues.apache.org/jira/browse/KAFKA-1408?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14313765#comment-14313765 ]
Dmitry Bugaychenko commented on KAFKA-1408: ------------------------------------------- When topic deletion is disabled the problem is not reproducable. Have not tried to enable deletion yet. And yes, I think its a duplicate of KAFKA-1317 > Kafk broker can not stop itself normaly after problems with connection to ZK > ---------------------------------------------------------------------------- > > Key: KAFKA-1408 > URL: https://issues.apache.org/jira/browse/KAFKA-1408 > Project: Kafka > Issue Type: Bug > Components: controller > Affects Versions: 0.8.1 > Reporter: Dmitry Bugaychenko > Assignee: Neha Narkhede > > After getting to inconsistence state due to short netwrok failure broker can > not stop itself. The last message in the log is: > {code} > INFO | jvm 1 | 2014/04/21 08:53:07 | [2014-04-21 09:53:06,999] INFO > [kafka-log-cleaner-thread-0], Stopped (kafka.log.LogCleaner) > INFO | jvm 1 | 2014/04/21 08:53:07 | [2014-04-21 09:53:06,999] INFO > [kafka-log-cleaner-thread-0], Shutdown completed (kafka.log.LogCleaner) > {code} > There is also a preceding error: > {code} > INFO | jvm 1 | 2014/04/21 08:52:55 | [2014-04-21 09:52:55,015] WARN > Controller doesn't exist (kafka.utils.Utils$) > INFO | jvm 1 | 2014/04/21 08:52:55 | kafka.common.KafkaException: > Controller doesn't exist > INFO | jvm 1 | 2014/04/21 08:52:55 | at > kafka.utils.ZkUtils$.getController(ZkUtils.scala:70) > INFO | jvm 1 | 2014/04/21 08:52:55 | at > kafka.server.KafkaServer.kafka$server$KafkaServer$$controlledShutdown(KafkaServer.scala:148) > INFO | jvm 1 | 2014/04/21 08:52:55 | at > kafka.server.KafkaServer$$anonfun$shutdown$1.apply$mcV$sp(KafkaServer.scala:220) > {code} > Here is a part of jstack (it looks like there is a deadlock between > delete-topics-thread and ZkClient-EventThread): > {code} > IWrapper-Connection id=10 state=WAITING > - waiting on <0x15d6aa44> (a > java.util.concurrent.locks.ReentrantLock$NonfairSync) > - locked <0x15d6aa44> (a > java.util.concurrent.locks.ReentrantLock$NonfairSync) > owned by ZkClient-EventThread-37-devlnx2:2181 id=37 > 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.shutdown(KafkaController.scala:641) > at > kafka.server.KafkaServer$$anonfun$shutdown$8.apply$mcV$sp(KafkaServer.scala:233) > at kafka.utils.Utils$.swallow(Utils.scala:167) > at kafka.utils.Logging$class.swallowWarn(Logging.scala:92) > at kafka.utils.Utils$.swallowWarn(Utils.scala:46) > at kafka.utils.Logging$class.swallow(Logging.scala:94) > at kafka.utils.Utils$.swallow(Utils.scala:46) > at kafka.server.KafkaServer.shutdown(KafkaServer.scala:233) > at odkl.databus.server.Main.stop(Main.java:184) > at > org.tanukisoftware.wrapper.WrapperManager.stopInner(WrapperManager.java:1982) > at > org.tanukisoftware.wrapper.WrapperManager.handleSocket(WrapperManager.java:2391) > at org.tanukisoftware.wrapper.WrapperManager.run(WrapperManager.java:2696) > at java.lang.Thread.run(Thread.java:744) > ZkClient-EventThread-37-devlnx2:2181 id=37 state=WAITING > - waiting on <0x3d5f9878> (a java.util.concurrent.CountDownLatch$Sync) > - locked <0x3d5f9878> (a java.util.concurrent.CountDownLatch$Sync) > 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.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:994) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1303) > at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:236) > at kafka.utils.ShutdownableThread.shutdown(ShutdownableThread.scala:36) > at > kafka.controller.TopicDeletionManager.shutdown(TopicDeletionManager.scala:93) > at > kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply$mcV$sp(KafkaController.scala:340) > 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@15d6aa44 > Controller-3-to-broker-3-send-thread id=2414 state=WAITING > - waiting on <0x4044618f> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > - locked <0x4044618f> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > at sun.misc.Unsafe.park(Native Method) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2043) > at > java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442) > at > kafka.controller.RequestSendThread.doWork(ControllerChannelManager.scala:121) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:51) > delete-topics-thread id=2416 state=WAITING > - waiting on <0x15d6aa44> (a > java.util.concurrent.locks.ReentrantLock$NonfairSync) > - locked <0x15d6aa44> (a > java.util.concurrent.locks.ReentrantLock$NonfairSync) > owned by ZkClient-EventThread-37-devlnx2:2181 id=37 > 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.TopicDeletionManager$DeleteTopicsThread.doWork(TopicDeletionManager.scala:333) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:51) > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)