Skip to content

Commit ed3da9a

Browse files
committed
modify isFollowerBootstrapping and isBootStrappingPeer
1 parent aa0ddf7 commit ed3da9a

4 files changed

Lines changed: 14 additions & 15 deletions

File tree

ratis-server-api/src/main/java/org/apache/ratis/server/leader/FollowerInfo.java

Lines changed: 0 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -112,10 +112,4 @@ public interface FollowerInfo {
112112

113113
/** Update lastRpcResponseTime and LastRespondedAppendEntriesSendTime */
114114
void updateLastRespondedAppendEntriesSendTime(Timestamp sendTime);
115-
116-
/** Set the caughtUp flag to true. */
117-
void catchUp();
118-
119-
/** @return true if this follower is caught up. */
120-
boolean isCaughtUp();
121115
}

ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -145,12 +145,12 @@ default SnapshotInfo shouldInstallSnapshot() {
145145
// we should install snapshot if the follower needs to catch up and:
146146
// 1. there is no local log entry but there is snapshot
147147
// 2. or the follower's next index is smaller than the log start index
148-
// 3. or the follower is bootstrapping and has not installed any snapshot yet
148+
// 3. or the follower is bootstrapping (i.e. not yet caught up) and has not installed any snapshot yet
149149
final FollowerInfo follower = getFollower();
150-
final boolean isFollowerCaughtUp = follower.isCaughtUp();
150+
final boolean isFollowerBootstrapping = getLeaderState().isFollowerBootstrapping(follower);
151151
final SnapshotInfo snapshot = getServer().getStateMachine().getLatestSnapshot();
152152

153-
if (!isFollowerCaughtUp && !follower.hasAttemptedToInstallSnapshot()) {
153+
if (isFollowerBootstrapping && !follower.hasAttemptedToInstallSnapshot()) {
154154
if (snapshot == null) {
155155
// Leader cannot send null snapshot to follower. Hence, acknowledge InstallSnapshot attempt (even though it
156156
// was not attempted) so that follower can come out of staging state after appending log entries.

ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerInfoImpl.java

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -173,13 +173,11 @@ public String toString() {
173173
", lastRpcResponseTime=" + lastRpcResponseTime.get().elapsedTimeMs() + ")";
174174
}
175175

176-
@Override
177-
public void catchUp() {
176+
void catchUp() {
178177
caughtUp = true;
179178
}
180179

181-
@Override
182-
public boolean isCaughtUp() {
180+
boolean isCaughtUp() {
183181
return caughtUp;
184182
}
185183

ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java

Lines changed: 9 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -820,7 +820,9 @@ public void onFollowerSuccessAppendEntries(FollowerInfo follower) {
820820

821821
@Override
822822
public boolean isFollowerBootstrapping(FollowerInfo follower) {
823-
return isBootStrappingPeer(follower.getId());
823+
// It is better to check caught up than staging state
824+
// since a follower may have already caught up but still in the staging state.
825+
return !isCaughtUp(follower);
824826
}
825827

826828
private void checkStaging() {
@@ -852,7 +854,12 @@ private void checkStaging() {
852854
}
853855

854856
boolean isBootStrappingPeer(RaftPeerId peerId) {
855-
return Optional.ofNullable(stagingState).map(s -> s.contains(peerId)).orElse(false);
857+
final Optional<LogAppender> info = getLogAppender(peerId);
858+
if (info.isPresent()) {
859+
return !isCaughtUp(info.get().getFollower());
860+
}
861+
final ConfigurationStagingState staging = stagingState;
862+
return staging != null && staging.contains(peerId);
856863
}
857864

858865
void submitUpdateCommitEvent() {

0 commit comments

Comments
 (0)