-
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
storage: transfer raft leadership and wait grace period when draining #22767
Conversation
Planning on cherry-picking this into 1.1 and 2.0 for rolling upgrades. |
Review status: 0 of 6 files reviewed at latest revision, 1 unresolved discussion, all commit checks successful. pkg/server/node.go, line 474 at r3 (raw file):
I don't think this is the right place for this wait. Don't we have something that waits for all leases to be transferred away? I was thinking we'd add an extra delay for raft operations after that has happened. This duration should also be a named constant (if not a cluster setting). Comments from Reviewable |
Is there anything we can do to make sure that Raft leadership does not transfer back to a draining replica, like we do for the range lease? With a 5 second sleep before we fully shut down, I can envision this being a problem. I don't think Reviewed 1 of 1 files at r1, 1 of 1 files at r2, 3 of 5 files at r3. pkg/server/node.go, line 474 at r3 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Yeah I was also thinking that the delay should be in pkg/storage/store.go, line 938 at r3 (raw file):
nit: Comments from Reviewable |
@nvanbenschoten that's a good point. What would the cleanest way to set an election timeout be? It's not clear to me how to change this after the initial election timeout is set through the config (i.e. election ticks). We could disable ticks but that seems disruptive. @bdarnell, what are your thoughts on this? Review status: 3 of 6 files reviewed at latest revision, 2 unresolved discussions. pkg/server/node.go, line 474 at r3 (raw file): Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
I don't think we want to do a wait once per pkg/storage/store.go, line 938 at r3 (raw file): Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Done. Comments from Reviewable |
It'd be nice if we could break out of the wait early if we've verified that all raft leaders have moved elsewhere (via the Reviewed 1 of 1 files at r1, 1 of 1 files at r2. Comments from Reviewable |
Added the state check in as I think the general consensus is for it. I lean towards having this check rather than a timeout but @bdarnell also suggested that timeout-based draining was preferable. @nvanbenschoten, regarding avoiding elections when draining I know we talked about delaying this PR until we could implement a solution upstream, but what do you think of getting this in as-is and revisiting this problem once the upstream changes are figured out? This PR provides us with a big enough win that I think it should be cherrypicked in and corner cases solved afterwards (especially after the elimination of the timeout-based approach). Review status: 3 of 6 files reviewed at latest revision, 2 unresolved discussions, all commit checks successful. pkg/server/node.go, line 474 at r3 (raw file): Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
Removed the cluster setting, every replica now waits for the leadership transfer if there was a lease transfer. Comments from Reviewable |
The rebalancer knows not to manually transfer things back because of the draining flag on the node descriptor. Other than rebalancer actions, leadership only moves when there are failures. If we get unlucky and our new target fails while we're draining and we become leader again, that's not ideal but I don't think it's a significant reduction in availability. (also note that the election timeout is currently 9s, so it couldn't currently happen within a 5s window) (If you do want to stop elections while we're draining, the simplest solution is for Replica.sendRaftMessage to drop MsgVotes)
Why so impatient? You need shutdown to take more than 5s in practice for load balancer health checks to take a node out of circulation anyway. I advocate for time-based draining instead of condition-based draining for several reasons. First, it's simpler if you don't have to write the conditions and hook them up in the right place. It also avoids the problem of choosing the wrong condition (if we had slept after transferring all leases instead of exiting as soon as that was finished, the leadership transfers would mostly work out on their own with no need for this change). Consistency between normal and abnormal cases is also important (if a 5s delay per node leads to timeouts in your deployment tools, for example, you want to discover this in normal operations instead of having it only show up when other problems are happening too) Review status: 3 of 5 files reviewed at latest revision, 4 unresolved discussions, all commit checks successful. pkg/storage/replica_proposal.go, line 423 at r4 (raw file):
Why can't we remove this pkg/storage/store.go, line 1003 at r4 (raw file):
Waiting on every replica inside the LimitedAsyncTask limits our parallelism (we'll have to wait for at least three additional round trips per replica). If we're going to have this condition, we should move it to the end of SetDraining with another store replica visitor inside a retry.ForDuration. Comments from Reviewable |
Use t.Helper() in a helper function that checks lease placement to get the calling line number on failure. Release note: None
The previous version of getRaftLeader would not return the leader if all replicas had the same term and the leader wasn't the first to be iterated over. Release note: None
These are all good points. I don't mind having the timeout. @a-robinson or @nvanbenschoten, are you strongly opposed to this? I'll move forward if not. Review status: 3 of 5 files reviewed at latest revision, 4 unresolved discussions, all commit checks successful. Comments from Reviewable |
I hadn't considered the effects of inconsistency on load balancer lag. I'm sold on the constant wait time. Review status: 3 of 5 files reviewed at latest revision, 4 unresolved discussions, all commit checks successful. Comments from Reviewable |
Really? I thought it was 3 seconds (
That's a good point. If we see this being an issue then we should pursue this approach.
Is there a suggested amount of time we should be waiting then?
No, I don't see any issue with it, as long as we make sure that a longer timeout doesn't undermine the rest of the work we do to gently drain a node. |
It's all dependent on your load balancer config. I'm aiming to provide another timeout (as a cluster setting) that fails readiness checks before we actually do any draining. |
Ah, you're right.
Anecdotally, the systems I've worked on have used 30s. The default health check interval for haproxy is 2s and for nginx is 5s, so we might be able to get away with as little as 10-15s. |
Previously, even though a draining replica would transfer its lease, when it came to `leasePostApply`, the raft leadership would not be transferred because the target was not up-to-date enough. This would result in the draining replica taking the leadership with it and producing a zero qps scenario until a new election was held after an election timeout. This change adds a draining flag to a replica so that it may skip this check. A draining store will also wait for a minimum of 5s after transferring away all of its replica's leases to allow for the raft leadership change to take place. Addresses #22573. Release note (bug fix): Fix a zero qps scenario when a node would be gracefully drained.
Review status: 0 of 5 files reviewed at latest revision, 4 unresolved discussions. pkg/storage/replica_proposal.go, line 423 at r4 (raw file): Previously, bdarnell (Ben Darnell) wrote…
#22573 (comment), I'm unsure as to the causes of this. We do check that the lease is valid before attempting to transfer leadership. pkg/storage/store.go, line 1003 at r4 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Done. Removed the condition check and reverted to timeout at the end of Comments from Reviewable |
Review status: 0 of 5 files reviewed at latest revision, 3 unresolved discussions, all commit checks successful. Comments from Reviewable |
Previously, even though a draining replica would transfer its lease,
when it came to
leasePostApply
, the raft leadership would not betransferred because the target was not up-to-date enough. This would
result in the draining replica taking the leadership with it and
producing a zero qps scenario until a new election was held after
an election timeout.
This change adds a draining flag to a replica so that it may skip this
check. A draining node will also wait for a minimum of 5s after
transferring away all of its replica's leases to allow for the raft
leadership change to take place.
Addresses #22573.
Release note (bug fix): Fix a zero qps scenario when a node would be
gracefully drained.
The first two commits improve some testing issues I encountered.
One thing I'm wondering about is how best to avoid the wait in
SetDraining
during a test run. We might also want to not wait in certain cases. I wasn't sure so I just kept it simple and we can iterate on it.cc @bdarnell