The main culprit is this thread which goes into "forever retry connection to a closed zookeeper" when I shutdown Kafka (via a Ctrl + C) after zookeeper has already been shutdown. I have attached the complete thread dump, but I don't know if it will be delivered to the mailing list.

"Thread-2" prio=10 tid=0xb3305000 nid=0x4758 waiting on condition [0x6ad69000]
   java.lang.Thread.State: TIMED_WAITING (parking)
    at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x70a93368> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) at java.util.concurrent.locks.LockSupport.parkUntil(LockSupport.java:267) at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitUntil(AbstractQueuedSynchronizer.java:2130)
    at org.I0Itec.zkclient.ZkClient.waitForKeeperState(ZkClient.java:636)
    at org.I0Itec.zkclient.ZkClient.waitUntilConnected(ZkClient.java:619)
    at org.I0Itec.zkclient.ZkClient.waitUntilConnected(ZkClient.java:615)
    at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:679)
    at org.I0Itec.zkclient.ZkClient.readData(ZkClient.java:766)
    at org.I0Itec.zkclient.ZkClient.readData(ZkClient.java:761)
    at kafka.utils.ZkUtils$.readDataMaybeNull(ZkUtils.scala:456)
    at kafka.utils.ZkUtils$.getController(ZkUtils.scala:65)
at kafka.server.KafkaServer.kafka$server$KafkaServer$$controlledShutdown(KafkaServer.scala:194) at kafka.server.KafkaServer$$anonfun$shutdown$1.apply$mcV$sp(KafkaServer.scala:269)
    at kafka.utils.Utils$.swallow(Utils.scala:172)
    at kafka.utils.Logging$class.swallowWarn(Logging.scala:92)
    at kafka.utils.Utils$.swallowWarn(Utils.scala:45)
    at kafka.utils.Logging$class.swallow(Logging.scala:94)
    at kafka.utils.Utils$.swallow(Utils.scala:45)
    at kafka.server.KafkaServer.shutdown(KafkaServer.scala:269)
at kafka.server.KafkaServerStartable.shutdown(KafkaServerStartable.scala:42)
    at kafka.Kafka$$anon$1.run(Kafka.scala:42)

-Jaikiran

On Monday 26 January 2015 05:46 AM, Neha Narkhede wrote:
For a clean shutdown, the broker tries to talk to the controller and also
issues reads to zookeeper. Possibly that is where it tries to reconnect to
zk. It will help to look at the thread dump.

Thanks
Neha

On Fri, Jan 23, 2015 at 8:53 PM, Jaikiran Pai <jai.forums2...@gmail.com>
wrote:

I was just playing around with the RC2 of 0.8.2 and noticed that if I
shutdown zookeeper first I can't shutdown Kafka server at all since it goes
into a never ending attempt to reconnect with zookeeper. I had to kill the
Kafka process to stop it. I tried it against trunk too and there too I see
the same issue. Should I file a JIRA for this and see if I can come up with
a patch?

FWIW, here's the unending (and IMO too frequent) attempts at trying to
reconnect. I've a thread dump too which shows that the other thread which
is trying to complete a controlled shutdown of Kafka is blocked forever for
the zookeeper to be up. I can attach it to the JIRA.

