Xinhao GU created RATIS-2283:
--------------------------------
Summary: GrpcLogAppender Thread Restart Leaves catchup=false,
Blocking Reconfiguration Progress
Key: RATIS-2283
URL: https://issues.apache.org/jira/browse/RATIS-2283
Project: Ratis
Issue Type: Bug
Reporter: Xinhao GU
Assignee: Xinhao GU
Attachments: image-2025-04-22-15-57-53-633.png
h3. *Description*
After the {{GrpcLogAppender}} thread restarts, the {{catchup}} flag in the
corresponding {{followerInfo}} remains {{{}false{}}}. This causes a critical
issue when performing a membership change (e.g., adding a new peer).
Specifically, {{checkProgress}} cannot complete, leading to indefinite blocking
during the reconfiguration phase.
h3.
related log
{code:java}
2025-03-14 17:57:43,342 [grpc-default-executor-41] WARN
o.a.r.g.s.GrpcAdminProtocolService:79 - Failed to setConfiguration:
SetConfigurationRequest:client-7BB03FC6049C->5@group-000200000014, cid=15,
seq=null, RW, null, SET_UNCONDITIONALLY, servers:[3|192.168.10.72:10750,
4|192.168.10.73:10750, 5|192.168.10.74:10750, 7|192.168.10.76:10750],
listeners:[]
java.util.concurrent.CompletionException:
org.apache.ratis.protocol.exceptions.ReconfigurationInProgressException:
Reconfiguration is already in progress: conf: {index: 1,
cur=peers:[3|192.168.10.72:10750, 4|192.168.10.73:10750,
5|192.168.10.74:10750]|listeners:[], old=null}
at org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:122)
at
org.apache.ratis.server.impl.RaftServerImpl.lambda$executeSubmitServerRequestAsync$10(RaftServerImpl.java:902)
at
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1768)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:833)
Caused by:
org.apache.ratis.protocol.exceptions.ReconfigurationInProgressException:
Reconfiguration is already in progress: conf: {index: 1,
cur=peers:[3|192.168.10.72:10750, 4|192.168.10.73:10750,
5|192.168.10.74:10750]|listeners:[], old=null}
at
org.apache.ratis.server.impl.RaftServerImpl.setConfigurationAsync(RaftServerImpl.java:1320)
at
org.apache.ratis.server.impl.RaftServerProxy.lambda$null$24(RaftServerProxy.java:623)
at org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:118)
... 5 common frames omitted {code}
h3. Cause Analysis
# *Direct Cause:*
When the {{GrpcLogAppender}} restarts, its {{catchup}} flag stays
{{{}false{}}}, and {{hasAttemptedToInstallSnapshot}} is unsatisfied. This
prevents the {{stagingState}} phase of reconfiguration from completing.
# *Code-Level Issue:*
In {{{}shouldInstallSnapshot(){}}}, the current logic only allows new followers
in the {{stagingState}} to trigger snapshot installation. Restarted _old
followers_ (e.g., after {{GrpcLogAppender}} restart) are excluded, causing
their {{catchup}} flag to remain {{false}} indefinitely.
h3. related code
{code:java}
private BootStrapProgress checkProgress(FollowerInfo follower, long committed) {
long confIndex = server.getRaftConf().getLogEntryIndex();
long matchIndex = follower.getMatchIndex();
LOG.info("{}: checkProgress for {}, commitIndex={}, conf'LogEntryIndex={}",
this, follower, committed, confIndex);
Preconditions.assertTrue(!isCaughtUp(follower));
final Timestamp progressTime =
Timestamp.currentTime().addTimeMs(-server.getMaxTimeoutMs());
final Timestamp timeoutTime =
Timestamp.currentTime().addTimeMs(-stagingTimeout.toLong(TimeUnit.MILLISECONDS));
if (follower.getLastRpcResponseTime().compareTo(timeoutTime) < 0) {
LOG.info("{} detects a follower {} timeout ({}ms) for bootstrapping", this,
follower,
follower.getLastRpcResponseTime().elapsedTimeMs());
return BootStrapProgress.NOPROGRESS;
} else if (matchIndex + stagingCatchupGap > committed
&& matchIndex >= confIndex
&& follower.getLastRpcResponseTime().compareTo(progressTime) > 0
&& follower.hasAttemptedToInstallSnapshot()) {
LOG.info("{} detects a follower {} is caught up for bootstrapping,
follower's matchIndex={}, Attempted={}", this, follower, matchIndex,
follower.hasAttemptedToInstallSnapshot());
return BootStrapProgress.CAUGHTUP;
} else {
LOG.info("{} detects a follower {} is making progress for bootstrapping,
follower's matchIndex={}, Attempted={}", this, follower, matchIndex,
follower.hasAttemptedToInstallSnapshot());
return BootStrapProgress.PROGRESSING;
}
} {code}
h3. *Proposed Fix:*
{*}Adjust {{shouldInstallSnapshot()}} Logic{*}: Allow snapshot installation for
both _new followers_ in {{stagingState}} and {_}restarted old followers{_}.
!image-2025-04-22-15-57-53-633.png!
--
This message was sent by Atlassian Jira
(v8.20.10#820010)