kvserver: add timeout for lease acquisitions #81136
kvserver: add timeout for lease acquisitions #81136craig[bot] merged 2 commits intocockroachdb:masterfrom
Conversation
688a39a to
c78e37e
Compare
tbg
left a comment
There was a problem hiding this comment.
Second commit LGTM, first commit I am concerned that we're making the API worse and not better by adding this, since we don't have a way to return a descriptive cancellation error.
Reviewed 2 of 2 files at r1, 1 of 1 files at r2, all commit messages.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker)
pkg/kv/kvserver/replica_range_lease.go line 362 at r2 (raw file):
// regular heartbeat. We must eventually return a NotLeaseHolderError, // otherwise we could prevent anyone else from acquiring the lease. Timeout: nlRenewal,
Let's say a range is unavailable, but the circuit breaker hasn't tripped (there isn't a lease). A request comes in and requests the lease. We will do so with a 4.5s timeout. I was worried that we wouldn't ever trip the breaker. But I think I'm wrong - the caller (this code) will abandon the lease proposal, but the proposal will stick around. So we will still tickle the breaker.
(Just rationalizing this for myself.)
pkg/util/stop/stopper.go line 503 at r1 (raw file):
// infallible. if opt.Timeout > 0 { _ = contextutil.RunWithTimeout(ctx, opt.TaskName, opt.Timeout,
This seems to defeat the purpose of using RunWithTimeout in the first place. If you specify a timeout, you want to pass a fallible closure. The nesting option that you alluded to would be better since it will actually surface the error massaged by RunWithTimeout.
erikgrinaker
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @tbg)
pkg/kv/kvserver/replica_range_lease.go line 362 at r2 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Let's say a range is unavailable, but the circuit breaker hasn't tripped (there isn't a lease). A request comes in and requests the lease. We will do so with a 4.5s timeout. I was worried that we wouldn't ever trip the breaker. But I think I'm wrong - the caller (this code) will abandon the lease proposal, but the proposal will stick around. So we will still tickle the breaker.
(Just rationalizing this for myself.)
Yeah, thanks for verifying. It seems unfortunate that all of these mechanisms (lease acquisitions, circuit breakers, etc) are driven by client requests.
pkg/util/stop/stopper.go line 503 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
This seems to defeat the purpose of using
RunWithTimeoutin the first place. If you specify a timeout, you want to pass a fallible closure. The nesting option that you alluded to would be better since it will actually surface the error massaged byRunWithTimeout.
If we do the call manually we're still in the same situation: we're in an infallible goroutine and have to handle the error from RunWithTimeout. The same is true if the caller simply passes in a cancellable context to RunAsyncTask: the context error must be handled inside the infallible goroutine. These errors must either be logged or passed to a different goroutine regardless of this option.
erikgrinaker
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @tbg)
pkg/util/stop/stopper.go line 503 at r1 (raw file):
Previously, erikgrinaker (Erik Grinaker) wrote…
If we do the call manually we're still in the same situation: we're in an infallible goroutine and have to handle the error from
RunWithTimeout. The same is true if the caller simply passes in a cancellable context toRunAsyncTask: the context error must be handled inside the infallible goroutine. These errors must either be logged or passed to a different goroutine regardless of this option.
Oh wait, yeah I see what you're saying. I'll drop it.
1ee0204 to
5a2cbde
Compare
erikgrinaker
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @tbg)
pkg/util/stop/stopper.go line 503 at r1 (raw file):
Previously, erikgrinaker (Erik Grinaker) wrote…
Oh wait, yeah I see what you're saying. I'll drop it.
Done, PTAL.
5a2cbde to
1086348
Compare
erikgrinaker
left a comment
There was a problem hiding this comment.
Second commit LGTM, first commit I am concerned that we're making the API worse and not better by adding this, since we don't have a way to return a descriptive cancellation error.
I think Reviewable ate the other comment thread, since I can't find it anywhere in the UI anymore now that it's resolved. Anyway, the new error handling requires a decode/encode roundtrip for roachpb.Error -- I'm a bit worried that those can be lossy, but let's see what CI says.
Reviewed 3 of 3 files at r3, all commit messages.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @tbg)
3a8260b to
2de8289
Compare
tbg
left a comment
There was a problem hiding this comment.
Reviewed 1 of 1 files at r4, all commit messages.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker)
|
I'm seeing two CI failures here:
Going to resolve these somehow before merging. |
2de8289 to
0f3f5d2
Compare
This patch deflakes `TestLeasePreferencesDuringOutage` by allowing nodes to acquire a lease even when they are not the Raft leader. Release note: None
This patch adds a timeout for lease acquisitions. It is set to twice the Raft election timeout (6 seconds total), since it may need to hold a Raft election and repropose the lease acquisition command, each of which may take up to one election timeout. Without this timeout, it's possible for a lease acquisition to stall indefinitely (e.g. in the case of a stalled disk). This prevents a `NotLeaseHolderError` from being returned to the client DistSender, which in turn prevents it from trying other replicas that could acquire the lease instead. This can cause a lease to remain invalid forever. Release note (bug fix): Fixed a bug where an unresponsive node (e.g. with a stalled disk) could prevent other nodes from acquiring its leases, effectively stalling these ranges until the node was shut down or recovered.
0f3f5d2 to
ab74b97
Compare
This is being resolved in cockroachdb/pebble#1719. We can merge this regardless, since this is already flaky on
I had to increase the timeout from the heartbeat timeout (4.5 seconds) to twice the Raft election timeout (6 seconds), since it can take up to two election timeouts to elect a new Raft leader and then repropose the lease acquisition command. I also deflaked the test in a separate commit. Mind giving this another look @tbg? |
tbg
left a comment
There was a problem hiding this comment.
Reviewed 2 of 2 files at r5, 1 of 1 files at r6, all commit messages.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker)
|
TFTR! bors r=tbg |
|
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 ab74b97 to blathers/backport-release-21.2-81136: 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 21.2.x failed. See errors above. error creating merge commit from ab74b97 to blathers/backport-release-22.1-81136: 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 22.1.x failed. See errors above. 🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is otan. |
kvserver: deflake
TestLeasePreferencesDuringOutageThis patch deflakes
TestLeasePreferencesDuringOutageby allowing nodesto acquire a lease even when they are not the Raft leader.
Release note: None
kvserver: add timeout for lease acquisitions
This patch adds a timeout for lease acquisitions. It is set to
twice the Raft election timeout (6 seconds total), since it may need to
hold a Raft election and repropose the lease acquisition command, each
of which may take up to one election timeout.
Without this timeout, it's possible for a lease acquisition to stall
indefinitely (e.g. in the case of a stalled disk). This prevents a
NotLeaseHolderErrorfrom being returned to the client DistSender,which in turn prevents it from trying other replicas that could acquire
the lease instead. This can cause a lease to remain invalid forever.
Touches #81100.
Release note (bug fix): Fixed a bug where an unresponsive node (e.g.
with a stalled disk) could prevent other nodes from acquiring its
leases, effectively stalling these ranges until the node was shut down
or recovered.