-
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
kvserver: unacquired leases during liveness failures #84655
Comments
cc @cockroachdb/replication |
Not sure how this'll go for me :), but I am trying to repro this bug by running a local three node cluster & sending a signal to one of the nodes that caused it to fail to heartbeat its liveness record. I think maybe I've reproed it; the other nodes never grab the expired leases. Here is my repro branch: #84859 In short:
And here is my
After sending SIGHUP to one of the CRDB nodes I ran with
There are multiple errors being returned here:
The second one we saw on the cloud cluster during the recent incident! I don't think we saw the first one. I'm not sure why; clearly my repro is different than what happened on the cloud cluster in some way. |
I believe there was another occurrence of this at https://cockroachlabs.atlassian.net/browse/SREOPS-5125. CC @mwang1026. Only medium confident due to GCed unredacted logs. |
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.
Is your feature request related to a problem? Please describe.
This is a follow up to an internal incident (https://github.com/cockroachlabs/support/issues/1697) and relates closely to #49220. It's possible for a node to not maintain its liveness (being unable to talk to the liveness leaseholder for ex.) and hold no range leases. Simultaneously it's possible for CRDB to not acquire range leases elsewhere. Few immediate reasons come to mind:
cockroach/pkg/kv/kvserver/replica_proposal_buf.go
Lines 450 to 452 in fbb50f5
Jira issue: CRDB-17786
Epic CRDB-19227
The text was updated successfully, but these errors were encountered: