Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

storage: use learner replicas for replica addition by default #39034

Merged
merged 1 commit into from
Aug 13, 2019

Conversation

danhhz
Copy link
Contributor

@danhhz danhhz commented Jul 22, 2019

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.

@cockroach-teamcity
Copy link
Member

This change is Reviewable

@danhhz
Copy link
Contributor Author

danhhz commented Jul 22, 2019

@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

mtc.replicateRange(1, 1, 2, 3, 4)

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?

Copy link
Member

@tbg tbg left a 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?

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:

} 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: :shipit: complete! 0 of 0 LGTMs obtained

@danhhz
Copy link
Contributor Author

danhhz commented Jul 23, 2019

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?

  1. Let them be duplicated. Wasteful (and causes flaky tests)
  2. Run the learner snap though the snap queue (probably requires a minor queue refactor. also i'd have to think through the various races, what if raft tries to queue it after we sent the learner snap?)
  3. Something similar to what we're doing now, but with a timeout since the replica got added (gross and how do you tune the timeout)

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

@danhhz
Copy link
Contributor Author

danhhz commented Jul 23, 2019

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

@danhhz
Copy link
Contributor Author

danhhz commented Jul 23, 2019

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

if _, expl, err := r.handleRaftReadyRaftMuLocked(ctx, inSnap); err != nil {
?

I190723 18:06:15.760863 39 storage/replica_command.go:1220  [s1,r1/1:/M{in-ax}] change replicas SUCCESS (ADD_REPLICA (n4,s4):4LEARNER): updated descriptor r1:/M{in-ax} [(n1,s1):1, (n2,s2):2, (n3,s3):3, (n4,s4):4LEARNER, next=5, gen=5]
I190723 18:06:15.761725 100 storage/raft.go:170  [s1,r1/1:/M{in-ax}] raft ready (must-sync=true)
  Outgoing Message[2]: 1->4 MsgApp Term:6 Log:6/31 Commit:31 Entries:[6/32 EntryNormal [62657b74dcd50745] [115]]
I190723 18:06:15.780203 101 storage/raft.go:170  [s1,r1/1:/M{in-ax}] raft ready (must-sync=false)
  Outgoing Message[1]: 1->4 MsgHeartbeat Term:6 Log:0/0
I190723 18:06:15.871591 39 storage/store_snapshot.go:775  [s1,r1/1:/M{in-ax}] sending LEARNER snapshot d55f9797 at applied index 31
I190723 18:06:15.871434 337 storage/store.go:3314  [s4,r1/4:{-}] incoming raft message:
1->4 MsgApp Term:6 Log:6/31 Commit:31 Entries:[6/32 EntryNormal [62657b74dcd50745] [115]]
I190723 18:06:15.871916 39 storage/store_snapshot.go:818  [s1,r1/1:/M{in-ax}] streamed snapshot to (n4,s4):4: kv pairs: 70, log entries: 0, rate-limit: 8.0 MiB/sec, 0.11s
I190723 18:06:15.871867 337 vendor/go.etcd.io/etcd/raft/raft.go:658  [s4,r1/4:{-}] 4 became follower at term 0
I190723 18:06:15.872103 337 vendor/go.etcd.io/etcd/raft/raft.go:380  [s4,r1/4:{-}] newRaft 4 [peers: [], term: 0, commit: 0, applied: 0, lastindex: 0, lastterm: 0]
I190723 18:06:15.872136 337 vendor/go.etcd.io/etcd/raft/raft.go:658  [s4,r1/4:{-}] 4 became follower at term 1
I190723 18:06:15.872172 337 vendor/go.etcd.io/etcd/raft/raft.go:808  [s4,r1/4:{-}] 4 [term: 1] received a MsgApp message with higher term from 1 [term: 6]
I190723 18:06:15.872206 337 vendor/go.etcd.io/etcd/raft/raft.go:658  [s4,r1/4:{-}] 4 became follower at term 6
I190723 18:06:15.872247 337 vendor/go.etcd.io/etcd/raft/raft.go:1297  [s4,r1/4:{-}] 4 [logterm: 0, index: 31] rejected msgApp [logterm: 6, index: 31] from 1
I190723 18:06:15.872295 337 storage/store.go:3314  [s4,r1/4:{-}] incoming raft message:
1->4 MsgHeartbeat Term:6 Log:0/0
I190723 18:06:15.872363 337 storage/raft.go:170  [s4,r1/4:{-}] raft ready (must-sync=true)
  Outgoing Message[0]: 4->1 MsgAppResp Term:6 Log:0/31 Rejected (Hint: 0)
  Outgoing Message[1]: 4->1 MsgHeartbeatResp Term:6 Log:0/0
I190723 18:06:15.872514 1315 vendor/go.etcd.io/etcd/raft/log.go:300  [s4,r1/4:{-}] log [committed=0, applied=0, unstable.offset=1, len(unstable.Entries)=0] starts to restore snapshot [index: 31, term: 6]
I190723 18:06:15.874499 102 storage/store.go:3314  [s1,r1/1:/M{in-ax}] incoming raft message:
4->1 MsgAppResp Term:6 Log:0/31 Rejected (Hint: 0)
I190723 18:06:15.874645 1315 vendor/go.etcd.io/etcd/raft/raft.go:1411  [s4,r1/4:{-}] 4 switched to configuration voters=(1)
I190723 18:06:15.874706 102 vendor/go.etcd.io/etcd/raft/raft.go:1040  [s1,r1/1:/M{in-ax}] 1 received msgApp rejection(lastindex: 0) from 4 for index 31
I190723 18:06:15.874882 1315 vendor/go.etcd.io/etcd/raft/raft.go:1411  [s4,r1/4:{-}] 4 switched to configuration voters=(1 2)
I190723 18:06:15.875020 1315 vendor/go.etcd.io/etcd/raft/raft.go:1411  [s4,r1/4:{-}] 4 switched to configuration voters=(1 2 3)
I190723 18:06:15.875059 1315 vendor/go.etcd.io/etcd/raft/raft.go:1411  [s4,r1/4:{-}] 4 switched to configuration voters=(1 2 3) learners=(4)
I190723 18:06:15.875074 102 vendor/go.etcd.io/etcd/raft/raft.go:1043  [s1,r1/1:/M{in-ax}] 1 decreased progress of 4 to [StateProbe match=0 next=1 learner]
I190723 18:06:15.875114 102 vendor/go.etcd.io/etcd/raft/raft.go:476  [s1,r1/1:/M{in-ax}] 1 [firstindex: 11, commit: 32] sent snapshot[index: 32, term: 6] to 4 [StateProbe match=0 next=1 learner]
I190723 18:06:15.875144 1315 vendor/go.etcd.io/etcd/raft/raft.go:1388  [s4,r1/4:{-}] 4 [commit: 31, lastindex: 31, lastterm: 6] restored snapshot [index: 31, term: 6]
I190723 18:06:15.875159 102 vendor/go.etcd.io/etcd/raft/raft.go:479  [s1,r1/1:/M{in-ax}] 1 paused sending replication messages to 4 [StateSnapshot match=0 next=1 learner paused pendingSnap=32]
I190723 18:06:15.875252 102 storage/raft.go:170  [s1,r1/1:/M{in-ax}] raft ready (must-sync=false)
  Outgoing Message[0]: 1->4 MsgSnap Term:6 Log:0/0 Snapshot:{[] {{[] [] []} 32 6 []} []}
I190723 18:06:15.875164 1315 vendor/go.etcd.io/etcd/raft/raft.go:1311  [s4,r1/4:{-}] 4 [commit: 31] restored snapshot [index: 31, term: 6]
I190723 18:06:15.875344 1315 storage/raft.go:170  [s4,r1/4:{-}] raft ready (must-sync=false)
  Outgoing Message[0]: 4->1 MsgAppResp Term:6 Log:0/31
I190723 18:06:15.875365 1315 storage/replica_raftstorage.go:827  [s4,r1/4:{-}] applying LEARNER snapshot at index 31 (id=d55f9797, encoded size=9562, 1 rocksdb batches, 0 log entries)
I190723 18:06:15.875907 1315 storage/replica_raftstorage.go:833  [s4,r1/4:/M{in-ax}] applied LEARNER snapshot in 0ms [clear=0ms batch=0ms entries=0ms commit=0ms]

Copy link
Member

@tbg tbg left a 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:

https://github.com/etcd-io/etcd/blob/b9c051e7a7de2973197ff12adce3da0ca3ae92ac/raft/raft.go#L1095-L1104

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:

https://github.com/etcd-io/etcd/blob/b9c051e7a7de2973197ff12adce3da0ca3ae92ac/raft/raft.go#L1159-L1180

(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:

// 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: :shipit: 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)

Copy link
Member

@tbg tbg left a 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: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @danhhz)

@danhhz
Copy link
Contributor Author

danhhz commented Jul 24, 2019

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.

Just making sure I'm on the same page, you mean the snapshotLogTruncationConstraints map (and not some other map)?

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 snapshotLogTruncationConstraints is a conceptual inter-dependance that I'd prefer to avoid. That said, I absolutely haven't worked through all the details yet. I'll try to brainstorm before our 1:1

@danhhz danhhz force-pushed the learner_on branch 2 times, most recently from 1e60f43 to 32274f1 Compare August 2, 2019 00:14
Copy link
Contributor Author

@danhhz danhhz left a 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: :shipit: 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

Copy link
Member

@tbg tbg left a 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: :shipit: 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.

@danhhz danhhz force-pushed the learner_on branch 2 times, most recently from 3f889de to c4dcc42 Compare August 7, 2019 22:46
Copy link
Contributor Author

@danhhz danhhz left a 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: :shipit: 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

Copy link
Member

@tbg tbg left a 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: :shipit: 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.

@danhhz danhhz force-pushed the learner_on branch 3 times, most recently from 8409a7f to 00cb1f7 Compare August 8, 2019 22:38
@danhhz danhhz changed the title [WIP] storage: default learners to on storage: use learner replicas for replica addition by default Aug 8, 2019
@danhhz danhhz marked this pull request as ready for review August 8, 2019 22:43
Copy link
Contributor Author

@danhhz danhhz left a 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: :shipit: 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

Copy link
Member

@tbg tbg left a 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: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @danhhz)

@danhhz
Copy link
Contributor Author

danhhz commented Aug 9, 2019

Thanks for all the reviews!

bors r=tbg

@craig
Copy link
Contributor

craig bot commented Aug 9, 2019

Build failed

@danhhz
Copy link
Contributor Author

danhhz commented Aug 9, 2019

eek

F190809 14:42:50.706904 176 vendor/go.etcd.io/etcd/raft/raft.go:1526  [n2,s2,r13/3:/Table/1{7-8}] 3 state.commit 45 is out of range [45, 27]
goroutine 176 [running]:
github.com/cockroachdb/cockroach/pkg/util/log.getStacks(0xc000436301, 0xc000436360, 0x0, 0xc00038c040)
	/go/src/github.com/cockroachdb/cockroach/pkg/util/log/clog.go:1016 +0xb1
github.com/cockroachdb/cockroach/pkg/util/log.(*loggingT).outputLogEntry(0x6c1dd20, 0xc000000004, 0x63f52a8, 0x23, 0x5f6, 0xc000a9d860, 0x46)
	/go/src/github.com/cockroachdb/cockroach/pkg/util/log/clog.go:874 +0x92b
github.com/cockroachdb/cockroach/pkg/util/log.addStructured(0x44e5220, 0xc000907e60, 0x4, 0x3, 0x3c8aef5, 0x2b, 0xc003741080, 0x4, 0x4)
	/go/src/github.com/cockroachdb/cockroach/pkg/util/log/structured.go:66 +0x2cc
github.com/cockroachdb/cockroach/pkg/util/log.logDepth(0x44e5220, 0xc000907e60, 0x2, 0xc000000004, 0x3c8aef5, 0x2b, 0xc003741080, 0x4, 0x4)
	/go/src/github.com/cockroachdb/cockroach/pkg/util/log/log.go:69 +0x8c
github.com/cockroachdb/cockroach/pkg/util/log.FatalfDepth(...)
	/go/src/github.com/cockroachdb/cockroach/pkg/util/log/log.go:199
github.com/cockroachdb/cockroach/pkg/storage.(*raftLogger).Panicf(0xc003544970, 0x3c8aef5, 0x2b, 0xc003741080, 0x4, 0x4)
	/go/src/github.com/cockroachdb/cockroach/pkg/storage/raft.go:102 +0xf1
github.com/cockroachdb/cockroach/vendor/go.etcd.io/etcd/raft.(*raft).loadState(0xc000808640, 0x7, 0x0, 0x2d, 0x0, 0x0, 0x0)
	/go/src/github.com/cockroachdb/cockroach/vendor/go.etcd.io/etcd/raft/raft.go:1526 +0x1be
github.com/cockroachdb/cockroach/vendor/go.etcd.io/etcd/raft.newRaft(0xc003825118, 0xc0039cd0c0)
	/go/src/github.com/cockroachdb/cockroach/vendor/go.etcd.io/etcd/raft/raft.go:368 +0xe8d
github.com/cockroachdb/cockroach/vendor/go.etcd.io/etcd/raft.NewRawNode(0xc0039cd118, 0xc003544970, 0xc0000560a0, 0x44e5220)
	/go/src/github.com/cockroachdb/cockroach/vendor/go.etcd.io/etcd/raft/rawnode.go:51 +0x39
github.com/cockroachdb/cockroach/pkg/storage.(*Replica).withRaftGroupLocked(0xc000423800, 0x3da2b00, 0xc003825778, 0xc003733778, 0x722101)
	/go/src/github.com/cockroachdb/cockroach/pkg/storage/replica_raft.go:1287 +0x297
github.com/cockroachdb/cockroach/pkg/storage.(*Replica).withRaftGroup(0xc000423800, 0xc003733700, 0xc003733778, 0x0, 0x0)
	/go/src/github.com/cockroachdb/cockroach/pkg/storage/replica_raft.go:1331 +0x8c
github.com/cockroachdb/cockroach/pkg/storage.(*Replica).stepRaftGroup(0xc000423800, 0xc0033cee00, 0x44a2ce0, 0xc000423800)
	/go/src/github.com/cockroachdb/cockroach/pkg/storage/replica_raft.go:385 +0x66
github.com/cockroachdb/cockroach/pkg/storage.(*Store).processRaftRequestWithReplica(0xc000c3e700, 0x44e5220, 0xc003743ec0, 0xc000423800, 0xc0033cee00, 0x80260a)
	/go/src/github.com/cockroachdb/cockroach/pkg/storage/store.go:3361 +0x1c0
github.com/cockroachdb/cockroach/pkg/storage.(*Store).processRequestQueue.func1(0x44e5220, 0xc003743ec0, 0xc000423800, 0x44e5220)
	/go/src/github.com/cockroachdb/cockroach/pkg/storage/store.go:3551 +0x9c
github.com/cockroachdb/cockroach/pkg/storage.(*Store).withReplicaForRequest(0xc000c3e700, 0x44e5220, 0xc003743ec0, 0xc0033cee00, 0xc003733e98, 0x0)
	/go/src/github.com/cockroachdb/cockroach/pkg/storage/store.go:3318 +0x150
github.com/cockroachdb/cockroach/pkg/storage.(*Store).processRequestQueue(0xc000c3e700, 0x44e5220, 0xc0004f8090, 0xd)
	/go/src/github.com/cockroachdb/cockroach/pkg/storage/store.go:3545 +0x200
github.com/cockroachdb/cockroach/pkg/storage.(*raftScheduler).worker(0xc000894380, 0x44e5220, 0xc0004f8090)
	/go/src/github.com/cockroachdb/cockroach/pkg/storage/scheduler.go:238 +0x214
github.com/cockroachdb/cockroach/pkg/storage.(*raftScheduler).Start.func2(0x44e5220, 0xc0004f8090)
	/go/src/github.com/cockroachdb/cockroach/pkg/storage/scheduler.go:161 +0x3e
github.com/cockroachdb/cockroach/pkg/util/stop.(*Stopper).RunWorker.func1(0xc0002b0250, 0xc0001b77a0, 0xc0002b0240)
	/go/src/github.com/cockroachdb/cockroach/pkg/util/stop/stopper.go:196 +0xfb
created by github.com/cockroachdb/cockroach/pkg/util/stop.(*Stopper).RunWorker
	/go/src/github.com/cockroachdb/cockroach/pkg/util/stop/stopper.go:189 +0xa8

@tbg
Copy link
Member

tbg commented Aug 9, 2019

Heh, I don't fully trust ./cockroach debug check-store, but some of those messages certainly mirror the crash:

$ ./cockroach  debug check-store /mnt/data1/cockroach/
W190809 19:55:58.618389 1 storage/engine/rocksdb.go:116  [rocksdb] [db/version_set.cc:3086] More existing levels in DB than needed. max_bytes_for_level_multiplier may not be guaranteed.
range 1: log index anomaly: 98 followed by 109
range 3: truncated index 130 should equal first index 117 - 1
range 3: applied index 130 should be between first index 117 and last index 119
range 3: committed index 130 ahead of last index  119
range 14: truncated index 23 should equal first index 0 - 1
range 14: applied index 23 should be between first index 0 and last index 0
range 14: committed index 23 ahead of last index  0
range 1: truncated index 108 should equal first index 85 - 1
range 11: truncated index 26 should equal first index 0 - 1
range 11: applied index 26 should be between first index 0 and last index 0
range 11: committed index 26 ahead of last index  0

@tbg
Copy link
Member

tbg commented Aug 9, 2019

@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.

@tbg
Copy link
Member

tbg commented Aug 9, 2019

(I checked - the raft log is there. It should not be).

@tbg
Copy link
Member

tbg commented Aug 9, 2019

@nvanbenschoten
Copy link
Member

This should be handled by https://github.com/cockroachdb/cockroach/pull/38932/files#diff-2a869a1b2e22f71ebc4ca49503fb0338R847. I wonder why that isn't working correctly.

@tbg
Copy link
Member

tbg commented Aug 9, 2019

It sure seems like we do try to cover the raft log with an SST:

func TestX(t *testing.T) {
    const rangeID = 1

    unreplicatedPrefixKey := keys.MakeRangeIDUnreplicatedPrefix(rangeID)
    unreplicatedStart := MakeMVCCMetadataKey(unreplicatedPrefixKey)
    unreplicatedEnd := MakeMVCCMetadataKey(unreplicatedPrefixKey.PrefixEnd())

    from, to := EncodeKey(unreplicatedStart), EncodeKey(unreplicatedEnd)
    k := EncodeKey(MakeMVCCMetadataKey(keys.RaftLogKey(rangeID, 1)))
    t.Errorf("from, k, to:\n%v\n%v\n%v", from, k, to)
}

    engine_test.go:834: from, k, to:
        [1 105 137 117 0]
        [1 105 137 117 114 102 116 108 0 0 0 0 0 0 0 1 0]
        [1 105 137 118 0]

Data dir attached, r3 is affected (r1 is also corrupted, as debug check-store will tell you).

repro-data.zip

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.
@danhhz
Copy link
Contributor Author

danhhz commented Aug 9, 2019

I verified this crash on master (0efea05), so you can fix up the vendor conflict and bors it again.

Awesome, thanks for looking into this! It would have taken me much longer to make heads or tails of it.

bors r=tbg

@craig
Copy link
Contributor

craig bot commented Aug 10, 2019

Build failed

@danhhz
Copy link
Contributor Author

danhhz commented Aug 10, 2019

lol what a terrifying pr. i'm done for the day, so this is going to have to go on ice until monday

F190809 23:48:49.155079 3886 storage/store.go:2142  [n2,s2,r40/3:/Table/60{-/1/3/4}] raft group deleted
goroutine 3886 [running]:
github.com/cockroachdb/cockroach/pkg/util/log.getStacks(0xc0003c6301, 0xc0003c6300, 0x0, 0xc0004432a0)
        /go/src/github.com/cockroachdb/cockroach/pkg/util/log/clog.go:1016 +0xb1
github.com/cockroachdb/cockroach/pkg/util/log.(*loggingT).outputLogEntry(0x6474b20, 0xc000000004, 0x5dec9c0, 0x10, 0x85e, 0xc0038ab600, 0x33)
        /go/src/github.com/cockroachdb/cockroach/pkg/util/log/clog.go:874 +0x92b
github.com/cockroachdb/cockroach/pkg/util/log.addStructured(0x40dae80, 0xc002afa780, 0xc000000004, 0x2, 0x0, 0x0, 0xc003320730, 0x1, 0x1)
        /go/src/github.com/cockroachdb/cockroach/pkg/util/log/structured.go:66 +0x2cc
github.com/cockroachdb/cockroach/pkg/util/log.logDepth(0x40dae80, 0xc002afa780, 0x1, 0x4, 0x0, 0x0, 0xc003320730, 0x1, 0x1)
        /go/src/github.com/cockroachdb/cockroach/pkg/util/log/log.go:69 +0x8c
github.com/cockroachdb/cockroach/pkg/util/log.Fatal(...)
        /go/src/github.com/cockroachdb/cockroach/pkg/util/log/log.go:189
github.com/cockroachdb/cockroach/pkg/storage.splitPostApply(0x40dae80, 0xc002afa780, 0x0, 0x15b9660798941a50, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
        /go/src/github.com/cockroachdb/cockroach/pkg/storage/store.go:2142 +0x217
github.com/cockroachdb/cockroach/pkg/storage.(*Replica).handleSplitResult(...)
        /go/src/github.com/cockroachdb/cockroach/pkg/storage/replica_application_result.go:233
github.com/cockroachdb/cockroach/pkg/storage.(*replicaStateMachine).handleNonTrivialReplicatedEvalResult(0xc004b150c0, 0x40dae80, 0xc002afa780, 0x0, 0x0, 0xc0043a8420, 0x0, 0x0, 0x0, 0x0, ...)
        /go/src/github.com/cockroachdb/cockroach/pkg/storage/replica_application_state_machine.go:904 +0x852
github.com/cockroachdb/cockroach/pkg/storage.(*replicaStateMachine).ApplySideEffects(0xc004b150c0, 0x40e7cc0, 0xc0035bb008, 0x0, 0x0, 0x0, 0x0)
        /go/src/github.com/cockroachdb/cockroach/pkg/storage/replica_application_state_machine.go:817 +0x72d
github.com/cockroachdb/cockroach/pkg/storage/apply.mapCheckedCmdIter(0x7ff14cbf47f0, 0xc004b15288, 0xc0033213d8, 0x0, 0x0, 0x0, 0x0)
[00:00:17] :                     [TestInitialPartitioning/multi_col_list_partitioning_-_DEFAULT_DEFAULT] [Test Output]
        /go/src/github.com/cockroachdb/cockroach/pkg/storage/apply/cmd.go:175 +0x11b
github.com/cockroachdb/cockroach/pkg/storage/apply.(*Task).applyOneBatch(0xc003321800, 0x40dae80, 0xc002afa780, 0x4107860, 0xc004b15228, 0x0, 0x0)
        /go/src/github.com/cockroachdb/cockroach/pkg/storage/apply/task.go:168 +0x21e
github.com/cockroachdb/cockroach/pkg/storage/apply.(*Task).ApplyCommittedEntries(0xc003321800, 0x40dae80, 0xc002afa780, 0x0, 0x0)
        /go/src/github.com/cockroachdb/cockroach/pkg/storage/apply/task.go:134 +0xcf
github.com/cockroachdb/cockroach/pkg/storage.(*Replica).handleRaftReadyRaftMuLocked(0xc004b15000, 0x40dae80, 0xc002afa780, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
        /go/src/github.com/cockroachdb/cockroach/pkg/storage/replica_raft.go:729 +0xa44
github.com/cockroachdb/cockroach/pkg/storage.(*Store).processRequestQueue.func1(0x40dae80, 0xc002afa780, 0xc004b15000, 0x40dae80)
        /go/src/github.com/cockroachdb/cockroach/pkg/storage/store.go:3557 +0x131
github.com/cockroachdb/cockroach/pkg/storage.(*Store).withReplicaForRequest(0xc001991100, 0x40dae80, 0xc002afa780, 0xc0029bfc00, 0xc005c33e98, 0x0)
        /go/src/github.com/cockroachdb/cockroach/pkg/storage/store.go:3318 +0x150
github.com/cockroachdb/cockroach/pkg/storage.(*Store).processRequestQueue(0xc001991100, 0x40dae80, 0xc000a56570, 0x28)
        /go/src/github.com/cockroachdb/cockroach/pkg/storage/store.go:3545 +0x200
github.com/cockroachdb/cockroach/pkg/storage.(*raftScheduler).worker(0xc0009e0380, 0x40dae80, 0xc000a56570)
        /go/src/github.com/cockroachdb/cockroach/pkg/storage/scheduler.go:238 +0x214
github.com/cockroachdb/cockroach/pkg/storage.(*raftScheduler).Start.func2(0x40dae80, 0xc000a56570)
        /go/src/github.com/cockroachdb/cockroach/pkg/storage/scheduler.go:161 +0x3e
github.com/cockroachdb/cockroach/pkg/util/stop.(*Stopper).RunWorker.func1(0xc0001fbb40, 0xc00139b320, 0xc0001fbb30)
        /go/src/github.com/cockroachdb/cockroach/pkg/util/stop/stopper.go:196 +0xfb
created by github.com/cockroachdb/cockroach/pkg/util/stop.(*Stopper).RunWorker
        /go/src/github.com/cockroachdb/cockroach/pkg/util/stop/stopper.go:189 +0xa8

@tbg
Copy link
Member

tbg commented Aug 12, 2019

^- 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

tbg added a commit to tbg/cockroach that referenced this pull request Aug 12, 2019
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
@tbg
Copy link
Member

tbg commented Aug 12, 2019

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.

craig bot pushed a commit that referenced this pull request Aug 12, 2019
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>
@tbg
Copy link
Member

tbg commented Aug 12, 2019

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).
r40@n1[replicate] adds n2 as a learner.
r40@n2 applies the learner snap.
r40@n1[split] inits a split (rhs=r68)
r40@n1[replicate] tries to upgrade n2 to voter, and fails since the descriptor changed.
r40@n1[split] finishes the split. n2 is now a learner on both r40 and r68, though it's uninitialized on the RHS.
r40@n1[replicate] tries to roll back the learner on n2, this fails due to desc change, so it abandons it for the next loop of the replicate queue (this is the last action of r40 before the crash)

r68@n1[replicate] removes the replica on n2 (abandoned learner). We're not talking replicaGC here, just removing from the descriptor.
Now r68@n1 repeatedly adds n2 as a learner, fails on the snap (because it intersects r40@n2) and removes the learner again, until the crash occurs.

The crash itself is caused by this

if replicaID < r.mu.minReplicaID {
return &roachpb.RaftGroupDeletedError{}
}

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

  1. set up three nodes and a scratch range
  2. make one node (say n2 as above) a learner
  3. partition off range 40 traffic to n2
  4. split the range (n2 won't know)
  5. remove the learner of the RHS from n2
  6. readd it (note that n2 will know because we're not blocking those msgs)
  7. unblock r40

The one snag is that *TestCluster doesn't let us do this selective dropping of traffic. multiTestContext does, but all that's needed to port this is the ability to set a store's raft Transport as done here:

// Start dropping all Raft traffic to the LHS replica on store2 so that it
// won't be aware that there is a merge in progress.
mtc.transport.Listen(store2.Ident.StoreID, &unreliableRaftHandler{
rangeID: lhsDesc.RangeID,
RaftMessageHandler: store2,
})
. I think that is straightforward to make available on the test cluster as well.

@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.

tbg added a commit to tbg/cockroach that referenced this pull request Aug 12, 2019
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.
craig bot pushed a commit that referenced this pull request Aug 13, 2019
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>
@tbg
Copy link
Member

tbg commented Aug 13, 2019

With #39571 merged and #39619 about to, I think we can get this in at last.

bors r+

craig bot pushed a commit that referenced this pull request Aug 13, 2019
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>
@craig
Copy link
Contributor

craig bot commented Aug 13, 2019

Build succeeded

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

storage: learners defaulted on
4 participants