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

Tom Byrne commented on CURATOR-172:
-----------------------------------

OK, I think I might have a line on what's happening. 

Thread A is making a call. It gets a CuratorZKClient, which calls checkTimeouts 
(getting the ConnectionState monitor). The connection is gone, and so reset() 
gets called, then HandleHolder calls ZK.close, which then calls 
ZKClientCnxn.close. 

This calls submitRequest which queues a "disconnect" packet, then wait()'s on 
the packet object. 

This wait() will only get notified when the packet is dequeued and notifyAll() 
is called in finishPacket(). When this thread calls wait(), it releases the 
monitor lock on the Packet object, but keeps the other two monitors that it 
has, one on the ConnectionState object, and another on the Zookeeper object 
itself. 

During this wait, the SendThread (either already running or wakes up, 
whatever). Throws an exception, which trickles through to cleanup().

inside cleanup(), the pending queue is drained, calling conLossPacket, then 
finishPacket(). 

Because they're pending aysnc packets, eventThread.queuePacket() is called on 
the pending async packet. We call the callback, which in turn trickles back up 
to (in this case) the CreateBuilderImpl AsyncCallback.StringCallback's 
processResult, which then tries to submit another event, which will never get 
processed, because in order to get the ZKClient again, we call 
getZookeeperClient, which will try to get the ConnectionState monitor lock that 
Thread A has.  

It will wait for that, never dequeuing, processing and notifying the 
"disconnect" packet that Thread A is waiting on.

I think I can reproduce this with a small test, I'll see what I can come up 
with in the next day or two.

