fix: enable leader to recover corrupted follower after data cleanup#929
Merged
mattisonchao merged 8 commits intomainfrom Mar 9, 2026
Merged
fix: enable leader to recover corrupted follower after data cleanup#929mattisonchao merged 8 commits intomainfrom
mattisonchao merged 8 commits intomainfrom
Conversation
When a follower restarts with clean data (NOT_MEMBER status), the leader could not recover it because: (1) AppendEntries rejected with generic ErrInvalidStatus that the leader couldn't distinguish, and (2) the leader's FollowerCursor retained a stale cursor position, skipping the snapshot. This introduces a dedicated ErrNodeIsNotMember error code so the leader can detect the situation, reset its cursor, and send a full snapshot to recover the follower. Fixes #924 Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>
…nnel Instead of stashing the receive-side error in an atomic.Value field, pass it back through a buffered channel so streamEntries() returns the actual follower error. The backoff handler can then check the error directly with status.Code(err). Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>
Have receiveAcks return error and capture it via a closure variable with wg.Go(), removing the channel. streamEntries waits on the WaitGroup then prefers the receive-side error over the send-side one. Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>
Ensures stream.Recv() is unblocked so wg.Wait() does not hang when streamEntriesLoop returns due to a non-context error. Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>
08fa8ba to
827c075
Compare
Replace errors.Join with multierr.Combine (consistent with the rest of the codebase). Filter out context.Canceled from send errors so that status.Code() works directly on the combined error for gRPC code detection. Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>
827c075 to
7a4928d
Compare
Move the context.Canceled filtering from streamEntries into streamEntriesLoop where the cancellation actually occurs, making the error handling more localized and removing the post-hoc filter. Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>
gRPC may wrap context.Canceled in a status error, so check both errors.Is(err, context.Canceled) and status.Code(err) == codes.Canceled in streamEntriesLoop, consistent with the rest of the codebase. Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>
mattisonchao
commented
Mar 8, 2026
Member
Author
mattisonchao
left a comment
There was a problem hiding this comment.
Self-review
Did a deep review of the full diff. Overall the changes are clean, well-scoped, and correct.
What this PR does
When a follower restarts with clean data (NOT_MEMBER status), the leader previously couldn't recover it because:
AppendEntriesreturned a genericErrInvalidStatusthe leader couldn't distinguish- The leader's
FollowerCursorretained a stale cursor position, soshouldSendSnapshot()skipped the snapshot
The fix introduces ErrNodeIsNotMember (code 112) so the leader can detect this, reset its cursor, and send a full snapshot.
Verified safe
- Status atomicity:
InstallSnapshotsets status toFOLLOWERwhile holdingrwMutex.Lock()(acquired at line 506, deferred unlock at 507), so no race withNewTermorAppendEntries. - Snapshot trigger: Resetting
ackOffsettoInvalidOffsetcorrectly triggersshouldSendSnapshot()via the first condition at line 171:ackOffset == wal.InvalidOffset && fc.ackTracker.CommitOffset() >= 0. - Goroutine safety:
receiveAckserror is captured via closure +wg.Wait(), which provides happens-before guarantee. No goroutine leak —cancel()unblocksstream.Recv(). - Error propagation through
multierr.Combine: WhenreceiveAcksreturnsCodeNodeIsNotMember, it callscancel(), which makesstreamEntriesLoopreturnnil(context.Canceled is now filtered). Somultierr.Combine(ErrNodeIsNotMember, nil)=ErrNodeIsNotMember, andstatus.Code()in the backoff handler correctly matches. toGRPCErrorordering:ErrNodeIsNotMemberis checked beforeErrInvalidStatusin the switch, so it takes precedence. Correct.
Note
status.Code() on a multierr.Combine result is order-dependent (it extracts the code from the first error). This works correctly here because when the receive side returns CodeNodeIsNotMember, the send side will be nil. But it's worth being aware of if this pattern is extended in the future.
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
Motivation
When a follower restarts with clean data (
NOT_MEMBERstatus), the leader cannot recover it and keeps failing. This happens becauseAppendEntriesreturns a genericErrInvalidStatusthat the leader can't distinguish from other errors, and the leader'sFollowerCursorretains a stale cursor position soshouldSendSnapshot()skips the snapshot.Modification
ErrNodeIsNotMember/CodeNodeIsNotMember(code 112) so the follower can report itsNOT_MEMBERstatus distinctly from other invalid-status errorsAppendEntriesnow returnsErrNodeIsNotMemberwhen the follower is inNOT_MEMBERstatus, instead of genericErrInvalidStatusInstallSnapshotsets the follower status toFOLLOWERafter a successful snapshot install, so subsequentAppendEntriescalls are acceptedFollowerCursor.runOnce()detectsCodeNodeIsNotMemberfrom the follower's ack stream and resets its ack offset to trigger a full snapshot on the next retrytoGRPCErrormaps the newErrNodeIsNotMemberdomain error to the correct gRPC status codeTestFollower_SnapshotRecoveryFromNotMemberto verify the end-to-end recovery pathFixes #924