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

Michael Han commented on ZOOKEEPER-2080:
----------------------------------------

Thanks a ton for your feedback Alex, and Flavio. Had more thoughts about this 
today. First to reply your comments posted earlier:

bq. If we pass the QV instance as part of the connectOne call, then can we 
remove the synchronized(self) block?
To pass QV instances we have to first retrieve them from a QuorumPeer object. 
Get QV instances on a QuorumPeer object requires synchronization on the 
QuorumPeer itself, either using existing methods or using what Alex was 
suggesting:
bq. To remove the synchronized block completely you could create a synchronized 
function in QuorumPeer that returns a pair

So in any case we will still possibly end up with a deadlock somewhere inside 
listener threads - we just moved the deadlock from one place to the other :) - 
because somewhere in the listener thread of QuorumCnxManager we are going to 
invoke synchronized version of getQV, which might block because the same 
QuorumPeer object which the getQV methods are invoking were acquired as a lock 
during restart of FLE.

To recap the deadlock problem - a simplified version description:
* Restart leader election acquires a QuorumPeer lock.
* Restart leader election requires shut down QuorumCnxManager as part of the 
process while holding the lock.
* Shutdown QuorumCnxManager requires synchronize on the same QuorumPeer.

The solution for each of these in order:
* Don't acquire a QuorumPeer lock when restart LE. This is less likely a 
solution due to the state changes to QuorumPeer during the process required the 
process has to be synchronized.
* Don't hold the lock while shutdown QuorumCnxManager. This is essentially the 
latest patch did by moving the shutdown out of the sync block. However, it does 
not fix the issue I spotted in previous comments (potential dead lock in 
QuorumPeer.restartLeaderElection - because this method synchronized implicitly 
on the intrinsic lock). 
* Don't synchronize on the QuorumPeer inside QuorumCnxManager's shutdown 
process: this implies we can't involve any of the QuorumPeer's synchronized 
methods in listener threads run loop - otherwise we will end up a deadlock. As 
previously commented, removing {{synchronized(self)}} block by passing QV 
instance to {{connectOne}} would not work because we still need to invoke the 
synchronized version of getQA somewhere in listener threads.

Other thoughts - we could possibly interrupt the connectOne which is 
synchronizing on the QuorumPeer. It should be fine to interrupt because at this 
point everything is shutting down and the server socket was already closed (in 
listener.halt) so no resource leak concerns. So we just need to find a way that 
after we call {{listener.halt()}} in {{QuorumCnxManager.halt()}}, we can signal 
the listener threads to bail out the blocking state - I need to think more 
about how to do this because it seems not possible to interrupt a synchronized 
code block - so we may have to use additional abstractions (i.e. 
ReentrantLock?) which adds more complexity.

 



> ReconfigRecoveryTest fails intermittently
> -----------------------------------------
>
>                 Key: ZOOKEEPER-2080
>                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2080
>             Project: ZooKeeper
>          Issue Type: Sub-task
>            Reporter: Ted Yu
>            Assignee: Michael Han
>             Fix For: 3.5.3, 3.6.0
>
>         Attachments: ZOOKEEPER-2080.patch, ZOOKEEPER-2080.patch, 
> ZOOKEEPER-2080.patch, ZOOKEEPER-2080.patch, ZOOKEEPER-2080.patch, 
> jacoco-ZOOKEEPER-2080.unzip-grows-to-70MB.7z, repro-20150816.log, 
> threaddump.log
>
>
> I got the following test failure on MacBook with trunk code:
> {code}
> Testcase: testCurrentObserverIsParticipantInNewConfig took 93.628 sec
>   FAILED
> waiting for server 2 being up
> junit.framework.AssertionFailedError: waiting for server 2 being up
>   at 
> org.apache.zookeeper.server.quorum.ReconfigRecoveryTest.testCurrentObserverIsParticipantInNewConfig(ReconfigRecoveryTest.java:529)
>   at 
> org.apache.zookeeper.JUnit4ZKTestRunner$LoggedInvokeMethod.evaluate(JUnit4ZKTestRunner.java:52)
> {code}



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

Reply via email to