This is an automated email from the ASF dual-hosted git repository. szetszwo pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/ratis.git
commit 21ce4e1fd8b39cd139a6821bd4de9cbf7d90a5b0 Author: GewuNewOne <89496957+rkg...@users.noreply.github.com> AuthorDate: Thu Apr 24 01:49:51 2025 +0800 RATIS-2283. GrpcLogAppender Thread Restart Leaves catchup=false, Blocking Reconfiguration Progress (#1250) --- .../main/java/org/apache/ratis/server/leader/LogAppender.java | 2 +- .../java/org/apache/ratis/server/impl/LeaderStateImpl.java | 11 +++++++++-- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java b/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java index 36331e3ab..cff5425d3 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java @@ -143,7 +143,7 @@ public interface LogAppender { // we should install snapshot if the follower needs to catch up and: // 1. there is no local log entry but there is snapshot // 2. or the follower's next index is smaller than the log start index - // 3. or the follower is bootstrapping and has not installed any snapshot yet + // 3. or the follower is bootstrapping (i.e. not yet caught up) and has not installed any snapshot yet final FollowerInfo follower = getFollower(); final boolean isFollowerBootstrapping = getLeaderState().isFollowerBootstrapping(follower); final SnapshotInfo snapshot = getServer().getStateMachine().getLatestSnapshot(); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index 9e39bd0ef..a84732d1c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -818,7 +818,9 @@ class LeaderStateImpl implements LeaderState { @Override public boolean isFollowerBootstrapping(FollowerInfo follower) { - return isBootStrappingPeer(follower.getId()); + // It is better to check caught up than staging state + // since a follower may have already caught up but still in the staging state. + return !isCaughtUp(follower); } private void checkStaging() { @@ -850,7 +852,12 @@ class LeaderStateImpl implements LeaderState { } boolean isBootStrappingPeer(RaftPeerId peerId) { - return Optional.ofNullable(stagingState).map(s -> s.contains(peerId)).orElse(false); + final Optional<LogAppender> info = getLogAppender(peerId); + if (info.isPresent()) { + return !isCaughtUp(info.get().getFollower()); + } + final ConfigurationStagingState staging = stagingState; + return staging != null && staging.contains(peerId); } void submitUpdateCommitEvent() {