You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by "Artem Golotin (JIRA)" <ji...@apache.org> on 2016/08/01 22:27:21 UTC

[jira] [Comment Edited] (ZOOKEEPER-1863) Race condition in commit processor leading to out of order request completion, xid mismatch on client.

    [ https://issues.apache.org/jira/browse/ZOOKEEPER-1863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15402819#comment-15402819 ] 

Artem Golotin edited comment on ZOOKEEPER-1863 at 8/1/16 10:26 PM:
-------------------------------------------------------------------

Hi! I was able to reproduce the same issue as mentioned in [ZOOKEEPER-2151|https://issues.apache.org/jira/browse/ZOOKEEPER-2151]. Here is the output from the stat command: 
{noformat}
Zookeeper version: 3.5.1-alpha--1, built on 07/08/2016 17:08 GMT
Clients:
 /0:0:0:0:0:0:0:1:59312[0](queued=0,recved=1,sent=0)
 /10.2.29.7:34266[0](queued=0,recved=1,sent=0)
 /10.2.29.7:55996[0](queued=0,recved=1,sent=0)
 /10.2.81.3:48640[0](queued=0,recved=1,sent=0)

Latency min/avg/max: 0/1/826
Received: 2199007
Sent: 935511
Connections: 4
Outstanding: 48112
Zxid: 0x100011d4e
Mode: leader
Node count: 171
{noformat}

Notice the number of outstanding connections.... 
After examining the code I found that the patch that have been submitted with this issue was applied to the code. With my limited experience I do not believe that I will be able to reproduce the issue, but I am saving the full stdout zookeeper output and can provide it if needed. 

Here's jstack dump:
{noformat}
Full thread dump OpenJDK 64-Bit Server VM (24.95-b01 mixed mode):

"NIOWorkerThread-4" daemon prio=10 tid=0x00007fbb9c007000 nid=0x1a7 waiting on condition [0x00007fbbd84ce000]
   java.lang.Thread.State: WAITING (parking)
	at sun.misc.Unsafe.park(Native Method)
	- parking to wait for  <0x00000000c18ed460> (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 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)

"NIOWorkerThread-3" daemon prio=10 tid=0x00007fbb9c005000 nid=0x1a6 waiting on condition [0x00007fbbd85cf000]
   java.lang.Thread.State: WAITING (parking)
	at sun.misc.Unsafe.park(Native Method)
	- parking to wait for  <0x00000000c18ed460> (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 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)

"NIOWorkerThread-2" daemon prio=10 tid=0x00007fbb9c003800 nid=0x1a4 waiting on condition [0x00007fbbd87d1000]
   java.lang.Thread.State: WAITING (parking)
	at sun.misc.Unsafe.park(Native Method)
	- parking to wait for  <0x00000000c18ed460> (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 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)

"ContainerManagerTask" daemon prio=10 tid=0x00007fbbb801b800 nid=0x1a3 in Object.wait() [0x00007fbbd88d2000]
   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.util.TimerThread.mainLoop(Timer.java:526)
	- locked <0x00000000c190fa68> (a java.util.TaskQueue)
	at java.util.TimerThread.run(Timer.java:505)

"SyncThread:1" prio=10 tid=0x00007fbbb8016000 nid=0x1a1 in Object.wait() [0x00007fbbd8ad4000]
   java.lang.Thread.State: WAITING (on object monitor)
	at java.lang.Object.wait(Native Method)
	at java.lang.Thread.join(Thread.java:1281)
	- locked <0x00000000c195c128> (a org.apache.zookeeper.server.SyncRequestProcessor)
	at java.lang.Thread.join(Thread.java:1355)
	at org.apache.zookeeper.server.SyncRequestProcessor.shutdown(SyncRequestProcessor.java:195)
	at org.apache.zookeeper.server.quorum.ProposalRequestProcessor.shutdown(ProposalRequestProcessor.java:90)
	at org.apache.zookeeper.server.PrepRequestProcessor.shutdown(PrepRequestProcessor.java:954)
	at org.apache.zookeeper.server.quorum.LeaderRequestProcessor.shutdown(LeaderRequestProcessor.java:78)
	at org.apache.zookeeper.server.ZooKeeperServer.shutdown(ZooKeeperServer.java:476)
	- locked <0x00000000c18ebd28> (a org.apache.zookeeper.server.quorum.LeaderZooKeeperServer)
	at org.apache.zookeeper.server.quorum.LeaderZooKeeperServer.shutdown(LeaderZooKeeperServer.java:102)
	- locked <0x00000000c18ebd28> (a org.apache.zookeeper.server.quorum.LeaderZooKeeperServer)
	at org.apache.zookeeper.server.ZooKeeperServer$ZooKeeperServerListenerImpl.notifyStopping(ZooKeeperServer.java:443)
	at org.apache.zookeeper.server.ZooKeeperCriticalThread.handleException(ZooKeeperCriticalThread.java:49)
	at org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:165)

"NIOWorkerThread-1" daemon prio=10 tid=0x00007fbb9c002000 nid=0x1a0 waiting on condition [0x00007fbbd8bd5000]
   java.lang.Thread.State: WAITING (parking)
	at sun.misc.Unsafe.park(Native Method)
	- parking to wait for  <0x00000000c18ed460> (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 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)

"LearnerCnxAcceptor-/10.2.88.8:2888" prio=10 tid=0x00007fbbb8007000 nid=0x19d runnable [0x00007fbbd8ed8000]
   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 org.apache.zookeeper.server.quorum.Leader$LearnerCnxAcceptor.run(Leader.java:364)

"QuorumPeer[myid=1](plain=/0:0:0:0:0:0:0:0:2181)(secure=disabled)" prio=10 tid=0x00007fbbe02fd800 nid=0x19c in Object.wait() [0x00007fbbd8fd9000]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)
	at java.lang.Object.wait(Native Method)
	at org.apache.zookeeper.server.quorum.Leader.lead(Leader.java:559)
	- locked <0x00000000c18ebe20> (a org.apache.zookeeper.server.quorum.Leader)
	at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:1036)

