You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by dr...@apache.org on 2022/09/09 06:15:08 UTC

[ratis] branch master updated: RATIS-1702. Make listener reject leader election (#739)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new df18d0b03 RATIS-1702. Make listener reject leader election (#739)
df18d0b03 is described below

commit df18d0b03e49575b97745dbf4b759cc4022bd562
Author: Yaolong Liu <ly...@163.com>
AuthorDate: Fri Sep 9 14:15:02 2022 +0800

    RATIS-1702. Make listener reject leader election (#739)
    
    * RATIS-1702. Make listener reject leader election
    
    * address review comments
    
    Co-authored-by: dragonyliu <dr...@tencent.com>
---
 .../java/org/apache/ratis/server/impl/VoteContext.java   |  4 ++++
 .../apache/ratis/server/impl/LeaderElectionTests.java    | 16 ++++++++++++++++
 2 files changed, 20 insertions(+)

diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/VoteContext.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/VoteContext.java
index 2b7d96975..6375d652c 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/VoteContext.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/VoteContext.java
@@ -17,6 +17,7 @@
  */
 package org.apache.ratis.server.impl;
 
+import org.apache.ratis.proto.RaftProtos;
 import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.server.DivisionInfo;
@@ -131,6 +132,9 @@ class VoteContext {
    * See Section 5.4.1 Election restriction
    */
   boolean decideVote(RaftPeer candidate, TermIndex candidateLastEntry) {
+    if (impl.getRole().getCurrentRole() == RaftProtos.RaftPeerRole.LISTENER) {
+      return reject("this server is a listener, who is a non-voting member");
+    }
     if (candidate == null) {
       return false;
     }
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java
index bf9938125..17b542994 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java
@@ -36,6 +36,7 @@ import org.apache.ratis.server.DivisionInfo;
 import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.metrics.LeaderElectionMetrics;
+import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogTestUtils;
 import org.apache.ratis.util.ExitUtils;
 import org.apache.ratis.util.JavaUtils;
@@ -498,6 +499,21 @@ public abstract class LeaderElectionTests<CLUSTER extends MiniRaftCluster>
     }
   }
 
+  @Test
+  public void testListenerRejectRequestVote() throws Exception {
+    runWithNewCluster(3, 2, this::runTestListenerRejectRequestVote);
+  }
+  void runTestListenerRejectRequestVote(CLUSTER cluster) throws IOException, InterruptedException {
+    final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster);
+    final TermIndex lastEntry = leader.getRaftLog().getLastEntryTermIndex();
+    RaftServer.Division listener = cluster.getListeners().get(0);
+    final RaftProtos.RequestVoteRequestProto r = ServerProtoUtils.toRequestVoteRequestProto(
+        leader.getMemberId(), listener.getId(),  leader.getRaftLog().getLastEntryTermIndex().getTerm() + 1, lastEntry, true);
+    RaftProtos.RequestVoteReplyProto listenerReply = listener.getRaftServer().requestVote(r);
+    Assert.assertFalse(listenerReply.getServerReply().getSuccess());
+  }
+
+
   @Test
   public void testPauseResumeLeaderElection() throws Exception {
     runWithNewCluster(3, this::runTestPauseResumeLeaderElection);