New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
RATIS-1446. Avoid leader election for invalid conf #560
Conversation
@szetszwo Actually, I am not sure whether we should make a specific check for each getPriority() use. Right now, I just prevent the NPE from the source. Currently, conf.getPeer().getPriority() only exists in leader election and leader state. |
// 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; | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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());
}
That's a good point. We should check. I just have walked through the code. There are two other cases needed to be fixed:
|
The check here in the getHigherPriorityPeers is not enough. The possible case I tested is that bootstrapped follower changes role from follower to candidate and start leader election, as it has an empty conf, it will become leader directly due to I sightly changed the check place. PTAL |
86e0c18
to
07e0a11
Compare
This is an invalid test case since the conf (i.e. the group) is incorrect in the beginning. We must either set the group correct before startup or use AdminApi.setConfiguration to change the conf after startup. |
The bootstrapped follower could start with empty conf and set conf through leader's appendEntries request or Installsnapshot request. Somehow caused election timeout would let this follower start leader with empty conf. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The additional changes are good although the empty-conf test case is invalid. Some comments inlined; see also https://issues.apache.org/jira/secure/attachment/13037236/560_reivew.patch
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java
Outdated
Show resolved
Hide resolved
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java
Outdated
Show resolved
Hide resolved
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java
Outdated
Show resolved
Hide resolved
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java
Outdated
Show resolved
Hide resolved
b85396a
to
7432712
Compare
Thanks for all the comments above. I have fixed all. @szetszwo With |
@Xushaohong , thanks for the update. TestLeaderElectionWithNetty.testTransferLeader failed. I also can reproduce it locally. Could you take a look? |
Currently, the setConf request would not update the |
We should fix the RaftReconfigurationBaseTest but not having the phase check. |
@@ -271,6 +272,9 @@ private boolean shouldRun(long electionTerm) { | |||
|
|||
private ResultAndTerm submitRequestAndWaitResult(Phase phase, RaftConfigurationImpl conf, long electionTerm) | |||
throws InterruptedException { | |||
if (!conf.containsInConf(server.getId()) && phase == Phase.ELECTION) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should fix the unit tests but not adding the phase check here.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should fix the unit tests but not adding the phase check here.
OK, let's find out what's wrong with UT. @szetszwo
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Could you help approve the workflow? : )
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@Xushaohong , the build finished. Please take a look.
I reviewed the failure UT of testRemovePeers. The fault is due to after removing the peers out of the conf, the division which should quit the raft group will become a candidate and start the election. During the prevote process, originally it will send out the request and then the other peers should send back |
This is a good idea. Let's shutdown the server when it is not in conf. Thanks. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1 the change looks good.
What changes were proposed in this pull request?
Candidate shall not start leader election in these cases in case of possible NPE caused by conf.getPeer().getPriority().
With this patch, the candidate will become a follower again when receiving the future appendEntries request.
Phenomenon:
The bootstrapped follower becomes the leader after the election timeout somehow, with an empty raft conf. Then in the process of
yieldLeaderToHigherPriorityPeer
, NPE happens as (null).getPriority().Setconfiguration action takes place at the end of appendEntriesAsync. If we make it possible to raise electiontimeout before this action, we can replay the NPE case.
Possible reason:
Network delay/loss? The leader did not send entries in time or the follower did not receive entries.
What is the link to the Apache JIRA
https://issues.apache.org/jira/browse/RATIS-1446
How was this patch tested?
Manual simulation in UT.