-
Notifications
You must be signed in to change notification settings - Fork 3.8k
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
kvserver: avoid unnecessary raft snapshots #97971
Labels
C-bug
Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior.
Comments
This comment was marked as resolved.
This comment was marked as resolved.
tbg
added
C-bug
Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior.
T-kv-replication
labels
Mar 3, 2023
cc @cockroachdb/replication |
tbg
added a commit
to tbg/cockroach
that referenced
this issue
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 issue
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 <[email protected]>
tbg
added a commit
to tbg/cockroach
that referenced
this issue
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 issue
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 <[email protected]> Co-authored-by: Andrew Baptist <[email protected]>
THardy98
pushed a commit
to THardy98/cockroach
that referenced
this issue
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 issue
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 <[email protected]>
erikgrinaker
pushed a commit
to erikgrinaker/raft
that referenced
this issue
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 <[email protected]> Signed-off-by: Tobias Grieger <[email protected]>
erikgrinaker
added a commit
to erikgrinaker/raft
that referenced
this issue
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 <[email protected]> Signed-off-by: Tobias Grieger <[email protected]>
erikgrinaker
added a commit
to erikgrinaker/raft
that referenced
this issue
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 <[email protected]> Signed-off-by: Tobias Grieger <[email protected]>
erikgrinaker
added a commit
to erikgrinaker/raft
that referenced
this issue
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 <[email protected]> Signed-off-by: Tobias Grieger <[email protected]>
erikgrinaker
added a commit
to erikgrinaker/raft
that referenced
this issue
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 <[email protected]> Signed-off-by: Tobias Grieger <[email protected]>
erikgrinaker
added a commit
to erikgrinaker/raft
that referenced
this issue
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 <[email protected]> Signed-off-by: Tobias Grieger <[email protected]>
erikgrinaker
added a commit
to erikgrinaker/raft
that referenced
this issue
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 <[email protected]> Signed-off-by: Tobias Grieger <[email protected]>
erikgrinaker
added a commit
to erikgrinaker/raft
that referenced
this issue
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 <[email protected]> Signed-off-by: Tobias Grieger <[email protected]>
erikgrinaker
added a commit
to erikgrinaker/raft
that referenced
this issue
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 <[email protected]> Signed-off-by: Tobias Grieger <[email protected]>
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Labels
C-bug
Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior.
Fix the reasons for unnecessary snapshots and land #87554. See that PR for discussion.
Originally posted by @andrewbaptist in #87554 (comment)
Jira issue: CRDB-24995
The text was updated successfully, but these errors were encountered: