-
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
kvserver: allow historical reads on subsumed ranges #51594
kvserver: allow historical reads on subsumed ranges #51594
Conversation
aeec5d9
to
1d2b8d3
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.
Drive-by comment: isn't there a lot of overlap between the LocalResult.MaybeWatchForMerge
flag and this new FreezeRangeForMerge
hook? We should try not to add more complexity here when we already have a way to communicate the result of a subsumption on a range through the standard LocalResult
mechanism. Maybe we should just change MaybeWatchForMerge
to carry the FreezeStart
timestamp instead of a boolean?
1d2b8d3
to
f21e2c7
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 needs a test, but I think the structure of the test will be quite similar to what I did for #50265, so I want to wait for that to land & maybe reuse some of its logic. Aside from that, this PR is RFAL.
@nvanbenschoten I moved FreezeStart
into LocalResult
.
In particular, PTAL at the hazard I have described in checkForPendingMergeRLocked
.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei and @nvanbenschoten)
f21e2c7
to
86d1858
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! This is looking good. We do need a test here, but I agree that we can sit on this until #50265 is merged and then follow some of the precedent set there.
Reviewed 5 of 5 files at r1.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15 and @andreimatei)
pkg/kv/kvserver/replica.go, line 1200 at r1 (raw file):
// `freezeStartTs` will be zero and we will effectively be blocking all read // requests. // TODO(aayush): If we permit co-operative lease transfers when a range is
What do you mean by "historical reads for those kinds of lease transfers"?
pkg/kv/kvserver/replica.go, line 1204 at r1 (raw file):
// transfers. if ba.Txn != nil && ba.Txn.MaxTimestamp.Less(freezeStartTs) {
ba.Txn.ReadTimestamp can actually be above MaxTimestamp, so we'll need to check that here. And also, what about non-transactional requests? I think we want logic similar to that in canServeFollowerRead
.
pkg/kv/kvserver/replica.go, line 1208 at r1 (raw file):
"in real time"
Is this true? The subsumption request did precede the current request in real time – that's why we're here. And keep in mind that this might be a request sent by an AOST query.
Your example below is what I was expecting to see here. That part is very good.
pkg/kv/kvserver/replica_read.go, line 183 at r1 (raw file):
// lead to `freezeStartTs` being overwritten with the new subsumption time. // This is fine. r.mu.freezeStartTs = lResult.FreezeStart
Let's avoid creating a new critical section and instead pass the timestamp into maybeWatchForMerge
and set it when we set r.mu.mergeComplete
.
Doing so also makes it clear that we're not unsetting r.mu.freezeStartTs
later on in maybeWatchForMerge
like we probably should be.
pkg/kv/kvserver/batcheval/cmd_subsume.go, line 136 at r1 (raw file):
reply.LeaseAppliedIndex = cArgs.EvalCtx.GetLeaseAppliedIndex() freezeStart := cArgs.EvalCtx.Clock().Now() reply.FreezeStart = freezeStart
nit: slight preference to just do:
reply.FreezeStart = cArgs.EvalCtx.Clock().Now()
return result.Result{
Local: result.LocalResult{FreezeStart: reply.FreezeStart},
}, nil
86d1858
to
936f93a
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/replica.go, line 1200 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
What do you mean by "historical reads for those kinds of lease transfers"?
Re-worded.
pkg/kv/kvserver/replica.go, line 1204 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
ba.Txn.ReadTimestamp can actually be above MaxTimestamp, so we'll need to check that here. And also, what about non-transactional requests? I think we want logic similar to that in
canServeFollowerRead
.
Done.
pkg/kv/kvserver/replica.go, line 1208 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
"in real time"
Is this true? The subsumption request did precede the current request in real time – that's why we're here. And keep in mind that this might be a request sent by an AOST query.
Your example below is what I was expecting to see here. That part is very good.
Done.
pkg/kv/kvserver/replica_read.go, line 183 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Let's avoid creating a new critical section and instead pass the timestamp into
maybeWatchForMerge
and set it when we setr.mu.mergeComplete
.Doing so also makes it clear that we're not unsetting
r.mu.freezeStartTs
later on inmaybeWatchForMerge
like we probably should be.
Done.
pkg/kv/kvserver/batcheval/cmd_subsume.go, line 136 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
nit: slight preference to just do:
reply.FreezeStart = cArgs.EvalCtx.Clock().Now() return result.Result{ Local: result.LocalResult{FreezeStart: reply.FreezeStart}, }, nil
Done
936f93a
to
6ccae64
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.
good stuff. Let's put that test in.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15 and @nvanbenschoten)
pkg/kv/kvserver/replica.go, line 265 at r2 (raw file):
mergeComplete chan struct{} // freezeStartTs indicates the subsumption time of this range when it is the // right-hand range in an ongoing merge.
pls put a small hint here about how this field is used
pkg/kv/kvserver/replica.go, line 1197 at r2 (raw file):
if ba.IsReadOnly() { freezeStartTs := r.getFreezeStartTs() // NB: If the subsumed range changes leaseholders after subsumption,
can we move or at least copy this comment and TODO to where the freezeStart
is set to zero on lease application?
If we leave a copy here, let's push it down a bit. At the current position, by the time I read it, I don't know what's going on around here; the key if ts.Less(freezeStartTs)
line is pretty far.
pkg/kv/kvserver/replica.go, line 1384 at r2 (raw file):
// neighbor is in its critical phase and, if so, arranges to block all requests // until the merge completes. func (r *Replica) maybeWatchForMerge(ctx context.Context, freezeStartTs hlc.Timestamp) error {
nit: s/freezeStartTs/freezeStart everywhere
pkg/kv/kvserver/replica.go, line 1384 at r2 (raw file):
// neighbor is in its critical phase and, if so, arranges to block all requests // until the merge completes. func (r *Replica) maybeWatchForMerge(ctx context.Context, freezeStartTs hlc.Timestamp) error {
pls comment the new arg
pkg/kv/kvserver/replica_proposal.go, line 614 at r2 (raw file):
} if !lResult.FreezeStart.IsEmpty() { log.Fatalf(ctx, "LocalEvalResult.FreezeStart should be zero: %s", lResult.FreezeStart)
nit: s/should be zero/should have been handled and reset
so that someone looking at just this line of code can make some sense of it without seeing the comment 10 lines above. Otherwise it looks like a bizarre assertion.
I know you're following less than great precedent.
pkg/kv/kvserver/replica_read.go, line 175 at r2 (raw file):
// docs/tech-notes/range-merges.md. This replica also needs to block read // traffic for all "recent" timestamps. Specifically, we can only let reads // through if they don't contain the freezeTimestamp in their uncertainty
s/freezeTimestamp/FreezeStart
But this comment seems to go too far with explaining the effects of maybeWatchForMerge()
. I'd rather keep most of it in maybeWatchForMerge
; I think leaving just the first sentence of the comment strikes a good balance.
pkg/kv/kvserver/batcheval/result/result.go, line 68 at r2 (raw file):
MaybeGossipNodeLiveness *roachpb.Span // FreezeStart indicates the high water mark timestamp beyond which the // range is guaranteed to not have served any requests, if set call
s/, if set call/. If set, call.
This comment should say that this is only set for merges.
pkg/kv/kvserver/batcheval/result/result.go, line 344 at r2 (raw file):
p.Local.FreezeStart = q.Local.FreezeStart } else if !q.Local.FreezeStart.IsEmpty() { return errors.New("conflicting FreezeStart")
errors.AssertionFailedf()
.
4f6dc99
to
c69e0d7
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/replica.go, line 265 at r2 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
pls put a small hint here about how this field is used
Done.
pkg/kv/kvserver/replica.go, line 1197 at r2 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
can we move or at least copy this comment and TODO to where the
freezeStart
is set to zero on lease application?If we leave a copy here, let's push it down a bit. At the current position, by the time I read it, I don't know what's going on around here; the key
if ts.Less(freezeStartTs)
line is pretty far.
Moving makes it clearer. Done.
pkg/kv/kvserver/replica.go, line 1384 at r2 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
pls comment the new arg
Done.
pkg/kv/kvserver/replica.go, line 1384 at r2 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
nit: s/freezeStartTs/freezeStart everywhere
Done
pkg/kv/kvserver/replica_proposal.go, line 614 at r2 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
nit: s/should be zero/should have been handled and reset
so that someone looking at just this line of code can make some sense of it without seeing the comment 10 lines above. Otherwise it looks like a bizarre assertion.
I know you're following less than great precedent.
Done.
pkg/kv/kvserver/replica_read.go, line 175 at r2 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
s/freezeTimestamp/FreezeStart
But this comment seems to go too far with explaining the effects of
maybeWatchForMerge()
. I'd rather keep most of it inmaybeWatchForMerge
; I think leaving just the first sentence of the comment strikes a good balance.
Agreed, done.
pkg/kv/kvserver/batcheval/result/result.go, line 68 at r2 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
s/, if set call/. If set, call.
This comment should say that this is only set for merges.
Done
pkg/kv/kvserver/batcheval/result/result.go, line 344 at r2 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
errors.AssertionFailedf()
.
Done
Rebased it on top of #50265 and added the test. RFAL. |
c69e0d7
to
5bedbc6
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.
Nit: If it's all the same to you, I'd squash the two commits. Introducing some field without using it yet should be done only if it really simplifies the review or if the respective commit is large enough. In this case the commit it tiny and splitting it actually hurts the review; I have to look across the two commits and see how the filter is used to form an opinion about it anyway.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15, @andreimatei, and @nvanbenschoten)
pkg/kv/kvserver/client_merge_test.go, line 3686 at r4 (raw file):
ctx := context.Background() var mtc multiTestContext
I hear we don't use this any more. Switch to TestServer
.
pkg/kv/kvserver/client_merge_test.go, line 3697 at r4 (raw file):
} storeCfg.TestingKnobs.TestingRequestFilter = state.suspendMergeTrigger mergeInProgressErrorCh := make(chan struct{}, 2)
please put add a comment to the 2
buffer size.
But I think something funky is going on. You're using this buffer across otherwise unrelated tests, and verifying at the end that the filter was called twice. First of all, I don't think you need a channel to verify that the filter was called twice; you can use a simpler atomic counter. But more importantly, I think that we'd separate the subtests and make them completely independent, this test overall would be a lot clearer. You wouldn't need the errGroup
any more for example, and also you wouldn't need buffering on this channel. It'd also be a lot clearer who blocks and who doesn't block.
pkg/kv/kvserver/client_merge_test.go, line 3698 at r4 (raw file):
storeCfg.TestingKnobs.TestingRequestFilter = state.suspendMergeTrigger mergeInProgressErrorCh := make(chan struct{}, 2) storeCfg.TestingKnobs.TestingConcurrencyRetryFilter = func(ctx context.Context, ba roachpb.BatchRequest, pErr *roachpb.Error) *roachpb.Error {
very long line
Style guidance is 100 chars.
pkg/kv/kvserver/client_merge_test.go, line 3699 at r4 (raw file):
mergeInProgressErrorCh := make(chan struct{}, 2) storeCfg.TestingKnobs.TestingConcurrencyRetryFilter = func(ctx context.Context, ba roachpb.BatchRequest, pErr *roachpb.Error) *roachpb.Error { if _, ok := pErr.GetDetail().(*roachpb.MergeInProgressError); ok && ba.Txn != nil {
I find it dubious that we care enough to test for ba.Txn != nil
, but we don't care enough to check the transaction ID or anything else about the request. What's the story? In any case, I think the story would probably improve if we separate subtests like I was saying above.
pkg/kv/kvserver/client_merge_test.go, line 3716 at r4 (raw file):
g, ctx := errgroup.WithContext(ctx) g.Go(func() error { if _, err := kv.SendWrapped(ctx, store.TestSender(), mergeArgs); err != nil {
I think you can do return kv.SendWrapped(...).GoError()
, and below.
pkg/kv/kvserver/client_merge_test.go, line 3724 at r4 (raw file):
// Unblock the merge so the blocked queries can continue close(state.finishMergeTxn) if err := g.Wait(); err != nil {
nit: require.NoError(g.Wait())
pkg/kv/kvserver/client_merge_test.go, line 3779 at r4 (raw file):
for count := 0; count < 2; count++ { select { case <-blockedRequestsTimer.C:
case <- time.After(20 * time.Second)
? Even with a loop around it, I don't think you need an explicit timer
.
pkg/kv/kvserver/kvserverbase/base.go, line 84 at r3 (raw file):
// ReplicaConcurrencyRetryFilter can be used to examine a concurrency retry // error before it is handled and its batch is re-executed. Return nil to
nit: s/re-executed/re-evaluated
I know that the function calling this filter talks about "execution", but we've gotten tighter with our language since that was written.
pkg/kv/kvserver/kvserverbase/base.go, line 87 at r3 (raw file):
// continue with regular processing or non-nil to terminate processing with the // returned error. type ReplicaConcurrencyRetryFilter func(context.Context, roachpb.BatchRequest, *roachpb.Error) *roachpb.Error
The returned error's semantics are unclear, and I see you don't need them. So rather than talk about it, I'd say just remove the ret val unless you had a particular reason to add it.
c4515e8
to
5b8252b
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 3686 at r4 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
I hear we don't use this any more. Switch to
TestServer
.
Done.
pkg/kv/kvserver/client_merge_test.go, line 3697 at r4 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
please put add a comment to the
2
buffer size.But I think something funky is going on. You're using this buffer across otherwise unrelated tests, and verifying at the end that the filter was called twice. First of all, I don't think you need a channel to verify that the filter was called twice; you can use a simpler atomic counter. But more importantly, I think that we'd separate the subtests and make them completely independent, this test overall would be a lot clearer. You wouldn't need the
errGroup
any more for example, and also you wouldn't need buffering on this channel. It'd also be a lot clearer who blocks and who doesn't block.
I separated queries into their own subtests and changed the test to be table-driven.
pkg/kv/kvserver/client_merge_test.go, line 3698 at r4 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
very long line
Style guidance is 100 chars.
Done
pkg/kv/kvserver/client_merge_test.go, line 3699 at r4 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
I find it dubious that we care enough to test for
ba.Txn != nil
, but we don't care enough to check the transaction ID or anything else about the request. What's the story? In any case, I think the story would probably improve if we separate subtests like I was saying above.
We didn't need that check. I removed it.
pkg/kv/kvserver/client_merge_test.go, line 3716 at r4 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
I think you can do
return kv.SendWrapped(...).GoError()
, and below.
Done
pkg/kv/kvserver/client_merge_test.go, line 3724 at r4 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
nit:
require.NoError(g.Wait())
Removed the errgroup
pkg/kv/kvserver/client_merge_test.go, line 3779 at r4 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
case <- time.After(20 * time.Second)
? Even with a loop around it, I don't think you need an explicittimer
.
Changed this bit to use SucceedsSoon
instead.
pkg/kv/kvserver/kvserverbase/base.go, line 84 at r3 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
nit: s/re-executed/re-evaluated
I know that the function calling this filter talks about "execution", but we've gotten tighter with our language since that was written.
Done.
pkg/kv/kvserver/kvserverbase/base.go, line 87 at r3 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
The returned error's semantics are unclear, and I see you don't need them. So rather than talk about it, I'd say just remove the ret val unless you had a particular reason to add it.
Had no real reason to add it, removed.
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! 1 of 0 LGTMs obtained (waiting on @aayushshah15, @andreimatei, and @nvanbenschoten)
pkg/kv/kvserver/replica.go, line 1135 at r5 (raw file):
); err != nil { return err } else if g.HoldingLatches() && st.State == kvserverpb.LeaseState_VALID {
was this wrong before? Or why did it change?
pkg/kv/kvserver/replica.go, line 1135 at r5 (raw file): Previously, andreimatei (Andrei Matei) wrote…
You and Nathan noticed it a little while ago: previously, we were checking for the state variable being nil, but this method is never called with a nil
|
927e970
to
6279bbd
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 5 of 6 files at r4, 3 of 4 files at r5, 1 of 1 files at r6.
Reviewable status: complete! 1 of 0 LGTMs obtained (and 1 stale) (waiting on @aayushshah15 and @andreimatei)
pkg/kv/kvserver/client_merge_test.go, line 3682 at r6 (raw file):
// can only serve read-only traffic for timestamps that precede the subsumption // time, but don't contain the subsumption time in their uncertainty interval. func TestHistoricalReadsAfterSubsume(t *testing.T) {
Nice test!
pkg/kv/kvserver/client_merge_test.go, line 3807 at r6 (raw file):
MaxOffset: testMaxOffset, TestingRequestFilter: state.suspendMergeTrigger, TestingConcurrencyRetryFilter: func(ctx context.Context,
nit: this formatting is pretty bizarre. Try:
TestingConcurrencyRetryFilter: func(
ctx context.Context, ba roachpb.BatchRequest, pErr *roachpb.Error,
) {
pkg/kv/kvserver/replica.go, line 1135 at r5 (raw file):
Previously, aayushshah15 (Aayush Shah) wrote…
You and Nathan noticed it a little while ago: previously, we were checking for the state variable being nil, but this method is never called with a nil
LeaseStatus
. The old implementation would incorrectly let follower read requests callcheckForPendingMergeRLocked
. Unless I'm missing something, this means we allowed the following scenario:
- RHS gets subsumed &
mergeCompleteCh
is instantiated on the leaseholder.- RHS's lease gets transferred, but the old leaseholder hasn't removed its
mergeCompleteCh
yet.- A follower read request gets routed to old leaseholder, and blocks until it removes its
mergeCompleteCh
channel (which will happen after the replica detects that the merge aborted).
+1 to what @aayushshah15 said. This had rotted at some point recently and this is the correct fix.
But do you mind updating the or st == nil
comment above?
pkg/kv/kvserver/replica.go, line 1553 at r6 (raw file):
r.mu.mergeComplete = nil close(mergeCompleteCh) r.mu.freezeStart.Reset()
nit: Reset
is inherited from this being a proto type. But it's a little awkward and hides the simplicity of this operation. I'd just do r.mu.freezeStart = hlc.Timestamp{}
.
Also, you might as well keep the unassignment of freezeStart
and mergeComplete
in the same order as their assignment. That kind of thing can prevent bugs in the future.
Currently, we block all requests on the RHS of a merge after it is subsumed. While we need to block all write & admin requests, we need only block "recent" read requests. In particular, if a read request for the RHS is old enough that its freeze timestamp is outside the request's uncertainty window, it is safe to let it through. This makes range merges a little bit less disruptive to foreground traffic. Release note: None
6279bbd
to
b192bba
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.
TFTR!
bors r+
Reviewable status: complete! 0 of 0 LGTMs obtained (and 2 stale) (waiting on @aayushshah15, @andreimatei, and @nvanbenschoten)
pkg/kv/kvserver/client_merge_test.go, line 3807 at r6 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
nit: this formatting is pretty bizarre. Try:
TestingConcurrencyRetryFilter: func( ctx context.Context, ba roachpb.BatchRequest, pErr *roachpb.Error, ) {
Done.
pkg/kv/kvserver/replica.go, line 1135 at r5 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
+1 to what @aayushshah15 said. This had rotted at some point recently and this is the correct fix.
But do you mind updating the
or st == nil
comment above?
Done.
pkg/kv/kvserver/replica.go, line 1553 at r6 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
nit:
Reset
is inherited from this being a proto type. But it's a little awkward and hides the simplicity of this operation. I'd just dor.mu.freezeStart = hlc.Timestamp{}
.Also, you might as well keep the unassignment of
freezeStart
andmergeComplete
in the same order as their assignment. That kind of thing can prevent bugs in the future.
Done.
bors r+ |
Build succeeded: |
This PR cleans up some handling around replica destruction that scared me when working on cockroachdb#46329 and cockroachdb#55293. Specifically, there were cases during merges where the destroy status on a replica would not be set until after that replicas data was destroyed. This was true of merges applied through entry application, although in those cases, no user data was removed so it seems mostly harmless. More concerning is that is was true of merges applied through snapshot application. This was extra concerning because snapshot application can delete user data if a subsumed range is only partially covered (see `clearSubsumedReplicaDiskData`). So we were hypothetically risking user-visible correctness problems, especially now that we allow follower reads through on subsumed ranges as of cockroachdb#51594. This PR patches up these issues and then adds a few extra assertions that enforce stricter preconditions for the functions at play during replica destruction. Specifically, we now assert that the destroy status is set to `destroyReasonRemoved` _before_ calling `preDestroyRaftMuLocked`. We also assert that if `RemoveOptions.DestroyData` is false when passed to `removeInitializedReplicaRaftMuLocked`, then the destroy status is also set. This unification allows us to remove the `ignoreDestroyStatus` flag on `RemoveOptions`, whose meaning is now exactly the inverse of `DestroyData`. In hindsight, pushing on why this extra flag was needed and what new information it was conveying to the function could have potentially caught these issues a little earlier. I think we'll want to backport this to v20.2, though probably in the first patch release because there is some risk here (especially without sufficient time to bake on master) and we aren't aware of seeing any correctness issues from the combination of the bug fixed here and cockroachdb#51594. Release note (bug fix): A hypothesized bug that could allow a follower read to miss data on a range in the middle of being merged away into its left-hand neighbor was fixed. The bug seems exceedingly unlikely to have materialized in practice.
This PR cleans up some handling around replica destruction that scared me when working on cockroachdb#46329 and cockroachdb#55293. Specifically, there were cases during merges where the destroy status on a replica would not be set until after that replicas data was destroyed. This was true of merges applied through entry application, although in those cases, no user data was removed so it seems mostly harmless. More concerning is that is was true of merges applied through snapshot application. This was extra concerning because snapshot application can delete user data if a subsumed range is only partially covered (see `clearSubsumedReplicaDiskData`). So we were hypothetically risking user-visible correctness problems, especially now that we allow follower reads through on subsumed ranges as of cockroachdb#51594. This PR patches up these issues and then adds a few extra assertions that enforce stricter preconditions for the functions at play during replica destruction. Specifically, we now assert that the destroy status is set to `destroyReasonRemoved` _before_ calling `preDestroyRaftMuLocked`. We also assert that if `RemoveOptions.DestroyData` is false when passed to `removeInitializedReplicaRaftMuLocked`, then the destroy status is also set. This unification allows us to remove the `ignoreDestroyStatus` flag on `RemoveOptions`, whose meaning is now exactly the inverse of `DestroyData`. In hindsight, pushing on why this extra flag was needed and what new information it was conveying to the function could have potentially caught these issues a little earlier. I think we'll want to backport this to v20.2, though probably in the first patch release because there is some risk here (especially without sufficient time to bake on master) and we aren't aware of seeing any correctness issues from the combination of the bug fixed here and cockroachdb#51594. Release note (bug fix): A hypothesized bug that could allow a follower read to miss data on a range in the middle of being merged away into its left-hand neighbor was fixed. The bug seems exceedingly unlikely to have materialized in practice.
55477: kv: set destroy status before destroying data on subsumed replicas r=nvanbenschoten a=nvanbenschoten This PR cleans up some handling around replica destruction that scared me when working on #46329 and #55293. Specifically, there were cases during merges where the destroy status on a replica would not be set until after that replicas data was destroyed. This was true of merges applied through entry application, although in those cases, no user data was removed so it seems mostly harmless. More concerning is that is was true of merges applied through snapshot application. This was extra concerning because snapshot application can delete user data if a subsumed range is only partially covered (see `clearSubsumedReplicaDiskData`). So we were hypothetically risking user-visible correctness problems, especially now that we allow follower reads through on subsumed ranges as of #51594. This PR patches up these issues and then adds a few extra assertions that enforce stricter preconditions for the functions at play during replica destruction. Specifically, we now assert that the destroy status is set to `destroyReasonRemoved` _before_ calling `preDestroyRaftMuLocked`. We also assert that if `RemoveOptions.DestroyData` is false when passed to `removeInitializedReplicaRaftMuLocked`, then the destroy status is also set. This unification allows us to remove the `ignoreDestroyStatus` flag on `RemoveOptions`, whose meaning is now exactly the inverse of `DestroyData`. In hindsight, pushing on why this extra flag was needed and what new information it was conveying to the function could have potentially caught these issues a little earlier. I think we'll want to backport this to v20.2, though probably in the first patch release because there is some risk here (especially without sufficient time to bake on master) and we aren't aware of seeing any correctness issues from the combination of the bug fixed here and #51594. Release note (bug fix): A hypothesized bug that could allow a follower read to miss data on a range in the middle of being merged away into its left-hand neighbor was fixed. The bug seems exceedingly unlikely to have materialized in practice. Co-authored-by: Nathan VanBenschoten <[email protected]>
This PR cleans up some handling around replica destruction that scared me when working on cockroachdb#46329 and cockroachdb#55293. Specifically, there were cases during merges where the destroy status on a replica would not be set until after that replicas data was destroyed. This was true of merges applied through entry application, although in those cases, no user data was removed so it seems mostly harmless. More concerning is that is was true of merges applied through snapshot application. This was extra concerning because snapshot application can delete user data if a subsumed range is only partially covered (see `clearSubsumedReplicaDiskData`). So we were hypothetically risking user-visible correctness problems, especially now that we allow follower reads through on subsumed ranges as of cockroachdb#51594. This PR patches up these issues and then adds a few extra assertions that enforce stricter preconditions for the functions at play during replica destruction. Specifically, we now assert that the destroy status is set to `destroyReasonRemoved` _before_ calling `preDestroyRaftMuLocked`. We also assert that if `RemoveOptions.DestroyData` is false when passed to `removeInitializedReplicaRaftMuLocked`, then the destroy status is also set. This unification allows us to remove the `ignoreDestroyStatus` flag on `RemoveOptions`, whose meaning is now exactly the inverse of `DestroyData`. In hindsight, pushing on why this extra flag was needed and what new information it was conveying to the function could have potentially caught these issues a little earlier. I think we'll want to backport this to v20.2, though probably in the first patch release because there is some risk here (especially without sufficient time to bake on master) and we aren't aware of seeing any correctness issues from the combination of the bug fixed here and cockroachdb#51594. Release note (bug fix): A hypothesized bug that could allow a follower read to miss data on a range in the middle of being merged away into its left-hand neighbor was fixed. The bug seems exceedingly unlikely to have materialized in practice.
Currently, we block all requests on the RHS of a merge after it is
subsumed. While we need to block all write & admin requests, we need
only block "recent" read requests. In particular, if a read request for
the RHS is old enough that its freeze timestamp is outside the request's
uncertainty window, it is safe to let it through. This makes range
merges a little bit less disruptive to foreground traffic.
Release note: None