You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ratis.apache.org by GitBox <gi...@apache.org> on 2021/12/09 11:33:48 UTC

[GitHub] [ratis] szetszwo commented on a change in pull request #560: RATIS-1446. Avoid leader election for invalid conf

szetszwo commented on a change in pull request #560:
URL: https://github.com/apache/ratis/pull/560#discussion_r765703801



##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
##########
@@ -583,6 +583,12 @@ synchronized void changeToCandidate(boolean forceStartLeaderElection) {
     if (state.shouldNotifyExtendedNoLeader()) {
       stateMachine.followerEvent().notifyExtendedNoLeader(getRoleInfoProto());
     }
+    // Candidate shall not start leader election in these cases in case of
+    // possible NPE caused by conf.getPeer().getPriority()
+    if (!getRaftConf().containsInBothConfs(getId())) {
+      LOG.warn("{} find invalid configuration {}, skip start leader election", this, getRaftConf());
+      return;
+    }

Review comment:
       The conf may be changed later on.  Therefore, let's don't check it here and check null in LeaderElection.getHigherPriorityPeers(..) as below.
   ```
     private Set<RaftPeerId> getHigherPriorityPeers(RaftConfiguration conf) {
       final Optional<Integer> priority = Optional.ofNullable(conf.getPeer(server.getId())).map(RaftPeer::getPriority);
       return conf.getAllPeers().stream()
           .filter(peer -> priority.filter(p -> peer.getPriority() > p).isPresent())
           .map(RaftPeer::getId)
           .collect(Collectors.toSet());
     }
   ```
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org