[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-2080?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Michael Han updated ZOOKEEPER-2080:
-----------------------------------
    Attachment: ZOOKEEPER-2080.patch

[~shralex] I have a working solution inspired from your comment that fixes the 
deadlock by moving 
[self.getElectionAlg().shutdown()|https://github.com/apache/zookeeper/blob/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java#L303]
 out of the [synchronized code 
block|https://github.com/apache/zookeeper/blob/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java#L289],
 while still keeping the 
[self.shuttingDownFEL|https://github.com/apache/zookeeper/blob/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java#L302]
 inside the block. 

I think the safety property is still satisfied because:
* 
[self.getElectionAlg().shutdown()|https://github.com/apache/zookeeper/blob/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java#L303]
 does not directly change any state of the 'self' QuorumPeer object. It only 
changes the state of the [election 
object|https://github.com/apache/zookeeper/blob/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java#L688]
 that the 'self' QuorumPeer holds. So we only need to take care of property 
synchronize the election object here.

* On the QuorumPeer side, when it is signaled shuttingDown from FEL, it will 
restart leader election (example: 
[here|https://github.com/apache/zookeeper/blob/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java#L1066]
 and 
[here|https://github.com/apache/zookeeper/blob/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java#L1083].
 
[startLeaderElection|https://github.com/apache/zookeeper/blob/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java#L811]
 however does not directly changing the current leader election algorithm 
object it holds (which is also the object we try to protect against concurrent 
access); it [recreates a new 
object|https://github.com/apache/zookeeper/blob/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java#L834]
 and use it from there. So we don't need to worry about synchronizing on the 
leader election object of the quorum peer, we just need keep a handle of it and 
shut it down properly so the resources it uses (like sockets) can be released 
and the object can be garbage collected.

* In the patch, the assignment of the shuttingDownLE flag and keeping old 
election object for the record is synchronized on the quorum peer.

This patch passed my pressure tests and does not cause any regressions.


> 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, 
> 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