"WorkerReceiver[myid=1]" daemon prio=10 tid=0x00007fbbe02fb800 nid=0x19b waiting on condition [0x00007fbbd90da000]
   java.lang.Thread.State: TIMED_WAITING (parking)
	at sun.misc.Unsafe.park(Native Method)
	- parking to wait for  <0x00000000c1820688> (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 org.apache.zookeeper.server.quorum.QuorumCnxManager.pollRecvQueue(QuorumCnxManager.java:1038)
	at org.apache.zookeeper.server.quorum.FastLeaderElection$Messenger$WorkerReceiver.run(FastLeaderElection.java:235)
	at java.lang.Thread.run(Thread.java:745)

"WorkerSender[myid=1]" daemon prio=10 tid=0x00007fbbe02fa800 nid=0x19a waiting on condition [0x00007fbbd91db000]
   java.lang.Thread.State: TIMED_WAITING (parking)
	at sun.misc.Unsafe.park(Native Method)
	- parking to wait for  <0x00000000c1820e98> (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.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
	at org.apache.zookeeper.server.quorum.FastLeaderElection$Messenger$WorkerSender.run(FastLeaderElection.java:457)
	at java.lang.Thread.run(Thread.java:745)

"/10.2.88.8:3888" prio=10 tid=0x00007fbbe02e9000 nid=0x199 runnable [0x00007fbbd92dc000]
   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 org.apache.zookeeper.server.quorum.QuorumCnxManager$Listener.run(QuorumCnxManager.java:635)

"2029193347@qtp-381175311-1 - Acceptor0 SocketConnector@0.0.0.0:8080" prio=10 tid=0x00007fbbe02d9000 nid=0x198 runnable [0x00007fbbd93dd000]
   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 org.mortbay.jetty.bio.SocketConnector.accept(SocketConnector.java:99)
	at org.mortbay.jetty.AbstractConnector$Acceptor.run(AbstractConnector.java:708)
	at org.mortbay.thread.QueuedThreadPool$PoolThread.run(QueuedThreadPool.java:582)

"1451939056@qtp-381175311-0" prio=10 tid=0x00007fbbe02dd800 nid=0x197 in Object.wait() [0x00007fbbd94de000]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)
	at java.lang.Object.wait(Native Method)
	at org.mortbay.thread.QueuedThreadPool$PoolThread.run(QueuedThreadPool.java:626)
	- locked <0x00000000c1821e40> (a org.mortbay.thread.QueuedThreadPool$PoolThread)

"ConnnectionExpirer" prio=10 tid=0x00007fbbe02d7800 nid=0x196 waiting on condition [0x00007fbbd95df000]
   java.lang.Thread.State: TIMED_WAITING (sleeping)
	at java.lang.Thread.sleep(Native Method)
	at org.apache.zookeeper.server.NIOServerCnxnFactory$ConnectionExpirerThread.run(NIOServerCnxnFactory.java:574)

"NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181" daemon prio=10 tid=0x00007fbbe02d6000 nid=0x195 runnable [0x00007fbbd96e0000]
   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 <0x00000000c19609b0> (a sun.nio.ch.Util$2)
	- locked <0x00000000c19609a0> (a java.util.Collections$UnmodifiableSet)
	- locked <0x00000000c19607a8> (a sun.nio.ch.EPollSelectorImpl)
	at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:98)
	at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:102)
	at org.apache.zookeeper.server.NIOServerCnxnFactory$AcceptThread.select(NIOServerCnxnFactory.java:229)
	at org.apache.zookeeper.server.NIOServerCnxnFactory$AcceptThread.run(NIOServerCnxnFactory.java:205)