> Deadlock when performing background operation
> ---------------------------------------------
>
>                 Key: CURATOR-172
>                 URL: https://issues.apache.org/jira/browse/CURATOR-172
>             Project: Apache Curator
>          Issue Type: Bug
>          Components: Client
>    Affects Versions: 2.4.2
>         Environment: Linux HOSTNAME-REMOVED 2.6.32-279.19.1.el6.x86_64 #1 SMP 
> Tue Dec 18 15:04:44 PST 2012 x86_64 x86_64 x86_64 GNU/Linux
> java version "1.7.0_60"
> Java(TM) SE Runtime Environment (build 1.7.0_60-b19)
> Java HotSpot(TM) 64-Bit Server VM (build 24.60-b09, mixed mode)
>            Reporter: Tom Byrne
>
> Had a box get into a state where our ZK connections were all deadlocked, 
> waiting on an object monitor. jstack shows that our background thread that 
> was creating a node was waiting on a lock that was held by the 
> CuratorFramework thread, who was waiting on an object monitor that looks like 
> it couldn't be completed until our other write was finished (packet.finish 
> would never return true.) 
> We have seen this happen twice, but don't notice it until afterwards, and 
> don't have enough logging to know what's triggering it (possible ZK 
> connections going away?) 
> Rest of the box is fine, network connections are not flapping, main IO 
> threads continue to accept and process connections, until we get backed up 
> waiting for ZK. 
> Here are the two stack traces:
> "ZooChangeWatcher-BackgroundReader--2-1-SendThread()" daemon prio=10 
> tid=0x00007fcf64108000 nid=0x88d waiting for monitor entry 
> [0x00007fcbf5d16000]
>    java.lang.Thread.State: BLOCKED (on object monitor)
>       at 
> org.apache.curator.ConnectionState.checkTimeouts(ConnectionState.java:177)
>       - waiting to lock <0x00000000d526bcc8> (a 
> org.apache.curator.ConnectionState)
>       at 
> org.apache.curator.ConnectionState.getZooKeeper(ConnectionState.java:88)
>       at 
> org.apache.curator.CuratorZookeeperClient.getZooKeeper(CuratorZookeeperClient.java:115)
>       at 
> org.apache.curator.framework.imps.CuratorFrameworkImpl.performBackgroundOperation(CuratorFrameworkImpl.java:763)
>       at 
> org.apache.curator.framework.imps.CuratorFrameworkImpl.processBackgroundOperation(CuratorFrameworkImpl.java:470)
>       at 
> org.apache.curator.framework.imps.CreateBuilderImpl.pathInBackground(CreateBuilderImpl.java:648)
>       at 
> org.apache.curator.framework.imps.CreateBuilderImpl.forPath(CreateBuilderImpl.java:427)
>       at 
> org.apache.curator.framework.imps.CreateBuilderImpl.forPath(CreateBuilderImpl.java:44)
>       at 
> org.apache.curator.framework.recipes.nodes.PersistentEphemeralNode.createNode(PersistentEphemeralNode.java:340)
>       at 
> org.apache.curator.framework.recipes.nodes.PersistentEphemeralNode.access$000(PersistentEphemeralNode.java:52)
>       at 
> org.apache.curator.framework.recipes.nodes.PersistentEphemeralNode$4.processResult(PersistentEphemeralNode.java:224)
>       at 
> org.apache.curator.framework.imps.CuratorFrameworkImpl.sendToBackgroundCallback(CuratorFrameworkImpl.java:686)
>       at 
> org.apache.curator.framework.imps.CuratorFrameworkImpl.checkBackgroundRetry(CuratorFrameworkImpl.java:659)
>       at 
> org.apache.curator.framework.imps.CuratorFrameworkImpl.processBackgroundOperation(CuratorFrameworkImpl.java:479)
>       at 
> org.apache.curator.framework.imps.CreateBuilderImpl.sendBackgroundResponse(CreateBuilderImpl.java:526)
>       at 
> org.apache.curator.framework.imps.CreateBuilderImpl.access$600(CreateBuilderImpl.java:44)
>       at 
> org.apache.curator.framework.imps.CreateBuilderImpl$6.processResult(CreateBuilderImpl.java:485)
>       at 
> org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:602)
>       at 
> org.apache.zookeeper.ClientCnxn$EventThread.queuePacket(ClientCnxn.java:475)
>       - locked <0x00000000fa8e16f8> (a 
> java.util.concurrent.LinkedBlockingQueue)
>       at org.apache.zookeeper.ClientCnxn.finishPacket(ClientCnxn.java:627)
>       at org.apache.zookeeper.ClientCnxn.conLossPacket(ClientCnxn.java:645)
>       at org.apache.zookeeper.ClientCnxn.access$2400(ClientCnxn.java:85)
>       at 
> org.apache.zookeeper.ClientCnxn$SendThread.cleanup(ClientCnxn.java:1160)
>       - locked <0x00000000fa8e1380> (a java.util.LinkedList)
>       at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1109)
> "CuratorFramework-0" daemon prio=10 tid=0x00007fd02cb57800 nid=0x4425 in 
> Object.wait() [0x00007fcfc507e000]
>    java.lang.Thread.State: WAITING (on object monitor)
>       at java.lang.Object.wait(Native Method)
>       at java.lang.Object.wait(Object.java:503)
>       at org.apache.zookeeper.ClientCnxn.submitRequest(ClientCnxn.java:1309)
>       - locked <0x00000000fa8e6750> (a org.apache.zookeeper.ClientCnxn$Packet)
>       at org.apache.zookeeper.ClientCnxn.close(ClientCnxn.java:1281)
>       at org.apache.zookeeper.ZooKeeper.close(ZooKeeper.java:677)
>       - locked <0x00000000fa8e0948> (a org.apache.zookeeper.ZooKeeper)
>       at org.apache.curator.HandleHolder.internalClose(HandleHolder.java:139)
>       at org.apache.curator.HandleHolder.closeAndReset(HandleHolder.java:77)
>       at org.apache.curator.ConnectionState.reset(ConnectionState.java:218)
>       - locked <0x00000000d526bcc8> (a org.apache.curator.ConnectionState)
>       at 
> org.apache.curator.ConnectionState.checkTimeouts(ConnectionState.java:194)
>       - locked <0x00000000d526bcc8> (a org.apache.curator.ConnectionState)
>       at 
> org.apache.curator.ConnectionState.getZooKeeper(ConnectionState.java:88)
>       at 
> org.apache.curator.CuratorZookeeperClient.getZooKeeper(CuratorZookeeperClient.java:115)
>       at 
> org.apache.curator.framework.imps.CuratorFrameworkImpl.performBackgroundOperation(CuratorFrameworkImpl.java:763)
>       at 
> org.apache.curator.framework.imps.CuratorFrameworkImpl.backgroundOperationsLoop(CuratorFrameworkImpl.java:749)
>       at 
> org.apache.curator.framework.imps.CuratorFrameworkImpl.access$300(CuratorFrameworkImpl.java:56)
>       at 
> org.apache.curator.framework.imps.CuratorFrameworkImpl$3.call(CuratorFrameworkImpl.java:244)
>       at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:166)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>       at java.lang.Thread.run(Thread.java:722)
> Help me Obi-Wan Kenobi, you're my only hope. 



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

Reply via email to