-
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
storage: use learner replicas for replica addition by default #39034
Conversation
@tbg I'm working through the test failures and wondering if you had thoughts on one of them. TestConcurrentRaftSnapshots will sometimes flake because it stalls out on line cockroach/pkg/storage/client_raft_test.go Line 1117 in e0044e8
It looks like the learner will get added, then the snapshot run, then the promotion txn will commit, all fine. But some of the replicas never get the new desc from the promotion txn. I put a log that prints every time a desc gets updated from ReplicaState and sure enough it will fire for 3 or 4 of the 5 replicas. I also printed out all the conf changes that get handed to raft and they work okay. Any thoughts on where it might be getting dropped? |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hmm, that does seem weird. Maybe one place things could be going wrong is replica quiescence?
cockroach/pkg/storage/replica_raft_quiesce.go
Lines 342 to 352 in 1d443c5
msg := raftpb.Message{ | |
From: uint64(r.mu.replicaID), | |
To: id, | |
Type: raftpb.MsgHeartbeat, | |
Term: status.Term, | |
Commit: commit, | |
} | |
if !r.maybeCoalesceHeartbeat(ctx, msg, toReplica, fromReplica, quiesce) { | |
log.Fatalf(ctx, "failed to coalesce known heartbeat: %v", msg) | |
} |
If "somehow" the range quiesces before all of the followers are caught up (maybe because it ignores the followers while they're learners and then doesn't realize that they're not any more)? Hmm that code does seem to do the right thing though:
cockroach/pkg/storage/replica_raft_quiesce.go
Lines 268 to 282 in 1d443c5
} else if progress.Match != status.Applied { | |
// Skip any node in the descriptor which is not live. | |
if livenessMap != nil && !livenessMap[rep.NodeID].IsLive { | |
if log.V(4) { | |
log.Infof(ctx, "skipping node %d because not live. Progress=%+v", | |
rep.NodeID, progress) | |
} | |
continue | |
} | |
if log.V(4) { | |
log.Infof(ctx, "not quiescing: replica %d match (%d) != applied (%d)", | |
rep.ReplicaID, progress.Match, status.Applied) | |
} | |
return nil, false | |
} |
Unless the newly added followers don't seem live for some reason, but I don't see why this is changing in this PR.
If it's not the quiescence I'd be interested in the raft traffic between leader and stuck followers, probably vmodule=raft=*
.
Reviewable status: complete! 0 of 0 LGTMs obtained
Aha! The replica somehow ends up needing a snap, which we now refuse to do because it's a learner, and it ends up permastuck. If I comment out the learner checks in the raft snap queue, the test is fine again. The next question I have is why the log is getting truncated to the point that it's cutting off this replica. IIRC, there's some code in the raft log queue to avoid truncating past an outstanding snapshot, I wonder how that plays with all the new learner stuff (new snap type, 2 txns, etc). We'd talked about revisiting the decision to not send raft snaps to learners for other reasons (https://reviewable.io/reviews/cockroachdb/cockroach/38149#-LjzozKd16T0XgR_r0zL), so it may be time to hash that out. What are our options?
I'm not thrilled with any of those. I'm going to look more closely at exactly why it needs the raft snap when this test flakes |
Okay, don't think anything is truncating the log. I threw a panic in batcheval.TruncateLog and it didn't fire. So maybe sometimes raft thinks it needs a snap after the learner snap is applied? I wonder why |
I think I have this figured out. Basically, raft sends a MsgApp to the learner before it finishes getting the learner snapshot and a MsgAppResp is enqueued in response with a rejection (Hint: 0). The snapshot then finishes applying. The MsgAppResp is received by the leader, which marks the learner as needing a snap (which we refuse to send from the raft log queue to a learner). Nothing ever tells the leader it no longer needs the snap. @tbg thoughts on what we should be doing here? How does raft normally find out that a snap got applied? Does some message normally get sent as part of cockroach/pkg/storage/store.go Line 3425 in 1db1dd6
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Interesting. So what usually happens is that upon applying a snapshot, the recipient generates a MsgAppResp
. When the leader receives that, it starts replicating to the follower again:
The reason this doesn't work in your example above is that Raft is waiting for a snap to apply at index 32, but the learner snapshot is at index 31. The follower could now catch up via the log, but Raft isn't smart enough to figure that out itself; it has decided that a snap at 32 will need to happen but it won't.
There is (*RawNode).ReportSnapshot()
which is basically a way to tell Raft that you did send a snapshot. What this does is move the follower back to probing status, which means the leader will try to append again:
(Followers that the leader tracks as needing a snapshot are never probed, see: https://github.com/etcd-io/etcd/blob/b9c051e7a7de2973197ff12adce3da0ca3ae92ac/raft/raft.go#L442-L446 where IsPaused()
is always true then)
If you called into ReportSnapshot()
after successfully sent snapshots (or we make raft smarter about switching out of StateSnapshot), it should avoid this particular problem, but I think that's just putting a band-aid on a fracture. I think at this point we want to walk back the decision to not send snaps to learners from the queue.
I looked at bit at synchronizing this via the map here:
cockroach/pkg/storage/replica.go
Lines 231 to 243 in 1d443c5
// A map of raft log index of pending preemptive snapshots to deadlines. | |
// Used to prohibit raft log truncations that would leave a gap between | |
// the snapshot and the new first index. The map entry has a zero | |
// deadline while the snapshot is being sent and turns nonzero when the | |
// snapshot has completed, preventing truncation for a grace period | |
// (since there is a race between the snapshot completing and its being | |
// reflected in the raft status used to make truncation decisions). | |
// | |
// NB: If we kept only one value, we could end up in situations in which | |
// we're either giving some snapshots no grace period, or keep an | |
// already finished snapshot "pending" for extended periods of time | |
// (preventing log truncation). | |
snapshotLogTruncationConstraints map[uuid.UUID]snapTruncationInfo |
but that's annoying and ideally we will also get rid of that map somehow. That leaves two options that I'm aware of:
I think that could be done. We stash the recipient replicaID in (the map is populated in GetSnapshot()
, so it's already active for any snap) and then we can tell the raft log queue to simply no-op on replicas that already have an inflight snap, handling the (desired) case in which the explicit learner snap goes out first. If the queue wins though (and I think you observed that this can happen frequently, which makes sense), instead of sending the learner snap we have to couple tightly with the queue again, which sucks.
I was thinking that we could grab the snapshot early (ie. so that we'd always beat the queue) but then we're back to preemptive snapshots, where we have a desc on the follower that doesn't contain the follower itself, and we want to get rid of that. (Which reminds me: we wanted to add a check that snaps that don't contain the recipient in their desc error out early). We could work around this by making a "preliminary" snapshot truncation constraint at index zero which can be upgraded only by the explicit snapshot send. Then we can grab the preliminary constraint before adding the learner (thus winning against the queue which won't attempt to upgrade, also this sort of constraint should not impact the truncation heuristics since it's at index zero, but that's something to verify), then upgrade it with the explicit learner snap so that the log queue understands what's going on, and then release everything at the end.
See if you like any of that, maybe we also need to bite the bullet and just let the log queue do the job, but I still feel that that'll be a bigger ball of hair than what I'm outlining above.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @danhhz)
pkg/sql/ambiguous_commit_test.go, line 60 at r1 (raw file):
// uniqueness constraints, or duplicate key violations. See #6053, #7604, and // #10023. func TestDanStressAmbiguousCommit(t *testing.T) {
Loving the new test names here. (I know why you're doing this and sadly it's temporary)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Heh, sorry about the non sequitur in the last comment. Just to be clear, the map is my preferred option at this moment. The other option is coupling tightly with the queue but I have lots of worries about that so I'd prefer not to.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @danhhz)
Just making sure I'm on the same page, you mean the I'm still feeling my way out here, so this isn't a firm opinion, but I initially lean toward trying to do this by calling into the queue. Reusing |
1e60f43
to
32274f1
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
slowing but surely making progress here
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @danhhz)
pkg/storage/client_raft_test.go, line 345 at r2 (raw file):
// a vote from r2. When r2 starts up, it still has itself as a learner in the // engine and refuses to vote. Deadlock. t.Skip(`WIP possible bug in etcd/raft`)
@tbg thoughts here?
pkg/storage/client_split_test.go, line 488 at r2 (raw file):
if err := tc.ServerConn(i).QueryRow(` SELECT count(*), sum(value) FROM crdb_internal.node_metrics WHERE name = 'range.snapshots.normal-applied'
this still doesn't work because a raft snap could happen during the initial replication (when the snapshot queue wins the race vs promoteToVoter). i think i could fix the test by asserting that range.snapshots.normal-applied
is the same before&after the splits. is that good enough?
pkg/storage/replica_command.go, line 1379 at r2 (raw file):
priority SnapshotRequest_Priority, ) (retErr error) { // WIP any reason not to do this here instead of in the callers?
question for ya
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Didn't give this an in-depth look today, but the snapshot index mutual exclusion looks good.
Reviewed 4 of 11 files at r1, 9 of 9 files at r2.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @danhhz)
pkg/storage/client_raft_test.go, line 345 at r2 (raw file):
Previously, danhhz (Daniel Harrison) wrote…
@tbg thoughts here?
Yep, that's Raft bug number two that you've found (first one is "learner can't receive snapshot"). I'll look into it.
pkg/storage/client_split_test.go, line 488 at r2 (raw file):
Previously, danhhz (Daniel Harrison) wrote…
this still doesn't work because a raft snap could happen during the initial replication (when the snapshot queue wins the race vs promoteToVoter). i think i could fix the test by asserting that
range.snapshots.normal-applied
is the same before&after the splits. is that good enough?
Yeah, that works for me.
pkg/storage/raft_snapshot_queue.go, line 118 at r2 (raw file):
// There is a snapshot being transferred. It's probably a LEARNER snap, so // bail for now and try again later. log.Infof(ctx, "not sending snapshot type RAFT to learner: %s", repDesc)
Info is pretty loud for this, I assume this is just your WIP debugging though (here and elsewhere)
pkg/storage/replica_command.go, line 1379 at r2 (raw file):
Previously, danhhz (Daniel Harrison) wrote…
question for ya
If both calls are basically sendSnapshot(); reportSnapshotStatus()
then yes, seems much better to do it here.
3f889de
to
c4dcc42
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
down to one test (TestMergeQueue), though it looks like we're getting some failures on teamcity that haven't flaked locally. i haven't looked into them yet. maybe i can get this ready for final review tomorrow 🤞
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @danhhz and @tbg)
pkg/storage/raft_snapshot_queue.go, line 118 at r2 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Info is pretty loud for this, I assume this is just your WIP debugging though (here and elsewhere)
nope, this one I intended. this is a reasonably infrequent race and there's been enough real bugs found around this when deflaking the tests that i think it's worth leaving in
which other ones are you concerned about?
pkg/storage/replica_command.go, line 1379 at r2 (raw file):
Previously, tbg (Tobias Grieger) wrote…
If both calls are basically
sendSnapshot(); reportSnapshotStatus()
then yes, seems much better to do it here.
Done
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewed 10 of 10 files at r3.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @danhhz)
pkg/storage/raft_log_queue.go, line 253 at r3 (raw file):
// (including only resetting when the follower resumes replicating). // // WIP anything to do here?
I was too optimistic when the TODO was written. PendingSnapshot
is initialized by the leader when it realizes the follower needs a snapshot, and it isn't initialized with the index of the snapshot that is actually sent by us (out of band), which likely is lower. So I don't think we should touch this now. Mind converting the TODO into a note on why we don't rely on PendingSnapshot (basically the above reason)?
pkg/storage/raft_snapshot_queue.go, line 118 at r2 (raw file):
Then make the message more descriptive, right now it'll be confusing for anyone who isn't either of us:
skipping snapshot; replica is likely a learner in the process of being added
which other ones are you concerned about?
I thought there were others, but doesn't look like it. Logging when we roll back a learner is definitely appropriate.
pkg/storage/raft_snapshot_queue.go, line 124 at r3 (raw file):
// learner snap at an index before the one raft wanted here. Raft 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 nothing ever
This shouldn't be what really happens. The follower would apply the snapshot, but the leader would keep thinking that it still needs a snapshot. The raft snap queue should thus eventually pick up the replica again and send another snapshot (this time not blocked by the pending snapshot map). The reason this "isn't happening" might be because unless someone requests a snapshot, it happens only driven by the scanner, so it might be minutes.
8409a7f
to
00cb1f7
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This should be ready to go
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @danhhz and @tbg)
pkg/sql/ambiguous_commit_test.go, line 60 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Loving the new test names here. (I know why you're doing this and sadly it's temporary)
RIP
pkg/storage/raft_log_queue.go, line 253 at r3 (raw file):
Previously, tbg (Tobias Grieger) wrote…
I was too optimistic when the TODO was written.
PendingSnapshot
is initialized by the leader when it realizes the follower needs a snapshot, and it isn't initialized with the index of the snapshot that is actually sent by us (out of band), which likely is lower. So I don't think we should touch this now. Mind converting the TODO into a note on why we don't rely on PendingSnapshot (basically the above reason)?
Done.
pkg/storage/raft_snapshot_queue.go, line 118 at r2 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Then make the message more descriptive, right now it'll be confusing for anyone who isn't either of us:
skipping snapshot; replica is likely a learner in the process of being added
which other ones are you concerned about?
I thought there were others, but doesn't look like it. Logging when we roll back a learner is definitely appropriate.
Done.
pkg/storage/raft_snapshot_queue.go, line 124 at r3 (raw file):
Previously, tbg (Tobias Grieger) wrote…
This shouldn't be what really happens. The follower would apply the snapshot, but the leader would keep thinking that it still needs a snapshot. The raft snap queue should thus eventually pick up the replica again and send another snapshot (this time not blocked by the pending snapshot map). The reason this "isn't happening" might be because unless someone requests a snapshot, it happens only driven by the scanner, so it might be minutes.
Done.
pkg/storage/replica_learner_test.go, line 418 at r4 (raw file):
testutils.SucceedsSoon(t, func() error { for _, p := range repl.RaftStatus().Progress { if p.State == tracker.StateSnapshot {
if you're curious, had to switch how this test worked because handing the failed snapshot status back to raft keeps it from getting into StateSnapshot
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
👍 glad to see that there weren't many changes outside of testing code.
It's likely appropriate to rename the log truncation pending snapshot index map to just the pending snapshot map, but honestly I'm pretty eager for you to get this merged, so I'd defer that to a follow-up.
Also, I have more work going into etcd/raft, but just bump to what's there and go ahead with the merge; I expect to need to bump repeatedly over the next few weeks.
Reviewed 9 of 14 files at r4, 5 of 5 files at r5.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @danhhz)
Thanks for all the reviews! bors r=tbg |
Build failed |
eek
|
Heh, I don't fully trust
|
@nvanbenschoten I'm pretty sure what's happening here is that the snapshot does not remove the existing raft log. That way the last index is computed from the raft log, but there shouldn't be a raft log. |
(I checked - the raft log is there. It should not be). |
Maybe this code here got lost https://github.com/cockroachdb/cockroach/pull/38932/files#diff-2a869a1b2e22f71ebc4ca49503fb0338L863 |
This should be handled by https://github.com/cockroachdb/cockroach/pull/38932/files#diff-2a869a1b2e22f71ebc4ca49503fb0338R847. I wonder why that isn't working correctly. |
It sure seems like we do try to cover the raft log with an SST:
Data dir attached, r3 is affected (r1 is also corrupted, as Only 2 SSTs in that directory. Maybe something got compacted in the split second before the crash or after startup? Somehow that doesn't sound likely. Anyway, it's too late here for me to poke around but I assume @nvanbenschoten and @jeffrey-xiao will figure this one out |
This was previously available by flipping a cluster setting that defaulted to off, this defaults it to on. With any luck, we'll be confident enough in this to remove the cluster setting in 19.2, which will allow us to rip out a bunch of code at the beginning of the 20.1 cycle. Closes cockroachdb#38902 Release note (general change): Replicas are now added using a raft learner and going through the normal raft snapshot process to catch them up, eliminating technical debt. No user facing changes are expected.
Awesome, thanks for looking into this! It would have taken me much longer to make heads or tails of it. bors r=tbg |
Build failed |
lol what a terrifying pr. i'm done for the day, so this is going to have to go on ice until monday
|
^- this was possibly just very bad luck, this looks like it's #21146 which I haven't seen in quite a while. Going to try repro'ing this on your branch. BTW master is busted right now due to some Rocks-level fallout so maybe we'll hold off for a day regardless while we get our ducks in a row on master |
This reverts commit 717c185. Apparently we violate the assertions. This needs to be fixed, but until then, let's keep the ball rolling. One likely culprit is cockroachdb#38932, see: cockroachdb#39034 (comment) Release note: None
Stressed this test for 96 minutes without a repro. I used your HEAD, not the result of the merge, so maybe that makes a difference. |
39562: Revert "c-deps: fix assertion-enabled builds" r=knz a=tbg This reverts commit 717c185. Apparently we violate the assertions. This needs to be fixed, but until then, let's keep the ball rolling. The assertion failures typically take the form > L0 file with seqno 90 90 vs. file with global_seqno 90 SIGABRT: abort See for example #39559 One likely culprit is #38932, see: #39034 (comment) Release note: None Co-authored-by: Tobias Schottdorf <tobias.schottdorf@gmail.com>
I think I understand the test failure to some degree, though I have to look more. There's hope that we can reliably nail this in a test, so we should attempt to do that. The scenario here is n1 is the leader and n2 is follower r40 is initially [a,z). r68@n1[replicate] removes the replica on n2 (abandoned learner). We're not talking replicaGC here, just removing from the descriptor. The crash itself is caused by this cockroach/pkg/storage/replica_init.go Lines 179 to 181 in 7dcf66d
We have a replica and give it a replicaID (determined by r40 when it ran the split), but the replica has since been readded multiple times, so its actual replicaID is higher. This is basically the problem in #21146 (comment). A test reproducing this would
The one snag is that cockroach/pkg/storage/client_merge_test.go Lines 2231 to 2236 in b320ff5
@danhhz, would you mind giving that a try? Solving this problem will ultimately need a bit more thought, unfortunately, so this PR is now blocked. |
The right hand side of a split can be readded before the split trigger fires, in which case the split trigger fails. See [bug description]. I [suggested] a test to reprduce this bug "properly", so we should look into that. In the meantime, it'll be good to see that this passes tests. I verified manually that setting `minReplicaID` to some large number before the call to `rightRng.initRaftMuLockedReplicaMuLocked` reproduces the symptoms prior to this commit, but that doesn't come as a surprise nor does it prove that the fix works flawlessly. [bug description]: cockroachdb#21146 (comment) [suggested]: cockroachdb#39034 (comment) Fixes cockroachdb#21146. Release note (bug fix): Fixed a rare panic (message: "raft group deleted") that could occur during splits.
39571: storage: avoid RaftGroupDeletedError from RHS in splitTrigger r=bdarnell a=tbg The right hand side of a split can be readded before the split trigger fires, in which case the split trigger fails. See [bug description]. I [suggested] a test to reprduce this bug "properly", so we should look into that. In the meantime, it'll be good to see that this passes tests. I verified manually that setting `minReplicaID` to some large number before the call to `rightRng.initRaftMuLockedReplicaMuLocked` reproduces the symptoms prior to this commit, but that doesn't come as a surprise nor does it prove that the fix works flawlessly. [bug description]: #21146 (comment) [suggested]: #39034 (comment) Fixes #21146. Release note (bug fix): Fixed a rare panic (message: "raft group deleted") that could occur during splits. Co-authored-by: Tobias Schottdorf <tobias.schottdorf@gmail.com>
39034: storage: use learner replicas for replica addition by default r=tbg a=danhhz This was previously available by flipping a cluster setting that defaulted to off, this defaults it to on. With any luck, we'll be confident enough in this to remove the cluster setting in 19.2, which will allow us to rip out a bunch of code at the beginning of the 20.1 cycle. Closes #38902 Release note (general change): Replicas are now added using a raft learner and going through the normal raft snapshot process to catch them up, eliminating technical debt. No user facing changes are expected. 39469: opt: fix panic due to incorrect type of ArrayFlatten r=rytaft a=rytaft This commit fixes a panic caused by incorrect typing of an `ArrayFlatten` expression. If the input to an `ArrayFlatten` expression is sorted, there may be more than one output column (although the columns used for sorting are hidden). If one of these hidden columns is chosen to infer the type of the expression, the type could be incorrect. This commit fixes the problem so that only the requested column is chosen for type inference. Fixes #38867 Release note (bug fix): Fixed a panic due to incorrect type inference of some ARRAY(...) expressions. Co-authored-by: Daniel Harrison <daniel.harrison@gmail.com> Co-authored-by: Rebecca Taft <becca@cockroachlabs.com>
Build succeeded |
This was previously available by flipping a cluster setting that
defaulted to off, this defaults it to on. With any luck, we'll be
confident enough in this to remove the cluster setting in 19.2, which
will allow us to rip out a bunch of code at the beginning of the 20.1
cycle.
Closes #38902
Release note (general change): Replicas are now added using a raft
learner and going through the normal raft snapshot process to catch them
up, eliminating technical debt. No user facing changes are expected.