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

Flavio Junqueira commented on ZOOKEEPER-975:
--------------------------------------------

Hi Vishal, What if we have a test with 3 servers with ids 1, 2, and 3. We start 
1 and 2, and let 2 be elected. Once 2 is elected, we start 3. With the current 
trunk code, 3 should declare itself LEADING, since it will receive the initial 
notifications of the other two processes. With your patch, this situation 
shouldn't happen.

What do you think?

> new peer goes in LEADING state even if ensemble is online
> ---------------------------------------------------------
>
>                 Key: ZOOKEEPER-975
>                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-975
>             Project: ZooKeeper
>          Issue Type: Bug
>    Affects Versions: 3.3.2
>            Reporter: Vishal K
>            Assignee: Vishal K
>             Fix For: 3.4.0
>
>         Attachments: ZOOKEEPER-975.patch, ZOOKEEPER-975.patch, 
> ZOOKEEPER-975.patch2, ZOOKEEPER-975.patch3
>
>
> Scenario:
> 1. 2 of the 3 ZK nodes are online
> 2. Third node is attempting to join
> 3. Third node unnecessarily goes in "LEADING" state
> 4. Then third goes back to LOOKING (no majority of followers) and finally 
> goes to FOLLOWING state.
> While going through the logs I noticed that a peer C that is trying to
> join an already formed cluster goes in LEADING state. This is because
> QuorumCnxManager of A and B sends the entire history of notification
> messages to C. C receives the notification messages that were
> exchanged between A and B when they were forming the cluster.
> In FastLeaderElection.lookForLeader(), due to the following piece of
> code, C quits lookForLeader assuming that it is supposed to lead.
> 740                             //If have received from all nodes, then 
> terminate
> 741                             if ((self.getVotingView().size() == 
> recvset.size()) &&
> 742                                     
> (self.getQuorumVerifier().getWeight(proposedLeader) != 0)){
> 743                                 self.setPeerState((proposedLeader == 
> self.getId()) ?
> 744                                         ServerState.LEADING: 
> learningState());
> 745                                 leaveInstance();
> 746                                 return new Vote(proposedLeader, 
> proposedZxid);
> 747
> 748                             } else if (termPredicate(recvset,
> This can cause:
> 1.  C to unnecessarily go in LEADING state and wait for tickTime * initLimit 
> and then restart the FLE.
> 2. C waits for 200 ms (finalizeWait) and then considers whatever
> notifications it has received to make a decision. C could potentially
> decide to follow an old leader, fail to connect to the leader, and
> then restart FLE. See code below.
> 752                             if (termPredicate(recvset,
> 753                                     new Vote(proposedLeader, proposedZxid,
> 754                                             logicalclock))) {
> 755 
> 756                                 // Verify if there is any change in the 
> proposed leader
> 757                                 while((n = recvqueue.poll(finalizeWait,
> 758                                         TimeUnit.MILLISECONDS)) != null){
> 759                                     if(totalOrderPredicate(n.leader, 
> n.zxid,
> 760                                             proposedLeader, 
> proposedZxid)){
> 761                                         recvqueue.put(n);
> 762                                         break;
> 763                                     }
> 764                                 }
> In general, this does not affect correctness of FLE since C will
> eventually go back to FOLLOWING state (A and B won't vote for
> C). However, this delays C from joining the cluster. This can in turn
> affect recovery time of an application.
> Proposal: A and B should send only the latest notification (most
> recent) instead of the entire history. Does this sound reasonable?

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to