"NIOServerCxnFactory.SelectorThread-0" daemon prio=10 tid=0x00007fbbe02d5800 nid=0x194 runnable [0x00007fbbd97e1000]
   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 <0x00000000c19479e8> (a sun.nio.ch.Util$2)
	- locked <0x00000000c19479d8> (a java.util.Collections$UnmodifiableSet)
	- locked <0x00000000c1947350> (a sun.nio.ch.EPollSelectorImpl)
	at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:98)
	at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:102)
	at org.apache.zookeeper.server.NIOServerCnxnFactory$SelectorThread.select(NIOServerCnxnFactory.java:426)
	at org.apache.zookeeper.server.NIOServerCnxnFactory$SelectorThread.run(NIOServerCnxnFactory.java:391)

"PurgeTask" daemon prio=10 tid=0x00007fbbe024d000 nid=0x193 in Object.wait() [0x00007fbbd98e2000]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)
	at java.lang.Object.wait(Native Method)
	at java.util.TimerThread.mainLoop(Timer.java:552)
	- locked <0x00000000c1822748> (a java.util.TaskQueue)
	at java.util.TimerThread.run(Timer.java:505)

"RMI TCP Accept-0" daemon prio=10 tid=0x00007fbbe01ed000 nid=0x191 runnable [0x00007fbbe4168000]
   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=0x00007fbbe00ab000 nid=0x190 runnable [0x0000000000000000]
   java.lang.Thread.State: RUNNABLE

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

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

"Signal Dispatcher" daemon prio=10 tid=0x00007fbbe00a3800 nid=0x18d waiting on condition [0x0000000000000000]
   java.lang.Thread.State: RUNNABLE

