Skip to content

Commit

Permalink
RATIS-1960. Follower may be incorrectly marked as having caught up (#983
Browse files Browse the repository at this point in the history
)
  • Loading branch information
adoroszlai authored Dec 10, 2023
1 parent 01b386b commit 9bd82aa
Showing 1 changed file with 34 additions and 19 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -500,16 +500,25 @@ PendingRequest startSetConfiguration(SetConfigurationRequest request, List<RaftP
peersToBootStrap, listenersToBootStrap, new PeerConfiguration(peersInNewConf, listenersInNewConf));
Collection<RaftPeer> newPeers = configurationStagingState.getNewPeers();
Collection<RaftPeer> newListeners = configurationStagingState.getNewListeners();
// set the staging state
this.stagingState = configurationStagingState;

if (newPeers.isEmpty() && newListeners.isEmpty()) {
applyOldNewConf();
Collection<RaftPeer> allNew = newListeners.isEmpty()
? newPeers
: newPeers.isEmpty()
? newListeners
: Stream.concat(newPeers.stream(), newListeners.stream())
.collect(Collectors.toList());

if (allNew.isEmpty()) {
applyOldNewConf(configurationStagingState);
} else {
// update the LeaderState's sender list
addAndStartSenders(newPeers);
addAndStartSenders(newListeners);
Collection<LogAppender> newAppenders = addSenders(allNew);

// set the staging state
stagingState = configurationStagingState;

newAppenders.forEach(LogAppender::start);
}

return pending;
}

Expand Down Expand Up @@ -579,14 +588,14 @@ private void commitIndexChanged() {
notifySenders();
}

private void applyOldNewConf() {
private void applyOldNewConf(ConfigurationStagingState stage) {
final ServerState state = server.getState();
final RaftConfigurationImpl current = state.getRaftConf();
final RaftConfigurationImpl oldNewConf= stagingState.generateOldNewConf(current, state.getLog().getNextIndex());
final long nextIndex = state.getLog().getNextIndex();
final RaftConfigurationImpl oldNewConf = stage.generateOldNewConf(current, nextIndex);
// apply the (old, new) configuration to log, and use it as the current conf
appendConfiguration(oldNewConf);

this.stagingState = null;
notifySenders();
}

Expand All @@ -607,7 +616,7 @@ void updateFollowerCommitInfos(CommitInfoCache cache, List<CommitInfoProto> prot
@Override
public AppendEntriesRequestProto newAppendEntriesRequestProto(FollowerInfo follower,
List<LogEntryProto> entries, TermIndex previous, long callId) {
final boolean initializing = isCaughtUp(follower);
final boolean initializing = !isCaughtUp(follower);
final RaftPeerId targetId = follower.getId();
return ServerProtoUtils.toAppendEntriesRequestProto(server.getMemberId(), targetId, currentTerm, entries,
ServerImplUtils.effectiveCommitIndex(raftLog.getLastCommittedIndex(), previous, entries.size()),
Expand All @@ -618,9 +627,13 @@ public AppendEntriesRequestProto newAppendEntriesRequestProto(FollowerInfo follo
* Update sender list for setConfiguration request
*/
private void addAndStartSenders(Collection<RaftPeer> newPeers) {
if (!newPeers.isEmpty()) {
addSenders(newPeers, RaftLog.LEAST_VALID_LOG_INDEX, false).forEach(LogAppender::start);
}
addSenders(newPeers).forEach(LogAppender::start);
}

private Collection<LogAppender> addSenders(Collection<RaftPeer> newPeers) {
return !newPeers.isEmpty()
? addSenders(newPeers, RaftLog.LEAST_VALID_LOG_INDEX, false)
: Collections.emptyList();
}

private RaftPeer getPeer(RaftPeerId id) {
Expand Down Expand Up @@ -811,20 +824,22 @@ private void checkStaging() {
} else {
final long commitIndex = server.getState().getLog().getLastCommittedIndex();
// check progress for the new followers
final EnumSet<BootStrapProgress> reports = getLogAppenders()
final List<FollowerInfoImpl> laggingFollowers = getLogAppenders()
.map(LogAppender::getFollower)
.filter(follower -> !isCaughtUp(follower))
.map(FollowerInfoImpl.class::cast)
.collect(Collectors.toList());
final EnumSet<BootStrapProgress> reports = laggingFollowers.stream()
.map(follower -> checkProgress(follower, commitIndex))
.collect(Collectors.toCollection(() -> EnumSet.noneOf(BootStrapProgress.class)));
if (reports.contains(BootStrapProgress.NOPROGRESS)) {
stagingState.fail(BootStrapProgress.NOPROGRESS);
} else if (!reports.contains(BootStrapProgress.PROGRESSING)) {
// all caught up!
applyOldNewConf();
getLogAppenders()
.map(LogAppender::getFollower)
applyOldNewConf(stagingState);
this.stagingState = null;
laggingFollowers.stream()
.filter(f -> server.getRaftConf().containsInConf(f.getId()))
.map(FollowerInfoImpl.class::cast)
.forEach(FollowerInfoImpl::catchUp);
}
}
Expand Down

0 comments on commit 9bd82aa

Please sign in to comment.