2015-01-24 10:15:46,278] WARN Session 0x14b1a4136800000 for server null,
unexpected error, closing socket connection and attempting reconnect
(org.apache.zookeeper.ClientCnxn)
java.net.ConnectException: Connection refused
     at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
     at sun.nio.ch.SocketChannelImpl.finishConnect(
SocketChannelImpl.java:739)
     at org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(
ClientCnxnSocketNIO.java:361)
     at org.apache.zookeeper.ClientCnxn$SendThread.run(
ClientCnxn.java:1081)
[2015-01-24 10:15:47,437] INFO Opening socket connection to server
localhost/127.0.0.1:2181. Will not attempt to authenticate using SASL
(unknown error) (org.apache.zookeeper.ClientCnxn)
[2015-01-24 10:15:47,438] WARN Session 0x14b1a4136800000 for server null,
unexpected error, closing socket connection and attempting reconnect
(org.apache.zookeeper.ClientCnxn)
java.net.ConnectException: Connection refused
     at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
     at sun.nio.ch.SocketChannelImpl.finishConnect(
SocketChannelImpl.java:739)
     at org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(
ClientCnxnSocketNIO.java:361)
     at org.apache.zookeeper.ClientCnxn$SendThread.run(
ClientCnxn.java:1081)
[2015-01-24 10:15:49,056] INFO Opening socket connection to server
localhost/127.0.0.1:2181. Will not attempt to authenticate using SASL
(unknown error) (org.apache.zookeeper.ClientCnxn)
[2015-01-24 10:15:49,057] WARN Session 0x14b1a4136800000 for server null,
unexpected error, closing socket connection and attempting reconnect
(org.apache.zookeeper.ClientCnxn)
java.net.ConnectException: Connection refused
     at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
     at sun.nio.ch.SocketChannelImpl.finishConnect(
SocketChannelImpl.java:739)
     at org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(
ClientCnxnSocketNIO.java:361)
     at org.apache.zookeeper.ClientCnxn$SendThread.run(
ClientCnxn.java:1081)
[2015-01-24 10:15:50,801] INFO Opening socket connection to server
localhost/127.0.0.1:2181. Will not attempt to authenticate using SASL
(unknown error) (org.apache.zookeeper.ClientCnxn)
[2015-01-24 10:15:50,802] WARN Session 0x14b1a4136800000 for server null,
unexpected error, closing socket connection and attempting reconnect
(org.apache.zookeeper.ClientCnxn)
java.net.ConnectException: Connection refused
     at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
     at sun.nio.ch.SocketChannelImpl.finishConnect(
SocketChannelImpl.java:739)
     at org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(
ClientCnxnSocketNIO.java:361)
     at org.apache.zookeeper.ClientCnxn$SendThread.run(
ClientCnxn.java:1081)




-Jaikiran




2015-01-26 19:56:16
Full thread dump Java HotSpot(TM) Server VM (24.71-b01 mixed mode):

"Attach Listener" daemon prio=10 tid=0xb2e04000 nid=0x4776 waiting on condition 
[0x00000000]
   java.lang.Thread.State: RUNNABLE