"Finalizer" daemon prio=10 tid=0x00007fbbe0078000 nid=0x18c in Object.wait() [0x00007fbbe4ea8000]
   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 <0x00000000c1861290> (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=0x00007fbbe0076000 nid=0x18b in Object.wait() [0x00007fbbe4fa9000]
   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 <0x00000000c1860ec0> (a java.lang.ref.Reference$Lock)

"main" prio=10 tid=0x00007fbbe000a800 nid=0x187 in Object.wait() [0x00007fbbe9a1a000]
   java.lang.Thread.State: WAITING (on object monitor)
	at java.lang.Object.wait(Native Method)
	- waiting on <0x00000000c18ebec0> (a org.apache.zookeeper.server.quorum.QuorumPeer)
	at java.lang.Thread.join(Thread.java:1281)
	- locked <0x00000000c18ebec0> (a org.apache.zookeeper.server.quorum.QuorumPeer)
	at java.lang.Thread.join(Thread.java:1355)
	at org.apache.zookeeper.server.quorum.QuorumPeerMain.runFromConfig(QuorumPeerMain.java:184)
	at org.apache.zookeeper.server.quorum.QuorumPeerMain.initializeAndRun(QuorumPeerMain.java:120)
	at org.apache.zookeeper.server.quorum.QuorumPeerMain.main(QuorumPeerMain.java:79)

"VM Thread" prio=10 tid=0x00007fbbe0071800 nid=0x18a runnable 

"GC task thread#0 (ParallelGC)" prio=10 tid=0x00007fbbe0020800 nid=0x188 runnable 

"GC task thread#1 (ParallelGC)" prio=10 tid=0x00007fbbe0022800 nid=0x189 runnable 

"VM Periodic Task Thread" prio=10 tid=0x00007fbbe01fa800 nid=0x192 waiting on condition 

JNI global references: 149

Heap
 PSYoungGen      total 26112K, used 8888K [0x00000000eb280000, 0x00000000ed880000, 0x0000000100000000)
  eden space 13312K, 45% used [0x00000000eb280000,0x00000000eb86e050,0x00000000ebf80000)
  from space 12800K, 22% used [0x00000000ecc00000,0x00000000ecec0000,0x00000000ed880000)
  to   space 12800K, 0% used [0x00000000ebf80000,0x00000000ebf80000,0x00000000ecc00000)
 ParOldGen       total 278528K, used 266898K [0x00000000c1800000, 0x00000000d2800000, 0x00000000eb280000)
  object space 278528K, 95% used [0x00000000c1800000,0x00000000d1ca48e8,0x00000000d2800000)
 PSPermGen       total 21504K, used 14831K [0x00000000b7200000, 0x00000000b8700000, 0x00000000c1800000)
  object space 21504K, 68% used [0x00000000b7200000,0x00000000b807bd50,0x00000000b8700000)
{noformat}

Unfortunately, the issue seems to have happened on Sunday when I was not able to check the health of my cluster or perform any immediate fixes. I do have several Kazoo clients + Kafka + Secor that are using zookeeper, and as all those are failing now with zookeeper outputting the following logs:

{noformat}
2016-08-01 20:45:51,828 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.81.3:50656
2016-08-01 20:45:51,829 - INFO  [NIOWorkerThread-4:ZooKeeperServer@931] - Client attempting to establish new session at /10.2.81.3:50656
2016-08-01 20:45:53,723 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.29.7:35862
2016-08-01 20:45:53,725 - INFO  [NIOWorkerThread-3:ZooKeeperServer@931] - Client attempting to establish new session at /10.2.29.7:35862
2016-08-01 20:45:59,171 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.81.3:50658
2016-08-01 20:45:59,172 - INFO  [NIOWorkerThread-1:ZooKeeperServer@931] - Client attempting to establish new session at /10.2.81.3:50658
2016-08-01 20:46:00,223 - INFO  [ConnnectionExpirer:NIOServerCnxn@1007] - Closed socket connection for client /10.2.29.7:35828 which had sessionid 0x10044f682c45bdd
2016-08-01 20:46:00,223 - INFO  [ConnnectionExpirer:NIOServerCnxn@1007] - Closed socket connection for client /10.2.81.3:50656 which had sessionid 0x10044f682c45bdf
2016-08-01 20:46:05,013 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.29.7:35898
2016-08-01 20:46:05,028 - INFO  [NIOWorkerThread-2:NIOServerCnxn@836] - Processing ruok command from /10.2.29.7:35898
2016-08-01 20:46:05,029 - INFO  [NIOWorkerThread-2:NIOServerCnxn@1007] - Closed socket connection for client /10.2.29.7:35898 (no session established for client)
2016-08-01 20:46:05,146 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.29.7:35900
2016-08-01 20:46:05,153 - INFO  [NIOWorkerThread-4:NIOServerCnxn@836] - Processing srvr command from /10.2.29.7:35900
2016-08-01 20:46:05,153 - INFO  [NIOWorkerThread-4:NIOServerCnxn@1007] - Closed socket connection for client /10.2.29.7:35900 (no session established for client)
2016-08-01 20:46:05,617 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.29.7:35904
2016-08-01 20:46:05,617 - INFO  [NIOWorkerThread-3:NIOServerCnxn@836] - Processing ruok command from /10.2.29.7:35904
2016-08-01 20:46:05,618 - INFO  [NIOWorkerThread-3:NIOServerCnxn@1007] - Closed socket connection for client /10.2.29.7:35904 (no session established for client)
2016-08-01 20:46:05,634 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.29.7:35906
2016-08-01 20:46:05,634 - INFO  [NIOWorkerThread-1:NIOServerCnxn@836] - Processing srvr command from /10.2.29.7:35906
2016-08-01 20:46:05,636 - INFO  [NIOWorkerThread-1:NIOServerCnxn@1007] - Closed socket connection for client /10.2.29.7:35906 (no session established for client)
2016-08-01 20:46:05,644 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.29.7:35908
2016-08-01 20:46:05,644 - INFO  [NIOWorkerThread-2:ZooKeeperServer@931] - Client attempting to establish new session at /10.2.29.7:35908
2016-08-01 20:46:06,718 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.81.3:50660
2016-08-01 20:46:06,719 - INFO  [NIOWorkerThread-4:ZooKeeperServer@931] - Client attempting to establish new session at /10.2.81.3:50660
2016-08-01 20:46:10,224 - INFO  [ConnnectionExpirer:NIOServerCnxn@1007] - Closed socket connection for client /10.2.81.3:50658 which had sessionid 0x10044f682c45be1
2016-08-01 20:46:10,225 - INFO  [ConnnectionExpirer:NIOServerCnxn@1007] - Closed socket connection for client /10.2.29.7:35862 which had sessionid 0x10044f682c45be0
2016-08-01 20:46:13,933 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.81.3:50662
2016-08-01 20:46:13,933 - INFO  [NIOWorkerThread-3:ZooKeeperServer@931] - Client attempting to establish new session at /10.2.81.3:50662
2016-08-01 20:46:16,030 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.29.7:35942
2016-08-01 20:46:16,032 - INFO  [NIOWorkerThread-1:ZooKeeperServer@931] - Client attempting to establish new session at /10.2.29.7:35942
2016-08-01 20:46:18,646 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.29.7:57696
2016-08-01 20:46:18,647 - INFO  [NIOWorkerThread-2:ZooKeeperServer@931] - Client attempting to establish new session at /10.2.29.7:57696
2016-08-01 20:46:20,223 - INFO  [ConnnectionExpirer:NIOServerCnxn@1007] - Closed socket connection for client /10.2.29.7:35908 which had sessionid 0x10044f682c45be2
2016-08-01 20:46:20,224 - INFO  [ConnnectionExpirer:NIOServerCnxn@1007] - Closed socket connection for client /10.2.81.3:50662 which had sessionid 0x10044f682c45be4
2016-08-01 20:46:20,225 - INFO  [ConnnectionExpirer:NIOServerCnxn@1007] - Closed socket connection for client /10.2.81.3:50660 which had sessionid 0x10044f682c45be3
2016-08-01 20:46:21,255 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.81.3:50664
2016-08-01 20:46:21,256 - INFO  [NIOWorkerThread-4:ZooKeeperServer@931] - Client attempting to establish new session at /10.2.81.3:50664
{noformat}

Also, I would like to point out that I am running zookeeper as a part of kubernetes cluster, so the issue may be caused by kubernetes internal proxy, but I do not know why that would cause an issue...


was (Author: agolotin):
Hi! I was able to reproduce the same issue as mentioned in [ZOOKEEPER-2151|https://issues.apache.org/jira/browse/ZOOKEEPER-2151]. Here is the output from the stat command: 
{noformat}
Zookeeper version: 3.5.1-alpha--1, built on 07/08/2016 17:08 GMT
Clients:
 /0:0:0:0:0:0:0:1:59312[0](queued=0,recved=1,sent=0)
 /10.2.29.7:34266[0](queued=0,recved=1,sent=0)
 /10.2.29.7:55996[0](queued=0,recved=1,sent=0)
 /10.2.81.3:48640[0](queued=0,recved=1,sent=0)

Latency min/avg/max: 0/1/826
Received: 2199007
Sent: 935511
Connections: 4
Outstanding: 48112
Zxid: 0x100011d4e
Mode: leader
Node count: 171
{noformat}

Notice the number of outstanding connections.... 
After examining the code I found that the patch that have been submitted with this issue was applied to the code. With my limited experience I do not believe that I will be able to reproduce the issue, but I am saving the full stdout zookeeper output and can provide it if needed. However, I cannot provide you with a jstack dump, because when I try to do so (even with -F option), it generates the following exception:
{noformat}
Error attaching to process: sun.jvm.hotspot.debugger.DebuggerException: Can't attach to the process
{noformat}

Unfortunately, the issue seems to have happened on Sunday when I was not able to check the health of my cluster or perform any immediate fixes. I do have several Kazoo clients + Kafka + Secor that are using zookeeper, and as all those are failing now with zookeeper outputting the following logs:

{noformat}
2016-08-01 20:45:51,828 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.81.3:50656
2016-08-01 20:45:51,829 - INFO  [NIOWorkerThread-4:ZooKeeperServer@931] - Client attempting to establish new session at /10.2.81.3:50656
2016-08-01 20:45:53,723 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.29.7:35862
2016-08-01 20:45:53,725 - INFO  [NIOWorkerThread-3:ZooKeeperServer@931] - Client attempting to establish new session at /10.2.29.7:35862
2016-08-01 20:45:59,171 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.81.3:50658
2016-08-01 20:45:59,172 - INFO  [NIOWorkerThread-1:ZooKeeperServer@931] - Client attempting to establish new session at /10.2.81.3:50658
2016-08-01 20:46:00,223 - INFO  [ConnnectionExpirer:NIOServerCnxn@1007] - Closed socket connection for client /10.2.29.7:35828 which had sessionid 0x10044f682c45bdd
2016-08-01 20:46:00,223 - INFO  [ConnnectionExpirer:NIOServerCnxn@1007] - Closed socket connection for client /10.2.81.3:50656 which had sessionid 0x10044f682c45bdf
2016-08-01 20:46:05,013 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.29.7:35898
2016-08-01 20:46:05,028 - INFO  [NIOWorkerThread-2:NIOServerCnxn@836] - Processing ruok command from /10.2.29.7:35898
2016-08-01 20:46:05,029 - INFO  [NIOWorkerThread-2:NIOServerCnxn@1007] - Closed socket connection for client /10.2.29.7:35898 (no session established for client)
2016-08-01 20:46:05,146 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.29.7:35900
2016-08-01 20:46:05,153 - INFO  [NIOWorkerThread-4:NIOServerCnxn@836] - Processing srvr command from /10.2.29.7:35900
2016-08-01 20:46:05,153 - INFO  [NIOWorkerThread-4:NIOServerCnxn@1007] - Closed socket connection for client /10.2.29.7:35900 (no session established for client)
2016-08-01 20:46:05,617 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.29.7:35904
2016-08-01 20:46:05,617 - INFO  [NIOWorkerThread-3:NIOServerCnxn@836] - Processing ruok command from /10.2.29.7:35904
2016-08-01 20:46:05,618 - INFO  [NIOWorkerThread-3:NIOServerCnxn@1007] - Closed socket connection for client /10.2.29.7:35904 (no session established for client)
2016-08-01 20:46:05,634 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.29.7:35906
2016-08-01 20:46:05,634 - INFO  [NIOWorkerThread-1:NIOServerCnxn@836] - Processing srvr command from /10.2.29.7:35906
2016-08-01 20:46:05,636 - INFO  [NIOWorkerThread-1:NIOServerCnxn@1007] - Closed socket connection for client /10.2.29.7:35906 (no session established for client)
2016-08-01 20:46:05,644 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.29.7:35908
2016-08-01 20:46:05,644 - INFO  [NIOWorkerThread-2:ZooKeeperServer@931] - Client attempting to establish new session at /10.2.29.7:35908
2016-08-01 20:46:06,718 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.81.3:50660
2016-08-01 20:46:06,719 - INFO  [NIOWorkerThread-4:ZooKeeperServer@931] - Client attempting to establish new session at /10.2.81.3:50660
2016-08-01 20:46:10,224 - INFO  [ConnnectionExpirer:NIOServerCnxn@1007] - Closed socket connection for client /10.2.81.3:50658 which had sessionid 0x10044f682c45be1
2016-08-01 20:46:10,225 - INFO  [ConnnectionExpirer:NIOServerCnxn@1007] - Closed socket connection for client /10.2.29.7:35862 which had sessionid 0x10044f682c45be0
2016-08-01 20:46:13,933 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.81.3:50662
2016-08-01 20:46:13,933 - INFO  [NIOWorkerThread-3:ZooKeeperServer@931] - Client attempting to establish new session at /10.2.81.3:50662
2016-08-01 20:46:16,030 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.29.7:35942
2016-08-01 20:46:16,032 - INFO  [NIOWorkerThread-1:ZooKeeperServer@931] - Client attempting to establish new session at /10.2.29.7:35942
2016-08-01 20:46:18,646 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.29.7:57696
2016-08-01 20:46:18,647 - INFO  [NIOWorkerThread-2:ZooKeeperServer@931] - Client attempting to establish new session at /10.2.29.7:57696
2016-08-01 20:46:20,223 - INFO  [ConnnectionExpirer:NIOServerCnxn@1007] - Closed socket connection for client /10.2.29.7:35908 which had sessionid 0x10044f682c45be2
2016-08-01 20:46:20,224 - INFO  [ConnnectionExpirer:NIOServerCnxn@1007] - Closed socket connection for client /10.2.81.3:50662 which had sessionid 0x10044f682c45be4
2016-08-01 20:46:20,225 - INFO  [ConnnectionExpirer:NIOServerCnxn@1007] - Closed socket connection for client /10.2.81.3:50660 which had sessionid 0x10044f682c45be3
2016-08-01 20:46:21,255 - INFO  [NIOServerCxnFactory.AcceptThread:/0.0.0.0:2181:NIOServerCnxnFactory$AcceptThread@296] - Accepted socket connection from /10.2.81.3:50664
2016-08-01 20:46:21,256 - INFO  [NIOWorkerThread-4:ZooKeeperServer@931] - Client attempting to establish new session at /10.2.81.3:50664
{noformat}

Also, I would like to point out that I am running zookeeper as a part of kubernetes cluster, so the issue may be caused by kubernetes internal proxy, but I do not know why that would cause an issue...

> Race condition in commit processor leading to out of order request completion, xid mismatch on client.
> ------------------------------------------------------------------------------------------------------
>
>                 Key: ZOOKEEPER-1863
>                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1863
>             Project: ZooKeeper
>          Issue Type: Bug
>          Components: server
>    Affects Versions: 3.5.0
>            Reporter: Dutch T. Meyer
>            Assignee: Dutch T. Meyer
>            Priority: Blocker
>             Fix For: 3.5.0
>
>         Attachments: ZOOKEEPER-1863.patch, ZOOKEEPER-1863.patch, ZOOKEEPER-1863.patch, ZOOKEEPER-1863.patch, ZOOKEEPER-1863.patch, ZOOKEEPER-1863.patch, ZOOKEEPER-1863.patch, stack.17512
>
>
> In CommitProcessor.java processor, if we are at the primary request handler on line 167:
> {noformat}
>                 while (!stopped && !isWaitingForCommit() &&
>                        !isProcessingCommit() &&
>                        (request = queuedRequests.poll()) != null) {
>                     if (needCommit(request)) {
>                         nextPending.set(request);
>                     } else {
>                         sendToNextProcessor(request);
>                     }
>                 }
> {noformat}
> A request can be handled in this block and be quickly processed and completed on another thread. If queuedRequests is empty, we then exit the block. Next, before this thread makes any more progress, we can get 2 more requests, one get_children(say), and a sync placed on queuedRequests for the processor. Then, if we are very unlucky, the sync request can complete and this object's commit() routine is called (from FollowerZookeeperServer), which places the sync request on the previously empty committedRequests queue. At that point, this thread continues.
> We reach line 182, which is a check on sync requests.
> {noformat}
>                 if (!stopped && !isProcessingRequest() &&
>                     (request = committedRequests.poll()) != null) {
> {noformat}
> Here we are not processing any requests, because the original request has completed. We haven't dequeued either the read or the sync request in this processor. Next, the poll above will pull the sync request off the queue, and in the following block, the sync will get forwarded to the next processor.
> This is a problem because the read request hasn't been forwarded yet, so requests are now out of order.
> I've been able to reproduce this bug reliably by injecting a Thread.sleep(5000) between the two blocks above to make the race condition far more likely, then in a client program.
> {noformat}
>         zoo_aget_children(zh, "/", 0, getchildren_cb, NULL);
>         //Wait long enough for queuedRequests to drain
>         sleep(1);
>         zoo_aget_children(zh, "/", 0, getchildren_cb, &th_ctx[0]);
>         zoo_async(zh, "/", sync_cb, &th_ctx[0]);
> {noformat}
> When this bug is triggered, 3 things can happen:
> 1) Clients will see requests complete out of order and fail on xid mismatches.
> 2) Kazoo in particular doesn't handle this runtime exception well, and can orphan outstanding requests.
> 3) I've seen zookeeper servers deadlock, likely because the commit cannot be completed, which can wedge the commit processor.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)