You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by lvfangmin <gi...@git.apache.org> on 2018/08/01 22:51:57 UTC

[GitHub] zookeeper pull request #588: [ZOOKEEPER-3109] Avoid long unavailable time du...

GitHub user lvfangmin opened a pull request:

    https://github.com/apache/zookeeper/pull/588

    [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 

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/lvfangmin/zookeeper ZOOKEEPER-3109

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/zookeeper/pull/588.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #588
    
----
commit 9611393b3d4d9e1a0327a5b8bf678e526c7fc5a7
Author: Fangmin Lyu <al...@...>
Date:   2018-08-01T22:49:57Z

    Avoid long unavailable time due to voter changed mind when activating the leader during election

----


---

[GitHub] zookeeper pull request #588: [ZOOKEEPER-3109] Avoid long unavailable time du...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/588#discussion_r207604144
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/Leader.java ---
    @@ -1165,6 +1165,58 @@ synchronized public long startForwarding(LearnerHandler handler,
         }
         // 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 MIN_TIME_WAIT_FOR_EPOCH = "zookeeper.leader.minTimeToWaitForEpoch";
    +    private static int minTimeToWaitForEpoch;
    +    static {
    +        minTimeToWaitForEpoch = Integer.getInteger(MIN_TIME_WAIT_FOR_EPOCH, -1);
    +        LOG.info(MIN_TIME_WAIT_FOR_EPOCH + " = " + minTimeToWaitForEpoch);
    +    }
    +
    +    // visible for test
    +    public static void setMinTimeToWaitForEpoch(int minTimeToWaitForEpoch) {
    +        Leader.minTimeToWaitForEpoch = minTimeToWaitForEpoch;
    +        LOG.info(MIN_TIME_WAIT_FOR_EPOCH + " set to " + minTimeToWaitForEpoch);
    +    }
    +
    +    /**
    +     * Quit condition:
    +     *
    +     * 1 voter goes to looking again and time waitForEpoch > minTimeToWaitForEpoch
    +     *
    +     * Note: the voter may go to looking again due to:
    +     * 1. change mind in the last minute when received a different notifications
    +     * 2. the leader hadn't started leading when it tried to connect to it.
    +     */
    +    private void quitLeading() {
    +        synchronized(connectingFollowers) {
    +            quitWaitForEpoch = true;
    +            connectingFollowers.notifyAll();
    +        }
    +        LOG.info("Quit leading due to disloyal voter.");
    --- End diff --
    
    It's one of the expected case which might happen based on the current FastLeaderElection code, but I think change to warning would catch more attention, I'll do that.


---

[GitHub] zookeeper issue #588: [ZOOKEEPER-3109] Avoid long unavailable time due to vo...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on the issue:

    https://github.com/apache/zookeeper/pull/588
  
    @anmolnar it's changing the leader activation but not election part, which should be less risky, we've run this on prod to avoid long time quorum activation time due to this. It would be great if more people can take a review on this one though.


---

[GitHub] zookeeper issue #588: [ZOOKEEPER-3109] Avoid long unavailable time due to vo...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on the issue:

    https://github.com/apache/zookeeper/pull/588
  
    merged to 3.6, thanks @lvfangmin ! 


---

[GitHub] zookeeper pull request #588: [ZOOKEEPER-3109] Avoid long unavailable time du...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/zookeeper/pull/588


---

[GitHub] zookeeper pull request #588: [ZOOKEEPER-3109] Avoid long unavailable time du...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/588#discussion_r207604429
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java ---
    @@ -86,6 +86,7 @@
     
         QuorumCnxManager manager;
     
    +    private SyncedLearnerTracker leadingVoteSet;
    --- End diff --
    
    It is used to pass into the Leader after the Leader instance is constructed.


---

[GitHub] zookeeper pull request #588: [ZOOKEEPER-3109] Avoid long unavailable time du...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/588#discussion_r210482560
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/Leader.java ---
    @@ -1172,6 +1172,58 @@ synchronized public long startForwarding(LearnerHandler handler,
         }
         // 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 MIN_TIME_WAIT_FOR_EPOCH = "zookeeper.leader.minTimeToWaitForEpoch";
    +    private static int minTimeToWaitForEpoch;
    +    static {
    +        minTimeToWaitForEpoch = Integer.getInteger(MIN_TIME_WAIT_FOR_EPOCH, -1);
    +        LOG.info(MIN_TIME_WAIT_FOR_EPOCH + " = " + minTimeToWaitForEpoch);
    +    }
    +
    +    // visible for test
    +    public static void setMinTimeToWaitForEpoch(int minTimeToWaitForEpoch) {
    +        Leader.minTimeToWaitForEpoch = minTimeToWaitForEpoch;
    +        LOG.info(MIN_TIME_WAIT_FOR_EPOCH + " set to " + minTimeToWaitForEpoch);
    +    }
    +
    +    /**
    +     * Quit condition:
    +     *
    +     * 1 voter goes to looking again and time waitForEpoch > minTimeToWaitForEpoch
    +     *
    +     * Note: the voter may go to looking again due to:
    +     * 1. change mind in the last minute when received a different notifications
    +     * 2. the leader hadn't started leading when it tried to connect to it.
    +     */
    --- End diff --
    
    If a connection between leader candidate and a vote is cut off right after leader acquires enough vote but before leader receives acks from voters, the voter will also change mind, right? If so should this also be added into comment if it does not fall into 1 or 2?


---

[GitHub] zookeeper pull request #588: [ZOOKEEPER-3109] Avoid long unavailable time du...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/588#discussion_r212073403
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/Leader.java ---
    @@ -1172,6 +1172,58 @@ synchronized public long startForwarding(LearnerHandler handler,
         }
         // 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 MIN_TIME_WAIT_FOR_EPOCH = "zookeeper.leader.minTimeToWaitForEpoch";
    +    private static int minTimeToWaitForEpoch;
    +    static {
    +        minTimeToWaitForEpoch = Integer.getInteger(MIN_TIME_WAIT_FOR_EPOCH, -1);
    +        LOG.info(MIN_TIME_WAIT_FOR_EPOCH + " = " + minTimeToWaitForEpoch);
    +    }
    +
    +    // visible for test
    +    public static void setMinTimeToWaitForEpoch(int minTimeToWaitForEpoch) {
    +        Leader.minTimeToWaitForEpoch = minTimeToWaitForEpoch;
    +        LOG.info(MIN_TIME_WAIT_FOR_EPOCH + " set to " + minTimeToWaitForEpoch);
    +    }
    +
    +    /**
    +     * Quit condition:
    +     *
    +     * 1 voter goes to looking again and time waitForEpoch > minTimeToWaitForEpoch
    +     *
    +     * Note: the voter may go to looking again due to:
    +     * 1. change mind in the last minute when received a different notifications
    +     * 2. the leader hadn't started leading when it tried to connect to it.
    +     */
    --- End diff --
    
    Yes, I haven't listed all the cases here, like voter being shutdown or restarted will also change vote, but in those cases the leader won't know until they finished restarting and trying to look for leader again. 
    



---

[GitHub] zookeeper issue #588: [ZOOKEEPER-3109] Avoid long unavailable time due to vo...

Posted by gmcdonald <gi...@git.apache.org>.
Github user gmcdonald commented on the issue:

    https://github.com/apache/zookeeper/pull/588
  
    take 2, trigger PR to Jenkins


---

[GitHub] zookeeper pull request #588: [ZOOKEEPER-3109] Avoid long unavailable time du...

Posted by breed <gi...@git.apache.org>.
Github user breed commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/588#discussion_r207597400
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/Leader.java ---
    @@ -1165,6 +1165,58 @@ synchronized public long startForwarding(LearnerHandler handler,
         }
         // 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 MIN_TIME_WAIT_FOR_EPOCH = "zookeeper.leader.minTimeToWaitForEpoch";
    --- End diff --
    
    should comment this. isn't it really MAX_TIME_TO_WAIT_FOR_EPOCH?


---

[GitHub] zookeeper issue #588: [ZOOKEEPER-3109] Avoid long unavailable time due to vo...

Posted by anmolnar <gi...@git.apache.org>.
Github user anmolnar commented on the issue:

    https://github.com/apache/zookeeper/pull/588
  
    @fpj @phunt  This patch has major changes in leader election.
    Do you have some spare cycles to take a look?


---

[GitHub] zookeeper pull request #588: [ZOOKEEPER-3109] Avoid long unavailable time du...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/588#discussion_r207605164
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/Leader.java ---
    @@ -1165,6 +1165,58 @@ synchronized public long startForwarding(LearnerHandler handler,
         }
         // 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 MIN_TIME_WAIT_FOR_EPOCH = "zookeeper.leader.minTimeToWaitForEpoch";
    --- End diff --
    
    From the quit decision maker, it means the minimum time we should give to the leader to wait for those epoch proposals.
    
    From the leader perspective, you can say this is the maximum time it can wait for those epoch proposals.
    
    Since this variable is used in the quit decision maker, that's why I called it minTimeToWaitForEpoch. 
    
    Maybe I'm wrong since I'm not native English speaker, I can change it to MAX_TIME_TO_WAIT_FOR_EPOCH if I get another one vote for this name :)


---

[GitHub] zookeeper pull request #588: [ZOOKEEPER-3109] Avoid long unavailable time du...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/588#discussion_r210483183
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/Leader.java ---
    @@ -1172,6 +1172,58 @@ synchronized public long startForwarding(LearnerHandler handler,
         }
         // 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 MIN_TIME_WAIT_FOR_EPOCH = "zookeeper.leader.minTimeToWaitForEpoch";
    +    private static int minTimeToWaitForEpoch;
    +    static {
    +        minTimeToWaitForEpoch = Integer.getInteger(MIN_TIME_WAIT_FOR_EPOCH, -1);
    +        LOG.info(MIN_TIME_WAIT_FOR_EPOCH + " = " + minTimeToWaitForEpoch);
    +    }
    +
    +    // visible for test
    +    public static void setMinTimeToWaitForEpoch(int minTimeToWaitForEpoch) {
    +        Leader.minTimeToWaitForEpoch = minTimeToWaitForEpoch;
    +        LOG.info(MIN_TIME_WAIT_FOR_EPOCH + " set to " + minTimeToWaitForEpoch);
    +    }
    +
    +    /**
    +     * Quit condition:
    +     *
    +     * 1 voter goes to looking again and time waitForEpoch > minTimeToWaitForEpoch
    +     *
    +     * Note: the voter may go to looking again due to:
    +     * 1. change mind in the last minute when received a different notifications
    +     * 2. the leader hadn't started leading when it tried to connect to it.
    +     */
    +    private void quitLeading() {
    +        synchronized(connectingFollowers) {
    +            quitWaitForEpoch = true;
    +            connectingFollowers.notifyAll();
    +        }
    +        LOG.info("Quit leading due to disloyal voter.");
    --- End diff --
    
    Should we avoid using 'disloyal' here, since it's not about Byzantine style fault, and the vote is not disloyal against the protocol? Maybe something like 'quite leading due to a voter previously voted me changed its vote because of ....'. ?  


---

[GitHub] zookeeper pull request #588: [ZOOKEEPER-3109] Avoid long unavailable time du...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/588#discussion_r212071584
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/Leader.java ---
    @@ -1172,6 +1172,58 @@ synchronized public long startForwarding(LearnerHandler handler,
         }
         // 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 MIN_TIME_WAIT_FOR_EPOCH = "zookeeper.leader.minTimeToWaitForEpoch";
    +    private static int minTimeToWaitForEpoch;
    +    static {
    +        minTimeToWaitForEpoch = Integer.getInteger(MIN_TIME_WAIT_FOR_EPOCH, -1);
    +        LOG.info(MIN_TIME_WAIT_FOR_EPOCH + " = " + minTimeToWaitForEpoch);
    --- End diff --
    
    Will do.


---

[GitHub] zookeeper pull request #588: [ZOOKEEPER-3109] Avoid long unavailable time du...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/588#discussion_r210482028
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/Leader.java ---
    @@ -1172,6 +1172,58 @@ synchronized public long startForwarding(LearnerHandler handler,
         }
         // 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 MIN_TIME_WAIT_FOR_EPOCH = "zookeeper.leader.minTimeToWaitForEpoch";
    +    private static int minTimeToWaitForEpoch;
    +    static {
    +        minTimeToWaitForEpoch = Integer.getInteger(MIN_TIME_WAIT_FOR_EPOCH, -1);
    +        LOG.info(MIN_TIME_WAIT_FOR_EPOCH + " = " + minTimeToWaitForEpoch);
    --- End diff --
    
    Use MessageFormat style {} substitutions.


---

[GitHub] zookeeper pull request #588: [ZOOKEEPER-3109] Avoid long unavailable time du...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/588#discussion_r212074255
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/Leader.java ---
    @@ -1172,6 +1172,58 @@ synchronized public long startForwarding(LearnerHandler handler,
         }
         // 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 MIN_TIME_WAIT_FOR_EPOCH = "zookeeper.leader.minTimeToWaitForEpoch";
    +    private static int minTimeToWaitForEpoch;
    +    static {
    +        minTimeToWaitForEpoch = Integer.getInteger(MIN_TIME_WAIT_FOR_EPOCH, -1);
    +        LOG.info(MIN_TIME_WAIT_FOR_EPOCH + " = " + minTimeToWaitForEpoch);
    +    }
    +
    +    // visible for test
    +    public static void setMinTimeToWaitForEpoch(int minTimeToWaitForEpoch) {
    +        Leader.minTimeToWaitForEpoch = minTimeToWaitForEpoch;
    +        LOG.info(MIN_TIME_WAIT_FOR_EPOCH + " set to " + minTimeToWaitForEpoch);
    +    }
    +
    +    /**
    +     * Quit condition:
    +     *
    +     * 1 voter goes to looking again and time waitForEpoch > minTimeToWaitForEpoch
    +     *
    +     * Note: the voter may go to looking again due to:
    +     * 1. change mind in the last minute when received a different notifications
    +     * 2. the leader hadn't started leading when it tried to connect to it.
    +     */
    +    private void quitLeading() {
    +        synchronized(connectingFollowers) {
    +            quitWaitForEpoch = true;
    +            connectingFollowers.notifyAll();
    +        }
    +        LOG.info("Quit leading due to disloyal voter.");
    --- End diff --
    
    I'll change it to something simpler like "Quit leading due to voter changed mind'.


---

[GitHub] zookeeper pull request #588: [ZOOKEEPER-3109] Avoid long unavailable time du...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/588#discussion_r210482115
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/Leader.java ---
    @@ -1172,6 +1172,58 @@ synchronized public long startForwarding(LearnerHandler handler,
         }
         // 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 MIN_TIME_WAIT_FOR_EPOCH = "zookeeper.leader.minTimeToWaitForEpoch";
    +    private static int minTimeToWaitForEpoch;
    +    static {
    +        minTimeToWaitForEpoch = Integer.getInteger(MIN_TIME_WAIT_FOR_EPOCH, -1);
    +        LOG.info(MIN_TIME_WAIT_FOR_EPOCH + " = " + minTimeToWaitForEpoch);
    +    }
    +
    +    // visible for test
    +    public static void setMinTimeToWaitForEpoch(int minTimeToWaitForEpoch) {
    +        Leader.minTimeToWaitForEpoch = minTimeToWaitForEpoch;
    +        LOG.info(MIN_TIME_WAIT_FOR_EPOCH + " set to " + minTimeToWaitForEpoch);
    --- End diff --
    
    Similar here for log style.


---

[GitHub] zookeeper pull request #588: [ZOOKEEPER-3109] Avoid long unavailable time du...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/588#discussion_r210481082
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/Leader.java ---
    @@ -1165,6 +1165,58 @@ synchronized public long startForwarding(LearnerHandler handler,
         }
         // 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 MIN_TIME_WAIT_FOR_EPOCH = "zookeeper.leader.minTimeToWaitForEpoch";
    --- End diff --
    
    MAX_TIME_TO_WAIT_FOR_EPOCH feels more intuitive for me. Also there should be some documentation of this new property in admin doc so users know its existence and how to configure it.


---

[GitHub] zookeeper issue #588: [ZOOKEEPER-3109] Avoid long unavailable time due to vo...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on the issue:

    https://github.com/apache/zookeeper/pull/588
  
    @gmcdonald what does "take 2, trigger PR to Jenkins" mean? Used to trigger PR to Jenkins? Any action item I need to take of?


---

[GitHub] zookeeper pull request #588: [ZOOKEEPER-3109] Avoid long unavailable time du...

Posted by breed <gi...@git.apache.org>.
Github user breed commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/588#discussion_r207597633
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java ---
    @@ -86,6 +86,7 @@
     
         QuorumCnxManager manager;
     
    +    private SyncedLearnerTracker leadingVoteSet;
    --- End diff --
    
    it's not totally clear what this is used for. can you comment it?


---

[GitHub] zookeeper pull request #588: [ZOOKEEPER-3109] Avoid long unavailable time du...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/588#discussion_r212071527
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/Leader.java ---
    @@ -1165,6 +1165,58 @@ synchronized public long startForwarding(LearnerHandler handler,
         }
         // 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 MIN_TIME_WAIT_FOR_EPOCH = "zookeeper.leader.minTimeToWaitForEpoch";
    --- End diff --
    
    Sounds good, I'll rename it to MAX_TIME_TO_WAIT_FOR_EPOCH, and add the documents.


---

[GitHub] zookeeper pull request #588: [ZOOKEEPER-3109] Avoid long unavailable time du...

Posted by breed <gi...@git.apache.org>.
Github user breed commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/588#discussion_r207596633
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/Leader.java ---
    @@ -1165,6 +1165,58 @@ synchronized public long startForwarding(LearnerHandler handler,
         }
         // 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 MIN_TIME_WAIT_FOR_EPOCH = "zookeeper.leader.minTimeToWaitForEpoch";
    +    private static int minTimeToWaitForEpoch;
    +    static {
    +        minTimeToWaitForEpoch = Integer.getInteger(MIN_TIME_WAIT_FOR_EPOCH, -1);
    +        LOG.info(MIN_TIME_WAIT_FOR_EPOCH + " = " + minTimeToWaitForEpoch);
    +    }
    +
    +    // visible for test
    +    public static void setMinTimeToWaitForEpoch(int minTimeToWaitForEpoch) {
    +        Leader.minTimeToWaitForEpoch = minTimeToWaitForEpoch;
    +        LOG.info(MIN_TIME_WAIT_FOR_EPOCH + " set to " + minTimeToWaitForEpoch);
    +    }
    +
    +    /**
    +     * Quit condition:
    +     *
    +     * 1 voter goes to looking again and time waitForEpoch > minTimeToWaitForEpoch
    +     *
    +     * Note: the voter may go to looking again due to:
    +     * 1. change mind in the last minute when received a different notifications
    +     * 2. the leader hadn't started leading when it tried to connect to it.
    +     */
    +    private void quitLeading() {
    +        synchronized(connectingFollowers) {
    +            quitWaitForEpoch = true;
    +            connectingFollowers.notifyAll();
    +        }
    +        LOG.info("Quit leading due to disloyal voter.");
    --- End diff --
    
    i think this should be warning. it is not expected.


---

[GitHub] zookeeper issue #588: [ZOOKEEPER-3109] Avoid long unavailable time due to vo...

Posted by gmcdonald <gi...@git.apache.org>.
Github user gmcdonald commented on the issue:

    https://github.com/apache/zookeeper/pull/588
  
    trigger jenkins with a comment ?


---

[GitHub] zookeeper issue #588: [ZOOKEEPER-3109] Avoid long unavailable time due to vo...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on the issue:

    https://github.com/apache/zookeeper/pull/588
  
    Rebased the code to resolve the conflict.


---