Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

kv: Test to verify raft snapshots are not needed #87554

Closed

Conversation

andrewbaptist
Copy link
Collaborator

@andrewbaptist andrewbaptist commented Sep 8, 2022

This commit introduces a simple test that attempts to create a single
range and add it to two followers. The expectation is that this should
succeed using the replicate queue to send the snapshots. However when
this test is run under --stress, it will fail occasionally due to an
incorrect state transition related to the StateProbe state.

Release note: None
Epic: none

@cockroach-teamcity
Copy link
Member

This change is Reviewable

@andrewbaptist
Copy link
Collaborator Author

This commit introduces a simple test that attempts to create a single
range and add it to two followers. The expectation is that this should
succeed using the replicate queue to send the snapshots. However when
this test is run under --stress, it will fail occasionally due to an
incorrect state transition related to the StateProbe state.

Release note: None
Epic: none
@andrewbaptist andrewbaptist marked this pull request as ready for review January 24, 2023 18:15
@andrewbaptist andrewbaptist requested a review from a team as a code owner January 24, 2023 18:15
@andrewbaptist
Copy link
Collaborator Author

@tbg I wanted to get this test pushed because it will be useful to have this fixed to handle some of the more strict tests related to when we need snapshots. While I understand there are cases where Raft Snapshots are sent unnecessarily, reducing these cases will help allow the writing of less flaky and more strict tests. Since you and @pavelkalinnikov are cleaning up some code related to Raft, having this test might be a useful thing to fix.

// From n2's perspective, it receives the MsgApp prior to the initial snapshot.
// This results in it responding with a rejected MsgApp. Later it receives the
// snapshot and correctly applies it. However, when n1 sees the rejected MsgApp,
// it moves n2 status to StateProbe and stops sending Raft updates to it as it
Copy link
Member

Choose a reason for hiding this comment

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

Not quite, in StateProbe the follower is contacted with each heartbeat and this would resolve. I think it's going straight to StateSnapshot.

Copy link
Member

Choose a reason for hiding this comment

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

Ah no, you are right - we do turn into a Probe first. But, since an uninited replica has lastIndex 0, the rejection hint is zero and we can't cover that from the log, so we end up in StateSnap when trying to append. This part works correctly.

@tbg
Copy link
Member

tbg commented Jan 30, 2023

Sorry you had to wrestle with this. I've cut my teeth on this too1 and back then wasn't able to put a band-aid on it. And I think over the years folks have spent time on it too, like here:

log.VEventf(ctx, 2, "%v", err)
// TODO(dan): This is super brittle and non-obvious. In the common case,
// this check avoids duplicate work, but in rare cases, we send the
// learner snap at an index before the one raft wanted here. The raft
// leader should be able to use logs to get the rest of the way, but it
// doesn't try. In this case, skipping the raft snapshot would mean that
// we have to wait for the next scanner cycle of the raft snapshot queue
// to pick it up again. So, punt the responsibility back to raft by
// telling it that the snapshot failed. If the learner snap ends up being
// sufficient, this message will be ignored, but if we hit the case
// described above, this will cause raft to keep asking for a snap and at
// some point the snapshot lock above will be released and we'll fall
// through to the logic below.
repl.reportSnapshotStatus(ctx, repDesc.ReplicaID, err)

I came away thinking that it would be better if raft continued probing followers even when they needed a snapshot, which would iron out this race in a reliable way. The interface raft offers right now is just not good. You have to call rawNode.ReportSnapshot2 which then moves the follower back to probing, but if an old rejection arrives, I think it'll go straight back to MsgSnap. I think we could do better if the index passed to ReportSnapshot were guaranteed to be durable, so we could move straight to StateReplicate, but this is a) a behavior change that causes bugs when forgotten about and b) just ... generally more brittle; we should not have to reason about durability in userspace.

Something I don't quite understand is - when the follower applies a snapshot, it sends an MsgAppResp for the snapshot index back. Yes this could be lost - but assuming it isn't, shouldn't this avoid any problem? An MsgAppResp should prove to raft that this follower is replicating. So regardless of ordering between the MsgApp(rej=true) and MsgApp(ack=N) we should be good: either raft (should) move the follower from StateSnap to StateReplicate, or we ignore the rejection since the rejection is not relevant to the index we're currently applying at.

So there's probably an inefficiency in the raft code handling the rejections. Looking at the code, I think it's here:

https://github.com/etcd-io/raft/blob/eb615dae0bdf57ca57494d870926a7bffdad0649/raft.go#L1410-L1414

We won't transition out of StateSnap into StateReplicate if the MsgAppResp(ack=N) is below the index of the snapshot raft requested. Because raft requests snapshots via the raft.Storage implementation, this basically takes the "most recent" state machine and so in case of the race in which the rejection arrives first the second requested snapshot is ~always ahead of the first sent (INITIAL) snapshot.

