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

Raul Gutierrez Segales commented on ZOOKEEPER-2080:
---------------------------------------------------

[~hanm]: thanks for tracking this down and for the patch! A few questions/asks, 
looking at the code:

{code}
Election election = null;
synchronized(self) {
    try {
        rqv = self.configFromString(new String(b));
        QuorumVerifier curQV = self.getQuorumVerifier();
        if (rqv.getVersion() > curQV.getVersion()) {
            LOG.info("{} Received version: {} my version: {}", self.getId(),
                    Long.toHexString(rqv.getVersion()),
                    Long.toHexString(self.getQuorumVerifier().getVersion()));
            if (self.getPeerState() == ServerState.LOOKING) {
                LOG.debug("Invoking processReconfig(), state: {}", 
self.getServerState());
                self.processReconfig(rqv, null, null, false);
                if (!rqv.equals(curQV)) {
                    LOG.info("restarting leader election");
                    // Signaling quorum peer to restart leader election.
                    self.shuttingDownLE = true;
                     // Get a hold of current leader election object of quorum 
peer,
                    // so we can clean it up later without holding the lock of 
quorum
                    // peer. If we shutdown current leader election we will run 
into
                    // potential deadlock. See ZOOKEEPER-2080 for more details.
                    election = self.getElectionAlg();
                }
            } else {
                LOG.debug("Skip processReconfig(), state: {}", 
self.getServerState());
            }
        }
    } catch (IOException e) {
        LOG.error("Something went wrong while processing config received from 
{}", response.sid);
   } catch (ConfigException e) {
       LOG.error("Something went wrong while processing config received from 
{}", response.sid);
   }
}
{code}

Do we really need to synchronize around self for the first part:

{code}
rqv = self.configFromString(new String(b));
QuorumVerifier curQV = self.getQuorumVerifier();
if (rqv.getVersion() > curQV.getVersion()) {

{code}

? Sounds like that can be done without synchronizing... no? 

Also, given you've spent a good amount of cycles untangling the dependencies 
around locking QuorumPeer, could you maybe add a comment before the 
synchronize(self) block noting why it is needed and who else might be 
contending for this lock. Thanks so much!

I think unit testing these things is a bit tricky, we might get a better return 
by just keeping better comments around synchronized regions and generally 
keeping them well maintained (imho). So I am happy to +1 without tests. 

> 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