"kafka-scheduler-9" daemon prio=10 tid=0xb312c800 nid=0x4762 waiting on 
condition [0x6adba000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x70a15ef0> (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)

"kafka-scheduler-8" daemon prio=10 tid=0xb3303800 nid=0x475b waiting on 
condition [0x6ae0b000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x70a15ef0> (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)

"Thread-2" prio=10 tid=0xb3305000 nid=0x4758 waiting on condition [0x6ad69000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x70a93368> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at 
java.util.concurrent.locks.LockSupport.parkUntil(LockSupport.java:267)
        at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitUntil(AbstractQueuedSynchronizer.java:2130)
        at org.I0Itec.zkclient.ZkClient.waitForKeeperState(ZkClient.java:636)
        at org.I0Itec.zkclient.ZkClient.waitUntilConnected(ZkClient.java:619)
        at org.I0Itec.zkclient.ZkClient.waitUntilConnected(ZkClient.java:615)
        at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:679)
        at org.I0Itec.zkclient.ZkClient.readData(ZkClient.java:766)
        at org.I0Itec.zkclient.ZkClient.readData(ZkClient.java:761)
        at kafka.utils.ZkUtils$.readDataMaybeNull(ZkUtils.scala:456)
        at kafka.utils.ZkUtils$.getController(ZkUtils.scala:65)
        at 
kafka.server.KafkaServer.kafka$server$KafkaServer$$controlledShutdown(KafkaServer.scala:194)
        at 
kafka.server.KafkaServer$$anonfun$shutdown$1.apply$mcV$sp(KafkaServer.scala:269)
        at kafka.utils.Utils$.swallow(Utils.scala:172)
        at kafka.utils.Logging$class.swallowWarn(Logging.scala:92)
        at kafka.utils.Utils$.swallowWarn(Utils.scala:45)
        at kafka.utils.Logging$class.swallow(Logging.scala:94)
        at kafka.utils.Utils$.swallow(Utils.scala:45)
        at kafka.server.KafkaServer.shutdown(KafkaServer.scala:269)
        at 
kafka.server.KafkaServerStartable.shutdown(KafkaServerStartable.scala:42)
        at kafka.Kafka$$anon$1.run(Kafka.scala:42)

"SIGINT handler" daemon prio=10 tid=0xb2e03000 nid=0x4755 in Object.wait() 
[0x6ae5c000]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        - waiting on <0x70a0b740> (a kafka.Kafka$$anon$1)
        at java.lang.Thread.join(Thread.java:1281)
        - locked <0x70a0b740> (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 <0x6ee6a7b8> (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)

"kafka-scheduler-7" daemon prio=10 tid=0xb312ac00 nid=0x4750 waiting on 
condition [0x6aead000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x70a15ef0> (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)

"Controller-0-to-broker-0-send-thread" prio=10 tid=0x6c935800 nid=0x4748 
waiting on condition [0x6aefe000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x745cabb8> (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:121)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:60)

"kafka-scheduler-0" daemon prio=10 tid=0x6b7e8c00 nid=0x4747 waiting on 
condition [0x6b059000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x73ab55a8> (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)

"kafka-scheduler-6" daemon prio=10 tid=0xb3301400 nid=0x4746 waiting on 
condition [0x6b0aa000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x70a15ef0> (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)

"kafka-scheduler-5" daemon prio=10 tid=0x6b7c1000 nid=0x4745 waiting on 
condition [0x6b0fb000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x70a15ef0> (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)

"kafka-request-handler-7" daemon prio=10 tid=0x6b7bf000 nid=0x4744 waiting on 
condition [0x6b14c000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x7126af80> (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.ArrayBlockingQueue.poll(ArrayBlockingQueue.java:389)
        at kafka.network.RequestChannel.receiveRequest(RequestChannel.scala:165)
        at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:47)
        at java.lang.Thread.run(Thread.java:745)

"kafka-request-handler-6" daemon prio=10 tid=0x6b7bd800 nid=0x4743 waiting on 
condition [0x6b19d000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x7126af80> (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.ArrayBlockingQueue.poll(ArrayBlockingQueue.java:389)
        at kafka.network.RequestChannel.receiveRequest(RequestChannel.scala:165)
        at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:47)
        at java.lang.Thread.run(Thread.java:745)

"kafka-request-handler-5" daemon prio=10 tid=0x6b7bc000 nid=0x4742 waiting on 
condition [0x6b1ee000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x7126af80> (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.ArrayBlockingQueue.poll(ArrayBlockingQueue.java:389)
        at kafka.network.RequestChannel.receiveRequest(RequestChannel.scala:165)
        at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:47)
        at java.lang.Thread.run(Thread.java:745)

"kafka-request-handler-4" daemon prio=10 tid=0x6b7bac00 nid=0x4741 waiting on 
condition [0x6b23f000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x7126af80> (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.ArrayBlockingQueue.poll(ArrayBlockingQueue.java:389)
        at kafka.network.RequestChannel.receiveRequest(RequestChannel.scala:165)
        at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:47)
        at java.lang.Thread.run(Thread.java:745)

"kafka-request-handler-3" daemon prio=10 tid=0x6b7b9400 nid=0x4740 waiting on 
condition [0x6b290000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x7126af80> (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.ArrayBlockingQueue.poll(ArrayBlockingQueue.java:389)
        at kafka.network.RequestChannel.receiveRequest(RequestChannel.scala:165)
        at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:47)
        at java.lang.Thread.run(Thread.java:745)

"kafka-request-handler-2" daemon prio=10 tid=0x6b7b7800 nid=0x473f waiting on 
condition [0x6b2e1000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x7126af80> (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.ArrayBlockingQueue.poll(ArrayBlockingQueue.java:389)
        at kafka.network.RequestChannel.receiveRequest(RequestChannel.scala:165)
        at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:47)
        at java.lang.Thread.run(Thread.java:745)

"kafka-request-handler-1" daemon prio=10 tid=0x6b7b5000 nid=0x473e waiting on 
condition [0x6b332000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x7126af80> (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.ArrayBlockingQueue.poll(ArrayBlockingQueue.java:389)
        at kafka.network.RequestChannel.receiveRequest(RequestChannel.scala:165)
        at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:47)
        at java.lang.Thread.run(Thread.java:745)

"kafka-request-handler-0" daemon prio=10 tid=0x6b7b6800 nid=0x473d waiting on 
condition [0x6b383000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x7126af80> (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.ArrayBlockingQueue.poll(ArrayBlockingQueue.java:389)
        at kafka.network.RequestChannel.receiveRequest(RequestChannel.scala:165)
        at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:47)
        at java.lang.Thread.run(Thread.java:745)

"request-expiration-task" prio=10 tid=0x6b7af000 nid=0x473c waiting on 
condition [0x6b3d4000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x71e00ac0> (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.DelayQueue.poll(DelayQueue.java:256)
        at 
kafka.server.RequestPurgatory$ExpiredRequestReaper.pollExpired(RequestPurgatory.scala:294)
        at 
kafka.server.RequestPurgatory$ExpiredRequestReaper.run(RequestPurgatory.scala:250)
        at java.lang.Thread.run(Thread.java:745)

"request-expiration-task" prio=10 tid=0x6b7b1c00 nid=0x473b waiting on 
condition [0x6b425000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x71de7c00> (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.DelayQueue.poll(DelayQueue.java:256)
        at 
kafka.server.RequestPurgatory$ExpiredRequestReaper.pollExpired(RequestPurgatory.scala:294)
        at 
kafka.server.RequestPurgatory$ExpiredRequestReaper.run(RequestPurgatory.scala:250)
        at java.lang.Thread.run(Thread.java:745)

"kafka-scheduler-4" daemon prio=10 tid=0x6bda4800 nid=0x473a waiting on 
condition [0x6b476000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x70a15ef0> (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)

"kafka-scheduler-3" daemon prio=10 tid=0x6b781c00 nid=0x4739 waiting on 
condition [0x6b4c7000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x70a15ef0> (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)

"kafka-socket-acceptor" prio=10 tid=0x6b760000 nid=0x4738 runnable [0x6b518000]
   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 <0x71b33df0> (a sun.nio.ch.Util$2)
        - locked <0x71b33de0> (a java.util.Collections$UnmodifiableSet)
        - locked <0x71b33cd0> (a sun.nio.ch.EPollSelectorImpl)
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:98)
        at kafka.network.Acceptor.run(SocketServer.scala:215)
        at java.lang.Thread.run(Thread.java:745)

"kafka-network-thread-9092-2" prio=10 tid=0x6b75dc00 nid=0x4737 runnable 
[0x6b569000]
   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 <0x71b251c8> (a sun.nio.ch.Util$2)
        - locked <0x71b251b8> (a java.util.Collections$UnmodifiableSet)
        - locked <0x71b250a8> (a sun.nio.ch.EPollSelectorImpl)
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:98)
        at kafka.network.Processor.run(SocketServer.scala:320)
        at java.lang.Thread.run(Thread.java:745)

"kafka-network-thread-9092-1" prio=10 tid=0x6b74f400 nid=0x4736 runnable 
[0x6b5ba000]
   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 <0x71b21b00> (a sun.nio.ch.Util$2)
        - locked <0x71b21af0> (a java.util.Collections$UnmodifiableSet)
        - locked <0x71b219e0> (a sun.nio.ch.EPollSelectorImpl)
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:98)
        at kafka.network.Processor.run(SocketServer.scala:320)
        at java.lang.Thread.run(Thread.java:745)

"kafka-network-thread-9092-0" prio=10 tid=0x6b740c00 nid=0x4735 runnable 
[0x6b60b000]
   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 <0x71b1d6a8> (a sun.nio.ch.Util$2)
        - locked <0x71b1d698> (a java.util.Collections$UnmodifiableSet)
        - locked <0x71b1d588> (a sun.nio.ch.EPollSelectorImpl)
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:98)
        at kafka.network.Processor.run(SocketServer.scala:320)
        at java.lang.Thread.run(Thread.java:745)

"metrics-meter-tick-thread-2" daemon prio=10 tid=0x6b733000 nid=0x4734 waiting 
on condition [0x6b65c000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x712bd4a8> (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=0x6b72f000 nid=0x4733 waiting 
on condition [0x6b6ad000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x712bd4a8> (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)

"kafka-scheduler-2" daemon prio=10 tid=0x6b725c00 nid=0x4732 waiting on 
condition [0x6b6fe000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x70a15ef0> (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)

"kafka-scheduler-1" daemon prio=10 tid=0x6b725400 nid=0x4731 waiting on 
condition [0x6b85c000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x70a15ef0> (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)

"kafka-scheduler-0" daemon prio=10 tid=0x6b724000 nid=0x4730 waiting on 
condition [0x6b8ad000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x70a15ef0> (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)

"main-EventThread" daemon prio=10 tid=0x6b9e8800 nid=0x472f waiting on 
condition [0x6b8fe000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x70fab280> (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(localhost:2181)" daemon prio=10 tid=0x6b9ecc00 nid=0x472e 
waiting on condition [0x6be69000]
   java.lang.Thread.State: TIMED_WAITING (sleeping)
        at java.lang.Thread.sleep(Native Method)
        at 
org.apache.zookeeper.client.StaticHostProvider.next(StaticHostProvider.java:101)
        at 
org.apache.zookeeper.ClientCnxn$SendThread.startConnect(ClientCnxn.java:940)
        at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1003)

"ZkClient-EventThread-13-localhost:2181" daemon prio=10 tid=0x6b9e0800 
nid=0x472d waiting on condition [0x6beba000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x70a98ac0> (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.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:67)

"RMI TCP Accept-0" daemon prio=10 tid=0x6c0c3400 nid=0x472b runnable 
[0x6bf8c000]
   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:388)
        at 
sun.rmi.transport.tcp.TCPTransport$AcceptLoop.run(TCPTransport.java:360)
        at java.lang.Thread.run(Thread.java:745)

"Service Thread" daemon prio=10 tid=0x6c8af800 nid=0x472a runnable [0x00000000]
   java.lang.Thread.State: RUNNABLE

"C2 CompilerThread1" daemon prio=10 tid=0x6c8ad800 nid=0x4729 waiting on 
condition [0x00000000]
   java.lang.Thread.State: RUNNABLE

"C2 CompilerThread0" daemon prio=10 tid=0x6c8abc00 nid=0x4728 waiting on 
condition [0x00000000]
   java.lang.Thread.State: RUNNABLE

"Signal Dispatcher" daemon prio=10 tid=0x6c8aa400 nid=0x4727 runnable 
[0x00000000]
   java.lang.Thread.State: RUNNABLE

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

"Finalizer" daemon prio=10 tid=0x6c898000 nid=0x4725 in Object.wait() 
[0x6c5ee000]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        - waiting on <0x6ee04240> (a java.lang.ref.ReferenceQueue$Lock)
        at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:135)
        - locked <0x6ee04240> (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=0x6c896800 nid=0x4724 in Object.wait() 
[0x6c63f000]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        - waiting on <0x6ee03ed8> (a java.lang.ref.Reference$Lock)
        at java.lang.Object.wait(Object.java:503)
        at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:133)
        - locked <0x6ee03ed8> (a java.lang.ref.Reference$Lock)

"main" prio=10 tid=0xb6608c00 nid=0x471d waiting on condition [0xb67aa000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x70a12a58> (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:307)
        at 
kafka.server.KafkaServerStartable.awaitShutdown(KafkaServerStartable.scala:60)
        at kafka.Kafka$.main(Kafka.scala:47)
        at kafka.Kafka.main(Kafka.scala)

"VM Thread" prio=10 tid=0x6c893c00 nid=0x4723 runnable 

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

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

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

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

"Concurrent Mark-Sweep GC Thread" prio=10 tid=0xb66de400 nid=0x4722 runnable 
"VM Periodic Task Thread" prio=10 tid=0x6c0cbc00 nid=0x472c waiting on 
condition 

JNI global references: 238

Reply via email to