This is an automated email from the ASF dual-hosted git repository.

andor pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/zookeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new 9e30955  ZOOKEEPER-3203: Tracking the number of non voting followers 
in ZK
9e30955 is described below

commit 9e309557c02315344d6b39012ff4c9633b54c3d3
Author: Fangmin Lyu <[email protected]>
AuthorDate: Mon Jan 7 14:55:43 2019 +0100

    ZOOKEEPER-3203: Tracking the number of non voting followers in ZK
    
    Author: Fangmin Lyu <[email protected]>
    
    Reviewers: [email protected]
    
    Closes #722 from lvfangmin/ZOOKEEPER-3203
---
 .../apache/zookeeper/server/admin/Commands.java    |  1 +
 .../zookeeper/server/command/MonitorCommand.java   |  1 +
 .../org/apache/zookeeper/server/quorum/Leader.java | 14 +++++++++++-
 .../apache/zookeeper/server/quorum/LeaderBean.java |  9 ++++++++
 .../zookeeper/server/quorum/LeaderMXBean.java      |  5 +++++
 .../zookeeper/server/quorum/LeaderBeanTest.java    | 26 +++++++++++++++++++++-
 6 files changed, 54 insertions(+), 2 deletions(-)

diff --git 
a/zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/Commands.java
 
