-
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
kv/kvserver: TestAdminRelocateRange failed #84242
Comments
kv/kvserver.TestAdminRelocateRange failed with artifacts on master @ 158ebe679bee1386df875ef8a175963bc6ac0cd1:
Parameters: |
Refs: cockroachdb#84242 Reason: flaky test Generated by bin/skip-test. Release justification: non-production code changes Release note: None
This test is fairly flaky even though we've not seen recent pings to this isse - https://teamcity.cockroachdb.com/project.html?projectId=Cockroach_Ci_TestsGcpLinuxX8664BigVm&buildTypeId=&tab=testDetails&testNameId=-5086970665568297757&order=TEST_STATUS_DESC&branch_Cockroach_Ci_TestsGcpLinuxX8664BigVm=__all_branches__&itemsCount=50, we're going to skip this until its stabilized. |
87280: kv/kvserver: skip TestAdminRelocateRange r=tbg a=adityamaru Refs: #84242 Reason: flaky test Generated by bin/skip-test. Release justification: non-production code changes Release note: None Co-authored-by: adityamaru <[email protected]>
When stressing, after a few hundred runs this test fails. The end result is correct:
However one of the atomic swaps fails
due to the incoming voter (s5) being in
This same behavior occurs with s2 being removed as the leaseholder, however with s6 being added instead. The response to a replicia being in state probe is expected, however why is the replica which swaps with the leaseholder (voter incoming) behind - when it was just streamed a snapshot. |
I don't know enough about the raft state machine, but it's possible that all replicas transition through Looking at |
I think @andrewbaptist is on the right track. When a snapshot is completed successfully, CRDB calls etcd/raft's So there does seem to be a race here if the snapshot response races ahead of the I don't understand why I ask because I wonder whether it would be worth avoiding this race by either synthesizing a |
That is a good question. This decision was made very early in the life of raft1 and I don't see a justification for it. At the time it was likely just as good to do either. Today, at least in CRDB, it seems a lot better to move to replicating state directly, which we already do (thanks to changes we made) on the MsgAppResp path2. Looking at this code I think there's an actual problem that is unrelated to the issue at hand but is worse. What raft really wants when it asks for a snapshot is for a snapshot at a particular index3. We ignore that and send a snapshot at whatever the index is when we call So if we ended up sending a snapshot that has a lower index than what raft requested, we'd have the leader believe that entries that are not in fact reflected on the followers state actually made it there. You can from that derive scenarios in which raft "commits" an entry that isn't on a quorum. Now without delegated follower snaps, this couldn't happen (applied index doesn't move backwards within an instance of the progress) but could it happen with delegation? The leaseholder is asked for a snapshot at 100, it asks a follower that is behind to send a snapshot and so it does, but maybe it only has applied the log up to index 50 and thus sends a snapshot at 50. To the best of my knowledge, there is no protection here - I like the first alternative in this issue, which is to fix it at the level of raft. If we end up deciding to do that, we should also move straight to StateReplicate. However, I think for 22.2 we need a more targeted solution. We could insert a bounded wait for the follower to leave StateProbe as part of the snapshot sending step. Footnotes
|
My recollection is that the split between ReportSnapshot and MsgAppResp was intended to permit things like delegated snapshots at a different index: raft asks for a snapshot without specifying exactly when, a snapshot is sent, and then the receiving node reports its new state. However, this was incompletely thought out and as @tbg says it got things exactly backwards. We should be getting the matched index from MsgAppResp instead of from when we sent the snapshot (or enforce that the delegated snapshot is not sent at an index that's too low), while it should be fine to optimistically go to StateReplicate (StateReplicate vs StateProbe is just a performance optimization). |
This is interesting, as it hints at an answer to the question of "why do both Would it be correct to say that the reason we need If so, then maybe the "sending back the corresponding This relates to:
If we had such a delay to allow the |
Yes, that's exactly right. In I would like to get to the point where
That seems like a good short-term solution that walks us towards adding that index to I think I still favor making raft transition immediately to After we call |
Most recent comment here: #87581 (comment) I'm back to being in favor of what Nathan originally suggested, which is transporting the MsgAppResp to the leader so that it can be |
re: the flakiness level, yeah it does seem extremely reduced (if not to practically zero). The historical flake reports in the issue above took ~2s to run, meaning they weren't on super overloaded machines. I'm currently stressing this test at an intensity that closely reflects that (~288 runs over 10 minutes) and have yet to see a failure, stressing with higher |
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.
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]>
The combination of #106793 and (for a possibly much rarer failure mode) etcd-io/raft#84 should likely fix this issue "holistically". Since both are on Pavel's docket for review and realistically speaking they may only merge after my departure, I'm assigning this issue to Pavel. |
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.
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]>
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.
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]>
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]>
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]>
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]>
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]>
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]>
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]>
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]>
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]>
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]>
kv/kvserver.TestAdminRelocateRange failed with artifacts on master @ 571bfa3afb3858ae84d8a8fcdbb0a38e058402a5:
Parameters:
TAGS=bazel,gss
Help
See also: How To Investigate a Go Test Failure (internal)
This test on roachdash | Improve this report!
Jira issue: CRDB-17542
The text was updated successfully, but these errors were encountered: