kvserver: synchronize replica removal with read-only requests#64324
kvserver: synchronize replica removal with read-only requests#64324craig[bot] merged 1 commit intocockroachdb:masterfrom
Conversation
41a0eaf to
257bd06
Compare
tbg
left a comment
There was a problem hiding this comment.
The code snippet that helped me understand this problem was this:
cockroach/pkg/kv/kvserver/replica_read.go
Lines 37 to 57 in 9c3ed73
We check the destroyStatus in checkExecutionCanProceed, but don't have an engine snapshot at this point; that only comes in line 57. So it's possible for a replicaGC to occur between lines 45 to 56 which would result in evaluating against an empty snapshot.
The fix looks good. There's a much worse fix that would probably fix the test but still have bugs, which is to grab an engine iterator before checkExecutionCanProceed. Since replica removal sets the destroy status before actually destroying, that should also make sure that reads under this race still get to see the pre-destroy state. But we'd still be evaluating on a destroyed replica, which is not supposed to happen and can have all kinds of issues.
The whole r.mu / readonlycmdmu / destroystatus dance seems ripe for a refactoring, but I understand that we're going to be backporting this quite a bit, so +1 on the minimal diff. But please a) update the comment on Replica.mu.destroyStatus to point out the full list of necessary locks, b) rename destroyStatus.Set to SetRaftMuReadOnlyMuLocked (I know, obnoxious, but safe). These can be done on master only to avoid backporting woes.
Let's also wait for @nvanbenschoten.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker and @nvanbenschoten)
pkg/kv/kvserver/client_relocate_range_test.go, line 381 at r1 (raw file):
if args.Ctx.Value(magicKey{}) != nil { resumeReadC := make(chan struct{}) delayReadC <- resumeReadC
You can also use a single channel:
// Read from *twice* by the goroutine we want to delay,
// once when arriving, second time just before exiting.
delayReadC := make(chan struct{})
func filter{} {
for i := 0; i < 2; i++ {
select {
case<-delayReadC
}
}
}
// main test
<-delayReadC
close(delayReadC)
I'll defer to you on whether this simplifies the test. I found the multiple channels harder to follow but that's just me. Mine is also more idempotent, though if the filter fires multiple times and the first time is for the wrong request we may not properly wait for the "correct" request to arrive; this should not be an issue here since you're doing a read request which hopefully doesn't have any internal retry errors in this test.
pkg/kv/kvserver/client_relocate_range_test.go, line 424 at r1 (raw file):
} readResC := make(chan reply) go func() {
Please use the stopper.
pkg/kv/kvserver/client_relocate_range_test.go, line 427 at r1 (raw file):
readCtx := context.WithValue(ctx, magicKey{}, struct{}{}) gArgs := getArgs(key) resp, pErr := kv.SendWrapped(readCtx, tc.Servers[0].DistSender(), gArgs)
You could go straight to the replica, as you're pinning the lease here, to avoid surprises. Or not, this is fine too.
pkg/kv/kvserver/client_relocate_range_test.go, line 441 at r1 (raw file):
tc.RemoveVotersOrFatal(t, key, tc.Target(0)) // Allow read to resume. Should return "foo".
How flaky is this without the fix? I wonder if you need to put it through the replicaGCQueue to make it very flaky. As is, I think we rely on reactive GC of tc.Target(0) following the removal. Making it extra flaky without the fix gives us extra confidence about the fix.
257bd06 to
2ff1305
Compare
erikgrinaker
left a comment
There was a problem hiding this comment.
There's a much worse fix that would probably fix the test but still have bugs, which is to grab an engine iterator
Not sure if this would help, since the replica removal is non-MVCC -- unless range clears are synchronized with active snapshots in Pebble.
The whole r.mu / readonlycmdmu / destroystatus dance seems ripe for a refactoring, but I understand that we're going to be backporting this quite a bit, so +1 on the minimal diff.
My thoughts exactly.
a) update the comment on Replica.mu.destroyStatus to point out the full list of necessary locks, b) rename destroyStatus.Set to SetRaftMuReadOnlyMuLocked (I know, obnoxious, but safe)
We don't always need to take out readOnlyCmdMu though. For example, we don't do this during creation or when removing uninitialized replicas (see comment). Although I suppose there's no harm in locking there as well, so I've updated these call sites. Will submit a separate PR with the method rename.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten and @tbg)
pkg/kv/kvserver/client_relocate_range_test.go, line 381 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
You can also use a single channel:
// Read from *twice* by the goroutine we want to delay, // once when arriving, second time just before exiting. delayReadC := make(chan struct{}) func filter{} { for i := 0; i < 2; i++ { select { case<-delayReadC } } } // main test <-delayReadC close(delayReadC)I'll defer to you on whether this simplifies the test. I found the multiple channels harder to follow but that's just me. Mine is also more idempotent, though if the filter fires multiple times and the first time is for the wrong request we may not properly wait for the "correct" request to arrive; this should not be an issue here since you're doing a read request which hopefully doesn't have any internal retry errors in this test.
Good idea, the fewer channels the better.
pkg/kv/kvserver/client_relocate_range_test.go, line 424 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Please use the stopper.
Done.
pkg/kv/kvserver/client_relocate_range_test.go, line 427 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
You could go straight to the replica, as you're pinning the lease here, to avoid surprises. Or not, this is fine too.
This stalled for some reason, will leave it as-is.
pkg/kv/kvserver/client_relocate_range_test.go, line 441 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
How flaky is this without the fix? I wonder if you need to put it through the replicaGCQueue to make it very flaky. As is, I think we rely on reactive GC of
tc.Target(0)following the removal. Making it extra flaky without the fix gives us extra confidence about the fix.
On master it's somewhat flaky, but it's pretty bad on 20.2.4 where I had to add sleeps to get it to trigger.
Fixing this is a bit iffy because with the fix we're going to be holding readOnlyCmdMu until the in-flight read request completes, which happens on close(resumeReadC). Waiting for the removal here (making it deterministic in the buggy case) will deadlock with the fix. Haven't come up with any clever ideas yet, might have to add testing knobs for it.
petermattis
left a comment
There was a problem hiding this comment.
There's a much worse fix that would probably fix the test but still have bugs, which is to grab an engine iterator
Not sure if this would help, since the replica removal is non-MVCC -- unless range clears are synchronized with active snapshots in Pebble.
Not sure I understand the point about range clears being synchronized with active snapshots in Pebble. Pebble snapshots respect every Pebble mutation operation. So if you have a Pebble snapshot open before doing a Pebble DeleteRange operation, the snapshot will not see the effect of the DeleteRange operation. This is true both of explicit snapshots, and the implicit snapshot that occurs when you open a Pebble iterator.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten and @tbg)
Right, I thought snapshots might refer to MVCC snapshots (which we can't use here), but I see that Pebble supports non-MVCC snapshots so that seems like a viable fix as well. |
2ff1305 to
814904f
Compare
nvb
left a comment
There was a problem hiding this comment.
There's a much worse fix that would probably fix the test but still have bugs, which is to grab an engine iterator before checkExecutionCanProceed.
I've wanted to do something like this for a while, as it would also make interactions with MVCC GC cleaner. But we need to make sure that we're actually capturing a stable LSM seqnum when we think we are. We've made some progress here over the past release in #58515, but there's still some work to do to eagerly capture these LSM snapshots if we want to address #55461.
The whole r.mu / readonlycmdmu / destroystatus dance seems ripe for a refactoring
I strongly agree with this. Even with this fix, it seems like requests that go through the (*Replica).executeWriteBatch path are still susceptible to some of these kinds of issues if they decide not to propose a Raft entry for one reason or another. We noticed this a little while ago in #46329 but never got around to doing anything here because we had no proof that anyone had ever actually hit this. I guess we still haven't seen this on the write path, though we could probably write a similar test to the one we added here to demonstrate issues for operations like ConditionalPuts, where a bad read would return before proposing.
There are some interesting concerns in that issue about how we need to be extra careful about the synchronization with read-write requests, because those requests establish a dependency on raft after proposing. But we see here that we grab the readOnlyCmdMu in various places below Raft. So if we weren't careful, we could introduce deadlocks.
Anyway, I'm also +1 on the minimal diff so we can backport something, but I'm hoping that we'll keep pulling on this thread, now that it's loose.
Reviewed 3 of 5 files at r1, 4 of 4 files at r2.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @erikgrinaker, and @tbg)
pkg/kv/kvserver/client_relocate_range_test.go, line 381 at r1 (raw file):
Previously, erikgrinaker (Erik Grinaker) wrote…
Good idea, the fewer channels the better.
This change seems fine. I do personally find that sending on a channel is a more readable hint as to the direction that control flow is passing (i.e. goroutine A is waiting on B, and can begin once B sends on the channel and is received by A), but last I checked, it doesn't make a difference. From the perspective of the go memory model and the go race detector, not only does an unbuffered channel_send_begin happen-before the corresponding channel_receive_end, but an unbuffered channel_receive_begin also happens-before the corresponding channel_send_end.
pkg/kv/kvserver/client_relocate_range_test.go, line 465 at r2 (raw file):
}) require.NoError(t, err) time.Sleep(3 * time.Second)
nit: 3 seconds seems a bit long. Do we lose anything with something closer to 500ms?
pkg/kv/kvserver/store_remove_replica.go, line 160 at r2 (raw file):
// Replica.raftMu and the replica is present in Store.mu.replicasByKey // (preventing any concurrent access to the replica's key range). rep.disconnectReplicationRaftMuLocked(ctx)
Do we need to grab the readOnlyCmdMu in this method anymore?
Replica removal did not synchronize with in-flight read-only requests, which could cause them to be evaluated on a removed (empty) replica, returning an empty result. This patch fixes the problem by locking `Replica.readOnlyCmdMu` during replica removal, thus either waiting for read-only requests to complete or not evaluating them. Release note (bug fix): Fixed a race condition where read-only requests during replica removal (e.g. during range merges or rebalancing) could be evaluated on the removed replica, returning an empty result.
814904f to
3357d2e
Compare
erikgrinaker
left a comment
There was a problem hiding this comment.
I'm hoping that we'll keep pulling on this thread, now that it's loose.
Agreed, these guarantees are pretty fundamental -- I'll poke a bit at that issue once this is in.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @nvanbenschoten, and @tbg)
pkg/kv/kvserver/client_relocate_range_test.go, line 381 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
This change seems fine. I do personally find that sending on a channel is a more readable hint as to the direction that control flow is passing (i.e. goroutine A is waiting on B, and can begin once B sends on the channel and is received by A), but last I checked, it doesn't make a difference. From the perspective of the go memory model and the go race detector, not only does an unbuffered channel_send_begin happen-before the corresponding channel_receive_end, but an unbuffered channel_receive_begin also happens-before the corresponding channel_send_end.
I don't really have a strong opinion, but think it's a fairly minor concern here, so will keep it as is.
pkg/kv/kvserver/client_relocate_range_test.go, line 465 at r2 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
nit: 3 seconds seems a bit long. Do we lose anything with something closer to 500ms?
Should be fine, done. CI tends to be more timing-sensitive, so I figured we'd make sure, but we're stressing these tests anyway. Will verify this for the 20.2 backport, which is a bit more timing-sensitive.
pkg/kv/kvserver/store_remove_replica.go, line 160 at r2 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Do we need to grab the
readOnlyCmdMuin this method anymore?
I was a bit hesitant to loosen any locks, but can't see why we'd need it here -- removed.
|
bors r=tbg,nvanbenschoten TFTRs! |
|
Build succeeded: |
Follow-up for cockroachdb#64324 that takes out `readOnlyCmdMu` to synchronize read-only requests with replica removals in a couple of additional spots. We now take them out at all `destroyStatus.Set()` sites, even though they may not all strictly need them. This change has already been included in backports for cockroachdb#64324. Release note: None
Follow-up for cockroachdb#64324 that takes out `readOnlyCmdMu` to synchronize read-only requests with replica removals in a couple of additional spots. We now take them out at all `destroyStatus.Set()` sites, even though they may not all strictly need them. Also fixes lock ordering wrt. `Store.mu` in `tryGetOrCreateReplica`. This change has already been included in backports for cockroachdb#64324. Release note: None
64378: kvserver: take out additional `readOnlyCmdMu` locks r=tbg,nvanbenschoten a=erikgrinaker Follow-up for #64324 that takes out `readOnlyCmdMu` to synchronize read-only requests with replica removals in a couple of additional spots. We now take them out at all `destroyStatus.Set()` sites, even though they may not all strictly need them. Also fixes lock ordering wrt. `Store.mu` in `tryGetOrCreateReplica`. This change has already been included in backports for #64324. Release note: None Co-authored-by: Erik Grinaker <grinaker@cockroachlabs.com>
Backports the test case from cockroachdb#64324 to check whether 19.2 has the replica removal read race condition. It does not appear so, as replica removal blocks until the in-flight read completes. Release note: None
64471: kvserver: synchronize replica removal with read-write requests r=tbg,nvanbenschoten a=erikgrinaker Replica removal did not synchronize with in-flight read-write requests, which could cause them to be evaluated on a removed (empty) replica. The request would not be able to persist any writes, since it's unable to submit Raft proposals. However, it can affect conditional writes, for example causing a `ConditionalPutRequest` to error because it finds a missing value instead of the expected one. This patch fixes the problem by taking out `Replica.readOnlyCmdMu` during pre-Raft evaluation, to synchronize with replica removal. This can cause such requests to return `AmbiguousResultError` as the write is evaluated. Resolves #46329, follow-up from #64324. Release note (bug fix): Fixed a race condition where read-write requests during replica removal (e.g. during range merges or rebalancing) could be evaluated on the removed replica. These will not have been able to write any data to persistent storage, but could behave unexpectedly, e.g. returning errors that they should not have returned. Co-authored-by: Erik Grinaker <grinaker@cockroachlabs.com>
Replica removal did not synchronize with in-flight read-only requests,
which could cause them to be evaluated on a removed (empty) replica,
returning an empty result.
This patch fixes the problem by locking
Replica.readOnlyCmdMuduringreplica removal, thus either waiting for read-only requests to complete
or not evaluating them.
Resolves #64325.
Release note (bug fix): Fixed a race condition where read-only requests
during replica removal (e.g. during range merges or rebalancing) could
be evaluated on the removed replica, returning an empty result.
/cc @cockroachdb/kv
We lock
readOnlyCmdMuduringremoveUninitializedReplicaRaftMuLockedandtryGetOrCreateReplicaas well, even though I don't believe reads can be routed to these replicas, following review comments.