b/zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/Commands.java
index 29e1845..f1e5500 100644
--- 
a/zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/Commands.java
+++ 
b/zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/Commands.java
@@ -369,6 +369,7 @@ public class Commands {
 
                 response.put("learners", leader.getLearners().size());
                 response.put("synced_followers", 
leader.getForwardingFollowers().size());
+                response.put("synced_non_voting_followers", 
leader.getNonVotingFollowers().size());
                 response.put("synced_observers", 
leader.getObservingLearners().size());
                 response.put("pending_syncs", leader.getNumPendingSyncs());
                 response.put("leader_uptime", leader.getUptime());
diff --git 
a/zookeeper-server/src/main/java/org/apache/zookeeper/server/command/MonitorCommand.java
 
b/zookeeper-server/src/main/java/org/apache/zookeeper/server/command/MonitorCommand.java
index e3ac230..aeca32d 100644
--- 
a/zookeeper-server/src/main/java/org/apache/zookeeper/server/command/MonitorCommand.java
+++ 
b/zookeeper-server/src/main/java/org/apache/zookeeper/server/command/MonitorCommand.java
@@ -73,6 +73,7 @@ public class MonitorCommand extends AbstractFourLetterCommand 
{
 
             print("learners", leader.getLearners().size());
             print("synced_followers", leader.getForwardingFollowers().size());
+            print("synced_non_voting_followers", 
leader.getNonVotingFollowers().size());
             print("pending_syncs", leader.getNumPendingSyncs());
 
             print("last_proposal_size", 
leader.getProposalStats().getLastBufferSize());
diff --git 
a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java 
b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java
index 397ea6d..c284deb 100644
--- 
a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java
+++ 
b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java
@@ -152,7 +152,19 @@ public class Leader implements LearnerMaster {
         }
     }
 
-    private void addForwardingFollower(LearnerHandler lh) {
+    public List<LearnerHandler> getNonVotingFollowers() {
+        List<LearnerHandler> nonVotingFollowers = new 
ArrayList<LearnerHandler>();
+        synchronized (forwardingFollowers) {
+            for (LearnerHandler lh : forwardingFollowers) {
+                if (!isParticipant(lh.getSid())) {
+                    nonVotingFollowers.add(lh);
+                }
+            }
+        }
+        return nonVotingFollowers;
+    }
+
+    void addForwardingFollower(LearnerHandler lh) {
         synchronized (forwardingFollowers) {
             forwardingFollowers.add(lh);
         }
diff --git 
a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderBean.java
 
b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderBean.java
index 0c3be4a..1c178f6 100644
--- 
a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderBean.java
+++ 
b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderBean.java
@@ -52,6 +52,15 @@ public class LeaderBean extends ZooKeeperServerBean 
implements LeaderMXBean {
     }
 
     @Override
+    public String nonVotingFollowerInfo() {
+        StringBuilder sb = new StringBuilder();
+        for (LearnerHandler handler : leader.getNonVotingFollowers()) {
+            sb.append(handler.toString()).append("\n");
+        }
+        return sb.toString();
+    }
+
+    @Override
     public long getElectionTimeTaken() {
         return leader.self.getElectionTimeTaken();
     }
diff --git 
a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderMXBean.java
 
b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderMXBean.java
index 7a1a439..4aed186 100644
--- 
a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderMXBean.java
+++ 
b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderMXBean.java
@@ -35,6 +35,11 @@ public interface LeaderMXBean extends ZooKeeperServerMXBean {
     public String followerInfo();
 
     /**
+     * @return information about current non-voting followers
+     */
+    public String nonVotingFollowerInfo();
+
+    /**
      * @return time taken for leader election in milliseconds.
      */
     public long getElectionTimeTaken();
diff --git 
a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java
 
b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java
index 69dac1f..38539b3 100644
--- 
a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java
+++ 
b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java
@@ -45,6 +45,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertNotEquals;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyString;
@@ -58,6 +59,7 @@ public class LeaderBeanTest {
     private FileTxnSnapLog fileTxnSnapLog;
     private LeaderZooKeeperServer zks;
     private QuorumPeer qp;
+    private QuorumVerifier quorumVerifierMock;
 
     @Before
     public void setUp() throws IOException, X509Exception {
@@ -73,7 +75,7 @@ public class LeaderBeanTest {
                         new InetSocketAddress(clientIP, 
PortAssignment.unique()),
                         new InetSocketAddress(clientIP, clientPort), 
LearnerType.PARTICIPANT));
 
-        QuorumVerifier quorumVerifierMock = mock(QuorumVerifier.class);
+        quorumVerifierMock = mock(QuorumVerifier.class);
         when(quorumVerifierMock.getAllMembers()).thenReturn(peersView);
 
         qp.setQuorumVerifier(quorumVerifierMock, false);
@@ -173,12 +175,21 @@ public class LeaderBeanTest {
 
     @Test
     public void testFollowerInfo() throws IOException {
+        Map<Long, QuorumServer> votingMembers = new HashMap<Long, 
QuorumServer>();
+        votingMembers.put(1L, null);
+        votingMembers.put(2L, null);
+        votingMembers.put(3L, null);
+        when(quorumVerifierMock.getVotingMembers()).thenReturn(votingMembers);
+
         LearnerHandler follower = mock(LearnerHandler.class);
         when(follower.getLearnerType()).thenReturn(LearnerType.PARTICIPANT);
         when(follower.toString()).thenReturn("1");
+        when(follower.getSid()).thenReturn(1L);
         leader.addLearnerHandler(follower);
+        leader.addForwardingFollower(follower);
 
         assertEquals("1\n", leaderBean.followerInfo());
+        assertEquals("", leaderBean.nonVotingFollowerInfo());
 
         LearnerHandler observer = mock(LearnerHandler.class);
         when(observer.getLearnerType()).thenReturn(LearnerType.OBSERVER);
@@ -186,5 +197,18 @@ public class LeaderBeanTest {
         leader.addLearnerHandler(observer);
 
         assertEquals("1\n", leaderBean.followerInfo());
+        assertEquals("", leaderBean.nonVotingFollowerInfo());
+
+        LearnerHandler nonVotingFollower = mock(LearnerHandler.class);
+        
when(nonVotingFollower.getLearnerType()).thenReturn(LearnerType.PARTICIPANT);
+        when(nonVotingFollower.toString()).thenReturn("5");
+        when(nonVotingFollower.getSid()).thenReturn(5L);
+        leader.addLearnerHandler(nonVotingFollower);
+        leader.addForwardingFollower(nonVotingFollower);
+
+        String followerInfo = leaderBean.followerInfo();
+        assertTrue(followerInfo.contains("1"));
+        assertTrue(followerInfo.contains("5"));
+        assertEquals("5\n", leaderBean.nonVotingFollowerInfo());
     }
 }

Reply via email to