Uploaded image for project: 'Ratis'
  1. Ratis
  2. RATIS-1960

Follower may be incorrectly marked as having caught up

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 3.0.1
    • server
    • None

    Description

      I think there is a race condition in LeaderStateImpl#checkStaging:

            // 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);
            }
      

      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):

          // set the staging state
          this.stagingState = configurationStagingState;
      
          if (newPeers.isEmpty() && newListeners.isEmpty()) {
            applyOldNewConf();
          } else {
            // update the LeaderState's sender list
            addAndStartSenders(newPeers);
            addAndStartSenders(newListeners);
      

      If the follower is incorrectly marked as having caught up, it will not transition from starting to running when it receives appendEntries:

            if (!initializing && lifeCycle.compareAndTransition(STARTING, RUNNING)) {
              role.startFollowerState(this, Op.APPEND_ENTRIES);
            }
      
      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: ...
      
      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: ...
      

      Attachments

        Issue Links

          Activity

            People

              adoroszlai Attila Doroszlai
              adoroszlai Attila Doroszlai
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 40m
                  40m