Skip to content
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

Closed
irfansharif opened this issue Jul 19, 2022 · 3 comments · Fixed by #87244
Closed

kvserver: unacquired leases during liveness failures #84655

irfansharif opened this issue Jul 19, 2022 · 3 comments · Fixed by #87244
Assignees
Labels
A-kv-replication Relating to Raft, consensus, and coordination. C-enhancement Solution expected to add code/behavior + preserve backward-compat (pg compat issues are exception) O-sre For issues SRE opened or otherwise cares about tracking. O-support Would prevent or help troubleshoot a customer escalation - bugs, missing observability/tooling, docs T-kv KV Team

Comments

@irfansharif
Copy link
Contributor

irfansharif commented Jul 19, 2022

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:

  • Follower replicas will reject lease acquisition attempts if there's a raft leader elsewhere. This check for raft leadership does not incorporate information about whether the raft leader node is able to actual acquire leases (in case it's failing liveness heartbeats).
    if leaderKnownAndEligible && !ownsCurrentLease && !b.testing.allowLeaseProposalWhenNotLeader {
    log.VEventf(ctx, 2, "not proposing lease acquisition because we're not the leader; replica %d is",
    leaderInfo.leader)
  • Reliance on distsenders routing requests away from the old, inactive leaseholder to other replicas which would in turn be able to transfer leases to themselves. If distsender caches never get updated (if the old leaseholder isn't hard crashed or requests to it just timeout), it's not clear that other replicas would ever receive such requests and be able to acquire leases.
  • Lack of active lease acquisition mechanisms from other nodes post heartbeat failures. Reliance on per-store replicate queue scanning to detect what ranges currently don't have leases can be a slow process.

Jira issue: CRDB-17786

Epic CRDB-19227

@irfansharif irfansharif added C-enhancement Solution expected to add code/behavior + preserve backward-compat (pg compat issues are exception) A-kv-replication Relating to Raft, consensus, and coordination. labels Jul 19, 2022
@blathers-crl
Copy link

blathers-crl bot commented Jul 19, 2022

cc @cockroachdb/replication

@joshimhoff
Copy link
Collaborator

joshimhoff commented Jul 21, 2022

Follower replicas will reject lease acquisition attempts if there's a raft leader elsewhere. This check for raft leadership does not incorporate information about whether the raft leader node is able to actual acquire leases (in case it's failing liveness heartbeats).

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:

~/g/s/g/c/cockroach [working] $ git diff
diff --git a/pkg/kv/kvserver/liveness/liveness.go b/pkg/kv/kvserver/liveness/liveness.go
index 215a45e6ad..00c64d597f 100644
--- a/pkg/kv/kvserver/liveness/liveness.go
+++ b/pkg/kv/kvserver/liveness/liveness.go
@@ -14,8 +14,12 @@ import (
        "bytes"
        "context"
        "fmt"
+       atomic2 "go.uber.org/atomic"
+       "os"
+       "os/signal"
        "strconv"
        "sync/atomic"
+       "syscall"
        "time"
 
        "github.com/cockroachdb/cockroach/pkg/base"
@@ -79,6 +83,14 @@ func (e *errRetryLiveness) Error() string {
        return fmt.Sprintf("%T: %s", *e, e.error)
 }
 
+var sigCh = func() chan os.Signal {
+       ch := make(chan os.Signal, 1)
+       signal.Notify(ch, syscall.SIGHUP)
+       return ch
+}()
+
+var weDead atomic2.Int32
+
 func isErrRetryLiveness(ctx context.Context, err error) bool {
        if errors.HasType(err, (*roachpb.AmbiguousResultError)(nil)) {
                // We generally want to retry ambiguous errors immediately, except if the
@@ -951,6 +963,19 @@ func (nl *NodeLiveness) heartbeatInternal(
                return errors.Errorf("proposed liveness update expires earlier than previous record")
        }
 
+       select {
+       case <-sigCh:
+                       weDead.Store(1)
+                       log.Warningf(ctx, "no liveness for u")
+                       return errors.Errorf("no liveness for u")
+       default:
+       }
+
+       if weDead.Load() == 1 {
+               log.Warningf(ctx, "no liveness for u")
+               return errors.Errorf("no liveness for u")
+       }
+
        update := livenessUpdate{
                oldLiveness: oldLiveness,
                newLiveness: newLiveness,

And here is my run.sh:

#!/bin/bash

./bin/roachprod destroy local || true
./bin/roachprod create -n 3 local
./bin/roachprod put local cockroach
./bin/roachprod start local
tail -F ~/local/*/logs/cockroach.log

After sending SIGHUP to one of the CRDB nodes I ran with bash run.sh, I get the following errors from kvprober, even after waiting a long time:

root@localhost:26257/defaultdb> select range_id, end_to_end_latency_ms, error from crdb_internal.probe_ranges(INTERVAL '1s', 'write');;
  range_id | end_to_end_latency_ms |                                                                                                                                                                    error
-----------+-----------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
         1 |                    77 |
         2 |                    97 |
         3 |                    76 |
         4 |                   172 |
         5 |                    55 |
         6 |         9223372036854 | operation "write probe" timed out after 1s (given timeout 1s): failed to send RPC: sending to all replicas failed; last error: [NotLeaseHolderError] failed to manipulate liveness record: no liveness for u; r6: replica (n3,s3):2 not lease holder; lease holder unknown
         7 |                    58 |
         8 |                    95 |
         9 |         9223372036854 | operation "write probe" timed out after 1.001s (given timeout 1s): failed to send RPC: sending to all replicas failed; last error: [NotLeaseHolderError] failed to manipulate liveness record: no liveness for u; r9: replica (n3,s3):2 not lease holder; lease holder unknown
        10 |                    59 |
        11 |         9223372036854 | operation "write probe" timed out after 1.001s (given timeout 1s): failed to send RPC: sending to all replicas failed; last error: [NotLeaseHolderError] failed to manipulate liveness record: no liveness for u; r11: replica (n3,s3):3 not lease holder; lease holder unknown
        12 |         9223372036854 | operation "write probe" timed out after 1.001s (given timeout 1s): failed to send RPC: sending to all replicas failed; last error: routing information detected to be stale: [NotLeaseHolderError] refusing to acquire lease on follower; r12: replica (n2,s2):3 not lease holder; current lease is repl=(n3,s3):2 seq=0 start=0,0 exp=<nil>
        13 |                    57 |
        14 |         9223372036854 | operation "write probe" timed out after 1s (given timeout 1s): failed to send RPC: sending to all replicas failed; last error: [NotLeaseHolderError] failed to manipulate liveness record: no liveness for u; r14: replica (n3,s3):3 not lease holder; lease holder unknown
        15 |                    44 |
        16 |                   114 |
        17 |                   134 |
        18 |                    78 |
        19 |                    75 |
        20 |         9223372036854 | operation "write probe" timed out after 1s (given timeout 1s): failed to send RPC: sending to all replicas failed; last error: [NotLeaseHolderError] failed to manipulate liveness record: no liveness for u; r20: replica (n3,s3):2 not lease holder; lease holder unknown
        21 |                    71 |
        22 |                   115 |
        23 |         9223372036854 | operation "write probe" timed out after 1.001s (given timeout 1s): failed to send RPC: sending to all replicas failed; last error: [NotLeaseHolderError] failed to manipulate liveness record: no liveness for u; r23: replica (n3,s3):2 not lease holder; lease holder unknown
        24 |                    61 |
        25 |         9223372036854 | operation "write probe" timed out after 1.001s (given timeout 1s): failed to send RPC: sending to all replicas failed; last error: [NotLeaseHolderError] failed to manipulate liveness record: no liveness for u; r25: replica (n3,s3):2 not lease holder; lease holder unknown
        26 |                    92 |
        27 |                    95 |
        28 |         9223372036854 | operation "write probe" timed out after 1s (given timeout 1s): failed to send RPC: sending to all replicas failed; last error: [NotLeaseHolderError] failed to manipulate liveness record: no liveness for u; r28: replica (n3,s3):2 not lease holder; lease holder unknown
        29 |                    60 |
        30 |                   114 |
        31 |         9223372036854 | operation "write probe" timed out after 1.001s (given timeout 1s): failed to send RPC: sending to all replicas failed; last error: [NotLeaseHolderError] failed to manipulate liveness record: no liveness for u; r31: replica (n3,s3):3 not lease holder; lease holder unknown
        32 |                    41 |
        33 |         9223372036854 | operation "write probe" timed out after 1.002s (given timeout 1s): failed to send RPC: sending to all replicas failed; last error: [NotLeaseHolderError] failed to manipulate liveness record: no liveness for u; r33: replica (n3,s3):2 not lease holder; lease holder unknown
        34 |                    60 |
        35 |                    56 |
        36 |         9223372036854 | operation "write probe" timed out after 1.001s (given timeout 1s): failed to send RPC: sending to all replicas failed; last error: [NotLeaseHolderError] failed to manipulate liveness record: no liveness for u; r36: replica (n3,s3):3 not lease holder; lease holder unknown
        37 |                    69 |
        38 |                    96 |
        39 |                    96 |
        40 |                   114 |
        41 |                    96 |
        42 |                   114 |
        43 |                   192 |
        44 |                    61 |
        45 |                   117 |
        46 |         9223372036854 | operation "write probe" timed out after 1.001s (given timeout 1s): failed to send RPC: sending to all replicas failed; last error: [NotLeaseHolderError] failed to manipulate liveness record: no liveness for u; r46: replica (n3,s3):3 not lease holder; lease holder unknown
        47 |                    60 |
        48 |         9223372036854 | operation "write probe" timed out after 1.001s (given timeout 1s): failed to send RPC: sending to all replicas failed; last error: [NotLeaseHolderError] failed to manipulate liveness record: no liveness for u; r48: replica (n3,s3):2 not lease holder; lease holder unknown
        49 |                    60 |
        50 |         9223372036854 | operation "write probe" timed out after 1s (given timeout 1s): failed to send RPC: sending to all replicas failed; last error: [NotLeaseHolderError] failed to manipulate liveness record: no liveness for u; r50: replica (n3,s3):3 not lease holder; lease holder unknown
(50 rows)

There are multiple errors being returned here:

  • operation "write probe" timed out after 1s (given timeout 1s): failed to send RPC: sending to all replicas failed; last error: [NotLeaseHolderError] failed to manipulate liveness record: no liveness for u; r50: replica (n3,s3):3 not lease holder; lease holder unknown
  • operation "write probe" timed out after 1.001s (given timeout 1s): failed to send RPC: sending to all replicas failed; last error: routing information detected to be stale: [NotLeaseHolderError] refusing to acquire lease on follower; r12: replica (n2,s2):3 not lease holder; current lease is repl=(n3,s3):2 seq=0 start=0,0 exp=<nil>

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.

@joshimhoff
Copy link
Collaborator

joshimhoff commented Aug 29, 2022

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.

@nvanbenschoten nvanbenschoten self-assigned this Sep 1, 2022
craig bot pushed a commit that referenced this issue Nov 1, 2022
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]>
@craig craig bot closed this as completed in 503c731 Nov 1, 2022
nvanbenschoten added a commit to nvanbenschoten/cockroach that referenced this issue Jan 26, 2023
…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.
nvanbenschoten added a commit to nvanbenschoten/cockroach that referenced this issue Jan 26, 2023
…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.
nathanstilwell pushed a commit that referenced this issue Feb 2, 2023
…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.
@shralex shralex added the O-support Would prevent or help troubleshoot a customer escalation - bugs, missing observability/tooling, docs label Feb 21, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
A-kv-replication Relating to Raft, consensus, and coordination. C-enhancement Solution expected to add code/behavior + preserve backward-compat (pg compat issues are exception) O-sre For issues SRE opened or otherwise cares about tracking. O-support Would prevent or help troubleshoot a customer escalation - bugs, missing observability/tooling, docs T-kv KV Team
Projects
None yet
4 participants