kvserver: eagerly extend expiration leases in Raft scheduler#100430
kvserver: eagerly extend expiration leases in Raft scheduler#100430craig[bot] merged 4 commits intocockroachdb:masterfrom
Conversation
88d1f21 to
e803c71
Compare
e803c71 to
c238665
Compare
|
@andrewbaptist @nvanbenschoten As we discussed, I've updated the PR to gate all of the new behavior here on |
c238665 to
4bbc2bb
Compare
andrewbaptist
left a comment
There was a problem hiding this comment.
Reviewed 4 of 4 files at r1.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten)
andrewbaptist
left a comment
There was a problem hiding this comment.
Thanks!
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten)
andrewbaptist
left a comment
There was a problem hiding this comment.
This will be great to have in place!
Reviewed 2 of 2 files at r2, 6 of 6 files at r3.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @erikgrinaker and @nvanbenschoten)
pkg/kv/kvserver/replica_raft.go line 1200 at r3 (raw file):
} if r.mu.quiescent {
I'm unsure if we would ever get here with an expiration lease, but if we did get here and it is an expiration lease (possibly upgrade path?) should we handle this case by creating/extending a lease? I'm thinking of cases like upgrades or setting changes.
pkg/kv/kvserver/replica_range_lease.go line 1417 at r3 (raw file):
// shouldExtendLeaseRLocked determines whether the lease should be extended // asynchronously, even if it is currently valid. The method returns true if // this range uses expiration-based leases, it is or was owned by this replica,
nit: This sentence is hard to parse now. Perhaps break it into conditions like:
The method returns true if the following conditions are all met:
- This range uses expiration-based leases.
- The most recent lease is owned by this replica.
- The lease is in need of renewal.
- There is no pending extension.
pkg/kv/kvserver/replica_lease_renewal_test.go line 55 at r3 (raw file):
RaftConfig: base.RaftConfig{ RangeLeaseDuration: time.Second, RaftTickInterval: 100 * time.Millisecond,
nit: No need to change anything here, but it would be nice if we could "manually" call tick on all ranges. Something similar to store.Enqueue. It could make these tests much more deterministic and faster and could use a ManualClock instead of having to deal with all these Eventually calls below.
This patch prevents quiescence with expiration-based leases, since we'll have to propose a lease extension shortly anyway. Out of caution, we only do this when `kv.expiration_leases_only.enabled` is `true`, with an eye towards 23.1 backports. For 23.2, we will always do this. Epic: none Release note: None
This patch eagerly extends expiration leases in the Raft scheduler, during Raft ticks, but only if `kv.expiration_leases_only.enabled` is `true`. This is possible because we no longer allow ranges with expiration leases to quiesce in this case. The old store lease renewer is disabled in this case, but still tracks local system ranges with expiration leases in case the setting is later changed. The store lease renewer is still retained for use by default, to allow backporting this change to 23.1 while keeping the existing default behavior. Doing this during Raft scheduling is compelling, since it already does efficient goroutine scheduling (including liveness range prioritization) and holds the replica mutex. Compared to an alternative implementation that relied on a separate, improved store lease renewer which also allowed ranges to quiesce between extensions, this has significantly better performance. The main reason for this is that unquiescing involves a Raft append to wake the leader. On an idle cluster with 20.000 ranges using only expiration leases, CPU usage was 26% vs. 30%, write IOPS was 500 vs. 900, and network traffic was 0.8 MB/s vs. 1.0 MB/s. KV benchmark results: ``` name old ops/sec new ops/sec delta kv0/enc=false/nodes=3 14.6k ± 3% 15.2k ± 4% +3.72% (p=0.003 n=8+8) kv95/enc=false/nodes=3 30.6k ± 3% 32.2k ± 3% +5.22% (p=0.000 n=8+8) name old p50 new p50 delta kv0/enc=false/nodes=3 11.3 ± 3% 11.0 ± 0% -2.76% (p=0.006 n=8+6) kv95/enc=false/nodes=3 4.59 ± 8% 4.55 ± 3% ~ (p=0.315 n=8+8) name old p95 new p95 delta kv0/enc=false/nodes=3 31.5 ± 0% 30.2 ± 4% -4.25% (p=0.003 n=6+8) kv95/enc=false/nodes=3 19.5 ± 7% 18.1 ± 4% -7.28% (p=0.000 n=8+7) name old p99 new p99 delta kv0/enc=false/nodes=3 48.7 ± 3% 47.4 ± 6% ~ (p=0.059 n=8+8) kv95/enc=false/nodes=3 32.8 ± 9% 30.2 ± 4% -8.04% (p=0.001 n=8+8) ``` Epic: none Release note: None
Previously, the replicate queue would only reacquire epoch leases, and allow expiration leases to expire. It now reacquires all leases, since they are eagerly extended. In the future, when quiescence is removed entirely, this responsibility should be moved to the Raft tick loop. Epic: none Release note: None
If `kv.expiration_leases_only.enabled´ is `true`, this patch disables expiration lease extension during request processing, since the Raft scheduler will eagerly extend leases in this case. This will eventually be removed completely, but is kept for now with an eye towards a 23.1 backport. Epic: none Release note: None
4bbc2bb to
2bae8bf
Compare
erikgrinaker
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @andrewbaptist and @nvanbenschoten)
pkg/kv/kvserver/replica_raft.go line 1200 at r3 (raw file):
Previously, andrewbaptist (Andrew Baptist) wrote…
I'm unsure if we would ever get here with an expiration lease, but if we did get here and it is an expiration lease (possibly upgrade path?) should we handle this case by creating/extending a lease? I'm thinking of cases like upgrades or setting changes.
If the setting is changed then the replicate queue will convert the lease to an expiration lease, which will wake up the Raft group, after which it will no longer quiesce. Beyond that, we don't currently attempt to eagerly acquire leases until the next client request arrives (or if the replicate queue gets to it within 10 minutes). This is equivalent to how we don't eagerly acquire epoch leases if they lapse either. We keep the scope of the Raft scheduler deliberately conservative and narrow for 23.1, so that it only extends leases once they're acquired, but we should improve this for 23.2 to make sure we always eagerly acquire leases on all ranges within seconds.
pkg/kv/kvserver/replica_range_lease.go line 1417 at r3 (raw file):
Previously, andrewbaptist (Andrew Baptist) wrote…
nit: This sentence is hard to parse now. Perhaps break it into conditions like:
The method returns true if the following conditions are all met:
- This range uses expiration-based leases.
- The most recent lease is owned by this replica.
- The lease is in need of renewal.
- There is no pending extension.
Done, thanks.
pkg/kv/kvserver/replica_lease_renewal_test.go line 55 at r3 (raw file):
Previously, andrewbaptist (Andrew Baptist) wrote…
nit: No need to change anything here, but it would be nice if we could "manually" call tick on all ranges. Something similar to
store.Enqueue. It could make these tests much more deterministic and faster and could use aManualClockinstead of having to deal with all theseEventuallycalls below.
Yeah, a lot of this code could be more amenable to unit-testing -- it would be nice to more easily test replica ticking and stepping in isolation, e.g. via a data-driven test or something. Although I also think there's value in having an integration test that wires everything up and makes sure it actually works. Either way, not a fight I'm going to pick now.
|
bors r+ |
|
Build succeeded: |
|
Encountered an error creating backports. Some common things that can go wrong:
You might need to create your backport manually using the backport tool. error creating merge commit from a71313d to blathers/backport-release-23.1-100430: POST https://api.github.com/repos/cockroachdb/cockroach/merges: 409 Merge conflict [] you may need to manually resolve merge conflicts with the backport tool. Backport to branch 23.1.x failed. See errors above. 🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is dev-inf. |
This is submitted with a 23.1 backport in mind, and is therefore limited in scope. More work is needed to fully address #98433 in the 23.2 timeframe.
kvserver: don't quiesce ranges with expiration-based leases
This patch prevents quiescence with expiration-based leases, since we'll have to propose a lease extension shortly anyway. Out of caution, we only do this when
kv.expiration_leases_only.enabledistrue, with an eye towards 23.1 backports. For 23.2, we will always do this.Touches #94592.
Epic: none
Release note: None
kvserver: eagerly extend expiration leases in Raft scheduler
This patch eagerly extends expiration leases in the Raft scheduler, during Raft ticks, but only if
kv.expiration_leases_only.enabledistrue. This is possible because we no longer allow ranges with expiration leases to quiesce in this case.The old store lease renewer is disabled in this case, but still tracks local system ranges with expiration leases in case the setting is later changed. The store lease renewer is still retained for use by default, to allow backporting this change to 23.1 while keeping the existing default behavior.
Doing this during Raft scheduling is compelling, since it already does efficient goroutine scheduling (including liveness range prioritization) and holds the replica mutex.
Compared to an alternative implementation that relied on a separate, improved store lease renewer which also allowed ranges to quiesce between extensions, this has significantly better performance. The main reason for this is that unquiescing involves a Raft append to wake the leader. On an idle cluster with 20.000 ranges using only expiration leases, CPU usage was 26% vs. 30%, write IOPS was 500 vs. 900, and network traffic was 0.8 MB/s vs. 1.0 MB/s. KV benchmark results:
Resolves #99812.
Touches #98433.
Epic: none
Release note: None
kvserver: also acquire expiration leases in replicate queue
Previously, the replicate queue would only reacquire epoch leases, and allow expiration leases to expire. It now reacquires all leases, since they are eagerly extended.
In the future, when quiescence is removed entirely, this responsibility should be moved to the Raft tick loop.
Epic: none
Release note: None
kvserver: conditionally extend expiration leases during requests
If
kv.expiration_leases_only.enabledistrue, this patch disables expiration lease extension during request processing, since the Raft scheduler will eagerly extend leases in this case. This will eventually be removed completely, but is kept for now with an eye towards a 23.1 backport.Epic: none
Release note: None