kvserver: acquire and switch leases during Raft ticks#101098
kvserver: acquire and switch leases during Raft ticks#101098craig[bot] merged 3 commits intocockroachdb:masterfrom
Conversation
836fb48 to
9389f73
Compare
9389f73 to
6554903
Compare
6554903 to
7c4c274
Compare
andrewbaptist
left a comment
There was a problem hiding this comment.
I had some minor comments / clarifications, but generally it all looks good. Thanks!
Reviewed 7 of 7 files at r1, 3 of 3 files at r2, 5 of 5 files at r3, all commit messages.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @erikgrinaker and @nvanbenschoten)
pkg/kv/kvserver/replica_range_lease.go line 1422 at r1 (raw file):
case kvserverpb.LeaseState_EXPIRED: // Attempt to acquire an expired lease if we're the Raft leader. return r.isRaftLeaderRLocked()
Why is this check isRaftLeaderRLocked instead of st.OwnerBy(r.StoreID())) I think they would give the same result here unless the leader and leaseholder are different, but there may be a case I'm missing. I guess I'm confused how to proceed in a leader/leaseholder split, should it be the leader or the previous leaseholder that gets the lease?
pkg/kv/kvserver/replica_range_lease.go line 1431 at r1 (raw file):
return st.OwnedBy(r.StoreID()) case kvserverpb.LeaseState_VALID, kvserverpb.LeaseState_UNUSABLE:
I'm a little confused about how we should handle UNUSABLE leases. Shouldn't these be treated more like EXPIRED leases since they are "about to" expire? Or is the problem that we still own it, so we can't let it go to the Raft leader? If we get to UNUSABLE with a leader/leaseholder split, we will be racing with the leader (who will try and get the lease once it thinks it's expired) and the leaseholder trying to get the lease when it is "almost expired". That's OK because only one will win, but it seems like we would want consistent behavior. Also it makes me wonder how we get out of the leader/leaseholder split at all here.
pkg/kv/kvserver/replicate_queue.go line 1024 at r1 (raw file):
// this lease transfer. // // TODO(erikgrinaker): This is also done more eagerly during Raft ticks, but
nit: Is this still a TODO?
pkg/kv/kvserver/store.go line 1982 at r2 (raw file):
// NB: cluster settings haven't propagated yet, so we have to check the last // known lease instead of relying on shouldUseExpirationLeaseRLocked(). if l, _ := rep.GetLease(); l.Type() == roachpb.LeaseExpiration && l.Sequence > 0 {
nit: Add a note about l.Sequence > 0. I would assume that all leases have a sequence > 0, so this should only happen for a newly created range that has never had a leaseholder. Alternatively, maybe we should just initialize all raft groups at startup and let the quiese again naturally. I don't love the idea of checking the previous setting. Another alternative is to check if we would get an expiration lease on this range by calling shouldUseExpirationLeaseRLocked. That seems a little better.
pkg/kv/kvserver/store.go line 246 at r3 (raw file):
settings.SystemOnly, "kv.store.concurrent_eager_lease_acquisitions", "the number of concurrent, eager lease acquisitions to make (0 disables eager acquisition)",
nit: What are the implications of setting this to 0. It seems better to not allow that as a user will do it by accident. I think the replicate_queue will still take care of this, but I'm not sure we would want anyone to do this.
erikgrinaker
left a comment
There was a problem hiding this comment.
TFTR!
I'm still a bit unsure about the PROSCRIBED policy here -- it makes sense after a restart, but I want to doublecheck that it won't be problematic after revoking our lease during a transfer. I think in that case we'll already see the new lease and not hit the PROSCRIBED path, and if we don't then the lease request's compare-and-swap will fail anyway.
I also just want to run some quick perf checks of Raft ticks to make sure the costs aren't increasing too much.
Reviewable status:
complete! 1 of 0 LGTMs obtained (waiting on @andrewbaptist and @nvanbenschoten)
pkg/kv/kvserver/replica_range_lease.go line 1422 at r1 (raw file):
Previously, andrewbaptist (Andrew Baptist) wrote…
Why is this check
isRaftLeaderRLockedinstead ofst.OwnerBy(r.StoreID()))I think they would give the same result here unless the leader and leaseholder are different, but there may be a case I'm missing. I guess I'm confused how to proceed in a leader/leaseholder split, should it be the leader or the previous leaseholder that gets the lease?
The common case here is when the old leaseholder died. In that case, st.OwnerBy(r.StoreID()) would mean that noone tries to acquire the lease, which is the opposite of what we want.
The reason we check the Raft leadership is because we also check below Raft that only the Raft leader can acquire the lease, to keep them colocated. Returning true for non-leaders is pointless here, because the lease request is guaranteed to fail below Raft. This also reduces the aggregate cost of these acquisition attempts by 2/3 since only 1/3 replicas will attempt it.
cockroach/pkg/kv/kvserver/replica_proposal_buf.go
Lines 615 to 628 in a9989a8
pkg/kv/kvserver/replica_range_lease.go line 1431 at r1 (raw file):
Only the leaseholder can request the lease during UNUSABLE, since it's in the stasis period of the current lease. We still own the lease, but we can't use it to serve read requests since it may overlap with a different leaseholder in real time (not logical time) due to clock skew. However, we know that the lease is about to expire, so we'd still like to try to extend it if we can instead of letting it expire and incurring lease acquisition latency for the next requests to the range.
it makes me wonder how we get out of the leader/leaseholder split at all here
In general, leader/leaseholder splits are handled by the leader actively transferring leadership to the leaseholder:
cockroach/pkg/kv/kvserver/replica_raft.go
Line 1216 in a9989a8
But once the lease expires, we try to acquire it on the current leader.
pkg/kv/kvserver/replicate_queue.go line 1024 at r1 (raw file):
Previously, andrewbaptist (Andrew Baptist) wrote…
nit: Is this still a TODO?
Yes, we still quiesce ranges with epoch leases, so we need the replicate queue to do this for ranges that use them (since they aren't ticked and therefore can't rely on the Raft scheduler to do this). Maybe in the future we can get rid of quiescence entirely for epoch leases too, but not yet.
pkg/kv/kvserver/store.go line 1982 at r2 (raw file):
Previously, andrewbaptist (Andrew Baptist) wrote…
nit: Add a note about l.Sequence > 0. I would assume that all leases have a sequence > 0, so this should only happen for a newly created range that has never had a leaseholder. Alternatively, maybe we should just initialize all raft groups at startup and let the quiese again naturally. I don't love the idea of checking the previous setting. Another alternative is to check if we would get an expiration lease on this range by calling
shouldUseExpirationLeaseRLocked. That seems a little better.
Yeah, the sequence check and only doing this for expiration leases is just to be conservative and avoid any unwanted regressions in the default case where we use epoch leases. I'll add a comment. I'd love to remove this and always initialize, but I think we should be conservative here until we get some more runtime with this in actual production environments.
As the comment points out, shouldUseExpirationLeaseRLocked doesn't work here -- it will always return false because cluster settings haven't propagated yet. The best we can do is to look at the last lease we know about, which in the worst case would give a false result if someone flipped the setting while the node was offline, and that doesn't seem terrible.
pkg/kv/kvserver/store.go line 246 at r3 (raw file):
Previously, andrewbaptist (Andrew Baptist) wrote…
nit: What are the implications of setting this to 0. It seems better to not allow that as a user will do it by accident. I think the replicate_queue will still take care of this, but I'm not sure we would want anyone to do this.
Oops, that description doesn't match reality. The actual implementation will disable the limit (i.e. infinity) when set to 0. I'm honestly not sure which behavior makes more sense. Maybe we'd want a way to disable this if it causes problems, so 0 is a useful escape hatch? I mean, setting it to 1 would be equally bad, and if someone did want infinity they can just use 1e6 or something.
erikgrinaker
left a comment
There was a problem hiding this comment.
Thanks again for the review! I've updated the PR, I'll give you a chance to respond to any of the comments or changes before borsing this.
I'm still a bit unsure about the
PROSCRIBEDpolicy here
We already do the same thing in redirectOnOrAcquireLease(), which will be called on every request processed by this replica in PROSCRIBED state, so this seems fine.
I also just want to run some quick perf checks of Raft ticks
Didn't see a significant effect. We'll do some more micro-optimization of Raft ticks later.
Reviewable status:
complete! 1 of 0 LGTMs obtained (waiting on @andrewbaptist and @nvanbenschoten)
pkg/kv/kvserver/store.go line 246 at r3 (raw file):
Previously, erikgrinaker (Erik Grinaker) wrote…
Oops, that description doesn't match reality. The actual implementation will disable the limit (i.e. infinity) when set to 0. I'm honestly not sure which behavior makes more sense. Maybe we'd want a way to disable this if it causes problems, so 0 is a useful escape hatch? I mean, setting it to 1 would be equally bad, and if someone did want infinity they can just use 1e6 or something.
I made this disable eager lease acquisition, it seems handy to have the ability if something goes wrong.
I also made the limit per-node rather than per-store, and renamed it to kv.lease.eager_acquisition_concurrency in the possibly naïve hope that we can move all lease-related settings under kv.lease.
f0bfc59 to
efe0b4b
Compare
|
Going to bors this to run some tests/benchmarks. TFTR! bors r+ |
|
Build failed: |
This patch eagerly acquires leases, and switches their type when appropriate, during Raft ticks. This avoids incurring lease acquisition latency on the next request to the range. However, it's only effective for unquiesced ranges (which includes all expiration-based ranges), so we retain the corresponding logic in the replicate queue as a fallback, taking effect within 10 minutes or so. Epic: none Release note: None
By default, replicas start out quiesced and only unquiesce in response to range traffic. Since expiration-based leases should not quiesce, and should eagerly acquire leases, this patch eagerly initializes and unquiesces replicas with expiration-based leases when loaded. Epic: none Release note: None
This patch limits the number of concurrent eager lease acquisitions done by the Raft scheduler to 256 per node (across all stores), configurable via `kv.lease.eager_acquisition_concurrency`. When the limit is reached, further lease acquisition requests are dropped and then retried on the next tick. This only applies to acquisition of expired leases and switching of lease types done by the Raft scheduler, not to lease extensions nor to lease acquisitions driven by client requests. Since expiration leases need to be extended regularly there is little point in throttling them and the cluster must be provisioned to handle them, and client-driven acquisitions are clearly high priority. A more sophisticated policy can be considered later if necessary. Epic: none Release note: None
efe0b4b to
5084670
Compare
|
bors r+ |
|
Build succeeded: |
kvserver: acquire and switch leases during Raft ticks
This patch eagerly acquires leases, and switches their type when appropriate, during Raft ticks. This avoids incurring lease acquisition latency on the next request to the range. However, it's only effective for unquiesced ranges (which includes all expiration-based ranges), so we retain the corresponding logic in the replicate queue as a fallback, taking effect within 10 minutes or so.
Resolves #98433.
Epic: none
Release note: None
kvserver: eagerly initialize replicas with expiration leases
By default, replicas start out quiesced and only unquiesce in response to range traffic. Since expiration-based leases should not quiesce, and should eagerly acquire leases, this patch eagerly initializes and unquiesces replicas with expiration-based leases when loaded.
Touches #98433.
Epic: none
Release note: None
kvserver: limit concurrent eager lease acquisitions
This patch limits the number of concurrent eager lease acquisitions done by the Raft scheduler to 256 per node (across all stores), configurable via
kv.lease.eager_acquisition_concurrency. When the limit is reached, further lease acquisition requests are dropped and then retried on the next tick.This only applies to acquisition of expired leases and switching of lease types done by the Raft scheduler, not to lease extensions nor to lease acquisitions driven by client requests. Since expiration leases need to be extended regularly there is little point in throttling them and the cluster must be provisioned to handle them, and client-driven acquisitions are clearly high priority.
A more sophisticated policy can be considered later if necessary.
Resolves #100426.
Epic: none
Release note: None