-
Notifications
You must be signed in to change notification settings - Fork 3.9k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
kvserver: perform initial upreplication of non-voters synchronously #63215
kvserver: perform initial upreplication of non-voters synchronously #63215
Conversation
fa5ea31
to
53a2875
Compare
53a2875
to
419b985
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.
Nice job on this. It looks like an overall simplification.
Reviewed 12 of 12 files at r1.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15 and @andreimatei)
pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go, line 581 at r1 (raw file):
n1.Exec(t, `ALTER TABLE test EXPERIMENTAL_RELOCATE NON_VOTERS VALUES (ARRAY[3], 1)`) // Wait until the new non-voter is upreplicated to n3.
💯
pkg/kv/kvserver/client_merge_test.go, line 4517 at r1 (raw file):
// We're controlling merge queue operation via // `store.SetMergeQueueActive`, so enable the cluster setting here. _, err := tc.ServerConn(0).Exec(`SET CLUSTER SETTING kv.range_merge.queue_enabled=true`)
Do we need this? Doesn't this default to true?
pkg/kv/kvserver/merge_queue.go, line 358 at r1 (raw file):
// TODO(aayush): Merges are indeed stable now, we can be smarter here about // which errors should be marked as purgatory-worthy. log.Warningf(ctx, "%v", err)
Don't we already log on non-benign queue errors in baseQueue.finishProcessingReplica
?
pkg/kv/kvserver/raft_snapshot_queue.go, line 113 at r1 (raw file):
snapType := SnapshotRequest_VIA_SNAPSHOT_QUEUE if typ := repDesc.GetType(); typ == roachpb.LEARNER || typ == roachpb.NON_VOTER {
Can you think of anything problematic that could come out of this new ability for the snapshot queue to skip NON_VOTER snapshots? I want to make sure we aren't accidentally introducing any new starvation conditions which could prevent an initialized NON_VOTER that has fallen behind and needs a snapshot from receiving one.
pkg/kv/kvserver/replica_command.go, line 1047 at r1 (raw file):
reason, details, targets.voterAdditions,
adds
?
pkg/kv/kvserver/replica_command.go, line 1061 at r1 (raw file):
} for _, target := range adds { r.tryRollBackLearnerReplica(ctx, r.Desc(), target, reason, details)
We discussed over a call that we're going to do this for NON_VOTERS as well, so we can pull this logic back inside addAndInitializeRaftLearners
.
pkg/kv/kvserver/replica_command.go, line 1112 at r1 (raw file):
reason, details, targets.nonVoterAdditions,
adds
?
pkg/kv/kvserver/replica_command.go, line 1544 at r1 (raw file):
// addAndInitializeRaftLearners adds etcd learners (LEARNERs or NON_VOTERs) to // the given replication targets and synchronously sends them an initial // snapshot to upreplicate.
Could you mention what a caller can expect when this method returns? Something about how the targets will all be initialized.
3e0b144
to
adebb0e
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.
The latest patch looks good, though I'm not sure it addressed all of the comments.
Reviewed 5 of 5 files at r2.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15 and @andreimatei)
I was going to write a test for this comment of yours before updating the PR:
|
a58d09d
to
4e0d106
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.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @andreimatei and @nvanbenschoten)
pkg/kv/kvserver/client_merge_test.go, line 4517 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Do we need this? Doesn't this default to true?
For some reason, I thought that ReplicationManual
disables the merge queue_enabled
cluster setting. Removed.
pkg/kv/kvserver/merge_queue.go, line 358 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Don't we already log on non-benign queue errors in
baseQueue.finishProcessingReplica
?
We do in general, but not for purgatory errors. I'm not sure why, though, since it seems like in most cases we do want these errors in the logs. See:
cockroach/pkg/kv/kvserver/queue.go
Lines 1084 to 1094 in ea9074b
if purgErr, ok := isPurgatoryError(err); ok { | |
bq.mu.Lock() | |
bq.addToPurgatoryLocked(ctx, stopper, repl, purgErr) | |
bq.mu.Unlock() | |
return | |
} | |
// If not a benign or purgatory error, log. | |
if !benign { | |
log.Errorf(ctx, "%v", err) | |
} |
pkg/kv/kvserver/raft_snapshot_queue.go, line 113 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Can you think of anything problematic that could come out of this new ability for the snapshot queue to skip NON_VOTER snapshots? I want to make sure we aren't accidentally introducing any new starvation conditions which could prevent an initialized NON_VOTER that has fallen behind and needs a snapshot from receiving one.
Added a test for this scenario. LMK if you dislike.
pkg/kv/kvserver/replica_command.go, line 1047 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
adds
?
Done.
pkg/kv/kvserver/replica_command.go, line 1061 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
We discussed over a call that we're going to do this for NON_VOTERS as well, so we can pull this logic back inside
addAndInitializeRaftLearners
.
Done.
pkg/kv/kvserver/replica_command.go, line 1112 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
adds
?
Done.
pkg/kv/kvserver/replica_command.go, line 1544 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Could you mention what a caller can expect when this method returns? Something about how the targets will all be initialized.
Done.
4f6a89f
to
55f3a9d
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.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @andreimatei and @nvanbenschoten)
pkg/kv/kvserver/client_merge_test.go, line 4517 at r1 (raw file):
Previously, aayushshah15 (Aayush Shah) wrote…
For some reason, I thought that
ReplicationManual
disables the mergequeue_enabled
cluster setting. Removed.
I kept missing this, but turns out ReplicationManual
does disable the merge_queue. See: https://github.com/cockroachdb/cockroach/blob/55f3a9d48537e019905b34b90e628d9ecfd4ddb8/pkg/testutils/testcluster/testcluster.go#L315-L325
55f3a9d
to
e975a19
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.
Reviewed 11 of 11 files at r3.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @aayushshah15 and @andreimatei)
pkg/kv/kvserver/merge_queue.go, line 358 at r1 (raw file):
Previously, aayushshah15 (Aayush Shah) wrote…
We do in general, but not for purgatory errors. I'm not sure why, though, since it seems like in most cases we do want these errors in the logs. See:
cockroach/pkg/kv/kvserver/queue.go
Lines 1084 to 1094 in ea9074b
if purgErr, ok := isPurgatoryError(err); ok { bq.mu.Lock() bq.addToPurgatoryLocked(ctx, stopper, repl, purgErr) bq.mu.Unlock() return } // If not a benign or purgatory error, log. if !benign { log.Errorf(ctx, "%v", err) }
Ack. Thanks for the explanation.
pkg/kv/kvserver/replica_learner_test.go, line 387 at r3 (raw file):
scratchStartKey := tc.ScratchRange(t) atomic.StoreInt64(&skipInitialSnapshot, 1) _, err := tc.AddNonVoters(scratchStartKey, false, tc.Target(1))
The waitForInit
argument seems really unfortunate. It's semantics aren't particularly easy to understand and it's only needed in this very specific case. Ideally, we wouldn't bloat the TestCluster
API to accommodate this one test, placing the burden on every other test to understand what this argument means. Instead, we'd be able to accomplish what it wants through some combination of testing knobs and maybe with some concurrency. For instance, could we put this in a goroutine, have some waiting mechanism in the main goroutine, manually enqueue, and then ensure that this eventually returns?
e975a19
to
6fca1ce
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.
Thanks for the reviews on this. I'll hold off on merging this until tomorrow in case you wanted one more look.
Reviewable status:
complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @aayushshah15, @andreimatei, and @nvanbenschoten)
pkg/kv/kvserver/replica_learner_test.go, line 387 at r3 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
The
waitForInit
argument seems really unfortunate. It's semantics aren't particularly easy to understand and it's only needed in this very specific case. Ideally, we wouldn't bloat theTestCluster
API to accommodate this one test, placing the burden on every other test to understand what this argument means. Instead, we'd be able to accomplish what it wants through some combination of testing knobs and maybe with some concurrency. For instance, could we put this in a goroutine, have some waiting mechanism in the main goroutine, manually enqueue, and then ensure that this eventually returns?
Heh, I was worried you wouldn't be a fan of this. Thanks for that suggestion!
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 8 of 8 files at r4.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @aayushshah15 and @andreimatei)
pkg/testutils/testcluster/testcluster.go, line 623 at r4 (raw file):
} if err := tc.waitForNewReplicas(startKey, false /* waitForVoter */, targets...); err != nil {
We seem to have lost a pair of comments in this file.
Resolves cockroachdb#63199 Before this commit, we relied on the raft snapshot queue to asynchronously perform the initial upreplication of non-voting replicas. This meant that by the time `AdminChangeReplicas` (and consequently, `AdminRelocateRange`) returned to its client, non-voters were not guaranteed to have been initialized. This was a deliberate decision and was, thus far, believed to be copacetic. However, this decision subtly made range merges (of ranges that have any number of non-voters) extremely unlikely to suceed, while causing severe disruption on foreground traffic on the right hand side of a merge. This was because the `mergeQueue` will first call `AdminRelocateRange` on the right hand side range in order to collocate its replicas with the replicas of the left hand side range. If the `mergeQueue` happened to relocate any non-voting replicas, they were likely to still be waiting for their initial snapshot by the time the `AdminMerge` attempted to subsume the RHS. Essentially, this meant that we were subsuming the RHS of a merge while some of its replicas weren't even initialized. This would cause the merge to fail and, in the interim, block all traffic over the RHS range for a 5 second window. This commit fixes the unfortunate sequence of events described above by making the behavior of `AdminChangeReplicas` more symmetric for voting and non-voting replicas. Now, if `AdminChangeReplicas` successfully returns, its client can safely assume that all new replicas have at least been upreplicated via an initial snapshot. Release note: None
6fca1ce
to
c52b01f
Compare
bors r+ |
Build failed (retrying...): |
bors r+ |
Already running a review |
Build succeeded: |
Resolves #63199
Before this commit, we relied on the raft snapshot queue to
asynchronously perform the initial upreplication of non-voting replicas.
This meant that by the time
AdminChangeReplicas
(and consequently,AdminRelocateRange
) returned to its client, non-voters were notguaranteed to have been initialized. This was a deliberate decision and
was, thus far, believed to be copacetic. However, this decision subtly
made range merges (of ranges that have any number of non-voters)
extremely unlikely to suceed, while causing severe disruption on
foreground traffic on the right hand side of a merge. This was because
the
mergeQueue
will first callAdminRelocateRange
on the right handside range in order to collocate its replicas with the replicas of the
left hand side range. If the
mergeQueue
happened to relocate anynon-voting replicas, they were likely to still be waiting for their
initial snapshot by the time the
AdminMerge
attempted to subsume theRHS. Essentially, this meant that we were subsuming the RHS of a merge
while some of its replicas weren't even initialized. This would cause
the merge to fail and, in the interim, block all traffic over the RHS
range for a 5 second window.
This commit fixes the unfortunate sequence of events described above by
making the behavior of
AdminChangeReplicas
more symmetric for votingand non-voting replicas. Now, if
AdminChangeReplicas
successfullyreturns, its client can safely assume that all new replicas have at
least been upreplicated via an initial snapshot.
Release note: None