[
https://issues.apache.org/jira/browse/RATIS-1960?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17794977#comment-17794977
]
Attila Doroszlai commented on RATIS-1960:
-----------------------------------------
In addition, I think there is a related bug in how {{initializing}} is
determined when sending append entries to followers.
Follower is currently considered {{initializing}} if it {{isCaughtUp}}:
{code:title=https://github.com/apache/ratis/blob/0d963e2ceec9045497bea1e4e2a939e84f36242a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java#L608-L610}
public AppendEntriesRequestProto newAppendEntriesRequestProto(FollowerInfo
follower,
List<LogEntryProto> entries, TermIndex previous, long callId) {
final boolean initializing = isCaughtUp(follower);
{code}
When follower is created (for setConfiguration), it starts out as not "caught
up", therefore not "initializing". The first append entry request will start
the follower (when not hitting the race condition), even it is still lagging.
Later, when it catches up, further appendEntries will come with
{{initializing=true}}, but that doesn't cause problems, follower is already
started.
However, starting follower too early may cause problem: follower may turn into
a candidate while snapshot is still being installed.
I think follower should be considered to be initializing _until_ it has caught
up, not _when_ it has.
Looking at git history, RATIS-1191 seems to have switched the {{initializing}}
flag when it was moved from being a parameter to a local variable:
{code}
- AppendEntriesRequestProto newAppendEntriesRequestProto(RaftPeerId targetId,
- TermIndex previous, List<LogEntryProto> entries, boolean initializing,
- long callId) {
+ AppendEntriesRequestProto newAppendEntriesRequestProto(FollowerInfo follower,
+ List<LogEntryProto> entries, TermIndex previous, long callId) {
+ final boolean initializing = isAttendingVote(follower);
+ final RaftPeerId targetId = follower.getPeer().getId();
{code}
and
{code}
- return leaderState.newAppendEntriesRequestProto(
- getFollowerId(), previous, protos, !follower.isAttendingVote(),
callId);
+ return leaderState.newAppendEntriesRequestProto(follower, protos,
previous, callId);
{code}
> Follower may be incorrectly marked as having caught up
> ------------------------------------------------------
>
> Key: RATIS-1960
> URL: https://issues.apache.org/jira/browse/RATIS-1960
> Project: Ratis
> Issue Type: Bug
> Components: server
> Reporter: Attila Doroszlai
> Assignee: Attila Doroszlai
> Priority: Major
> Time Spent: 10m
> Remaining Estimate: 0h
>
> I think there is a race condition in {{LeaderStateImpl#checkStaging}}:
> {code:title=https://github.com/apache/ratis/blob/0d963e2ceec9045497bea1e4e2a939e84f36242a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java#L813-L829}
> // check progress for the new followers
> final EnumSet<BootStrapProgress> reports = getLogAppenders()
> .map(LogAppender::getFollower)
> .filter(follower -> !isCaughtUp(follower))
> .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)
> .filter(f -> server.getRaftConf().containsInConf(f.getId()))
> .map(FollowerInfoImpl.class::cast)
> .forEach(FollowerInfoImpl::catchUp);
> }
> {code}
> Followers are collected/iterated twice:
> * check progress status
> * mark as having caught up
> The race condition is between the thread executing {{checkStaging}}
> ({{LeaderStateImpl}}), and the thread setting the stage and adding new
> followers in {{startSetConfiguration}} (which can be client thread):
> {code:title=https://github.com/apache/ratis/blob/0d963e2ceec9045497bea1e4e2a939e84f36242a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java#L503-L511}
> // set the staging state
> this.stagingState = configurationStagingState;
> if (newPeers.isEmpty() && newListeners.isEmpty()) {
> applyOldNewConf();
> } else {
> // update the LeaderState's sender list
> addAndStartSenders(newPeers);
> addAndStartSenders(newListeners);
> {code}
> If the follower is incorrectly marked as having caught up, it will not
> transition from starting to running when it receives {{appendEntries}}:
> {code:title=bad (initializing=true)}
> [omNode-bootstrap-1-server-thread2] DEBUG server.RaftServer$Division
> (RaftServerImpl.java:logAppendEntries(1504)) -
> omNode-bootstrap-1@group-0AAC5367B30E: receive appendEntries(omNode-1, 1,
> (t:1, i:0), 8, true, commits:[omNode-1:c8, omNode-bootstrap-1:c0], cId:1,
> entries: ...
> {code}
> {code:title=good (initializing=false)}
> [omNode-bootstrap-1-server-thread1] DEBUG util.LifeCycle
> (LifeCycle.java:validate(116)) - omNode-bootstrap-1: STARTING -> RUNNING
> [omNode-bootstrap-1-server-thread1] DEBUG server.RaftServer$Division
> (RaftServerImpl.java:logAppendEntries(1504)) -
> omNode-bootstrap-1@group-0AAC5367B30E: receive appendEntries(omNode-1, 1,
> (t:1, i:0), 8, false, commits:[omNode-1:c8, omNode-bootstrap-1:c0], cId:1,
> entries: ...
> {code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)