Hi all,
        We have a kafka cluster with 11 nodes, and we found there are some 
partition’s replica num is not equal to isr’s num,because our data traffic is 
small,we think it should isr’s num should equal to replica’s num at last,
but it can not recovery to normal,so we try to shutdown a broker that have 
follower partition and not catch up with leader.
        before we shutdown the broker,we found the broker’s id is not in 
zookeeper’s ids children list,so I think it is disconnected to zookeeper again 
for some network traffic,but the procedure is alive. we have found this 
phenomenon for several
times, we think it is the zookeeper callback missed, so the zkclient cannot 
register it again.
        but it is not my point,my point is,we stop the kafka,but it cannot 
normal exit,because it will stop at kafkacontroller.shutdown() for very very 
long time,and we cannot exit the broker util we use kill -9.

for solve this problem,we jstack the procedure and we found it stop at 
autoRebalanceScheduler.shutdown(),my stack’s result is in my email’s attachment.

can someone help it, thank you very much.
2016-10-28 22:22:44
Full thread dump Java HotSpot(TM) 64-Bit Server VM (24.76-b04 mixed mode):

"Attach Listener" daemon prio=10 tid=0x00007f29b0002000 nid=0x27d6 waiting on 
condition [0x0000000000000000]
   java.lang.Thread.State: RUNNABLE

"Thread-2" prio=10 tid=0x00007f299049b800 nid=0x241f waiting on condition 
[0x00007f2888a0c000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000bd411f20> (a 
java.util.concurrent.locks.ReentrantLock$NonfairSync)
        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.CoreUtils$.inLock(CoreUtils.scala:260)
        at kafka.controller.KafkaController.shutdown(KafkaController.scala:692)
        at 
kafka.server.KafkaServer$$anonfun$shutdown$10.apply$mcV$sp(KafkaServer.scala:543)
        at kafka.utils.CoreUtils$.swallow(CoreUtils.scala:79)
        at kafka.utils.Logging$class.swallowWarn(Logging.scala:92)
        at kafka.utils.CoreUtils$.swallowWarn(CoreUtils.scala:51)
        at kafka.utils.Logging$class.swallow(Logging.scala:94)
        at kafka.utils.CoreUtils$.swallow(CoreUtils.scala:51)
        at kafka.server.KafkaServer.shutdown(KafkaServer.scala:543)
        at 
kafka.server.KafkaServerStartable.shutdown(KafkaServerStartable.scala:49)
        at kafka.Kafka$$anon$1.run(Kafka.scala:63)

"SIGTERM handler" daemon prio=10 tid=0x00007f29b0001000 nid=0x241c in 
Object.wait() [0x00007f2889059000]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.Thread.join(Thread.java:1281)
        - locked <0x00000000bc7fa470> (a kafka.Kafka$$anon$1)
        at java.lang.Thread.join(Thread.java:1355)
        at 
java.lang.ApplicationShutdownHooks.runHooks(ApplicationShutdownHooks.java:106)
        at 
java.lang.ApplicationShutdownHooks$1.run(ApplicationShutdownHooks.java:46)
        at java.lang.Shutdown.runHooks(Shutdown.java:123)
        at java.lang.Shutdown.sequence(Shutdown.java:167)
        at java.lang.Shutdown.exit(Shutdown.java:212)
        - locked <0x00000000bc70ada0> (a java.lang.Class for java.lang.Shutdown)
        at java.lang.Terminator$1.handle(Terminator.java:52)
        at sun.misc.Signal$1.run(Signal.java:212)
        at java.lang.Thread.run(Thread.java:745)

"main-EventThread" daemon prio=10 tid=0x00007f2984027800 nid=0x46d3 waiting on 
condition [0x00007f2888c55000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000dc11ae78> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        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 org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:494)

"main-SendThread(10.4.232.86:2181)" daemon prio=10 tid=0x00007f2984026800 
nid=0x46d2 runnable [0x00007f2888d56000]
   java.lang.Thread.State: RUNNABLE
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:87)
        - locked <0x00000000dc123f28> (a sun.nio.ch.Util$2)
        - locked <0x00000000dc123f18> (a java.util.Collections$UnmodifiableSet)
        - locked <0x00000000dc123f38> (a sun.nio.ch.EPollSelectorImpl)
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:98)
        at 
org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:349)
        at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1081)

"Controller-1-to-broker-2-send-thread" prio=10 tid=0x00007f2980150800 
nid=0x45b4 waiting on condition [0x00007f2888f58000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000e3c9a9c0> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        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:162)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)

"kafka-scheduler-0" daemon prio=10 tid=0x00007f2980168800 nid=0x45ad waiting on 
condition [0x00007f288915a000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000bd411f20> (a 
java.util.concurrent.locks.ReentrantLock$NonfairSync)
        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.CoreUtils$.inLock(CoreUtils.scala:260)
        at 
kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$18.apply(KafkaController.scala:1217)
        at 
kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$18.apply(KafkaController.scala:1215)
        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:1215)
        at 
kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1194)
        at 
scala.collection.immutable.HashMap$HashMap1.foreach(HashMap.scala:224)
        at 
scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:403)
        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:471)
        at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:304)
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178)
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:745)

"Controller-1-to-broker-0-send-thread" prio=10 tid=0x00007f2980136000 
nid=0x4581 waiting on condition [0x00007f288925b000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000da2c7eb0> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        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:162)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)

"Controller-1-to-broker-6-send-thread" prio=10 tid=0x00007f2980134000 
nid=0x4580 waiting on condition [0x00007f288935c000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000da2ce478> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        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:162)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)

"Controller-1-to-broker-3-send-thread" prio=10 tid=0x00007f2980131800 
nid=0x457f waiting on condition [0x00007f288945d000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000da2c7fe8> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        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:162)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)

"Controller-1-to-broker-9-send-thread" prio=10 tid=0x00007f298012f800 
nid=0x457e waiting on condition [0x00007f288955e000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000da2ce5b0> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        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:162)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)

"Controller-1-to-broker-10-send-thread" prio=10 tid=0x00007f298012d800 
nid=0x457d waiting on condition [0x00007f288965f000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000da2d4a40> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        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:162)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)

"Controller-1-to-broker-1-send-thread" prio=10 tid=0x00007f298012b800 
nid=0x457c waiting on condition [0x00007f2889760000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000da2c8120> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        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:162)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)

"Controller-1-to-broker-7-send-thread" prio=10 tid=0x00007f2980129800 
nid=0x457b waiting on condition [0x00007f2889861000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000da2ce6e8> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        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:162)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)

"Controller-1-to-broker-4-send-thread" prio=10 tid=0x00007f2980128000 
nid=0x457a waiting on condition [0x00007f2889962000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000da2c8258> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        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:162)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)

"Controller-1-to-broker-5-send-thread" prio=10 tid=0x00007f2980127000 
nid=0x4579 waiting on condition [0x00007f2660ed4000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000da2db008> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        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:162)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)

"Controller-1-to-broker-8-send-thread" prio=10 tid=0x00007f2980044000 
nid=0x4578 waiting on condition [0x00007f26612d8000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000da2ce820> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        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:162)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)

"metrics-meter-tick-thread-2" daemon prio=10 tid=0x00007f2a08702000 nid=0x3f74 
waiting on condition [0x00007f29d4f8c000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000be000fb8> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        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.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1085)
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:807)
        at 
java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1068)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1130)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:745)

"metrics-meter-tick-thread-1" daemon prio=10 tid=0x00007f2a08701000 nid=0x3f73 
waiting on condition [0x00007f2a043a1000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000be000fb8> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        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.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1090)
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:807)
        at 
java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1068)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1130)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:745)

"ZkClient-EventThread-19-10.4.232.86:2181,10.4.232.242:2181,10.4.235.92:2181/config/mobile/mq/yf-mafka2-common"
 daemon prio=10 tid=0x00007f2a0895c800 nid=0x3ead waiting on condition 
[0x00007f2a044a2000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000e3cb99d8> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        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: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:735)
        at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)

"SensorExpiryThread" daemon prio=10 tid=0x00007f2a0894d000 nid=0x3ea9 waiting 
on condition [0x00007f2a045a3000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000be3e1ae0> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        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.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1090)
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:807)
        at 
java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1068)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1130)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:745)

"RMI TCP Accept-0" daemon prio=10 tid=0x00007f2a086d4800 nid=0x3e9e runnable 
[0x00007f2a047a5000]
   java.lang.Thread.State: RUNNABLE
        at java.net.PlainSocketImpl.socketAccept(Native Method)
        at 
java.net.AbstractPlainSocketImpl.accept(AbstractPlainSocketImpl.java:398)
        at java.net.ServerSocket.implAccept(ServerSocket.java:530)
        at java.net.ServerSocket.accept(ServerSocket.java:498)
        at 
sun.management.jmxremote.LocalRMIServerSocketFactory$1.accept(LocalRMIServerSocketFactory.java:52)
        at 
sun.rmi.transport.tcp.TCPTransport$AcceptLoop.executeAcceptLoop(TCPTransport.java:399)
        at 
sun.rmi.transport.tcp.TCPTransport$AcceptLoop.run(TCPTransport.java:371)
        at java.lang.Thread.run(Thread.java:745)

"RMI TCP Accept-6066" daemon prio=10 tid=0x00007f2a086ce000 nid=0x3e9d runnable 
[0x00007f2a048a6000]
   java.lang.Thread.State: RUNNABLE
        at java.net.PlainSocketImpl.socketAccept(Native Method)
        at 
java.net.AbstractPlainSocketImpl.accept(AbstractPlainSocketImpl.java:398)
        at java.net.ServerSocket.implAccept(ServerSocket.java:530)
        at java.net.ServerSocket.accept(ServerSocket.java:498)
        at 
sun.rmi.transport.tcp.TCPTransport$AcceptLoop.executeAcceptLoop(TCPTransport.java:399)
        at 
