[
https://issues.apache.org/jira/browse/RATIS-1960?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17795009#comment-17795009
]
Tsz-wo Sze commented on RATIS-1960:
-----------------------------------
Good catch on both bugs!
The pull request is now merged. Thanks, [~adoroszlai]!
> 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: 20m
> 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=https://github.com/apache/ratis/blob/0d963e2ceec9045497bea1e4e2a939e84f36242a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java#L1583-L1585}
> if (!initializing && lifeCycle.compareAndTransition(STARTING, RUNNING))
> {
> role.startFollowerState(this, Op.APPEND_ENTRIES);
> }
> {code}
> {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)