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

Flavio Junqueira commented on ZOOKEEPER-2080:
---------------------------------------------

On the race involving {{shuttingDownLE}}, say that FLE sets it to true right 
after the {{QuorumPeer}} thread has checked that it is false here:

{noformat}
if (shuttingDownLE) {
     shuttingDownLE = false;
     startLeaderElection();
}
setCurrentVote(makeLEStrategy().lookForLeader());

{noformat}

The quorum peer thread will skip starting leader election, and when it calls 
{{lookForLeader}}, we might end up getting an empty vote because the FLE 
instance is stopped. This doesn't look like it is fatal, though. The server 
will remain in the looking state, which will force it to try again. Do you 
agree?

bq. Do we need to synchronize on this code in QuorumPeer? 

I think this is the case I'm mentioning above, but let me know.

bq. I think we have another potential dead lock in 
QuorumPeer.restartLeaderElection

Sounds right, good catch. I'm wondering why {{QuorumCnxManager.connectOne}} is 
synchronized on self, all it is trying to do is to connect to another server.

bq. While looking through code, I find it is hard to reason about current 
behavior due to the mixed state changes that could be done by many parties.

A refactor of this code makes perfect sense to me. {{QuorumPeer}} and 
{{QuorumCnxManager}}} could highly benefit from a redesign. FLE could also 
benefit from a refactor, but I find it better encapsulated because in principle 
all we need is to call {{lookForLeader}} and let it send and receive messages.

One thing that would be awesome to do with respect to leader election is to 
remove the need of an additional port. It would be great if we had a single 
port for all server communication.

Sorry about the daydreaming, you got me going...

> 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