sun.rmi.transport.tcp.TCPTransport$AcceptLoop.run(TCPTransport.java:371)
        at java.lang.Thread.run(Thread.java:745)

"RMI TCP Accept-0" daemon prio=10 tid=0x00007f2a086bf000 nid=0x3e9a runnable 
[0x00007f2a049a7000]
   java.lang.Thread.State: RUNNABLE
        at java.net.PlainSocketImpl.socketAccept(Native Method)
        at 
java.net.AbstractPlainSocketImpl.accept(AbstractPlainSocketImpl.java:398)
        at java.net.ServerSocket.implAccept(ServerSocket.java:530)
        at java.net.ServerSocket.accept(ServerSocket.java:498)
        at 
sun.rmi.transport.tcp.TCPTransport$AcceptLoop.executeAcceptLoop(TCPTransport.java:399)
        at 
sun.rmi.transport.tcp.TCPTransport$AcceptLoop.run(TCPTransport.java:371)
        at java.lang.Thread.run(Thread.java:745)

"Service Thread" daemon prio=10 tid=0x00007f2a0846e800 nid=0x3e98 runnable 
[0x0000000000000000]
   java.lang.Thread.State: RUNNABLE

"C2 CompilerThread1" daemon prio=10 tid=0x00007f2a0846b800 nid=0x3e97 waiting 
on condition [0x0000000000000000]
   java.lang.Thread.State: RUNNABLE

"C2 CompilerThread0" daemon prio=10 tid=0x00007f2a08469800 nid=0x3e96 waiting 
on condition [0x0000000000000000]
   java.lang.Thread.State: RUNNABLE

"Signal Dispatcher" daemon prio=10 tid=0x00007f2a08467000 nid=0x3e95 runnable 
[0x0000000000000000]
   java.lang.Thread.State: RUNNABLE

"Surrogate Locker Thread (Concurrent GC)" daemon prio=10 tid=0x00007f2a08465000 
nid=0x3e94 waiting on condition [0x0000000000000000]
   java.lang.Thread.State: RUNNABLE

"Finalizer" daemon prio=10 tid=0x00007f2a08443000 nid=0x3e8e in Object.wait() 
[0x00007f2a0c7b1000]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:135)
        - locked <0x00000000be3e2718> (a java.lang.ref.ReferenceQueue$Lock)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:151)
        at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:209)

"Reference Handler" daemon prio=10 tid=0x00007f2a08441000 nid=0x3e8d in 
Object.wait() [0x00007f2a0c8b2000]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.Object.wait(Object.java:503)
        at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:133)
        - locked <0x00000000be3e3550> (a java.lang.ref.Reference$Lock)

"main" prio=10 tid=0x00007f2a0800c000 nid=0x3e7d waiting on condition 
[0x00007f2a109cd000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000be0038e0> (a 
java.util.concurrent.CountDownLatch$Sync)
        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.server.KafkaServer.awaitShutdown(KafkaServer.scala:569)
        at 
kafka.server.KafkaServerStartable.awaitShutdown(KafkaServerStartable.scala:68)
        at kafka.Kafka$.main(Kafka.scala:68)
        at kafka.Kafka.main(Kafka.scala)

"VM Thread" prio=10 tid=0x00007f2a0843c800 nid=0x3e8c runnable 

"Gang worker#0 (Parallel GC Threads)" prio=10 tid=0x00007f2a08020000 nid=0x3e7e 
runnable 

"Gang worker#1 (Parallel GC Threads)" prio=10 tid=0x00007f2a08022000 nid=0x3e7f 
runnable 

"Gang worker#2 (Parallel GC Threads)" prio=10 tid=0x00007f2a08024000 nid=0x3e80 
runnable 

"Gang worker#3 (Parallel GC Threads)" prio=10 tid=0x00007f2a08025800 nid=0x3e81 
runnable 

"G1 Main Concurrent Mark GC Thread" prio=10 tid=0x00007f2a08037800 nid=0x3e87 
runnable 

"Gang worker#0 (G1 Parallel Marking Threads)" prio=10 tid=0x00007f2a08042000 
nid=0x3e88 runnable 

"G1 Concurrent Refinement Thread#0" prio=10 tid=0x00007f2a08030000 nid=0x3e86 
runnable 

"G1 Concurrent Refinement Thread#1" prio=10 tid=0x00007f2a0802e000 nid=0x3e85 
runnable 

"G1 Concurrent Refinement Thread#2" prio=10 tid=0x00007f2a0802c000 nid=0x3e84 
runnable 

"G1 Concurrent Refinement Thread#3" prio=10 tid=0x00007f2a0802a800 nid=0x3e83 
runnable 

"G1 Concurrent Refinement Thread#4" prio=10 tid=0x00007f2a08028800 nid=0x3e82 
runnable 


"VM Periodic Task Thread" prio=10 tid=0x00007f2a086d7000 nid=0x3e9f waiting on 
condition 

JNI global references: 318

Reply via email to