kvserver: remove unnecessary special casing of lease for LAI#74810
kvserver: remove unnecessary special casing of lease for LAI#74810craig[bot] merged 4 commits intocockroachdb:masterfrom
Conversation
0c6abe5 to
ef4f571
Compare
nvb
left a comment
There was a problem hiding this comment.
Reviewed 5 of 5 files at r1, 1 of 1 files at r2, 1 of 1 files at r3, 1 of 1 files at r4, all commit messages.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @tbg)
-- commits, line 4 at r1:
s/MaxLeaseAppliedIndex/MaxLeaseIndex/ throughout
Also, s/MaxAppliedLeaseIndex/MaxLeaseIndex/ below
pkg/kv/kvserver/replica_proposal_buf.go, line 603 at r1 (raw file):
) (uint64, hlc.Timestamp, error) { // Assign a MaxLeaseAppliedIndex (see checkForcedErr). These provide replay
MaxLeaseAppliedIndex isn't actually a thing, even though it sounds like it should be.
pkg/kv/kvserver/replica_proposal_buf.go, line 607 at r1 (raw file):
// // Lease-related proposals have their own replay protection (via the lease // sequence) and will not check the lease applied index (since they are
"via the lease sequence and previous lease's proposal timestamp"
pkg/kv/kvserver/replica_proposal_buf.go, line 615 at r1 (raw file):
// lease proposals (both transfers and requests). var lai uint64 if !p.command.ReplicatedEvalResult.IsLeaseRequest {
nit: we use p.Request.IsLeaseRequest() down below in this function and also elsewhere in this file, so we shouldn't switch this over to look at the ReplicatedEvalResult unless we're willing to do so everywhere in the propBuf.
We were previously "not" assigning a LeaseAppliedIndex to lease request proposals. But we were doing so very half-heartedly: instead of always assigning zero, we assigned "whatever was assigned to the previous command". This meant that in practice, the first lease on a range would get zero, and any subsequent leases would get some nonzero numbers. This wasn't particularly principled and raises eyebrows. For testing convenience and clarity it is helpful to assign a zero MLAI to leases, which is what this commit does. We then turn to the special-casing in refreshProposalsLocked. `(*Replica).refreshProposalsLocked` previously refused to repropose commands that had a zero `AppliedLeaseIndex` (LAI). This was done on grounds of needing to provide replay protection for these requests. Upon inspection it became clear that a zero MLAI could only ever apply to lease requests; they are the only proposals with an exception, and as we saw above it would not usually result in a zero, but could (and especially in testing, where most leases are the first lease on their respective range). We [discussed] this internally and concluded that leases can in fact be reproposed, since they have their own replay protection mediated through the lease's sequence number. This is great since as stated above, we did in fact repropose (some) leases and have for years. This commit removes the special casing. Fixes cockroachdb#74711. As suggested by @nvanbenschoten I ran the `kvserver` tests with this diff that reproposes every lease request: ```diff diff --git a/pkg/kv/kvserver/replica_proposal_buf.go b/pkg/kv/kvserver/replica_proposal_buf.go index 95765d8..1b66745866 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -576,6 +576,11 @@ func (b *propBuf) FlushLockedWithRaftGroup( ents = append(ents, raftpb.Entry{ Data: p.encodedCommand, }) + if p.Request.IsLeaseRequest() { + ents = append(ents, raftpb.Entry{ + Data: p.encodedCommand, + }) + } } } if firstErr != nil { ``` This inspired the subsequent commits. [discussed]: https://cockroachlabs.slack.com/archives/G01G8LK77DK/p1641559185021100 Release note: None
I found this while sanity-checking the preceding commits with a custom code change that appended each lease request to the raft log twice, which would cause an NPE during `TestReplicaLeaseCounters`. Release note: None
This would otherwise trigger the "command already applied" assertion in `ApplySideEffects` when run with artificial reproposals of lease commands. Release note: None
Release note: None
ef4f571 to
39d4244
Compare
tbg
left a comment
There was a problem hiding this comment.
TFTR, RFAL.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten)
pkg/kv/kvserver/replica_proposal_buf.go, line 615 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
nit: we use
p.Request.IsLeaseRequest()down below in this function and also elsewhere in this file, so we shouldn't switch this over to look at theReplicatedEvalResultunless we're willing to do so everywhere in thepropBuf.
Done. Also cleared up a misconception I had between lease requests and transfers. I thought some of the special casing also applied to transfers, but this is incorrect. Transfers use the LAI because they're only ever proposed by the leaseholder. This is now pointed out.
|
Friendly ping, @nvanbenschoten. |
nvb
left a comment
There was a problem hiding this comment.
Reviewed 2 of 4 files at r5, 1 of 1 files at r6, 1 of 1 files at r8, all commit messages.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @tbg)
|
bors r=nvanbenschoten |
|
Build failed (retrying...): |
|
Build succeeded: |
Touches #33007 (this simplifies #71806).
We were previously "not" assigning a LeaseAppliedIndex to lease
request proposals. But we were doing so very half-heartedly: instead of
always assigning zero, we assigned "whatever was assigned to the
previous command". This meant that in practice, the first lease on a
range would get zero, and any subsequent leases would get some nonzero
numbers. This wasn't particularly principled and raises eyebrows. For
testing convenience and clarity it is helpful to assign a zero MLAI
to leases, which is what this commit does.
We then turn to the special-casing in refreshProposalsLocked.
(*Replica).refreshProposalsLockedpreviously refused to reproposecommands that had a zero
AppliedLeaseIndex(LAI). This was done ongrounds of needing to provide replay protection for these requests.
Upon inspection it became clear that a zero MLAI could only ever apply
to lease requests; they are the only proposals with an exception, and as
we saw above it would not usually result in a zero, but could (and
especially in testing, where most leases are the first lease on their
respective range).
We discussed this internally and concluded that leases can in fact be
reproposed, since they have their own replay protection mediated through
the lease's sequence number. This is great since as stated above, we did
in fact repropose (some) leases and have for years.
This commit removes the special casing.
Fixes #74711.
As suggested by @nvanbenschoten I ran the
kvservertests with thisdiff that reproposes every lease request:
This inspired the subsequent commits.
Release note: None