We should check if this test still flakes with the TODO addressed (which is a very small change, though to land it we'd need to write a test). But I'd still advocate for sending probes to followers in StateSnap, for resilience. There is a lot riding on this one MsgAppResp the follower will generate upon application of the snapshot, and we shouldn't be requiring a second raft snapshot even if that MsgAppResp got dropped. We want to send a snapshot only if it is truly necessary.

Footnotes

  1. https://github.com/cockroachdb/cockroach/blob/74107c97238cc04c13bf2a61aa96a744758ebe1a/pkg/kv/kvserver/replica_probe_test.go#L61-L94

  2. https://github.com/cockroachdb/cockroach/blob/ed9928ff27dc70c6d423ec1cebb3840610e26390/pkg/kv/kvserver/replica_raft.go#L1597-L1612

@tbg
Copy link
Member

tbg commented Jan 30, 2023

Filed #96198 to track this on our end.

@tbg
Copy link
Member

tbg commented Jul 14, 2023

Have been running this test on top of #106793, where it is at least fairly reliable:

2337 runs so far, 0 failures, over 10m0s

Without that PR, it failed after 75s with a 60-s circuit breaker timeout, meaning things went wrong ~15s into the stress attempt. In other word, this was really easy to reproduce before.

edit: still going strong:

6122 runs so far, 0 failures, over 26m10s

edit2:

18067 runs so far, 0 failures, over 1h17m10s

edit3:

334186 runs completed, 0 failures, over 24h0m0s

tbg added a commit to tbg/cockroach that referenced this pull request Jul 14, 2023
This addresses the following race:

- n1 runs a ConfChange that adds n2 as a learner.
- n1 sends MsgApp to the learner.
- n1 starts the INITIAL snapshot, say at index 100.
- n2 receives n1's MsgApp. Since it's an uninitialized
  Replica and its log is empty, it rejects this MsgApp.
- n2 receives and applies the INITIAL snapshot, which prompts it to send an
  affirmative MsgAppResp to n1.
- n1's RawNode now tracks n2 as StateProbe (via call to ReportSnapshot(success))
- n1 receives the MsgApp rejection; n2 regresses to StateSnapshot because the
  rejection comes with a RejectHint (suggested next index to try) of zero,
  which is not in n1's log. In particular, the SnapshotIndex will likely be
  higher than the index of the snapshot actually sent, say 101.
- n1 receives the affirmative MsgAppResp (for index 100). However, 100 < 101
  so this is ignored and the follower remains in StateSnapshot.

With this commit, the last two steps cannot happen: n2 transitions straight to
StateReplicate because we step a copy of the affirmative MsgAppResp in. The
later rejection will be dropped, since it is stale (you can't hint at index zero
when you already have a positive index confirmed).

I will add that there is no great testing for the above other than stressing
the test with additional logging, noting the symptoms, and noting that they
disappear with this commit. Scripted testing of this code is within reach[^1]
but is outside of the scope of this PR.

[^1]: cockroachdb#105177

There is an additional bit of brittleness that is silently suppressed by this
commit, but which deserves to be fixed independently because how the problem
gets avoided seems accidental and incomplete. When raft requests a snapshot, it notes its
current LastIndex and uses it as the PendingSnapshot for the follower's
Progress.

At the time of writing, MsgAppResp that reconnect the follower to the log but
which are not greater than or equal to PendingSnapshot are ignored. In effect,
this means that perfectly good snapshots are thrown away if they happen to be a
little bit stale. In the example above, the snapshot is stale: PendingSnapshot
is 101, but the snapshot is at index 100. Then how does this commit (mostly)
fix the problem, i.e. why isn't the snapshot discarded? The key is that when we
synchronously step the MsgAppResp(100) into the leader's RawNode, the rejection
hasn't arrived yet and so the follower transitions into StateReplicate with a
Match of 100. This is then enough so that raft recognizes the rejected MsgApp
as stale (since it would regress on durably stored entries). However, there is
an alternative example where the rejection arrives earlier: after the snapshot
index has been picked, but before the follower has been transitioned into
StateReplicate. For this to have a negative effect, an entry has to be appended
to the leader's log between generating the snapshot and handling the rejection.
Without the combination of delegated snapshots and sustained write activity on
the leader, this window is small, and this combination is usually not present
in tests but it may well be relevant in "real" clusters. We track addressing
this in cockroachdb#106813.

Closes cockroachdb#87554.
Closes cockroachdb#97971.
Closes cockroachdb#84242.

Epic: None
Release note (bug fix): removed a source of unnecessary Raft snapshots during
replica movement.
tbg added a commit to tbg/raft that referenced this pull request Jul 14, 2023
Prior to this commit, the leader would not take into account snapshots reported
by a follower unless they matched or exceeded the tracked PendingSnapshot index
(which is the leader's last index at the time of requesting the snapshot). This
is too inflexible: the leader should take into account any snapshot that
reconnects the follower to its log. This PR makes that change.

In doing so, it addresses long-standing problems that we've encountered in
CockroachDB. Unless you create the snapshot immediately and locally when raft
emits an MsgSnap, it's difficult/impossible to later synthesize a snapshot at
the requested index. It is possible to get one above the requested index which
raft always accepted, but CockroachDB delegates snapshots to followers who
might be behind on applying the log, and it is awkward to have wait for log
application to send the snapshot just to satisfy an overly strict condition in
raft. Additionally, CockroachDB also sends snapshots preemptively when adding a
new replica since there are qualitative differences between an initial snapshot
and one needed to reconnect to the log and one does not want to wait for raft
to round-trip to the follower to realize that a snapshot is needed. In this
case, the sent snapshot is commonly behind the PendingSnapshot since the leader
transitions the follower into StateProbe when a snapshot is already in flight.

Touches cockroachdb/cockroach#84242.
Touches cockroachdb/cockroach#87553.
Touches cockroachdb/cockroach#87554.
Touches cockroachdb/cockroach#97971.
See also https://github.com/cockroachdb/cockroach/blob/2b91c3829270eb512c5380201c26a3d838fc567a/pkg/kv/kvserver/raft_snapshot_queue.go#L131-L143.

Signed-off-by: Tobias Grieger <tobias.b.grieger@gmail.com>
tbg added a commit to tbg/cockroach that referenced this pull request Jul 20, 2023
This addresses the following race:

- n1 runs a ConfChange that adds n2 as a learner.
- n1 sends MsgApp to the learner.
- n1 starts the INITIAL snapshot, say at index 100.
- n2 receives n1's MsgApp. Since it's an uninitialized
  Replica and its log is empty, it rejects this MsgApp.
- n2 receives and applies the INITIAL snapshot, which prompts it to send an
  affirmative MsgAppResp to n1.
- n1's RawNode now tracks n2 as StateProbe (via call to ReportSnapshot(success))
- n1 receives the MsgApp rejection; n2 regresses to StateSnapshot because the
  rejection comes with a RejectHint (suggested next index to try) of zero,
  which is not in n1's log. In particular, the SnapshotIndex will likely be
  higher than the index of the snapshot actually sent, say 101.
- n1 receives the affirmative MsgAppResp (for index 100). However, 100 < 101
  so this is ignored and the follower remains in StateSnapshot.

With this commit, the last two steps cannot happen: n2 transitions straight to
StateReplicate because we step a copy of the affirmative MsgAppResp in. The
later rejection will be dropped, since it is stale (you can't hint at index zero
when you already have a positive index confirmed).

I will add that there is no great testing for the above other than stressing
the test with additional logging, noting the symptoms, and noting that they
disappear with this commit. Scripted testing of this code is within reach[^1]
but is outside of the scope of this PR.

[^1]: cockroachdb#105177

There is an additional bit of brittleness that is silently suppressed by this
commit, but which deserves to be fixed independently because how the problem
gets avoided seems accidental and incomplete. When raft requests a snapshot, it notes its
current LastIndex and uses it as the PendingSnapshot for the follower's
Progress.

At the time of writing, MsgAppResp that reconnect the follower to the log but
which are not greater than or equal to PendingSnapshot are ignored. In effect,
this means that perfectly good snapshots are thrown away if they happen to be a
little bit stale. In the example above, the snapshot is stale: PendingSnapshot
is 101, but the snapshot is at index 100. Then how does this commit (mostly)
fix the problem, i.e. why isn't the snapshot discarded? The key is that when we
synchronously step the MsgAppResp(100) into the leader's RawNode, the rejection
hasn't arrived yet and so the follower transitions into StateReplicate with a
Match of 100. This is then enough so that raft recognizes the rejected MsgApp
as stale (since it would regress on durably stored entries). However, there is
an alternative example where the rejection arrives earlier: after the snapshot
index has been picked, but before the follower has been transitioned into
StateReplicate. For this to have a negative effect, an entry has to be appended
to the leader's log between generating the snapshot and handling the rejection.
Without the combination of delegated snapshots and sustained write activity on
the leader, this window is small, and this combination is usually not present
in tests but it may well be relevant in "real" clusters. We track addressing
this in cockroachdb#106813.

Closes cockroachdb#87554.
Closes cockroachdb#97971.
Closes cockroachdb#84242.

Epic: None
Release note (bug fix): removed a source of unnecessary Raft snapshots during
replica movement.
craig bot pushed a commit that referenced this pull request Jul 21, 2023
106793: kvserver: communicate snapshot index back along with snapshot response r=erikgrinaker a=tbg

This addresses the following race:

- n1 runs a ConfChange that adds n2 as a learner.
- n1 sends MsgApp to the learner.
- n1 starts the INITIAL snapshot, say at index 100.
- n2 receives n1's MsgApp. Since it's an uninitialized
  Replica and its log is empty, it rejects this MsgApp.
- n2 receives and applies the INITIAL snapshot, which prompts it to send an
  affirmative MsgAppResp to n1.
- n1's RawNode now tracks n2 as StateProbe (via call to ReportSnapshot(success))
- n1 receives the MsgApp rejection; n2 regresses to StateSnapshot because the
  rejection comes with a RejectHint (suggested next index to try) of zero,
  which is not in n1's log. In particular, the SnapshotIndex will likely be
  higher than the index of the snapshot actually sent, say 101.
- n1 receives the affirmative MsgAppResp (for index 100). However, 100 < 101
  so this is ignored and the follower remains in StateSnapshot.

With this commit, the last two steps cannot happen: n2 transitions straight to
StateReplicate because we step a copy of the affirmative MsgAppResp in. The
later rejection will be dropped, since it is stale (you can't hint at index zero
when you already have a positive index confirmed).

I will add that there is no great testing for the above other than stressing
the test with additional logging, noting the symptoms, and noting that they
disappear with this commit. Scripted testing of this code is within reach[^1]
but is outside of the scope of this PR.

[^1]: #105177

There is an additional bit of brittleness that is silently suppressed by this
commit, but which deserves to be fixed independently because how the problem
gets avoided seems accidental and incomplete. When raft requests a snapshot, it notes its
current LastIndex and uses it as the PendingSnapshot for the follower's
Progress.

At the time of writing, MsgAppResp that reconnect the follower to the log but
which are not greater than or equal to PendingSnapshot are ignored. In effect,
this means that perfectly good snapshots are thrown away if they happen to be a
little bit stale. In the example above, the snapshot is stale: PendingSnapshot
is 101, but the snapshot is at index 100. Then how does this commit (mostly)
fix the problem, i.e. why isn't the snapshot discarded? The key is that when we
synchronously step the MsgAppResp(100) into the leader's RawNode, the rejection
hasn't arrived yet and so the follower transitions into StateReplicate with a
Match of 100. This is then enough so that raft recognizes the rejected MsgApp
as stale (since it would regress on durably stored entries). However, there is
an alternative example where the rejection arrives earlier: after the snapshot
index has been picked, but before the follower has been transitioned into
StateReplicate. For this to have a negative effect, an entry has to be appended
to the leader's log between generating the snapshot and handling the rejection.
Without the combination of delegated snapshots and sustained write activity on
the leader, this window is small, and this combination is usually not present
in tests but it may well be relevant in "real" clusters. We track addressing
this in #106813.

Closes #87554.
Closes #97971.
Closes #84242.

Epic: None
Release note (bug fix): removed a source of unnecessary Raft snapshots during
replica movement.


Co-authored-by: Tobias Grieger <tobias.b.grieger@gmail.com>
Co-authored-by: Andrew Baptist <baptist@cockroachlabs.com>
@craig craig bot closed this in e968604 Jul 21, 2023
THardy98 pushed a commit to THardy98/cockroach that referenced this pull request Jul 24, 2023
This addresses the following race:

- n1 runs a ConfChange that adds n2 as a learner.
- n1 sends MsgApp to the learner.
- n1 starts the INITIAL snapshot, say at index 100.
- n2 receives n1's MsgApp. Since it's an uninitialized
  Replica and its log is empty, it rejects this MsgApp.
- n2 receives and applies the INITIAL snapshot, which prompts it to send an
  affirmative MsgAppResp to n1.
- n1's RawNode now tracks n2 as StateProbe (via call to ReportSnapshot(success))
- n1 receives the MsgApp rejection; n2 regresses to StateSnapshot because the
  rejection comes with a RejectHint (suggested next index to try) of zero,
  which is not in n1's log. In particular, the SnapshotIndex will likely be
  higher than the index of the snapshot actually sent, say 101.
- n1 receives the affirmative MsgAppResp (for index 100). However, 100 < 101
  so this is ignored and the follower remains in StateSnapshot.

With this commit, the last two steps cannot happen: n2 transitions straight to
StateReplicate because we step a copy of the affirmative MsgAppResp in. The
later rejection will be dropped, since it is stale (you can't hint at index zero
when you already have a positive index confirmed).

I will add that there is no great testing for the above other than stressing
the test with additional logging, noting the symptoms, and noting that they
disappear with this commit. Scripted testing of this code is within reach[^1]
but is outside of the scope of this PR.

[^1]: cockroachdb#105177

There is an additional bit of brittleness that is silently suppressed by this
commit, but which deserves to be fixed independently because how the problem
gets avoided seems accidental and incomplete. When raft requests a snapshot, it notes its
current LastIndex and uses it as the PendingSnapshot for the follower's
Progress.

At the time of writing, MsgAppResp that reconnect the follower to the log but
which are not greater than or equal to PendingSnapshot are ignored. In effect,
this means that perfectly good snapshots are thrown away if they happen to be a
little bit stale. In the example above, the snapshot is stale: PendingSnapshot
is 101, but the snapshot is at index 100. Then how does this commit (mostly)
fix the problem, i.e. why isn't the snapshot discarded? The key is that when we
synchronously step the MsgAppResp(100) into the leader's RawNode, the rejection
hasn't arrived yet and so the follower transitions into StateReplicate with a
Match of 100. This is then enough so that raft recognizes the rejected MsgApp
as stale (since it would regress on durably stored entries). However, there is
an alternative example where the rejection arrives earlier: after the snapshot
index has been picked, but before the follower has been transitioned into
StateReplicate. For this to have a negative effect, an entry has to be appended
to the leader's log between generating the snapshot and handling the rejection.
Without the combination of delegated snapshots and sustained write activity on
the leader, this window is small, and this combination is usually not present
in tests but it may well be relevant in "real" clusters. We track addressing
this in cockroachdb#106813.

Closes cockroachdb#87554.
Closes cockroachdb#97971.
Closes cockroachdb#84242.

Epic: None
Release note (bug fix): removed a source of unnecessary Raft snapshots during
replica movement.
erikgrinaker pushed a commit to erikgrinaker/raft that referenced this pull request Nov 17, 2023
Prior to this commit, the leader would not take into account snapshots reported
by a follower unless they matched or exceeded the tracked PendingSnapshot index
(which is the leader's last index at the time of requesting the snapshot). This
is too inflexible: the leader should take into account any snapshot that
reconnects the follower to its log. This PR makes that change.

In doing so, it addresses long-standing problems that we've encountered in
CockroachDB. Unless you create the snapshot immediately and locally when raft
emits an MsgSnap, it's difficult/impossible to later synthesize a snapshot at
the requested index. It is possible to get one above the requested index which
raft always accepted, but CockroachDB delegates snapshots to followers who
might be behind on applying the log, and it is awkward to have wait for log
application to send the snapshot just to satisfy an overly strict condition in
raft. Additionally, CockroachDB also sends snapshots preemptively when adding a
new replica since there are qualitative differences between an initial snapshot
and one needed to reconnect to the log and one does not want to wait for raft
to round-trip to the follower to realize that a snapshot is needed. In this
case, the sent snapshot is commonly behind the PendingSnapshot since the leader
transitions the follower into StateProbe when a snapshot is already in flight.

Touches cockroachdb/cockroach#84242.
Touches cockroachdb/cockroach#87553.
Touches cockroachdb/cockroach#87554.
Touches cockroachdb/cockroach#97971.
See also https://github.com/cockroachdb/cockroach/blob/2b91c3829270eb512c5380201c26a3d838fc567a/pkg/kv/kvserver/raft_snapshot_queue.go#L131-L143.

Signed-off-by: Tobias Grieger <tobias.b.grieger@gmail.com>
erikgrinaker pushed a commit to erikgrinaker/raft that referenced this pull request Nov 17, 2023
Prior to this commit, the leader would not take into account snapshots
reported by a follower unless they matched or exceeded the tracked
PendingSnapshot index (which is the leader's last index at the time of
requesting the snapshot). This is too inflexible: the leader should take
into account any snapshot that reconnects the follower to its log. This
PR adds a config option ResumeReplicateBelowPendingSnapshot that enables
this behavior.

In doing so, it addresses long-standing problems that we've encountered
in CockroachDB. Unless you create the snapshot immediately and locally
when raft emits an MsgSnap, it's difficult/impossible to later
synthesize a snapshot at the requested index. It is possible to get one
above the requested index which raft always accepted, but CockroachDB
delegates snapshots to followers who might be behind on applying the
log, and it is awkward to have to wait for log application to send the
snapshot just to satisfy an overly strict condition in raft.
Additionally, CockroachDB also sends snapshots preemptively when adding
a new replica since there are qualitative differences between an initial
snapshot and one needed to reconnect to the log and one does not want to
wait for raft to round-trip to the follower to realize that a snapshot
is needed. In this case, the sent snapshot is commonly behind the
PendingSnapshot since the leader transitions the follower into
StateProbe when a snapshot is already in flight.

Touches cockroachdb/cockroach#84242.
Touches cockroachdb/cockroach#87553.
Touches cockroachdb/cockroach#87554.
Touches cockroachdb/cockroach#97971.
Touches cockroachdb/cockroach#114349.
See also https://github.com/cockroachdb/cockroach/blob/2b91c3829270eb512c5380201c26a3d838fc567a/pkg/kv/kvserver/raft_snapshot_queue.go#L131-L143.

Signed-off-by: Erik Grinaker <grinaker@cockroachlabs.com>
Signed-off-by: Tobias Grieger <tobias.b.grieger@gmail.com>
erikgrinaker added a commit to erikgrinaker/raft that referenced this pull request Nov 17, 2023
A leader will not take into account snapshots reported by a follower
unless they match or exceed the tracked PendingSnapshot index (which is
the leader's last index at the time of requesting the snapshot). This is
too inflexible: the leader should take into account any snapshot that
reconnects the follower to its log. This PR adds a config option
ResumeReplicateBelowPendingSnapshot that enables this behavior.

In doing so, it addresses long-standing problems that we've encountered
in CockroachDB. Unless you create the snapshot immediately and locally
when raft emits an MsgSnap, it's difficult/impossible to later
synthesize a snapshot at the requested index. It is possible to get one
above the requested index which raft always accepted, but CockroachDB
delegates snapshots to followers who might be behind on applying the
log, and it is awkward to have to wait for log application to send the
snapshot just to satisfy an overly strict condition in raft.
Additionally, CockroachDB also sends snapshots preemptively when adding
a new replica since there are qualitative differences between an initial
snapshot and one needed to reconnect to the log and one does not want to
wait for raft to round-trip to the follower to realize that a snapshot
is needed. In this case, the sent snapshot is commonly behind the
PendingSnapshot since the leader transitions the follower into
StateProbe when a snapshot is already in flight.

Touches cockroachdb/cockroach#84242.
Touches cockroachdb/cockroach#87553.
Touches cockroachdb/cockroach#87554.
Touches cockroachdb/cockroach#97971.
Touches cockroachdb/cockroach#114349.
See also https://github.com/cockroachdb/cockroach/blob/2b91c3829270eb512c5380201c26a3d838fc567a/pkg/kv/kvserver/raft_snapshot_queue.go#L131-L143.

Signed-off-by: Erik Grinaker <grinaker@cockroachlabs.com>
Signed-off-by: Tobias Grieger <tobias.b.grieger@gmail.com>
erikgrinaker added a commit to erikgrinaker/raft that referenced this pull request Nov 17, 2023
A leader will not take into account snapshots reported by a follower
unless they match or exceed the tracked PendingSnapshot index (which is
the leader's last indexat the time of requesting the snapshot). This is
too inflexible: the leader should take into account any snapshot that
reconnects the follower to its log. This PR adds a config option
ResumeReplicateBelowPendingSnapshot that enables this behavior.

In doing so, it addresses long-standing problems that we've encountered
in CockroachDB. Unless you create the snapshot immediately and locally
when raft emits an MsgSnap, it's difficult/impossible to later
synthesize a snapshot at the requested index. It is possible to get one
above the requested index which raft always accepted, but CockroachDB
delegates snapshots to followers who might be behind on applying the
log, and it is awkward to have to wait for log application to send the
snapshot just to satisfy an overly strict condition in raft.
Additionally, CockroachDB also sends snapshots preemptively when adding
a new replica since there are qualitative differences between an initial
snapshot and one needed to reconnect to the log and one does not want to
wait for raft to round-trip to the follower to realize that a snapshot
is needed. In this case, the sent snapshot is commonly behind the
PendingSnapshot since the leader transitions the follower into
StateProbe when a snapshot is already in flight.

Touches cockroachdb/cockroach#84242.
Touches cockroachdb/cockroach#87553.
Touches cockroachdb/cockroach#87554.
Touches cockroachdb/cockroach#97971.
Touches cockroachdb/cockroach#114349.
See also https://github.com/cockroachdb/cockroach/blob/2b91c3829270eb512c5380201c26a3d838fc567a/pkg/kv/kvserver/raft_snapshot_queue.go#L131-L143.

Signed-off-by: Erik Grinaker <grinaker@cockroachlabs.com>
Signed-off-by: Tobias Grieger <tobias.b.grieger@gmail.com>
erikgrinaker added a commit to erikgrinaker/raft that referenced this pull request Nov 17, 2023
A leader will not take into account snapshots reported by a follower
unless they match or exceed the tracked PendingSnapshot index (which is
the leader's last indexat the time of requesting the snapshot). This is
too inflexible: the leader should take into account any snapshot that
reconnects the follower to its log. This PR adds a config option
ResumeReplicateBelowPendingSnapshot that enables this behavior.

In doing so, it addresses long-standing problems that we've encountered
in CockroachDB. Unless you create the snapshot immediately and locally
when raft emits an MsgSnap, it's difficult/impossible to later
synthesize a snapshot at the requested index. It is possible to get one
above the requested index which raft always accepted, but CockroachDB
delegates snapshots to followers who might be behind on applying the
log, and it is awkward to have to wait for log application to send the
snapshot just to satisfy an overly strict condition in raft.
Additionally, CockroachDB also sends snapshots preemptively when adding
a new replica since there are qualitative differences between an initial
snapshot and one needed to reconnect to the log and one does not want to
wait for raft to round-trip to the follower to realize that a snapshot
is needed. In this case, the sent snapshot is commonly behind the
PendingSnapshot since the leader transitions the follower into
StateProbe when a snapshot is already in flight.

Touches cockroachdb/cockroach#84242.
Touches cockroachdb/cockroach#87553.
Touches cockroachdb/cockroach#87554.
Touches cockroachdb/cockroach#97971.
Touches cockroachdb/cockroach#114349.
See also https://github.com/cockroachdb/cockroach/blob/2b91c3829270eb512c5380201c26a3d838fc567a/pkg/kv/kvserver/raft_snapshot_queue.go#L131-L143.

Signed-off-by: Erik Grinaker <grinaker@cockroachlabs.com>
Signed-off-by: Tobias Grieger <tobias.b.grieger@gmail.com>
erikgrinaker added a commit to erikgrinaker/raft that referenced this pull request Jan 4, 2024
A leader will not take into account snapshots reported by a follower
unless they match or exceed the tracked PendingSnapshot index (which is
the leader's last indexat the time of requesting the snapshot). This is
too inflexible: the leader should take into account any snapshot that
reconnects the follower to its log. This PR adds a config option
ResumeReplicateBelowPendingSnapshot that enables this behavior.

In doing so, it addresses long-standing problems that we've encountered
in CockroachDB. Unless you create the snapshot immediately and locally
when raft emits an MsgSnap, it's difficult/impossible to later
synthesize a snapshot at the requested index. It is possible to get one
above the requested index which raft always accepted, but CockroachDB
delegates snapshots to followers who might be behind on applying the
log, and it is awkward to have to wait for log application to send the
snapshot just to satisfy an overly strict condition in raft.
Additionally, CockroachDB also sends snapshots preemptively when adding
a new replica since there are qualitative differences between an initial
snapshot and one needed to reconnect to the log and one does not want to
wait for raft to round-trip to the follower to realize that a snapshot
is needed. In this case, the sent snapshot is commonly behind the
PendingSnapshot since the leader transitions the follower into
StateProbe when a snapshot is already in flight.

Touches cockroachdb/cockroach#84242.
Touches cockroachdb/cockroach#87553.
Touches cockroachdb/cockroach#87554.
Touches cockroachdb/cockroach#97971.
Touches cockroachdb/cockroach#114349.
See also https://github.com/cockroachdb/cockroach/blob/2b91c3829270eb512c5380201c26a3d838fc567a/pkg/kv/kvserver/raft_snapshot_queue.go#L131-L143.

Signed-off-by: Erik Grinaker <grinaker@cockroachlabs.com>
Signed-off-by: Tobias Grieger <tobias.b.grieger@gmail.com>
erikgrinaker added a commit to erikgrinaker/raft that referenced this pull request Jan 4, 2024
A leader will not take into account snapshots reported by a follower
unless they match or exceed the tracked PendingSnapshot index (which is
the leader's last indexat the time of requesting the snapshot). This is
too inflexible: the leader should take into account any snapshot that
reconnects the follower to its log. This PR adds a config option
ResumeReplicateBelowPendingSnapshot that enables this behavior.

In doing so, it addresses long-standing problems that we've encountered
in CockroachDB. Unless you create the snapshot immediately and locally
when raft emits an MsgSnap, it's difficult/impossible to later
synthesize a snapshot at the requested index. It is possible to get one
above the requested index which raft always accepted, but CockroachDB
delegates snapshots to followers who might be behind on applying the
log, and it is awkward to have to wait for log application to send the
snapshot just to satisfy an overly strict condition in raft.
Additionally, CockroachDB also sends snapshots preemptively when adding
a new replica since there are qualitative differences between an initial
snapshot and one needed to reconnect to the log and one does not want to
wait for raft to round-trip to the follower to realize that a snapshot
is needed. In this case, the sent snapshot is commonly behind the
PendingSnapshot since the leader transitions the follower into
StateProbe when a snapshot is already in flight.

Touches cockroachdb/cockroach#84242.
Touches cockroachdb/cockroach#87553.
Touches cockroachdb/cockroach#87554.
Touches cockroachdb/cockroach#97971.
Touches cockroachdb/cockroach#114349.
See also https://github.com/cockroachdb/cockroach/blob/2b91c3829270eb512c5380201c26a3d838fc567a/pkg/kv/kvserver/raft_snapshot_queue.go#L131-L143.

Signed-off-by: Erik Grinaker <grinaker@cockroachlabs.com>
Signed-off-by: Tobias Grieger <tobias.b.grieger@gmail.com>
erikgrinaker added a commit to erikgrinaker/raft that referenced this pull request Jan 5, 2024
A leader will not take into account snapshots reported by a follower
unless they match or exceed the tracked PendingSnapshot index (which is
the leader's last indexat the time of requesting the snapshot). This is
too inflexible: the leader should take into account any snapshot that
reconnects the follower to its log. This PR adds a config option
ResumeReplicateBelowPendingSnapshot that enables this behavior.

In doing so, it addresses long-standing problems that we've encountered
in CockroachDB. Unless you create the snapshot immediately and locally
when raft emits an MsgSnap, it's difficult/impossible to later
synthesize a snapshot at the requested index. It is possible to get one
above the requested index which raft always accepted, but CockroachDB
delegates snapshots to followers who might be behind on applying the
log, and it is awkward to have to wait for log application to send the
snapshot just to satisfy an overly strict condition in raft.
Additionally, CockroachDB also sends snapshots preemptively when adding
a new replica since there are qualitative differences between an initial
snapshot and one needed to reconnect to the log and one does not want to
wait for raft to round-trip to the follower to realize that a snapshot
is needed. In this case, the sent snapshot is commonly behind the
PendingSnapshot since the leader transitions the follower into
StateProbe when a snapshot is already in flight.

Touches cockroachdb/cockroach#84242.
Touches cockroachdb/cockroach#87553.
Touches cockroachdb/cockroach#87554.
Touches cockroachdb/cockroach#97971.
Touches cockroachdb/cockroach#114349.
See also https://github.com/cockroachdb/cockroach/blob/2b91c3829270eb512c5380201c26a3d838fc567a/pkg/kv/kvserver/raft_snapshot_queue.go#L131-L143.

Signed-off-by: Erik Grinaker <grinaker@cockroachlabs.com>
Signed-off-by: Tobias Grieger <tobias.b.grieger@gmail.com>
erikgrinaker added a commit to erikgrinaker/raft that referenced this pull request Jan 9, 2024
A leader will not take into account snapshots reported by a follower
unless they match or exceed the tracked PendingSnapshot index (which is
the leader's last indexat the time of requesting the snapshot). This is
too inflexible: the leader should take into account any snapshot that
reconnects the follower to its log. This PR makes that change.

In doing so, it addresses long-standing problems that we've encountered
in CockroachDB. Unless you create the snapshot immediately and locally
when raft emits an MsgSnap, it's difficult/impossible to later
synthesize a snapshot at the requested index. It is possible to get one
above the requested index which raft always accepted, but CockroachDB
delegates snapshots to followers who might be behind on applying the
log, and it is awkward to have to wait for log application to send the
snapshot just to satisfy an overly strict condition in raft.
Additionally, CockroachDB also sends snapshots preemptively when adding
a new replica since there are qualitative differences between an initial
snapshot and one needed to reconnect to the log and one does not want to
wait for raft to round-trip to the follower to realize that a snapshot
is needed. In this case, the sent snapshot is commonly behind the
PendingSnapshot since the leader transitions the follower into
StateProbe when a snapshot is already in flight.

Touches cockroachdb/cockroach#84242.
Touches cockroachdb/cockroach#87553.
Touches cockroachdb/cockroach#87554.
Touches cockroachdb/cockroach#97971.
Touches cockroachdb/cockroach#114349.
See also https://github.com/cockroachdb/cockroach/blob/2b91c3829270eb512c5380201c26a3d838fc567a/pkg/kv/kvserver/raft_snapshot_queue.go#L131-L143.

Signed-off-by: Erik Grinaker <grinaker@cockroachlabs.com>
Signed-off-by: Tobias Grieger <tobias.b.grieger@gmail.com>
erikgrinaker added a commit to erikgrinaker/raft that referenced this pull request Jan 9, 2024
A leader will not take into account snapshots reported by a follower
unless they match or exceed the tracked PendingSnapshot index (which is
the leader's last indexat the time of requesting the snapshot). This is
too inflexible: the leader should take into account any snapshot that
reconnects the follower to its log. This PR makes that change.

In doing so, it addresses long-standing problems that we've encountered
in CockroachDB. Unless you create the snapshot immediately and locally
when raft emits an MsgSnap, it's difficult/impossible to later
synthesize a snapshot at the requested index. It is possible to get one
above the requested index which raft always accepted, but CockroachDB
delegates snapshots to followers who might be behind on applying the
log, and it is awkward to have to wait for log application to send the
snapshot just to satisfy an overly strict condition in raft.
Additionally, CockroachDB also sends snapshots preemptively when adding
a new replica since there are qualitative differences between an initial
snapshot and one needed to reconnect to the log and one does not want to
wait for raft to round-trip to the follower to realize that a snapshot
is needed. In this case, the sent snapshot is commonly behind the
PendingSnapshot since the leader transitions the follower into
StateProbe when a snapshot is already in flight.

Touches cockroachdb/cockroach#84242.
Touches cockroachdb/cockroach#87553.
Touches cockroachdb/cockroach#87554.
Touches cockroachdb/cockroach#97971.
Touches cockroachdb/cockroach#114349.
See also https://github.com/cockroachdb/cockroach/blob/2b91c3829270eb512c5380201c26a3d838fc567a/pkg/kv/kvserver/raft_snapshot_queue.go#L131-L143.

Signed-off-by: Erik Grinaker <grinaker@cockroachlabs.com>
Signed-off-by: Tobias Grieger <tobias.b.grieger@gmail.com>
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.

None yet

3 participants