Hi Neha,

Here is the thread dump. The process has been up for almost 7 hours and 
Nothing is in the log but numerous " INFO Closing socket connection to ...."
Thanks 

Libo

Deadlock Detection:

No deadlocks found.

Thread 20075: (state = BLOCKED)
 - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information may 
be imprecise)
 - java.util.concurrent.locks.LockSupport.parkNanos(java.lang.Object, long) 
@bci=20, line=226 (Interpreted frame)
 - 
java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(java.util.concurrent.SynchronousQueue$TransferStack$SNode,
 boolean, long) @bci=174, line=460 (Interpreted frame)
 - 
java.util.concurrent.SynchronousQueue$TransferStack.transfer(java.lang.Object, 
boolean, long) @bci=102, line=359 (Interpreted frame)
 - java.util.concurrent.SynchronousQueue.poll(long, 
java.util.concurrent.TimeUnit) @bci=11, line=942 (Interpreted frame)
 - java.util.concurrent.ThreadPoolExecutor.getTask() @bci=141, line=1043 
(Compiled frame)
 - 
java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
 @bci=17, line=1103 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=603 
(Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 9516: (state = BLOCKED)


Thread 22490: (state = IN_NATIVE)
 - java.net.SocketInputStream.socketRead0(java.io.FileDescriptor, byte[], int, 
int, int) @bci=0 (Compiled frame; information may be imprecise)
 - java.net.SocketInputStream.read(byte[], int, int, int) @bci=79, line=150 
(Compiled frame)
 - java.net.SocketInputStream.read(byte[], int, int) @bci=11, line=121 
(Compiled frame)
 - java.io.BufferedInputStream.fill() @bci=175, line=235 (Compiled frame)
 - java.io.BufferedInputStream.read() @bci=12, line=254 (Compiled frame)
 - java.io.FilterInputStream.read() @bci=4, line=83 (Compiled frame)
 - 
sun.rmi.transport.tcp.TCPTransport.handleMessages(sun.rmi.transport.Connection, 
boolean) @bci=25, line=535 (Compiled frame)
 - sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0() @bci=685, 
line=808 (Interpreted frame)
 - sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run() @bci=50, line=667 
(Interpreted frame)
 - 
java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
 @bci=46, line=1110 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=603 
(Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 14658: (state = IN_NATIVE)
 - java.net.SocketInputStream.socketRead0(java.io.FileDescriptor, byte[], int, 
int, int) @bci=0 (Compiled frame; information may be imprecise)
 - java.net.SocketInputStream.read(byte[], int, int, int) @bci=79, line=150 
(Compiled frame)
 - java.net.SocketInputStream.read(byte[], int, int) @bci=11, line=121 
(Compiled frame)
 - java.io.BufferedInputStream.fill() @bci=175, line=235 (Compiled frame)
 - java.io.BufferedInputStream.read() @bci=12, line=254 (Compiled frame)
 - java.io.FilterInputStream.read() @bci=4, line=83 (Compiled frame)
 - 
sun.rmi.transport.tcp.TCPTransport.handleMessages(sun.rmi.transport.Connection, 
boolean) @bci=25, line=535 (Compiled frame)
 - sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0() @bci=685, 
line=808 (Interpreted frame)
 - sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run() @bci=50, line=667 
(Interpreted frame)
 - 
java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
 @bci=46, line=1110 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=603 
(Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 16336: (state = IN_NATIVE)
 - java.net.SocketInputStream.socketRead0(java.io.FileDescriptor, byte[], int, 
int, int) @bci=0 (Compiled frame; information may be imprecise)
 - java.net.SocketInputStream.read(byte[], int, int, int) @bci=79, line=150 
(Compiled frame)
 - java.net.SocketInputStream.read(byte[], int, int) @bci=11, line=121 
(Compiled frame)
 - java.io.BufferedInputStream.fill() @bci=175, line=235 (Compiled frame)
 - java.io.BufferedInputStream.read() @bci=12, line=254 (Compiled frame)
 - java.io.FilterInputStream.read() @bci=4, line=83 (Compiled frame)
 - 
sun.rmi.transport.tcp.TCPTransport.handleMessages(sun.rmi.transport.Connection, 
boolean) @bci=25, line=535 (Compiled frame)
 - sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0() @bci=685, 
line=808 (Interpreted frame)
 - sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run() @bci=50, line=667 
(Interpreted frame)
 - 
java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
 @bci=46, line=1110 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=603 
(Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 12630: (state = IN_NATIVE)
 - java.net.SocketInputStream.socketRead0(java.io.FileDescriptor, byte[], int, 
int, int) @bci=0 (Compiled frame; information may be imprecise)
 - java.net.SocketInputStream.read(byte[], int, int, int) @bci=79, line=150 
(Compiled frame)
 - java.net.SocketInputStream.read(byte[], int, int) @bci=11, line=121 
(Compiled frame)
 - java.io.BufferedInputStream.fill() @bci=175, line=235 (Compiled frame)
 - java.io.BufferedInputStream.read() @bci=12, line=254 (Compiled frame)
 - java.io.FilterInputStream.read() @bci=4, line=83 (Compiled frame)
 - 
sun.rmi.transport.tcp.TCPTransport.handleMessages(sun.rmi.transport.Connection, 
boolean) @bci=25, line=535 (Compiled frame)
 - sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0() @bci=685, 
line=808 (Interpreted frame)
 - sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run() @bci=50, line=667 
(Interpreted frame)
 - 
java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
 @bci=46, line=1110 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=603 
(Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 30477: (state = BLOCKED)
 - java.lang.Object.wait(long) @bci=0 (Interpreted frame)
 - 
com.sun.jmx.remote.internal.ArrayNotificationBuffer.fetchNotifications(com.sun.jmx.remote.internal.NotificationBufferFilter,
 long, long, int) @bci=697, line=435 (Interpreted frame)
 - 
com.sun.jmx.remote.internal.ArrayNotificationBuffer$ShareBuffer.fetchNotifications(com.sun.jmx.remote.internal.NotificationBufferFilter,
 long, long, int) @bci=14, line=227 (Interpreted frame)
 - com.sun.jmx.remote.internal.ServerNotifForwarder.fetchNotifs(long, long, 
int) @bci=78, line=274 (Interpreted frame)
 - javax.management.remote.rmi.RMIConnectionImpl$3.run() @bci=19, line=1288 
(Interpreted frame)
 - javax.management.remote.rmi.RMIConnectionImpl$3.run() @bci=1, line=1286 
(Interpreted frame)
 - javax.management.remote.rmi.RMIConnectionImpl.fetchNotifications(long, int, 
long) @bci=145, line=1292 (Interpreted frame)
 - sun.reflect.GeneratedMethodAccessor86.invoke(java.lang.Object, 
java.lang.Object[]) @bci=320 (Interpreted frame)
 - sun.reflect.DelegatingMethodAccessorImpl.invoke(java.lang.Object, 
java.lang.Object[]) @bci=6, line=43 (Compiled frame)
 - java.lang.reflect.Method.invoke(java.lang.Object, java.lang.Object[]) 
@bci=57, line=601 (Compiled frame)
 - sun.rmi.server.UnicastServerRef.dispatch(java.rmi.Remote, 
java.rmi.server.RemoteCall) @bci=242, line=322 (Compiled frame)
 - sun.rmi.transport.Transport$1.run() @bci=23, line=177 (Compiled frame)
 - sun.rmi.transport.Transport$1.run() @bci=1, line=174 (Compiled frame)
 - 
java.security.AccessController.doPrivileged(java.security.PrivilegedExceptionAction,
 java.security.AccessControlContext) @bci=0 (Compiled frame)
 - sun.rmi.transport.Transport.serviceCall(java.rmi.server.RemoteCall) 
@bci=163, line=173 (Compiled frame)
 - 
sun.rmi.transport.tcp.TCPTransport.handleMessages(sun.rmi.transport.Connection, 
boolean) @bci=185, line=553 (Compiled frame)
 - sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0() @bci=685, 
line=808 (Interpreted frame)
 - sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run() @bci=50, line=667 
(Interpreted frame)
 - 
java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
 @bci=46, line=1110 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=603 
(Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 30475: (state = BLOCKED)
 - java.lang.Object.wait(long) @bci=0 (Interpreted frame)
 - com.sun.jmx.remote.internal.ServerCommunicatorAdmin$Timeout.run() @bci=227, 
line=168 (Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 25600: (state = BLOCKED)
 - java.lang.Object.wait(long) @bci=0 (Interpreted frame)
 - 
com.sun.jmx.remote.internal.ArrayNotificationBuffer.fetchNotifications(com.sun.jmx.remote.internal.NotificationBufferFilter,
 long, long, int) @bci=697, line=435 (Interpreted frame)
 - 
com.sun.jmx.remote.internal.ArrayNotificationBuffer$ShareBuffer.fetchNotifications(com.sun.jmx.remote.internal.NotificationBufferFilter,
 long, long, int) @bci=14, line=227 (Interpreted frame)
 - com.sun.jmx.remote.internal.ServerNotifForwarder.fetchNotifs(long, long, 
int) @bci=78, line=274 (Interpreted frame)
 - javax.management.remote.rmi.RMIConnectionImpl$3.run() @bci=19, line=1288 
(Interpreted frame)
 - javax.management.remote.rmi.RMIConnectionImpl$3.run() @bci=1, line=1286 
(Interpreted frame)
 - javax.management.remote.rmi.RMIConnectionImpl.fetchNotifications(long, int, 
long) @bci=145, line=1292 (Interpreted frame)
 - sun.reflect.GeneratedMethodAccessor86.invoke(java.lang.Object, 
java.lang.Object[]) @bci=320 (Interpreted frame)
 - sun.reflect.DelegatingMethodAccessorImpl.invoke(java.lang.Object, 
java.lang.Object[]) @bci=6, line=43 (Compiled frame)
 - java.lang.reflect.Method.invoke(java.lang.Object, java.lang.Object[]) 
@bci=57, line=601 (Compiled frame)
 - sun.rmi.server.UnicastServerRef.dispatch(java.rmi.Remote, 
java.rmi.server.RemoteCall) @bci=242, line=322 (Compiled frame)
 - sun.rmi.transport.Transport$1.run() @bci=23, line=177 (Compiled frame)
 - sun.rmi.transport.Transport$1.run() @bci=1, line=174 (Compiled frame)
 - 
java.security.AccessController.doPrivileged(java.security.PrivilegedExceptionAction,
 java.security.AccessControlContext) @bci=0 (Compiled frame)
 - sun.rmi.transport.Transport.serviceCall(java.rmi.server.RemoteCall) 
@bci=163, line=173 (Compiled frame)
 - 
sun.rmi.transport.tcp.TCPTransport.handleMessages(sun.rmi.transport.Connection, 
boolean) @bci=185, line=553 (Interpreted frame)
 - sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0() @bci=685, 
line=808 (Interpreted frame)
 - sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run() @bci=50, line=667 
(Interpreted frame)
 - 
java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
 @bci=46, line=1110 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=603 
(Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 25597: (state = BLOCKED)
 - java.lang.Object.wait(long) @bci=0 (Interpreted frame)
 - com.sun.jmx.remote.internal.ServerCommunicatorAdmin$Timeout.run() @bci=227, 
line=168 (Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 29033: (state = BLOCKED)
 - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information may 
be imprecise)
 - java.util.concurrent.locks.LockSupport.parkNanos(java.lang.Object, long) 
@bci=20, line=226 (Compiled frame)
 - 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(long)
 @bci=68, line=2082 (Compiled frame)
 - java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take() 
@bci=122, line=1090 (Compiled frame)
 - java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take() 
@bci=1, line=807 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor.getTask() @bci=156, line=1043 
(Compiled frame)
 - 
java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
 @bci=17, line=1103 (Interpreted frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=603 
(Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 6502: (state = BLOCKED)
 - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information may 
be imprecise)
 - java.util.concurrent.locks.LockSupport.parkNanos(java.lang.Object, long) 
@bci=20, line=226 (Compiled frame)
 - 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(long)
 @bci=68, line=2082 (Compiled frame)
 - java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take() 
@bci=122, line=1090 (Compiled frame)
 - java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take() 
@bci=1, line=807 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor.getTask() @bci=156, line=1043 
(Compiled frame)
 - 
java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
 @bci=17, line=1103 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=603 
(Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 6501: (state = BLOCKED)
 - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information may 
be imprecise)
 - java.util.concurrent.locks.LockSupport.parkNanos(java.lang.Object, long) 
@bci=20, line=226 (Compiled frame)
 - 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(long)
 @bci=68, line=2082 (Compiled frame)
 - java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take() 
@bci=122, line=1090 (Compiled frame)
 - java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take() 
@bci=1, line=807 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor.getTask() @bci=156, line=1043 
(Compiled frame)
 - 
java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
 @bci=17, line=1103 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=603 
(Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 6500: (state = BLOCKED)
 - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information may 
be imprecise)
 - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
line=186 (Compiled frame)
 - 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await() 
@bci=42, line=2043 (Compiled frame)
 - java.util.concurrent.ArrayBlockingQueue.take() @bci=20, line=374 
(Interpreted frame)
 - kafka.network.RequestChannel.receiveRequest() @bci=4, line=132 (Interpreted 
frame)
 - kafka.server.KafkaRequestHandler.run() @bci=22, line=34 (Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 6499: (state = BLOCKED)
 - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information may 
be imprecise)
 - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
line=186 (Compiled frame)
 - 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await() 
@bci=42, line=2043 (Compiled frame)
 - java.util.concurrent.ArrayBlockingQueue.take() @bci=20, line=374 
(Interpreted frame)
 - kafka.network.RequestChannel.receiveRequest() @bci=4, line=132 (Interpreted 
frame)
 - kafka.server.KafkaRequestHandler.run() @bci=22, line=34 (Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 6498: (state = BLOCKED)
 - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information may 
be imprecise)
 - java.util.concurrent.locks.LockSupport.parkNanos(java.lang.Object, long) 
@bci=20, line=226 (Compiled frame)
 - 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(long)
 @bci=68, line=2082 (Compiled frame)
 - java.util.concurrent.DelayQueue.poll(long, java.util.concurrent.TimeUnit) 
@bci=85, line=256 (Compiled frame)
 - kafka.server.RequestPurgatory$ExpiredRequestReaper.pollExpired() @bci=10, 
line=256 (Compiled frame)
 - kafka.server.RequestPurgatory$ExpiredRequestReaper.run() @bci=41, line=213 
(Compiled frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 6497: (state = BLOCKED)
 - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information may 
be imprecise)
 - java.util.concurrent.locks.LockSupport.parkNanos(java.lang.Object, long) 
@bci=20, line=226 (Compiled frame)
 - 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(long)
 @bci=68, line=2082 (Compiled frame)
 - java.util.concurrent.DelayQueue.poll(long, java.util.concurrent.TimeUnit) 
@bci=85, line=256 (Compiled frame)
 - kafka.server.RequestPurgatory$ExpiredRequestReaper.pollExpired() @bci=10, 
line=256 (Compiled frame)
 - kafka.server.RequestPurgatory$ExpiredRequestReaper.run() @bci=41, line=213 
(Compiled frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 6496: (state = BLOCKED)
 - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information may 
be imprecise)
 - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
line=186 (Compiled frame)
 - 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await() 
@bci=42, line=2043 (Compiled frame)
 - java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take() 
@bci=98, line=1085 (Compiled frame)
 - java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take() 
@bci=1, line=807 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor.getTask() @bci=156, line=1043 
(Compiled frame)
 - 
java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
 @bci=17, line=1103 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=603 
(Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 6495: (state = BLOCKED)
 - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information may 
be imprecise)
 - java.util.concurrent.locks.LockSupport.parkNanos(java.lang.Object, long) 
@bci=20, line=226 (Compiled frame)
 - 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(long)
 @bci=68, line=2082 (Compiled frame)
 - java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take() 
@bci=122, line=1090 (Compiled frame)
 - java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take() 
@bci=1, line=807 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor.getTask() @bci=156, line=1043 
(Compiled frame)
 - 
java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
 @bci=17, line=1103 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=603 
(Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 6494: (state = BLOCKED)
 - sun.misc.Unsafe.park(boolean, long) @bci=0 (Interpreted frame)
 - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
line=186 (Interpreted frame)
 - 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await() 
@bci=42, line=2043 (Interpreted frame)
 - java.util.concurrent.LinkedBlockingQueue.take() @bci=29, line=442 
(Interpreted frame)
 - org.apache.zookeeper.ClientCnxn$EventThread.run() @bci=9, line=503 
(Interpreted frame)


Thread 6493: (state = IN_NATIVE)
 - sun.nio.ch.EPollArrayWrapper.epollWait(long, int, long, int) @bci=0 
(Compiled frame; information may be imprecise)
 - sun.nio.ch.EPollArrayWrapper.poll(long) @bci=18, line=228 (Compiled frame)
 - sun.nio.ch.EPollSelectorImpl.doSelect(long) @bci=28, line=81 (Compiled frame)
 - sun.nio.ch.SelectorImpl.lockAndDoSelect(long) @bci=37, line=87 (Compiled 
frame)
 - sun.nio.ch.SelectorImpl.select(long) @bci=30, line=98 (Compiled frame)
 - org.apache.zookeeper.ClientCnxn$SendThread.run() @bci=233, line=1134 
(Compiled frame)


Thread 6492: (state = BLOCKED)
 - sun.misc.Unsafe.park(boolean, long) @bci=0 (Interpreted frame)
 - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
line=186 (Interpreted frame)
 - 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await() 
@bci=42, line=2043 (Interpreted frame)
 - java.util.concurrent.LinkedBlockingQueue.take() @bci=29, line=442 
(Interpreted frame)
 - org.I0Itec.zkclient.ZkEventThread.run() @bci=19, line=67 (Interpreted frame)


Thread 6491: (state = IN_NATIVE)
 - sun.nio.ch.EPollArrayWrapper.epollWait(long, int, long, int) @bci=0 
(Compiled frame; information may be imprecise)
 - sun.nio.ch.EPollArrayWrapper.poll(long) @bci=18, line=228 (Compiled frame)
 - sun.nio.ch.EPollSelectorImpl.doSelect(long) @bci=28, line=81 (Compiled frame)
 - sun.nio.ch.SelectorImpl.lockAndDoSelect(long) @bci=37, line=87 (Compiled 
frame)
 - sun.nio.ch.SelectorImpl.select(long) @bci=30, line=98 (Compiled frame)
 - kafka.network.Acceptor.run() @bci=147, line=144 (Compiled frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 6490: (state = IN_NATIVE)
 - sun.nio.ch.EPollArrayWrapper.epollWait(long, int, long, int) @bci=0 
(Compiled frame; information may be imprecise)
 - sun.nio.ch.EPollArrayWrapper.poll(long) @bci=18, line=228 (Compiled frame)
 - sun.nio.ch.EPollSelectorImpl.doSelect(long) @bci=28, line=81 (Compiled frame)
 - sun.nio.ch.SelectorImpl.lockAndDoSelect(long) @bci=37, line=87 (Compiled 
frame)
 - sun.nio.ch.SelectorImpl.select(long) @bci=30, line=98 (Compiled frame)
 - kafka.network.Processor.run() @bci=292, line=234 (Compiled frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 6489: (state = IN_NATIVE)
 - sun.nio.ch.EPollArrayWrapper.epollWait(long, int, long, int) @bci=0 
(Compiled frame; information may be imprecise)
 - sun.nio.ch.EPollArrayWrapper.poll(long) @bci=18, line=228 (Compiled frame)
 - sun.nio.ch.EPollSelectorImpl.doSelect(long) @bci=28, line=81 (Compiled frame)
 - sun.nio.ch.SelectorImpl.lockAndDoSelect(long) @bci=37, line=87 (Compiled 
frame)
 - sun.nio.ch.SelectorImpl.select(long) @bci=30, line=98 (Compiled frame)
 - kafka.network.Processor.run() @bci=292, line=234 (Compiled frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 6486: (state = BLOCKED)
 - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information may 
be imprecise)
 - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
line=186 (Compiled frame)
 - 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await() 
@bci=42, line=2043 (Compiled frame)
 - java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take() 
@bci=98, line=1085 (Compiled frame)
 - java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take() 
@bci=1, line=807 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor.getTask() @bci=156, line=1043 
(Compiled frame)
 - 
java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
 @bci=17, line=1103 (Interpreted frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=603 
(Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 6485: (state = BLOCKED)
 - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information may 
be imprecise)
 - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
line=186 (Compiled frame)
 - 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await() 
@bci=42, line=2043 (Compiled frame)
 - java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take() 
@bci=98, line=1085 (Compiled frame)
 - java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take() 
@bci=1, line=807 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor.getTask() @bci=156, line=1043 
(Compiled frame)
 - 
java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
 @bci=17, line=1103 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=603 
(Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 5146: (state = IN_NATIVE)
 - java.net.PlainSocketImpl.socketAccept(java.net.SocketImpl) @bci=0 
(Interpreted frame)
 - java.net.AbstractPlainSocketImpl.accept(java.net.SocketImpl) @bci=7, 
line=398 (Interpreted frame)
 - java.net.ServerSocket.implAccept(java.net.Socket) @bci=60, line=522 
(Interpreted frame)
 - java.net.ServerSocket.accept() @bci=48, line=490 (Interpreted frame)
 - sun.management.jmxremote.LocalRMIServerSocketFactory$1.accept() @bci=1, 
line=52 (Interpreted frame)
 - sun.rmi.transport.tcp.TCPTransport$AcceptLoop.executeAcceptLoop() @bci=55, 
line=387 (Interpreted frame)
 - sun.rmi.transport.tcp.TCPTransport$AcceptLoop.run() @bci=1, line=359 
(Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 5143: (state = IN_NATIVE)
 - java.net.PlainSocketImpl.socketAccept(java.net.SocketImpl) @bci=0 
(Interpreted frame)
 - java.net.AbstractPlainSocketImpl.accept(java.net.SocketImpl) @bci=7, 
line=398 (Interpreted frame)
 - java.net.ServerSocket.implAccept(java.net.Socket) @bci=60, line=522 
(Interpreted frame)
 - java.net.ServerSocket.accept() @bci=48, line=490 (Interpreted frame)
 - sun.rmi.transport.tcp.TCPTransport$AcceptLoop.executeAcceptLoop() @bci=55, 
line=387 (Interpreted frame)
 - sun.rmi.transport.tcp.TCPTransport$AcceptLoop.run() @bci=1, line=359 
(Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 5142: (state = IN_NATIVE)
 - java.net.PlainSocketImpl.socketAccept(java.net.SocketImpl) @bci=0 
(Interpreted frame)
 - java.net.AbstractPlainSocketImpl.accept(java.net.SocketImpl) @bci=7, 
line=398 (Interpreted frame)
 - java.net.ServerSocket.implAccept(java.net.Socket) @bci=60, line=522 
(Interpreted frame)
 - java.net.ServerSocket.accept() @bci=48, line=490 (Interpreted frame)
 - sun.rmi.transport.tcp.TCPTransport$AcceptLoop.executeAcceptLoop() @bci=55, 
line=387 (Interpreted frame)
 - sun.rmi.transport.tcp.TCPTransport$AcceptLoop.run() @bci=1, line=359 
(Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=722 (Interpreted frame)


Thread 5135: (state = BLOCKED)


Thread 5134: (state = BLOCKED)
 - java.lang.Object.wait(long) @bci=0 (Interpreted frame)
 - java.lang.ref.ReferenceQueue.remove(long) @bci=44, line=135 (Interpreted 
frame)
 - java.lang.ref.ReferenceQueue.remove() @bci=2, line=151 (Interpreted frame)
 - java.lang.ref.Finalizer$FinalizerThread.run() @bci=3, line=177 (Interpreted 
frame)


Thread 5133: (state = BLOCKED)
 - java.lang.Object.wait(long) @bci=0 (Interpreted frame)
 - java.lang.Object.wait() @bci=2, line=503 (Interpreted frame)
 - java.lang.ref.Reference$ReferenceHandler.run() @bci=46, line=133 
(Interpreted frame)


Thread 5113: (state = BLOCKED)
 - sun.misc.Unsafe.park(boolean, long) @bci=0 (Interpreted frame)
 - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
line=186 (Interpreted frame)
 - 
java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt() 
@bci=1, line=834 (Interpreted frame)
 - 
java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(int)
 @bci=72, line=994 (Interpreted frame)
 - 
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(int)
 @bci=24, line=1303 (Interpreted frame)
 - java.util.concurrent.CountDownLatch.await() @bci=5, line=236 (Interpreted 
frame)
 - kafka.server.KafkaServer.awaitShutdown() @bci=4, line=140 (Interpreted frame)
 - kafka.server.KafkaServerStartable.awaitShutdown() @bci=4, line=56 
(Interpreted frame)
 - kafka.Kafka$.main(java.lang.String[]) @bci=115, line=47 (Interpreted frame)
 - kafka.Kafka.main(java.lang.String[]) @bci=4 (Interpreted frame)


Regards,

Libo


-----Original Message-----
From: Neha Narkhede [mailto:neha.narkh...@gmail.com] 
Sent: Wednesday, August 21, 2013 11:53 AM
To: users@kafka.apache.org
Subject: RE: broker never comes back to ISR

Could you take a thread dump on that broker and send it across? One of the 
possibilities is the replica fetcher thread is somehow dead.

Thanks,
Neha
On Aug 21, 2013 8:00 AM, "Yu, Libo" <libo...@citi.com> wrote:

> I checked the log of normal restart. The replication manager should 
> start to handle leader and isr request after the server is up. What 
> may stop it from doing that?
> Is it because of missing mx4j-tools.jar?
>
> Regards,
>
> Libo
>
> From: Yu, Libo [ICG-IT]
> Sent: Wednesday, August 21, 2013 10:51 AM
> To: 'users@kafka.apache.org'
> Subject: broker never comes back to ISR
>
> Hi team,
>
> We have three kafka brokers in a production cluster. We use 
> replication factor 3 for all topics.
> We notice quite frequently one broker is not in isr. Sometimes after 
> it is restarted, it will go back to isr. Sometimes even after it is 
> restarted, it will not go back to isr.
>
> In today's case, after a broker is restarted, this is what we found 
> from the log:
>
> [2013-08-21 08:22:55,524] INFO [Kafka Server 2], started
> (kafka.server.KafkaServer)
> [2013-08-21 08:25:06,621] INFO Closing socket connection to /xxx.xx.xx.xx.
> (kafka.network.Processor)
> [2013-08-21 08:25:06,716] INFO Closing socket connection to / 
> xxx.xx.xx.xx. (kafka.network.Processor)
> [2013-08-21 08:27:19,824] INFO Closing socket connection to / 
> xxx.xx.xx.xx. (kafka.network.Processor)
> [2013-08-21 08:28:16,711] INFO Closing socket connection to / 
> xxx.xx.xx.xx. (kafka.network.Processor)
> [2013-08-21 08:28:17,978] INFO Closing socket connection to / 
> xxx.xx.xx.xx. (kafka.network.Processor) ...
> Numerous "Closing socket connection" and nothing else.
>
> Any guidance will be appreciated.
>
> Regards,
>
> Libo
>
>

Reply via email to