-
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: restarted node in need of snapshots can be wedged for long time #37906
Comments
Can the leader detect if one or more of the Raft followers is behind, and if so, delay Raft log truncation until the dead node timeout expires (i.e. 5 minutes)? |
Raft log truncation at the time of writing takes into account node liveness in the sense that it will immediately become willing to "sacrifice" followers that are not live (i.e. after ~4-5s of downtime). As we discussed in the weekly, this amounts to a historical artifact (because we had to be very careful to not let the Raft log grow large historically due to its inclusion in snapshots) that isn't a concern any more as apparent from this issue the current behavior is undesirable. I suggest the following heuristics:
The second step may seem unexpectedly simple, but I think it checks out. If we sacrifice followers the moment they're marked as dead they will still have most of their replicas (removing them requires sending snapshots elsewhere, so it'll go at a leisurely pace compared to how fast the log queue can truncate). A dead node's replicas are supposedly going to be transferred away completely over time OR the node will restart, at which point log truncation will become possible again (since no dead follower exists any more). And the first criterion protects us from amassing a prohibitively long Raft log. At the same time, it decouples the log queue from the store dead timeout, which is a nice property. |
The other problem we have here is that replicas in need of a snapshot translate the latency of queueing for and finally receiving the snapshot to any clients waiting on the replica. If we can detect the situation in which a replica is waiting for a snapshot without a timeout, we're in a good place to prevent this from becoming a problem because we can refuse the requests with a However, one spicy problem is that when a request first arrives at such a range, the result will be a lease request proposed to Raft. This lease request's outcome will only be known after the snapshot arrives (and the replica can catch up all the way to the result of the lease request). As a result, if the replica actually succeeds in acquiring the lease (it may well if no other lease has been active since the replica went down), no other replica can serve traffic until the stuck replica receives its snapshot, effectively stalling that range for that duration (which can be many hours depending on the amount of data that needs to be transferred). We could "probe" Raft (with a timeout) before issuing any lease requests and could build a small state machine around the probes that can translate into a circuit breaker to solve the first problem above. "Probing" could either mean submitting a noop through Raft, or sending a one-off RPC to a quorum of replicas and noting the highest committed index, which would be required to be surpassed by that of the local replica before declaring the probe a success. I think to defuse this issue to 80% we need to fix the log truncation and come up with a relatively reliable indicator for returning a NotLeaseholderError, falling back to a timeout on the lease request to prevent the worst case of requests getting stuck indefinitely. The situation in which the down node gets a lease is hopefully rare because it needing a snapshot sort of implies that there was another leader in the meantime that wrote more log entries. Recording the outcome of what @danhhz and I just discussed: we're prioritizing getting something in place for 19.1.3 which is going to be cut in ~ a week, so time is of the essence. We're conjecturing that we'll get most bang for our buck by relaxing the raft log queue truncation heuristics to avoid sacrificing dead followers until the size of the raft log gets bigger than a few mb (already true today). At worst this will lead to few-mb Raft logs which aren't a concern (for example, during IMPORT/RESTORE we regularly dozens of mbs in the Raft log, though concentrated in only a handful of large entries). This will achieve the absence of snapshots for any but the "hot" ranges (that write more than X mb during the downtime) where previously even moderately lukewarm ranges would require snapshots. We'll get this landed on master ASAP and let it bake, and additionally get some more manual testing in using the steps in the first issue. Additionally we'll return a NotLeaseholderError after a timeout from getting the lease as a failsafe. |
Could the quota pool cause this to escalate into even more severe failures? If the node is up and responding to heartbeats but can't accept new log entries until it gets a snapshot, it will prevent the release of proposal quota and then block writes even on the live nodes. |
@bdarnell the quota pool handles this case, though we'll need to check that it actually works as advertised, I don't think it ever mattered before. cockroach/pkg/storage/replica_proposal_quota.go Lines 204 to 209 in c097a16
|
I can reproduce this readily with a ~10m roachtest that's basically Tobi's original instructions. In addition to absurdly long response times on SQL queries, I sometimes can't even get a SQL connection to the just restarted node (getting user info?). (I'm also curious if this effect is the underlying cause of what I was seeing in #36879.) I've looked at this today and I think we need two mitigations. First, when a node is down for a bit (~minutes), we make it less likely for it to need a snapshot when it is restarted. Second, we lessen the impact if it does need snapshots. I've focused first on things that are hopefully safe enough to backport into 19.1.3, to unblock our affected customer. 19.1.3 is cut a week from Monday, so I'm hoping to get fixes in soon so they can bake for a week and shake out any unforeseen consequences. To avoid snapshots when a node comes back, we'll make the log truncation less aggressive. I think something like what's described in #37906 (comment) is a good long term solution, but to me it feels too risky to backport without more baking time than we have. A good compromise is to simply raise the threshold for what the raft log queue considers "recently active". (The raft log queue doesn't truncate when a follower is missing but has been recently active.) Recent currently has a threshold of 10s and shares the same MaxQuotaReplicaLivenessDuration constant as the quota pool. This sharing seems to have been introduced in #32439 but not for any particular principled reason. A natural place to raise the threshold to is the Experimentally this seems to work decently well. There are still a small number of snapshots, but it's small enough that the node recovers quickly when it comes back up. We should investigate why there are still this small number of snapshots, but in the meantime the second mitigation (lessening the impact of shapshots on a node that's just come up) will cover the rest. Note that this only helps if the node is down for less than For the second mitigation, I think we need to have some sort of timeout on redirectOnOrAcquireLease, will will return NotLeaseholderError when it fires. redirectOnOrAcquireLease currently blocks on getting the snapshot, which can be hours if every replica is getting one and this replica is at the end of the line. Relying on this timeout will have an affect on tail latencies, but so far, it's the least risky thing I've thought of. Some more sophisticated ideas have been tossed around to do this without the tail latency, but it's nice to have a low-risk, obvious-how-it-works CYA here. I have yet to run experiments for this. The two big open questions are what threshold to use for the timeout and what happens if the timeout fires on the leaseholder? |
Previously, we'd hold off on truncating the raft log if a replica was missing but contactable in the last 10 seconds. This meant that if a node was down for *more* than 10 seconds, there was a good chance that we'd truncate logs for some of its replicas (especially for its high-traffic ones) and it would need snapshots for them when it came back up. This was for two reasons. First, we've historically assumed that it's cheaper to catch a far-behind node up with a snapshot than with entries. Second, when we replicated the raft log truncation, we'd have to include it in snapshots anyway, but that's changed recently. The problem is when a node is down for a short time but more than 10 seconds (think a couple minutes). It might come back up to find that it needs a snapshot for most ranges. We rate limit snapshots fairly aggressively because they've been disruptive in the past, but this means that it could potentially take hours for a node to recover from a 2 minute outage. This would be merely unfortunate if there wasn't a second compounding issue. A recently restarted node has a cold leaseholder cache. When it gets traffic for one of its replicas, it first tries itself as the leaseholder (maybe it will get lucky and won't need the network hop). Unfortunately, if the replica needs a snapshot, this decision currently blocks on it. This means that requests sent to the recently started node could block for as long as the heavily-throttled snapshots take, hours or even days. Short outages of more than 10 seconds are reasonably common with routine maintenance (rolling to a new version, swapping hardware, etc), so it's likely that customers will hit this (and one did). This commit ties the threshold that the raft log queue uses for recent activity to `server.time_until_store_dead`, which is already the threshold where we give up on a down node and start transferring away its replicas. It defaults to 5 minutes, which is still short, but it's more reasonable than 10 seconds. Crucially, it also is a cluster setting, so it can be overridden. We'd like to move even further in the direction of leniency about raft log truncation while a replica is missing, but this is a compromise that's intended to be less risky to backport to 19.1.x. Partial mitigation for cockroachdb#37906 Potentially also helps with cockroachdb#36879 Release note (bug fix): Nodes that have been down for less than `server.time_until_store_dead` now recover more quickly when they rejoin.
I posted some discussion of the log truncation approach on the review in #38484. The TL;DR is that the most intuitive safe code change seems to not care about the 5 minute distinction, which is nice. We leave nodes alone when they're up, and when they're not up we also leave them alone unless the log is large, in which case we sacrifice them. (Today the "leave them alone unless" isn't there, we go straight to sacrificing by pretending the followers aren't even there). For the timeout in For implementing this, do you really have to do anything more than change this code cockroach/pkg/storage/replica_range_lease.go Lines 1028 to 1032 in c097a16
to early return with the |
A long-term solution to the redirectOnOrAcquireLease problem would be to replace raft-level proposal forwarding with an RPC that could return the index of the new proposal. Then we could know to stop waiting if our index is way behind the index of our proposed lease. (or we could forward the proposal with a maxIndex so it could be dropped if the leader is ahead). Replacing the proposal flow is also contemplated in #21849. |
Previously, we'd hold off on truncating the raft log if a replica was missing but contactable in the last 10 seconds. This meant that if a node was down for *more* than 10 seconds, there was a good chance that we'd truncate logs for some of its replicas (especially for its high-traffic ones) and it would need snapshots for them when it came back up. This was for two reasons. First, we've historically assumed that it's cheaper to catch a far-behind node up with a snapshot than with entries. Second, snapshots historically had to include the Raft log which implied a need to keep the size of the Raft log tightly controlled due to being pulled into memory at the snapshot receiver, but that's changed recently. The problem is when a node is down for longer than 10 seconds but shorter than the time it takes to upreplicate all of its ranges onto new nodes. It might come back up to find that it needs a snapshot for most ranges. We rate limit snapshots fairly aggressively because they've been disruptive in the past, but this means that it could potentially take hours for a node to recover from a 2 minute outage. This would be merely unfortunate if there wasn't a second compounding issue. A recently restarted node has a cold leaseholder cache. When it gets traffic for one of its replicas, it first tries itself as the leaseholder (maybe it will get lucky and won't need the network hop). Unfortunately, if the replica needs a snapshot, this decision currently blocks on it. This means that requests sent to the recently started node could block for as long as the heavily-throttled snapshots take, hours or even days. Short outages of more than 10 seconds are reasonably common with routine maintenance (rolling to a new version, swapping hardware, etc), so it's likely that customers will hit this (and one did). This commit avoids truncating the log past any follower's position when all replicas have recently been active (the quota pool keeps it from growing without bound in this case). If at least one replica hasn't recently been active, it holds off any truncation until the log reaches a size threshold. Partial mitigation for cockroachdb#37906 Potentially also helps with cockroachdb#36879 Release note (bug fix): Nodes that have been down for less than `server.time_until_store_dead` now recover more quickly when they rejoin.
I proposed something similar in #37906 (comment), but I like the idea of making this about Raft proposal forwarding. If the follower includes its commit index into the proposal the leader will be in a good position to make a reasonable call via heuristics, though this necessarily involves some magic values which would be good to avoid. Instead, the leader could refuse proposals unless the follower is in |
Previously, we'd hold off on truncating the raft log if a replica was missing but contactable in the last 10 seconds. This meant that if a node was down for *more* than 10 seconds, there was a good chance that we'd truncate logs for some of its replicas (especially for its high-traffic ones) and it would need snapshots for them when it came back up. This was for two reasons. First, we've historically assumed that it's cheaper to catch a far-behind node up with a snapshot than with entries. Second, snapshots historically had to include the Raft log which implied a need to keep the size of the Raft log tightly controlled due to being pulled into memory at the snapshot receiver, but that's changed recently. The problem is when a node is down for longer than 10 seconds but shorter than the time it takes to upreplicate all of its ranges onto new nodes. It might come back up to find that it needs a snapshot for most ranges. We rate limit snapshots fairly aggressively because they've been disruptive in the past, but this means that it could potentially take hours for a node to recover from a 2 minute outage. This would be merely unfortunate if there wasn't a second compounding issue. A recently restarted node has a cold leaseholder cache. When it gets traffic for one of its replicas, it first tries itself as the leaseholder (maybe it will get lucky and won't need the network hop). Unfortunately, if the replica needs a snapshot, this decision currently blocks on it. This means that requests sent to the recently started node could block for as long as the heavily-throttled snapshots take, hours or even days. Short outages of more than 10 seconds are reasonably common with routine maintenance (rolling to a new version, swapping hardware, etc), so it's likely that customers will hit this (and one did). This commit avoids truncating the log past any follower's position when all replicas have recently been active (the quota pool keeps it from growing without bound in this case). If at least one replica hasn't recently been active, it holds off any truncation until the log reaches a size threshold. Partial mitigation for cockroachdb#37906 Potentially also helps with cockroachdb#36879 Release note (bug fix): Nodes that have been down for less than `server.time_until_store_dead` now recover more quickly when they rejoin.
Previously, we'd hold off on truncating the raft log if a replica was missing but contactable in the last 10 seconds. This meant that if a node was down for *more* than 10 seconds, there was a good chance that we'd truncate logs for some of its replicas (especially for its high-traffic ones) and it would need snapshots for them when it came back up. This was for two reasons. First, we've historically assumed that it's cheaper to catch a far-behind node up with a snapshot than with entries. Second, snapshots historically had to include the Raft log which implied a need to keep the size of the Raft log tightly controlled due to being pulled into memory at the snapshot receiver, but that's changed recently. The problem is when a node is down for longer than 10 seconds but shorter than the time it takes to upreplicate all of its ranges onto new nodes. It might come back up to find that it needs a snapshot for most ranges. We rate limit snapshots fairly aggressively because they've been disruptive in the past, but this means that it could potentially take hours for a node to recover from a 2 minute outage. This would be merely unfortunate if there wasn't a second compounding issue. A recently restarted node has a cold leaseholder cache. When it gets traffic for one of its replicas, it first tries itself as the leaseholder (maybe it will get lucky and won't need the network hop). Unfortunately, if the replica needs a snapshot, this decision currently blocks on it. This means that requests sent to the recently started node could block for as long as the heavily-throttled snapshots take, hours or even days. Short outages of more than 10 seconds are reasonably common with routine maintenance (rolling to a new version, swapping hardware, etc), so it's likely that customers will hit this (and one did). This commit avoids truncating the log past any follower's position when all replicas have recently been active (the quota pool keeps it from growing without bound in this case). If at least one replica hasn't recently been active, it holds off any truncation until the log reaches a size threshold. Partial mitigation for cockroachdb#37906 Potentially also helps with cockroachdb#36879 Release note (bug fix): Nodes that have been down now recover more quickly when they rejoin, assuming they weren't down for much more than the value of the `server.time_until_store_dead` cluster setting (which defaults to 5 minutes).
38484: storage: truncate raft log less aggressively when replica is missing r=tbg a=danhhz Previously, we'd hold off on truncating the raft log if a replica was missing but contactable in the last 10 seconds. This meant that if a node was down for *more* than 10 seconds, there was a good chance that we'd truncate logs for some of its replicas (especially for its high-traffic ones) and it would need snapshots for them when it came back up. This was for two reasons. First, we've historically assumed that it's cheaper to catch a far-behind node up with a snapshot than with entries. Second, when we replicated the raft log truncation, we'd have to include it in snapshots anyway, but that's changed recently. The problem is when a node is down for a short time but more than 10 seconds (think a couple minutes). It might come back up to find that it needs a snapshot for most ranges. We rate limit snapshots fairly aggressively because they've been disruptive in the past, but this means that it could potentially take hours for a node to recover from a 2 minute outage. This would be merely unfortunate if there wasn't a second compounding issue. A recently restarted node has a cold leaseholder cache. When it gets traffic for one of its replicas, it first tries itself as the leaseholder (maybe it will get lucky and won't need the network hop). Unfortunately, if the replica needs a snapshot, this decision currently blocks on it. This means that requests sent to the recently started node could block for as long as the heavily-throttled snapshots take, hours or even days. Short outages of more than 10 seconds are reasonably common with routine maintenance (rolling to a new version, swapping hardware, etc), so it's likely that customers will hit this (and one did). This commit ties the threshold that the raft log queue uses for recent activity to `server.time_until_store_dead`, which is already the threshold where we give up on a down node and start transferring away its replicas. It defaults to 5 minutes, which is still short, but it's more reasonable than 10 seconds. Crucially, it also is a cluster setting, so it can be overridden. We'd like to move even further in the direction of leniency about raft log truncation while a replica is missing, but this is a compromise that's intended to be less risky to backport to 19.1.x. Partial mitigation for #37906 Potentially also helps with #36879 Release note (bug fix): Nodes that have been down now recover more quickly when they rejoin, assuming they weren't down for much more than the value of the `server.time_until_store_dead` cluster setting (which defaults to 5 minutes). Co-authored-by: Daniel Harrison <[email protected]> Co-authored-by: Tobias Schottdorf <[email protected]>
Previously, we'd hold off on truncating the raft log if a replica was missing but contactable in the last 10 seconds. This meant that if a node was down for *more* than 10 seconds, there was a good chance that we'd truncate logs for some of its replicas (especially for its high-traffic ones) and it would need snapshots for them when it came back up. This was for two reasons. First, we've historically assumed that it's cheaper to catch a far-behind node up with a snapshot than with entries. Second, snapshots historically had to include the Raft log which implied a need to keep the size of the Raft log tightly controlled due to being pulled into memory at the snapshot receiver, but that's changed recently. The problem is when a node is down for longer than 10 seconds but shorter than the time it takes to upreplicate all of its ranges onto new nodes. It might come back up to find that it needs a snapshot for most ranges. We rate limit snapshots fairly aggressively because they've been disruptive in the past, but this means that it could potentially take hours for a node to recover from a 2 minute outage. This would be merely unfortunate if there wasn't a second compounding issue. A recently restarted node has a cold leaseholder cache. When it gets traffic for one of its replicas, it first tries itself as the leaseholder (maybe it will get lucky and won't need the network hop). Unfortunately, if the replica needs a snapshot, this decision currently blocks on it. This means that requests sent to the recently started node could block for as long as the heavily-throttled snapshots take, hours or even days. Short outages of more than 10 seconds are reasonably common with routine maintenance (rolling to a new version, swapping hardware, etc), so it's likely that customers will hit this (and one did). This commit avoids truncating the log past any follower's position when all replicas have recently been active (the quota pool keeps it from growing without bound in this case). If at least one replica hasn't recently been active, it holds off any truncation until the log reaches a size threshold. Partial mitigation for cockroachdb#37906 Potentially also helps with cockroachdb#36879 Release note (bug fix): Nodes that have been down now recover more quickly when they rejoin, assuming they weren't down for much more than the value of the `server.time_until_store_dead` cluster setting (which defaults to 5 minutes).
A first mitigation has landed on master and is very likely to be included in the next 19.1.x patch release. While this first mitigation has solved the problem in our testing and should greatly reduce it in most real-world scenarios too, we are working on additional safeguards related to this issue:
|
Should we backport #38484 to 2.1 as well? |
Does 2.1 have the work that removes the raft log from snapshots? If not, we might want to do 2.1-specific stability testing of that PR before backporting. |
No, that's 19.1. With 2.1 we run a higher risk of seeing that snapshot refusal loop again (I think snapshots with 8mb are refused, and truncation would kick in at 4mb). I believe the refusal code was often triggered by SSTs which clock in at ~22mb each from what I remember, so changing the truncation heuristics likely wouldn't make a big difference for the worse (but it should make one for the better from what we can tell). But we'd need to do our due diligence there, I'm thinking that it would pass the new roachtest 5x and a reasonably sized import (1+ hour) that passes reliably (are there any of them?) before the cherry pick should continue to do so after. |
This patch aims to improve the behavior in scenarios where a follower replica is behind, unaware of the latest lease, and it tries to acquire a lease in its ignorant state. That lease acquisition request is bound to fail (because the lease that it's based on is stale), but while it fails (or, rather, until the behind replica finds out that it failed) local requests are blocked. This blocking can last for a very long time in situations where a snapshot is needed to catch up the follower, and the snapshot is queued up behind many other snapshots (e.g. after a node has been down for a while and gets restarted). This patch tries an opinionated solution: never allow followers to acquire leases. If there is a leader, it's a better idea for the leader to acquire the lease. The leader might have a lease anyway or, even if it doesn't, having the leader acquire it saves a leadership transfer (leadership follows the lease). We change the proposal path to recognize lease requests and reject them early if the current replica is a follower and the leader is known. The rejection points to the leader, which causes the request that triggered the lease acquisition to make its way to the leader and attempt to acquire a lease over there. Fixes cockroachdb#37906 As described in cockroachdb#37906, the badness caused by requests blocking behind a doomed lease acq request could be reproduced with a 100-warehouse tpcc workload (--no-wait) on a 3 node cluster. Turning off a node for 10 minutes and then turning it back on would result in the cluster being wedged for a few minutes until all the snapshots are transferred. I've verified that this patch fixes it. Release note (bug fix): A bug causing queries sent to a freshly-restarted node to sometimes hang for a long time while the node catches up with replication has been fixed.
This reverts commit 8f98ade. This is a revert of the main commit in cockroachdb#57789 (which was a backport of cockroachdb#55148). It turns out that the commit in question introduced a deadlock: if replicas that are not the leader refuse to take the lease (because their not the leader) and the leader is a VOTER_INCOMING, VOTER_DEMOTING or VOTER_OUTGOING replica which also refuses to take the lease because it's not a VOTER_FULL [0] => deadlock. This deadlock was found in cockroachdb#57798. The patch will return with some massaging. This revert is original work on the 20.2 branch; I'm not reverting it on master in the hope of just fixing the issue. Touches cockroachdb#37906 Release note: This voids the previous release note reading "A bug causing queries sent to a freshly-restarted node to sometimes hang for a long time while the node catches up with replication has been fixed."
…iveness Fixes cockroachdb#84655. Related to cockroachdb#49220. This commit extends the logic introduced in 8aa1c14 to simultaneously campaign for Raft leadership when rejecting a lease request on a Raft follower that observes that the current Raft leader is not live according to node liveness. These kinds of cases are often associated with asymmetric network partitions. In such cases, the Raft leader will be unable to acquire an epoch-based lease until it heartbeats its liveness record. As a result, the range can only regain availability if a different replica acquires the lease. However, the protection added in 8aa1c14 prevents followers from acquiring leases to protect against a different form of unavailability. After this commit, the followers will attempt to acquire Raft leadership when detecting such cases by campaigning. This allows these ranges to recover availability once Raft leadership moves off the partitioned Raft leader to one of the followers that can reach node liveness and can subsequently acquire the lease. Campaigning for Raft leadership is safer than blindly allowing the lease request to be proposed (through a redirected proposal). This is because the follower may be arbitrarily far behind on its Raft log and acquiring the lease in such cases could cause unavailability (the kind we saw in cockroachdb#37906). By instead calling a Raft pre-vote election, the follower can determine whether it is behind on its log without risking disruption. If so, we don't want it to acquire the lease — one of the other followers that is caught up on its log can. If not, it will eventually become leader and can proceed with a future attempt to acquire the lease. The commit adds a test that reproduces the failure mode described in cockroachdb#84655. It creates an asymmetric network partition scenario that looks like: ``` [0] raft leader / initial leaseholder ^ / \ / \ v v [1]<--->[2] raft followers ^ ^ \ / \ / v [3] liveness range ``` It then waits for the raft leader's lease to expire and demonstrates that one of the raft followers will now call a Raft election, which allows it to safely grab Raft leadership, acquire the lease, and recover availability. Without the change, the test failed. Release justification: None. Too risky for the stability period. Potential backport candidate after sufficient baking on master. Release note (bug fix): A bug causing ranges to remain without a leaseholder in cases of asymmetric network partitions has been resolved.
87244: kv: campaign on rejected lease request when leader not live in node liveness r=erikgrinaker a=nvanbenschoten Fixes #84655. Related to #49220. This commit extends the logic introduced in 8aa1c14 to simultaneously campaign for Raft leadership when rejecting a lease request on a Raft follower that observes that the current Raft leader is not live according to node liveness. These kinds of cases are often associated with asymmetric network partitions. In such cases, the Raft leader will be unable to acquire an epoch-based lease until it heartbeats its liveness record. As a result, the range can only regain availability if a different replica acquires the lease. However, the protection added in 8aa1c14 prevents followers from acquiring leases to protect against a different form of unavailability. After this commit, the followers will attempt to acquire Raft leadership when detecting such cases by campaigning. This allows these ranges to recover availability once Raft leadership moves off the partitioned Raft leader to one of the followers that can reach node liveness and can subsequently acquire the lease. Campaigning for Raft leadership is safer than blindly allowing the lease request to be proposed (through a redirected proposal). This is because the follower may be arbitrarily far behind on its Raft log and acquiring the lease in such cases could cause unavailability (the kind we saw in #37906). By instead calling a Raft pre-vote election, the follower can determine whether it is behind on its log without risking disruption. If so, we don't want it to acquire the lease — one of the other followers that is caught up on its log can. If not, it will eventually become leader and can proceed with a future attempt to acquire the lease. The commit adds a test that reproduces the failure mode described in #84655. It creates an asymmetric network partition scenario that looks like: ``` [0] raft leader / leaseholder ^ / \ / \ v v [1]<--->[2] raft followers ^ ^ \ / \ / v [3] liveness range ``` It then waits for the raft leader's lease to expire and demonstrates that one of the raft followers will now call a Raft election, which allows it to safely grab Raft leadership, acquire the lease, and recover availability. Without the change, the test failed. ---- Release justification: None. Too risky for the stability period. Potential backport candidate after sufficient baking on master. Release note (bug fix): A bug causing ranges to remain without a leaseholder in cases of asymmetric network partitions has been resolved. 88101: kv,sql: simplify the Txn API by removing 2 cleanup functions r=lidorcarmel a=lidorcarmel Txn.CleanupOnError() basically does a rollback, and in addition takes an error only for the purpose of logging it. Txn.CommitOrCleanup() tries to commit and if unsuccessful it tries a rollback. The error from the rollback is logged but not returned, the error from the commit is returned. Removing these 2 functions means that the caller should call Commit and Rollback directly when needed, and handle the returned errors. For example, sql may need to log errors to a different channel from the one used but Txn, and tests may want to fail when a Rollback fails unexpectedly. This PR removes those functions. Release note: None Epic: None 91011: storage: adjust range tombstone language to encourage caution r=nicktrav a=jbowens Adjust the MVCC range tombstone cluster setting description to highlight its experimental nature and to be appropriately cautious about the consequence of enabling range tombstones. Cockroach 22.2.0 will ship with the cluster setting and likely at least one bug that may induce corruption if the setting is enabled (#90948). Epic: None Release note: None Close #91001. 91077: sql/schemachange: revert TableZoneConfig field ID r=fqazi a=fqazi Fixes: #91053 Backwards compatibility with 22.2 was accidentally regressed, when the field ID for tabe_zone_config was changed for older declarative schema changer states. To address this, this patch will revert the field ID back to the same value as 22.2 Release note: None Co-authored-by: Nathan VanBenschoten <[email protected]> Co-authored-by: Lidor Carmel <[email protected]> Co-authored-by: Jackson Owens <[email protected]> Co-authored-by: Faizan Qazi <[email protected]>
…iveness Fixes cockroachdb#84655. Related to cockroachdb#49220. This commit extends the logic introduced in 8aa1c14 to simultaneously campaign for Raft leadership when rejecting a lease request on a Raft follower that observes that the current Raft leader is not live according to node liveness. These kinds of cases are often associated with asymmetric network partitions. In such cases, the Raft leader will be unable to acquire an epoch-based lease until it heartbeats its liveness record. As a result, the range can only regain availability if a different replica acquires the lease. However, the protection added in 8aa1c14 prevents followers from acquiring leases to protect against a different form of unavailability. After this commit, the followers will attempt to acquire Raft leadership when detecting such cases by campaigning. This allows these ranges to recover availability once Raft leadership moves off the partitioned Raft leader to one of the followers that can reach node liveness and can subsequently acquire the lease. Campaigning for Raft leadership is safer than blindly allowing the lease request to be proposed (through a redirected proposal). This is because the follower may be arbitrarily far behind on its Raft log and acquiring the lease in such cases could cause unavailability (the kind we saw in cockroachdb#37906). By instead calling a Raft pre-vote election, the follower can determine whether it is behind on its log without risking disruption. If so, we don't want it to acquire the lease — one of the other followers that is caught up on its log can. If not, it will eventually become leader and can proceed with a future attempt to acquire the lease. The commit adds a test that reproduces the failure mode described in cockroachdb#84655. It creates an asymmetric network partition scenario that looks like: ``` [0] raft leader / initial leaseholder ^ / \ / \ v v [1]<--->[2] raft followers ^ ^ \ / \ / v [3] liveness range ``` It then waits for the raft leader's lease to expire and demonstrates that one of the raft followers will now call a Raft election, which allows it to safely grab Raft leadership, acquire the lease, and recover availability. Without the change, the test failed. Release justification: None. Too risky for the stability period. Potential backport candidate after sufficient baking on master. Release note (bug fix): A bug causing ranges to remain without a leaseholder in cases of asymmetric network partitions has been resolved.
…iveness Fixes cockroachdb#84655. Related to cockroachdb#49220. This commit extends the logic introduced in 8aa1c14 to simultaneously campaign for Raft leadership when rejecting a lease request on a Raft follower that observes that the current Raft leader is not live according to node liveness. These kinds of cases are often associated with asymmetric network partitions. In such cases, the Raft leader will be unable to acquire an epoch-based lease until it heartbeats its liveness record. As a result, the range can only regain availability if a different replica acquires the lease. However, the protection added in 8aa1c14 prevents followers from acquiring leases to protect against a different form of unavailability. After this commit, the followers will attempt to acquire Raft leadership when detecting such cases by campaigning. This allows these ranges to recover availability once Raft leadership moves off the partitioned Raft leader to one of the followers that can reach node liveness and can subsequently acquire the lease. Campaigning for Raft leadership is safer than blindly allowing the lease request to be proposed (through a redirected proposal). This is because the follower may be arbitrarily far behind on its Raft log and acquiring the lease in such cases could cause unavailability (the kind we saw in cockroachdb#37906). By instead calling a Raft pre-vote election, the follower can determine whether it is behind on its log without risking disruption. If so, we don't want it to acquire the lease — one of the other followers that is caught up on its log can. If not, it will eventually become leader and can proceed with a future attempt to acquire the lease. The commit adds a test that reproduces the failure mode described in cockroachdb#84655. It creates an asymmetric network partition scenario that looks like: ``` [0] raft leader / initial leaseholder ^ / \ / \ v v [1]<--->[2] raft followers ^ ^ \ / \ / v [3] liveness range ``` It then waits for the raft leader's lease to expire and demonstrates that one of the raft followers will now call a Raft election, which allows it to safely grab Raft leadership, acquire the lease, and recover availability. Without the change, the test failed. Release justification: None. Too risky for the stability period. Potential backport candidate after sufficient baking on master. Release note (bug fix): A bug causing ranges to remain without a leaseholder in cases of asymmetric network partitions has been resolved.
…iveness Fixes #84655. Related to #49220. This commit extends the logic introduced in 8aa1c14 to simultaneously campaign for Raft leadership when rejecting a lease request on a Raft follower that observes that the current Raft leader is not live according to node liveness. These kinds of cases are often associated with asymmetric network partitions. In such cases, the Raft leader will be unable to acquire an epoch-based lease until it heartbeats its liveness record. As a result, the range can only regain availability if a different replica acquires the lease. However, the protection added in 8aa1c14 prevents followers from acquiring leases to protect against a different form of unavailability. After this commit, the followers will attempt to acquire Raft leadership when detecting such cases by campaigning. This allows these ranges to recover availability once Raft leadership moves off the partitioned Raft leader to one of the followers that can reach node liveness and can subsequently acquire the lease. Campaigning for Raft leadership is safer than blindly allowing the lease request to be proposed (through a redirected proposal). This is because the follower may be arbitrarily far behind on its Raft log and acquiring the lease in such cases could cause unavailability (the kind we saw in #37906). By instead calling a Raft pre-vote election, the follower can determine whether it is behind on its log without risking disruption. If so, we don't want it to acquire the lease — one of the other followers that is caught up on its log can. If not, it will eventually become leader and can proceed with a future attempt to acquire the lease. The commit adds a test that reproduces the failure mode described in #84655. It creates an asymmetric network partition scenario that looks like: ``` [0] raft leader / initial leaseholder ^ / \ / \ v v [1]<--->[2] raft followers ^ ^ \ / \ / v [3] liveness range ``` It then waits for the raft leader's lease to expire and demonstrates that one of the raft followers will now call a Raft election, which allows it to safely grab Raft leadership, acquire the lease, and recover availability. Without the change, the test failed. Release justification: None. Too risky for the stability period. Potential backport candidate after sufficient baking on master. Release note (bug fix): A bug causing ranges to remain without a leaseholder in cases of asymmetric network partitions has been resolved.
repro steps:
Restart the dead node and run a SELECT COUNT(*) on the tpcc tables through it
Chances are this will hang for minutes before returning. The reason is that none of the replicas will have been moved off that node while it was down (since there are only three nodes in the cluster), but now all of them will need Raft snapshots. There is no particular ordering in which these snapshots are requested. Some of our COUNT(*) requests (or even worse, liveness requests etc) may hit a replica on the local node in need of a snapshot. The lease request will hang until the snapshot is applied, which can take a long time.
We should do something here, like short-circuit lease request (for a blank NotLeaseholderErr) if we can with reasonable accuracy detect that the node is in need of a snapshot. This isn't trivial to detect on a follower (would be easier on the leader, but alas). We could also prioritize snapshots for system ranges higher to resolve the somewhat orthogonal problem that such hung requests on the system range can prevent the node from becoming live in the first place, which causes hard-to-diagnose problems.
The text was updated successfully, but these errors were encountered: