Repository: zookeeper
Updated Branches:
  refs/heads/master 176bd6822 -> 4d7b9e8f3


ZOOKEEPER-3109: Avoid long unavailable time due to voter changed mind during 
leader election

For more details, please check descriptions in 
https://issues.apache.org/jira/browse/ZOOKEEPER-3109

Author: Fangmin Lyu <allen...@fb.com>

Reviewers: breed, hanm

Closes #588 from lvfangmin/ZOOKEEPER-3109


Project: http://git-wip-us.apache.org/repos/asf/zookeeper/repo
Commit: http://git-wip-us.apache.org/repos/asf/zookeeper/commit/4d7b9e8f
Tree: http://git-wip-us.apache.org/repos/asf/zookeeper/tree/4d7b9e8f
Diff: http://git-wip-us.apache.org/repos/asf/zookeeper/diff/4d7b9e8f

Branch: refs/heads/master
Commit: 4d7b9e8f34775261fed517d232094f0247a4d3cf
Parents: 176bd68
Author: Fangmin Lyu <allen...@fb.com>
Authored: Tue Aug 28 21:26:58 2018 -0700
Committer: Michael Han <h...@apache.org>
Committed: Tue Aug 28 21:26:58 2018 -0700

----------------------------------------------------------------------
 .../server/quorum/FastLeaderElection.java       | 94 +++++++++++++-------
 .../apache/zookeeper/server/quorum/Leader.java  | 59 +++++++++++-
 .../server/quorum/SyncedLearnerTracker.java     | 19 ++--
 .../server/quorum/QuorumPeerMainTest.java       | 90 +++++++++++++++++++
 .../content/xdocs/zookeeperAdmin.xml            | 92 +++++++++++--------
 5 files changed, 280 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/zookeeper/blob/4d7b9e8f/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java
