Skip to content

fix: enable leader to recover corrupted follower after data cleanup#929

Merged
mattisonchao merged 8 commits intomainfrom
fix/follower-recovery-924
Mar 9, 2026
Merged

fix: enable leader to recover corrupted follower after data cleanup#929
mattisonchao merged 8 commits intomainfrom
fix/follower-recovery-924

Conversation

@mattisonchao
Copy link
Copy Markdown
Member

@mattisonchao mattisonchao commented Mar 7, 2026

Motivation

When a follower restarts with clean data (NOT_MEMBER status), the leader cannot recover it and keeps failing. This happens because AppendEntries returns a generic ErrInvalidStatus that the leader can't distinguish from other errors, and the leader's FollowerCursor retains a stale cursor position so shouldSendSnapshot() skips the snapshot.

Modification

  • Introduce ErrNodeIsNotMember / CodeNodeIsNotMember (code 112) so the follower can report its NOT_MEMBER status distinctly from other invalid-status errors
  • AppendEntries now returns ErrNodeIsNotMember when the follower is in NOT_MEMBER status, instead of generic ErrInvalidStatus
  • InstallSnapshot sets the follower status to FOLLOWER after a successful snapshot install, so subsequent AppendEntries calls are accepted
  • Leader's FollowerCursor.runOnce() detects CodeNodeIsNotMember from the follower's ack stream and resets its ack offset to trigger a full snapshot on the next retry
  • toGRPCError maps the new ErrNodeIsNotMember domain error to the correct gRPC status code
  • Add TestFollower_SnapshotRecoveryFromNotMember to verify the end-to-end recovery path

Fixes #924

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>
mattisonchao and others added 3 commits March 8, 2026 03:07
…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>
@mattisonchao mattisonchao force-pushed the fix/follower-recovery-924 branch 2 times, most recently from 08fa8ba to 827c075 Compare March 7, 2026 19:27
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>
@mattisonchao mattisonchao force-pushed the fix/follower-recovery-924 branch from 827c075 to 7a4928d Compare March 7, 2026 19:27
@mattisonchao mattisonchao marked this pull request as draft March 7, 2026 19:56
@mattisonchao mattisonchao marked this pull request as ready for review March 8, 2026 15:08
@mattisonchao mattisonchao self-assigned this Mar 8, 2026
mattisonchao and others added 3 commits March 8, 2026 23:43
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>
Copy link
Copy Markdown
Member Author

@mattisonchao mattisonchao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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:

  1. AppendEntries returned a generic ErrInvalidStatus the leader couldn't distinguish
  2. The leader's FollowerCursor retained a stale cursor position, so shouldSendSnapshot() 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: InstallSnapshot sets status to FOLLOWER while holding rwMutex.Lock() (acquired at line 506, deferred unlock at 507), so no race with NewTerm or AppendEntries.
  • Snapshot trigger: Resetting ackOffset to InvalidOffset correctly triggers shouldSendSnapshot() via the first condition at line 171: ackOffset == wal.InvalidOffset && fc.ackTracker.CommitOffset() >= 0.
  • Goroutine safety: receiveAcks error is captured via closure + wg.Wait(), which provides happens-before guarantee. No goroutine leak — cancel() unblocks stream.Recv().
  • Error propagation through multierr.Combine: When receiveAcks returns CodeNodeIsNotMember, it calls cancel(), which makes streamEntriesLoop return nil (context.Canceled is now filtered). So multierr.Combine(ErrNodeIsNotMember, nil) = ErrNodeIsNotMember, and status.Code() in the backoff handler correctly matches.
  • toGRPCError ordering: ErrNodeIsNotMember is checked before ErrInvalidStatus in 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.

@mattisonchao mattisonchao merged commit afafc14 into main Mar 9, 2026
9 checks passed
@mattisonchao mattisonchao deleted the fix/follower-recovery-924 branch March 9, 2026 00:39
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Leader cannot recover corrupted follower after data cleanup

1 participant