----------------------------------------------------------------------
diff --git 
a/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java 
b/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java
index 1bd0fbf..5e8a6ca 100644
--- a/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java
+++ b/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java
@@ -69,12 +69,12 @@ public class FastLeaderElection implements Election {
      */
 
     final static int maxNotificationInterval = 60000;
-    
+
     /**
      * This value is passed to the methods that check the quorum
      * majority of an established ensemble for those values that
-     * should not be taken into account in the comparison 
-     * (electionEpoch and zxid). 
+     * should not be taken into account in the comparison
+     * (electionEpoch and zxid).
      */
     final static int IGNOREVALUE = -1;
 
@@ -86,6 +86,7 @@ public class FastLeaderElection implements Election {
 
     QuorumCnxManager manager;
 
+    private SyncedLearnerTracker leadingVoteSet;
 
     /**
      * Notifications are messages that let other peers know that
@@ -126,7 +127,7 @@ public class FastLeaderElection implements Election {
          * Address of sender
          */
         long sid;
-        
+
         QuorumVerifier qv;
         /*
          * epoch of the proposed leader
@@ -247,7 +248,7 @@ public class FastLeaderElection implements Election {
 
                         // this is the backwardCompatibility mode for no 
version information
                         boolean backCompatibility40 = 
(response.buffer.capacity() == 40);
-                        
+
                         response.buffer.clear();
 
                         // Instantiate Notification and set its attributes
@@ -266,7 +267,7 @@ public class FastLeaderElection implements Election {
                                 /*
                                  * Version added in 3.4.6
                                  */
-                                
+
                                 version = response.buffer.getInt();
                             } else {
                                 LOG.info("Backward compatibility mode (36 
bits), server id: {}", response.sid);
@@ -284,7 +285,7 @@ public class FastLeaderElection implements Election {
                             byte b[] = new byte[configLength];
 
                             response.buffer.get(b);
-                                                       
+
                             synchronized(self) {
                                 try {
                                     rqv = self.configFromString(new String(b));
@@ -312,11 +313,11 @@ public class FastLeaderElection implements Election {
                                } catch (ConfigException e) {
                                    LOG.error("Something went wrong while 
processing config received from {}", response.sid);
                                }
-                            }                          
+                            }
                         } else {
                             LOG.info("Backward compatibility mode (before 
reconfig), server id: {}", response.sid);
                         }
-                       
+
                         /*
                          * If it is from a non-voting server (such as an 
observer or
                          * a non-voting follower), respond right away.
@@ -408,6 +409,13 @@ public class FastLeaderElection implements Election {
                                  */
                                 Vote current = self.getCurrentVote();
                                 if(ackstate == QuorumPeer.ServerState.LOOKING){
+                                    if (self.leader != null) {
+                                        if (leadingVoteSet != null) {
+                                            
self.leader.setLeadingVoteSet(leadingVoteSet);
+                                            leadingVoteSet = null;
+                                        }
+                                        
self.leader.reportLookingSid(response.sid);
+                                    }
                                     if(LOG.isDebugEnabled()){
                                         LOG.debug("Sending new notification. 
My id ={} recipient={} zxid=0x{} leader={} config version = {}",
                                                 self.getId(),
@@ -653,6 +661,7 @@ public class FastLeaderElection implements Election {
         stop = true;
         proposedLeader = -1;
         proposedZxid = -1;
+        leadingVoteSet = null;
         LOG.debug("Shutting down connection manager");
         manager.halt();
         LOG.debug("Shutting down messenger");
@@ -724,15 +733,16 @@ public class FastLeaderElection implements Election {
     }
 
     /**
-     * Termination predicate. Given a set of votes, determines if have
-     * sufficient to declare the end of the election round.
-     * 
+     * Given a set of votes, return the SyncedLearnerTracker which is used to
+     * determines if have sufficient to declare the end of the election round.
+     *
      * @param votes
      *            Set of votes
      * @param vote
      *            Identifier of the vote received last
+     * @return the SyncedLearnerTracker with vote details
      */
-    private boolean termPredicate(Map<Long, Vote> votes, Vote vote) {
+    private SyncedLearnerTracker getVoteTracker(Map<Long, Vote> votes, Vote 
vote) {
         SyncedLearnerTracker voteSet = new SyncedLearnerTracker();
         voteSet.addQuorumVerifier(self.getQuorumVerifier());
         if (self.getLastSeenQuorumVerifier() != null
@@ -751,7 +761,7 @@ public class FastLeaderElection implements Election {
             }
         }
 
-        return voteSet.hasAllQuorums();
+        return voteSet;
     }
 
     /**
@@ -828,7 +838,7 @@ public class FastLeaderElection implements Election {
      * @return long
      */
     private long getInitId(){
-        
if(self.getQuorumVerifier().getVotingMembers().containsKey(self.getId()))       
+        
if(self.getQuorumVerifier().getVotingMembers().containsKey(self.getId()))
             return self.getId();
         else return Long.MIN_VALUE;
     }
@@ -862,6 +872,19 @@ public class FastLeaderElection implements Election {
     }
 
     /**
+     * Update the peer state based on the given proposedLeader. Also update
+     * the leadingVoteSet if it becomes the leader.
+     */
+    private void setPeerState(long proposedLeader, SyncedLearnerTracker 
voteSet) {
+        ServerState ss = (proposedLeader == self.getId()) ?
+                ServerState.LEADING: learningState();
+        self.setPeerState(ss);
+        if (ss == ServerState.LEADING) {
+            leadingVoteSet = voteSet;
+        }
+    }
+
+    /**
      * Starts a new round of leader election. Whenever our QuorumPeer
      * changes its state to LOOKING, this method is invoked, and it
      * sends notifications to all other peers.
@@ -894,6 +917,8 @@ public class FastLeaderElection implements Election {
                     ", proposed zxid=0x" + Long.toHexString(proposedZxid));
             sendNotifications();
 
+            SyncedLearnerTracker voteSet;
+
             /*
              * Loop in which we exchange notifications until we find a leader
              */
@@ -925,7 +950,7 @@ public class FastLeaderElection implements Election {
                     notTimeout = (tmpTimeOut < maxNotificationInterval?
                             tmpTimeOut : maxNotificationInterval);
                     LOG.info("Notification time out: " + notTimeout);
-                } 
+                }
                 else if (validVoter(n.sid) && validVoter(n.leader)) {
                     /*
                      * Only proceed if the vote comes from a replica in the 
current or next
@@ -976,9 +1001,11 @@ public class FastLeaderElection implements Election {
 
                         recvset.put(n.sid, new Vote(n.leader, n.zxid, 
n.electionEpoch, n.peerEpoch));
 
-                        if (termPredicate(recvset,
-                                new Vote(proposedLeader, proposedZxid,
-                                        logicalclock.get(), proposedEpoch))) {
+                        voteSet = getVoteTracker(
+                                recvset, new Vote(proposedLeader, proposedZxid,
+                                        logicalclock.get(), proposedEpoch));
+
+                        if (voteSet.hasAllQuorums()) {
 
                             // Verify if there is any change in the proposed 
leader
                             while((n = recvqueue.poll(finalizeWait,
@@ -995,8 +1022,7 @@ public class FastLeaderElection implements Election {
                              * relevant message from the reception queue
                              */
                             if (n == null) {
-                                self.setPeerState((proposedLeader == 
self.getId()) ?
-                                        ServerState.LEADING: learningState());
+                                setPeerState(proposedLeader, voteSet);
 
                                 Vote endVote = new Vote(proposedLeader,
                                         proposedZxid, proposedEpoch);
@@ -1016,11 +1042,12 @@ public class FastLeaderElection implements Election {
                          */
                         if(n.electionEpoch == logicalclock.get()){
                             recvset.put(n.sid, new Vote(n.leader, n.zxid, 
n.electionEpoch, n.peerEpoch));
-                            if(termPredicate(recvset, new Vote(n.leader,
-                                            n.zxid, n.electionEpoch, 
n.peerEpoch, n.state))
+                            voteSet = getVoteTracker(
+                                    recvset, new Vote(n.leader, n.zxid,
+                                    n.electionEpoch, n.peerEpoch, n.state));
+                            if(voteSet.hasAllQuorums()
                                             && checkLeader(outofelection, 
n.leader, n.electionEpoch)) {
-                                self.setPeerState((n.leader == self.getId()) ?
-                                        ServerState.LEADING: learningState());
+                                setPeerState(n.leader, voteSet);
 
                                 Vote endVote = new Vote(n.leader, n.zxid, 
n.peerEpoch);
                                 leaveInstance(endVote);
@@ -1036,21 +1063,22 @@ public class FastLeaderElection implements Election {
                          * least one corner case in which the ensemble can be
                          * created with inconsistent zxid and election epoch
                          * info. However, given that only one ensemble can be
-                         * running at a single point in time and that each 
-                         * epoch is used only once, using only the epoch to 
+                         * running at a single point in time and that each
+                         * epoch is used only once, using only the epoch to
                          * compare the votes is sufficient.
-                         * 
+                         *
                          * @see 
https://issues.apache.org/jira/browse/ZOOKEEPER-1732
                          */
-                        outofelection.put(n.sid, new Vote(n.leader, 
+                        outofelection.put(n.sid, new Vote(n.leader,
+                                IGNOREVALUE, IGNOREVALUE, n.peerEpoch, 
n.state));
+                        voteSet = getVoteTracker(
+                                outofelection, new Vote(n.leader,
                                 IGNOREVALUE, IGNOREVALUE, n.peerEpoch, 
n.state));
-                        if (termPredicate(outofelection, new Vote(n.leader,
-                                IGNOREVALUE, IGNOREVALUE, n.peerEpoch, 
n.state))
+                        if (voteSet.hasAllQuorums()
                                 && checkLeader(outofelection, n.leader, 
IGNOREVALUE)) {
                             synchronized(this){
                                 logicalclock.set(n.electionEpoch);
-                                self.setPeerState((n.leader == self.getId()) ?
-                                        ServerState.LEADING: learningState());
+                                setPeerState(n.leader, voteSet);
                             }
                             Vote endVote = new Vote(n.leader, n.zxid, 
n.peerEpoch);
                             leaveInstance(endVote);

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/4d7b9e8f/src/java/main/org/apache/zookeeper/server/quorum/Leader.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/quorum/Leader.java 
b/src/java/main/org/apache/zookeeper/server/quorum/Leader.java
index 190d98d..6ff5e80 100644
--- a/src/java/main/org/apache/zookeeper/server/quorum/Leader.java
+++ b/src/java/main/org/apache/zookeeper/server/quorum/Leader.java
@@ -1172,6 +1172,60 @@ public class Leader {
     }
     // VisibleForTesting
     protected final Set<Long> connectingFollowers = new HashSet<Long>();
+
+    private volatile boolean quitWaitForEpoch = false;
+    private volatile long timeStartWaitForEpoch = -1;
+    private volatile SyncedLearnerTracker voteSet;
+
+    public static final String MAX_TIME_TO_WAIT_FOR_EPOCH = 
"zookeeper.leader.maxTimeToWaitForEpoch";
+    private static int maxTimeToWaitForEpoch;
+    static {
+        maxTimeToWaitForEpoch = Integer.getInteger(MAX_TIME_TO_WAIT_FOR_EPOCH, 
-1);
+        LOG.info("{} = {}ms", MAX_TIME_TO_WAIT_FOR_EPOCH, 
maxTimeToWaitForEpoch);
+    }
+
+    // visible for test
+    public static void setMaxTimeToWaitForEpoch(int maxTimeToWaitForEpoch) {
+        Leader.maxTimeToWaitForEpoch = maxTimeToWaitForEpoch;
+        LOG.info("Set {} to {}ms", MAX_TIME_TO_WAIT_FOR_EPOCH, 
Leader.maxTimeToWaitForEpoch);
+    }
+
+    /**
+     * Quit condition:
+     *
+     * 1 voter goes to looking again and time waitForEpoch > 
maxTimeToWaitForEpoch
+     *
+     * Note: the voter may go to looking again in case of:
+     * 1. change mind in the last minute when received a different notification
+     * 2. the leader hadn't started leading when it tried to connect to it
+     * 3. connection broken between the voter and leader
+     * 4. voter being shutdown or restarted
+     */
+    private void quitLeading() {
+        synchronized(connectingFollowers) {
+            quitWaitForEpoch = true;
+            connectingFollowers.notifyAll();
+        }
+        LOG.info("Quit leading due to voter changed mind.");
+    }
+
+    public void setLeadingVoteSet(SyncedLearnerTracker voteSet) {
+        this.voteSet = voteSet;
+    }
+
+    public void reportLookingSid(long sid) {
+        if (maxTimeToWaitForEpoch < 0 || timeStartWaitForEpoch < 0
+                || !waitingForNewEpoch) {
+            return;
+        }
+        if (voteSet == null || !voteSet.hasSid(sid)) {
+            return;
+        }
+        if (Time.currentElapsedTime() - timeStartWaitForEpoch > 
maxTimeToWaitForEpoch) {
+            quitLeading();
+        }
+    }
+
     public long getEpochToPropose(long sid, long lastAcceptedEpoch) throws 
InterruptedException, IOException {
         synchronized(connectingFollowers) {
             if (!waitingForNewEpoch) {
@@ -1191,9 +1245,12 @@ public class Leader {
                 connectingFollowers.notifyAll();
             } else {
                 long start = Time.currentElapsedTime();
+                if (sid == self.getId()) {
+                    timeStartWaitForEpoch = start;
+                }
                 long cur = start;
                 long end = start + self.getInitLimit()*self.getTickTime();
-                while(waitingForNewEpoch && cur < end) {
+                while(waitingForNewEpoch && cur < end  && !quitWaitForEpoch) {
                     connectingFollowers.wait(end - cur);
                     cur = Time.currentElapsedTime();
                 }

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/4d7b9e8f/src/java/main/org/apache/zookeeper/server/quorum/SyncedLearnerTracker.java
----------------------------------------------------------------------
diff --git 
a/src/java/main/org/apache/zookeeper/server/quorum/SyncedLearnerTracker.java 
b/src/java/main/org/apache/zookeeper/server/quorum/SyncedLearnerTracker.java
index 3482fcb..d773119 100644
--- a/src/java/main/org/apache/zookeeper/server/quorum/SyncedLearnerTracker.java
+++ b/src/java/main/org/apache/zookeeper/server/quorum/SyncedLearnerTracker.java
@@ -25,7 +25,7 @@ import 
org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
 
 public class SyncedLearnerTracker {
 
-    protected ArrayList<QuorumVerifierAcksetPair> qvAcksetPairs = 
+    protected ArrayList<QuorumVerifierAcksetPair> qvAcksetPairs =
                 new ArrayList<QuorumVerifierAcksetPair>();
 
     public void addQuorumVerifier(QuorumVerifier qv) {
@@ -44,6 +44,15 @@ public class SyncedLearnerTracker {
         return change;
     }
 
+    public boolean hasSid(long sid) {
+        for (QuorumVerifierAcksetPair qvAckset : qvAcksetPairs) {
+            if 
(!qvAckset.getQuorumVerifier().getVotingMembers().containsKey(sid)) {
+                return false;
+            }
+        }
+        return true;
+    }
+
     public boolean hasAllQuorums() {
         for (QuorumVerifierAcksetPair qvAckset : qvAcksetPairs) {
             if 
(!qvAckset.getQuorumVerifier().containsQuorum(qvAckset.getAckset()))
@@ -51,14 +60,14 @@ public class SyncedLearnerTracker {
         }
         return true;
     }
-        
+
     public String ackSetsToString(){
         StringBuilder sb = new StringBuilder();
-            
+
         for (QuorumVerifierAcksetPair qvAckset : qvAcksetPairs) {
             sb.append(qvAckset.getAckset().toString()).append(",");
         }
-            
+
         return sb.substring(0, sb.length()-1);
     }
 
@@ -66,7 +75,7 @@ public class SyncedLearnerTracker {
         private final QuorumVerifier qv;
         private final HashSet<Long> ackset;
 
-        public QuorumVerifierAcksetPair(QuorumVerifier qv, HashSet<Long> 
ackset) {                
+        public QuorumVerifierAcksetPair(QuorumVerifier qv, HashSet<Long> 
ackset) {
             this.qv = qv;
             this.ackset = ackset;
         }

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/4d7b9e8f/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
----------------------------------------------------------------------
diff --git 
a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java 
b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
index bbdd56c..5928ea9 100644
--- a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
+++ b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
@@ -1407,6 +1407,85 @@ public class QuorumPeerMainTest extends 
QuorumPeerTestBase {
         }
     }
 
+    /**
+     * Test leader election finished  with 1 disloyal voter and without
+     * majority followers, expecting to see the quorum stablized only
+     * after waiting for maxTimeToWaitForEpoch.
+     */
+    @Test
+    public void testLeaderElectionWithDisloyalVoter() throws IOException {
+        testLeaderElection(5, 3, 1000, 10000);
+    }
+
+    /**
+     * Test leader election finished  with 1 disloyal voter and majority
+     * followers, expecting to see the quorum stablized immediately even
+     * there is 1 disloyal voter.
+     *
+     * Set the maxTimeToWaitForEpoch to 3s and maxTimeWaitForServerUp to
+     * 2s to confirm this.
+     */
+    @Test
+    public void testLeaderElectionWithDisloyalVoter_stillHasMajority()
+            throws IOException {
+        testLeaderElection(5, 5, 3000, 2000);
+    }
+
+    void testLeaderElection(int totalServers, int serversToStart,
+            int maxTimeToWaitForEpoch, int maxTimeWaitForServerUp)
+            throws IOException {
+        Leader.setMaxTimeToWaitForEpoch(maxTimeToWaitForEpoch);
+
+        // set up config for an ensemble with given number of servers
+        servers = new Servers();
+        int ENSEMBLE_SERVERS = totalServers;
+        final int clientPorts[] = new int[ENSEMBLE_SERVERS];
+        StringBuilder sb = new StringBuilder();
+        String server;
+
+        for (int i = 0; i < ENSEMBLE_SERVERS; i++) {
+            clientPorts[i] = PortAssignment.unique();
+            server = "server." + i + "=127.0.0.1:" + PortAssignment.unique()
+                    + ":" + PortAssignment.unique() + ":participant;127.0.0.1:"
+                    + clientPorts[i];
+            sb.append(server + "\n");
+        }
+        String currentQuorumCfgSection = sb.toString();
+
+        // start servers
+        int SERVERS_TO_START = serversToStart;
+        MainThread[] mt = new MainThread[SERVERS_TO_START];
+        Context[] contexts = new Context[SERVERS_TO_START];
+        servers.mt = mt;
+        numServers = SERVERS_TO_START;
+        for (int i = 0; i < SERVERS_TO_START; i++) {
+            // hook the 1st follower to quit following after leader election
+            // simulate the behavior of changing voting during looking
+            final Context context = new Context();
+            if (i == 0) {
+                context.quitFollowing = true;
+            }
+            contexts[i] = context;
+            mt[i] = new MainThread(i, clientPorts[i], currentQuorumCfgSection,
+                    false) {
+                @Override
+                public TestQPMain getTestQPMain() {
+                    return new CustomizedQPMain(context);
+                }
+            };
+            mt[i].start();
+        }
+
+        // make sure the quorum can be formed within initLimit * tickTime
+        // the default setting is 10 * 4000 = 40000 ms
+        for (int i = 0; i < SERVERS_TO_START; i++) {
+            Assert.assertTrue(
+                "Server " + i + " should have joined quorum by now",
+                ClientBase.waitForServerUp(
+                        "127.0.0.1:" + clientPorts[i], 
maxTimeWaitForServerUp));
+        }
+    }
+
     static class Context {
         boolean quitFollowing = false;
         boolean exitWhenAckNewLeader = false;
@@ -1465,6 +1544,17 @@ public class QuorumPeerMainTest extends 
QuorumPeerTestBase {
                 throws IOException {
             return new Follower(this, new FollowerZooKeeperServer(logFactory,
                     this, this.getZkDb())) {
+                @Override
+                void followLeader() throws InterruptedException {
+                    if (context.quitFollowing) {
+                        // reset the flag
+                        context.quitFollowing = false;
+                        LOG.info("Quit following");
+                        return;
+                    } else {
+                        super.followLeader();
+                    }
+                }
 
                 @Override
                 void writePacket(QuorumPacket pp, boolean flush) throws 
IOException {

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/4d7b9e8f/zookeeper-docs/src/documentation/content/xdocs/zookeeperAdmin.xml
----------------------------------------------------------------------
diff --git a/zookeeper-docs/src/documentation/content/xdocs/zookeeperAdmin.xml 
b/zookeeper-docs/src/documentation/content/xdocs/zookeeperAdmin.xml
index afd068b..ade4e74 100644
--- a/zookeeper-docs/src/documentation/content/xdocs/zookeeperAdmin.xml
+++ b/zookeeper-docs/src/documentation/content/xdocs/zookeeperAdmin.xml
@@ -296,11 +296,11 @@ server.3=zoo3:2888:3888</programlisting>
           <para><computeroutput>$ java -cp 
zookeeper.jar:lib/slf4j-api-1.7.5.jar:lib/slf4j-log4j12-1.7.5.jar:lib/log4j-1.2.17.jar:conf
 \
               org.apache.zookeeper.server.quorum.QuorumPeerMain zoo.cfg
           </computeroutput></para>
-          
+
           <para>QuorumPeerMain starts a ZooKeeper server,
             <ulink 
url="http://java.sun.com/javase/technologies/core/mntr-mgmt/javamanagement/";>JMX</ulink>
             management beans are also registered which allows
-            management through a JMX management console. 
+            management through a JMX management console.
             The <ulink url="zookeeperJMX.html">ZooKeeper JMX
             document</ulink> contains details on managing ZooKeeper with JMX.
           </para>
@@ -428,7 +428,7 @@ server.3=zoo3:2888:3888</programlisting>
             components that perform consistently.</para>
         </listitem>
       </orderedlist>
-    
+
     <para>The sections below contain considerations for ZooKeeper
       administrators to maximize the probability for these assumptions
       to hold true. Some of these are cross-machines considerations,
@@ -437,7 +437,7 @@ server.3=zoo3:2888:3888</programlisting>
 
     <section id="sc_CrossMachineRequirements">
       <title>Cross Machine Requirements</title>
-    
+
       <para>For the ZooKeeper service to be active, there must be a
         majority of non-failing machines that can communicate with
         each other. To create a deployment that can tolerate the
@@ -653,9 +653,9 @@ server.3=zoo3:2888:3888</programlisting>
       <ulink url="http://www.slf4j.org/manual.html";>its manual</ulink>.</para>
 
     <para>For more information about LOG4J, see
-      <ulink 
url="http://logging.apache.org/log4j/1.2/manual.html#defaultInit";>Log4j Default 
Initialization Procedure</ulink> 
+      <ulink 
url="http://logging.apache.org/log4j/1.2/manual.html#defaultInit";>Log4j Default 
Initialization Procedure</ulink>
       of the log4j manual.</para>
-      
+
     </section>
 
     <section id="sc_troubleshooting">
@@ -664,10 +664,10 @@ server.3=zoo3:2888:3888</programlisting>
                <varlistentry>
                <term> Server not coming up because of file corruption</term>
                <listitem>
-               <para>A server might not be able to read its database and fail 
to come up because of 
+               <para>A server might not be able to read its database and fail 
to come up because of
                some file corruption in the transaction logs of the ZooKeeper 
server. You will
                see some IOException on loading ZooKeeper database. In such a 
case,
-               make sure all the other servers in your ensemble are up and  
working. Use "stat" 
+               make sure all the other servers in your ensemble are up and  
working. Use "stat"
                command on the command port to see if they are in good health. 
After you have verified that
                all the other servers of the ensemble are up, you can go ahead 
and clean the database
                of the corrupt server. Delete all the files in 
datadir/version-2 and datalogdir/version-2/.
@@ -875,7 +875,7 @@ server.3=zoo3:2888:3888</programlisting>
               by snapCount. In order to prevent all of the machines in the 
quorum
               from taking a snapshot at the same time, each ZooKeeper server
               will take a snapshot when the number of transactions in the 
transaction log
-              reaches a runtime generated random value in the [snapCount/2+1, 
snapCount] 
+              reaches a runtime generated random value in the [snapCount/2+1, 
snapCount]
               range.The default snapCount is 100,000.</para>
             </listitem>
           </varlistentry>
@@ -885,10 +885,10 @@ server.3=zoo3:2888:3888</programlisting>
             <listitem>
               <para>(No Java system property)</para>
 
-              <para>Limits the number of concurrent connections (at the socket 
+              <para>Limits the number of concurrent connections (at the socket
               level) that a single client, identified by IP address, may make
-              to a single member of the ZooKeeper ensemble. This is used to 
-              prevent certain classes of DoS attacks, including file 
+              to a single member of the ZooKeeper ensemble. This is used to
+              prevent certain classes of DoS attacks, including file
               descriptor exhaustion. The default is 60. Setting this to 0
               entirely removes the limit on concurrent connections.</para>
             </listitem>
@@ -932,7 +932,7 @@ server.3=zoo3:2888:3888</programlisting>
               the <emphasis role="bold">tickTime</emphasis>.</para>
             </listitem>
            </varlistentry>
-           
+
            <varlistentry>
              <term>fsync.warningthresholdms</term>
              <listitem>
@@ -954,16 +954,16 @@ server.3=zoo3:2888:3888</programlisting>
             <listitem>
               <para>(No Java system property)</para>
 
-              <para><emphasis role="bold">New in 3.4.0:</emphasis> 
+              <para><emphasis role="bold">New in 3.4.0:</emphasis>
               When enabled, ZooKeeper auto purge feature retains
               the <emphasis role="bold">autopurge.snapRetainCount</emphasis> 
most
-              recent snapshots and the corresponding transaction logs in the 
-              <emphasis role="bold">dataDir</emphasis> and <emphasis 
+              recent snapshots and the corresponding transaction logs in the
+              <emphasis role="bold">dataDir</emphasis> and <emphasis
               role="bold">dataLogDir</emphasis> respectively and deletes the 
rest.
               Defaults to 3. Minimum value is 3.</para>
             </listitem>
           </varlistentry>
-          
+
           <varlistentry>
             <term>autopurge.purgeInterval</term>
 
@@ -1046,18 +1046,40 @@ server.3=zoo3:2888:3888</programlisting>
               corresponds to the authenticated UDP-based version of fast
               leader election, and "3" corresponds to TCP-based version of
               fast leader election. Currently, algorithm 3 is the 
default.</para>
-              
+
               <note>
               <para> The implementations of leader election 1, and 2 are now
               <emphasis role="bold"> deprecated </emphasis>. We have the 
intention
-              of removing them in the next release, at which point only the 
-              FastLeaderElection will be available. 
+              of removing them in the next release, at which point only the
+              FastLeaderElection will be available.
               </para>
               </note>
             </listitem>
           </varlistentry>
 
           <varlistentry>
+            <term>maxTimeToWaitForEpoch</term>
+
+            <listitem>
+              <para>(Java system property: zookeeper.leader.<emphasis
+                      role="bold">maxTimeToWaitForEpoch</emphasis>)</para>
+
+              <para><emphasis role="bold">New in 3.6.0:</emphasis>
+                The maximum time to wait for epoch from voters when activating
+                leader. If leader received a LOOKING notification from one of
+                it's voters, and it hasn't received epoch packets from majority
+                within maxTimeToWaitForEpoch, then it will goto LOOKING and
+                elect leader again.
+
+                This can be tuned to reduce the quorum or server unavailable
+                time, it can be set to be much smaller than initLimit * 
tickTime.
+                In cross datacenter environment, it can be set to something
+                like 2s.
+              </para>
+            </listitem>
+          </varlistentry>
+
+          <varlistentry>
             <term>initLimit</term>
 
             <listitem>
@@ -1109,8 +1131,8 @@ server.3=zoo3:2888:3888</programlisting>
               used by the clients must match the list of ZooKeeper servers
               that each ZooKeeper server has.</para>
 
-              <para>There are two port numbers <emphasis 
role="bold">nnnnn</emphasis>. 
-              The first followers use to connect to the leader, and the second 
is for 
+              <para>There are two port numbers <emphasis 
role="bold">nnnnn</emphasis>.
+              The first followers use to connect to the leader, and the second 
is for
               leader election. If you want to test multiple servers on a 
single machine, then
               different ports can be used for each server.</para>
             </listitem>
@@ -1136,11 +1158,11 @@ server.3=zoo3:2888:3888</programlisting>
               <para>(No Java system property)</para>
 
               <para>Enables a hierarchical quorum construction."x" is a group 
identifier
-              and the numbers following the "=" sign correspond to server 
identifiers. 
+              and the numbers following the "=" sign correspond to server 
identifiers.
               The left-hand side of the assignment is a colon-separated list 
of server
               identifiers. Note that groups must be disjoint and the union of 
all groups
               must be the ZooKeeper ensemble. </para>
-              
+
               <para> You will find an example <ulink 
url="zookeeperHierarchicalQuorums.html">here</ulink>
               </para>
             </listitem>
@@ -1157,14 +1179,14 @@ server.3=zoo3:2888:3888</programlisting>
               when voting. There are a few parts of ZooKeeper that require 
voting
               such as leader election and the atomic broadcast protocol. By 
default
               the weight of server is 1. If the configuration defines groups, 
but not
-              weights, then a value of 1 will be assigned to all servers.  
+              weights, then a value of 1 will be assigned to all servers.
               </para>
-              
+
               <para> You will find an example <ulink 
url="zookeeperHierarchicalQuorums.html">here</ulink>
               </para>
             </listitem>
           </varlistentry>
-          
+
           <varlistentry>
             <term>cnxTimeout</term>
 
@@ -1172,8 +1194,8 @@ server.3=zoo3:2888:3888</programlisting>
               <para>(Java system property: zookeeper.<emphasis
               role="bold">cnxTimeout</emphasis>)</para>
 
-              <para>Sets the timeout value for opening connections for leader 
election notifications. 
-              Only applicable if you are using electionAlg 3. 
+              <para>Sets the timeout value for opening connections for leader 
election notifications.
+              Only applicable if you are using electionAlg 3.
               </para>
 
               <note>
@@ -1356,7 +1378,7 @@ server.3=zoo3:2888:3888</programlisting>
               role="bold">zookeeper.superUser</emphasis>)</para>
 
               <para>Similar to <emphasis 
role="bold">zookeeper.X509AuthenticationProvider.superUser</emphasis>
-              but is generic for SASL based logins. It stores the name of 
+              but is generic for SASL based logins. It stores the name of
               a user that can access the znode hierarchy as a "super" user.
               </para>
             </listitem>
@@ -1498,10 +1520,10 @@ server.3=zoo3:2888:3888</programlisting>
             <term>quorumListenOnAllIPs</term>
 
             <listitem>
-              <para>When set to true the ZooKeeper server will listen  
+              <para>When set to true the ZooKeeper server will listen
               for connections from its peers on all available IP addresses,
               and not only the address configured in the server list of the
-              configuration file. It affects the connections handling the 
+              configuration file. It affects the connections handling the
               ZAB protocol and the Fast Leader Election protocol. Default
               value is <emphasis role="bold">false</emphasis>.</para>
             </listitem>
@@ -1764,7 +1786,7 @@ server.3=zoo3:2888:3888</programlisting>
                     <para>(Java system property: <emphasis
                             
role="bold">zookeeper.admin.idleTimeout</emphasis>)</para>
 
-                    <para>Set the maximum idle time in milliseconds that a 
connection can wait 
+                    <para>Set the maximum idle time in milliseconds that a 
connection can wait
                           before sending or receiving data. Defaults to 30000 
ms.</para>
                 </listitem>
             </varlistentry>
@@ -1950,7 +1972,7 @@ server.3=zoo3:2888:3888</programlisting>
             <term>mntr</term>
 
             <listitem>
-              <para><emphasis role="bold">New in 3.4.0:</emphasis> Outputs a 
list 
+              <para><emphasis role="bold">New in 3.4.0:</emphasis> Outputs a 
list
               of variables that could be used for monitoring the health of the 
cluster.</para>
 
               <programlisting>$ echo mntr | nc localhost 2185
@@ -1978,7 +2000,7 @@ server.3=zoo3:2888:3888</programlisting>
               zk_max_proposal_size 64
               </programlisting>
 
-              <para>The output is compatible with java properties format and 
the content 
+              <para>The output is compatible with java properties format and 
the content
               may change over time (new keys added). Your scripts should 
expect changes.</para>
 
               <para>ATTENTION: Some of the keys are platform specific and some 
of the keys are only exported by the Leader. </para>

Reply via email to