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

kv: partial/asymmetric partition that does not isolate node liveness does not force range lease revocation #49220

Open
kalman5 opened this issue May 18, 2020 · 28 comments
Labels
A-kv Anything in KV that doesn't belong in a more specific category. A-kv-replication Relating to Raft, consensus, and coordination. O-community Originated from the community 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 P-3 Issues/test failures with no fix SLA T-kv KV Team

Comments

@kalman5
Copy link

kalman5 commented May 18, 2020

ERROR: internal error while retrieving user account

Please describe the issue you observed, and any steps we can take to reproduce it:

Setup a 6 nodes cluster (3 on "datacenter" A, 3 on "datacenter" B (172.31.2.0/24)), set default replication zone to num_replicas = 6, create a table with a num_replica = 6 as well.

Create a iptables rule on a node of A such that it can not reach any node on B.

sudo iptables -I OUTPUT -d 172.31.2.0/24 -j DROP

Trying to connect to that node I get:

$ cockroach-v20.1.0.linux-amd64/cockroach sql --insecure --url postgresql://kalman@192.168.26.157:26257/testdb
#
# Welcome to the CockroachDB SQL shell.
# All statements must be terminated by a semicolon.
# To exit, type: \q.
#

Connecting to another node works, but issuing a command like:
select count(*) from test2;

it hangs forever, as soon I dropped that rule with:

sudo iptables -D OUTPUT -d 172.31.2.0/24 -j DROP

the query in hang completed and now I'm able to connect to that node without issues

Environment:

  • CockroachDB version: 20.1
  • Server OS: Linux Ubuntu 18.04
  • Client app: cockroach sql or psql give the same error

Additional context
What was the impact? 1 Node isolated from other 3 nodes the entire cluster looks
like is not working

For the dashboard all 6 nodes are available.

gz#7690

gz#8203

gz#8949

gz#10844

Jira issue: CRDB-4243

gz#14119

gz#14290

Epic CRDB-25199

gz#16890

@blathers-crl
Copy link

blathers-crl bot commented May 18, 2020

Hello, I am Blathers. I am here to help you get the issue triaged.

It looks like you have not filled out the issue in the format of any of our templates. To best assist you, we advise you to use one of these templates.

I have CC'd a few people who may be able to assist you:

  • @ricardocrdb (member of the technical support engineering team)

If we have not gotten back to your issue within a few business days, you can try the following:

  • Join our community slack channel and ask on #cockroachdb.
  • Try find someone from here if you know they worked closely on the area and CC them.

🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is otan.

@blathers-crl blathers-crl bot added O-community Originated from the community X-blathers-oncall labels May 18, 2020
@kalman5 kalman5 changed the title [20.1][Internal error] ERROR: internal error while retrieving user account [20.1][Internal error] 1 Node isolated => ERROR: internal error while retrieving user account, cluster down as well May 18, 2020
@yuzefovich
Copy link
Member

cc @RichardJCai

@RichardJCai
Copy link
Contributor

Do you happen to have the logs up until the error is given?

@kalman5
Copy link
Author

kalman5 commented May 19, 2020

Do you happen to have the logs up until the error is given?

I can repeat the experiment whenever you want, you can find me on slack "Gaetano", what bothers
me is the entire cluster unavailability

@kalman5
Copy link
Author

kalman5 commented May 19, 2020

This is what I see in the log of that node isolated (it can still reach 2 nodes on the same datacenter).

crdb_report1.txt

I'm not able to get an internal error while connecting to the cluster, however a simple
select
SELECT count(*) FROM test;
on my table hangs forever untill I don't remove the firewall rule

@darinpp darinpp self-assigned this May 20, 2020
@darinpp
Copy link
Contributor

darinpp commented May 20, 2020

I'm able to reproduce this.

@nvanbenschoten nvanbenschoten self-assigned this May 20, 2020
@nvanbenschoten
Copy link
Member

I'm able to reproduce this as well and this seems like a real issue. The problem occurs because at least one range in the table you are querying, test2, has a leaseholder on the partially partitioned node (let's call it P). The gateway you are querying is unable to reach this leaseholder on P, so the query hangs.

The question is - why isn't this leaseholder moving to a different node that is not partially partitioned? At first, I thought this was due to the quorum size of 6. Typically, even-numbered quorums are fragile because they can result in voting ties during leader elections. If all nodes in DC A voted for P and all nodes in DC B voted for one of the other nodes, the election would result in a draw and would need to be run again. However, I tested this out on a 7 node cluster and saw the same thing. That's because we aren't even calling an election for the inaccessible range on P.

The problem here seems to be that we delegate liveness tracking to a separate range in the system ("node liveness"). This NodeLiveness range may have a different set of replicas than the table range. More importantly, it may have a different leaseholder. It turns out that there are two cases we can get into here:

  1. the node liveness leaseholder in DC A
  2. the node liveness leaseholder in DC B

In the case where the node liveness leaseholder is in DC B, the partitioned node P is unable to heartbeat its liveness record. Its lease on the table range then effectively expires, which allows one of the other nodes to call an election and then acquire the lease. The whole thing recovers in about 15s and then we're back to normal.

However, in the case where the node liveness leaseholder is in DC A, things are more tricky. P is still able to heartbeat its liveness record, so it keeps effectively extending its lease over the table range. The NodeLiveness leaseholder doesn't know that P is partitioned, so it happy allows it to continue heartbeating its record. Meanwhile, the data range itself is quiesced and no-one is trying to write to it, so none of the replicas are expecting Raft heartbeats. So the leaseholder just stays put and the query hangs while trying to connect to P.

I'm not actually sure how this class of partial failures is supposed to be handled. It's easy enough to understand in standard Raft, but the two-level liveness hierarchy we have makes it harder to reason about. @bdarnell do you know what we expect to happen here? Should we be restricting the conditions under which we allow a node to heartbeat its liveness record - similar to what we did with nodes that are unable to fsync? We currently use node-liveness as a proxy to determine suitability to hold leases, but it doesn't do a good enough job recognizing partial failures like this, and I'm not sure that it can.

Note that I don't think we have this kind of issue with ranges that use standard expiration-based leases.

@nvanbenschoten nvanbenschoten added the A-kv-replication Relating to Raft, consensus, and coordination. label May 20, 2020
@bdarnell
Copy link
Contributor

Unfortunately we can't handle this like we did with nodes that can't fsync by forbidding liveness heartbeats from nodes that are partitioned. What exactly would make a node consider itself partitioned? Even a single unreachable peer could cause hanging queries if that peer happened to be the gateway. In this case node P can only reach a minority of peers, which seems on its face to be a reasonable threshold, but I think if we require a node to have a majority of its peers live in order to come back online we'll probably have trouble recovering from outages.

Solving network partitions by detecting the partition is a losing game - a sufficiently complex partition can always defy your efforts to figure out what's going on (and in the limit, we're OK with that - we're a CAP-consistent database, not a CAP-available one, and there will be situations in which we're unable to remain available. We just want to handle "reasonable" classes of partitions, for some definition of "reasonable").

We used to have a general mechanism for this, the SendNextTimeout in DistSender. This would cause DistSender to try a different node if the one that it thought was the leaseholder was failing to respond. This was removed in #16088 and some follow-on commits because (IIRC) the additional retries were leading to cascading failures in overloaded clusters, and it was redundant with GRPC-level heartbeats.

Assuming those GRPC-level heartbeats are working, the gateway node should be periodically dropping its connection to P and trying on another replica. But as long as P is still live, it'll just get redirected back to P. We should be using these requests as a trigger to unquiesce the range, allow the other replicas of the range to detect P's failure, and then forcibly invalidate its liveness record.

@kalman5
Copy link
Author

kalman5 commented May 21, 2020

To be honest when I did this experiment I just expected CRDB to form a mesh network such that P would reach nodes in the other datacenter via not partitioned nodes.
A mesh network would be a nice abstraction in CRDB solving this kind of problems by construction.

@darinpp
Copy link
Contributor

darinpp commented May 26, 2020

CC @tbg

@nvanbenschoten nvanbenschoten changed the title [20.1][Internal error] 1 Node isolated => ERROR: internal error while retrieving user account, cluster down as well kv: partial partition that does not isolate node liveness does not force range lease revocation May 26, 2020
@andreimatei
Copy link
Contributor

Unfortunately we can't handle this like we did with nodes that can't fsync by forbidding liveness heartbeats from nodes that are partitioned. What exactly would make a node consider itself partitioned? Even a single unreachable peer could cause hanging queries if that peer happened to be the gateway. In this case node P can only reach a minority of peers, which seems on its face to be a reasonable threshold, but I think if we require a node to have a majority of its peers live in order to come back online we'll probably have trouble recovering from outages.

We have an option in between a node not heartbeating itself and the node continuing to hold its leases: the node in question can increment its own epoch, thereby relinquishing all its leases. But the twist is that it'd immediately take back all its leases, except for the range(s) it believes to have an inaccessible quorum. I mean, the node would attempt to take back all the leases, but it wouldn't succeed for the partitioned ranges.

@RoachietheSupportRoach RoachietheSupportRoach added the A-kv Anything in KV that doesn't belong in a more specific category. label Mar 9, 2021
@daniel-crlabs
Copy link
Contributor

Hi guys, is there an update on this? This is impacting another customer (please see zendesk 8203), I'm not putting their name here as this is a public repo.

@andreimatei
Copy link
Contributor

No update. I think it's unlikely that there's going to be any movement here soon, cause it seems hard.
However, note that this issue is about partial partitions. Zendesk 8203 seems to me to be about a full partition; with full partitions things are supposed to be more clear.

@jlinder jlinder added the T-kv KV Team label Jun 16, 2021
@joshimhoff joshimhoff added the O-sre For issues SRE opened or otherwise cares about tracking. label Aug 9, 2021
nvanbenschoten added a commit to nvanbenschoten/cockroach that referenced this issue Nov 1, 2022
…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.
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 <nvanbenschoten@gmail.com>
Co-authored-by: Lidor Carmel <lidor@cockroachlabs.com>
Co-authored-by: Jackson Owens <jackson@cockroachlabs.com>
Co-authored-by: Faizan Qazi <faizan@cockroachlabs.com>
@smcvey
Copy link
Contributor

smcvey commented Nov 30, 2022

A variant of this issue affected another customer yesterday. In this instance, there was an 8 node cluster. Due to a networking issue, nodes 3 and 7 could not communicate with each other, but both nodes could communicate with the rest of the cluster. The debug shows the usual error messages regarding RPC timeouts, unable to server requests for ranges etc. ZD ticket is 14899.

@andrewbaptist
Copy link
Collaborator

For async partitions - there is work in progress for 23.1. This doesn't handle all cases of partial partitions, but will handle 1-way partitions much better than today: #84289

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.
craig bot pushed a commit that referenced this issue Jun 28, 2023
104042: base: enable Raft CheckQuorum by default  r=erikgrinaker a=erikgrinaker

This patch enables Raft CheckQuorum by default. In etcd/raft, this also has the effect of fully enabling PreVote, such that followers won't grant prevotes if they've heard from a leader in the past election timeout interval.

This is more robust against partial and asymmetric network partitions. Otherwise, a partitioned node may be able to hold spurious elections and steal leadership away from an established leader. This can cause the leader to become unreachable by the leaseholder, resulting in permanent range unavailability.

We are still able to hold immediate elections, e.g. when unquiescing a range to find a dead leader. If a quorum of followers consider the leader dead and forget it (becoming leaderless followers), they will grant prevotes despite having seen the leader recently (i.e. before quiescing), and can hold an election immediately.

This is compatibile with 23.1 in mixed-version clusters:

* Leaders with mixed `CheckQuorum` settings is fine: they only apply the step-down logic to themselves, and register follower activity regardless of the followers' settings.

* Voters with mixed `CheckQuorum` settings if fine: the leader recency criterion is only applied to their own vote, so either they'll enforce it or not.

* Campaigning on leader removal is fine-ish: before 23.2 finalization, the first range replica will campaign -- if this replica is 23.2 it will bypass pre-vote and call an immediate election, if it is 23.1 then it will use pre-vote. However, upon receiving the 23.1 pre-vote request, 23.2 nodes will check if the leader is still in the descriptor, and if it isn't they will forget it and grant the pre-vote. A quorum will likely apply the leader removal before receiving pre-vote requests. Otherwise, we will recover after an election timeout.

* Campaigning after unquiescing is fine: the logic remains unchanged, and 23.2 nodes will forget the leader and grant prevotes if they find the leader dead according to liveness.

* Campaigning during lease acquisitions is fine: this is needed to steal leadership away from an active leader that can't itself acquire an epoch lease because it's failing liveness heartbeats. If a 23.2 node also finds the leader dead in liveness, it will forget it and grant the prevote.

Resolves #92088.
Touches #49220.
Epic: none.

Release note (bug fix): The Raft PreVote and CheckQuorum mechanisms are now fully enabled. These prevent spurious elections when followers already have an active leader, and cause leaders to step down if they don't hear back from a quorum of followers. This improves reliability under partial and asymmetric network partitions, by avoiding spurious elections and preventing unavailability where a partially partitioned node could steal leadership away from an established leaseholder who would then no longer be able to reach the leader and submit writes.

Co-authored-by: Erik Grinaker <grinaker@cockroachlabs.com>
@exalate-issue-sync exalate-issue-sync bot added the O-support Would prevent or help troubleshoot a customer escalation - bugs, missing observability/tooling, docs label Feb 29, 2024
@shralex shralex added the P-3 Issues/test failures with no fix SLA label Mar 2, 2024
@erikgrinaker erikgrinaker removed their assignment Mar 27, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
A-kv Anything in KV that doesn't belong in a more specific category. A-kv-replication Relating to Raft, consensus, and coordination. O-community Originated from the community 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 P-3 Issues/test failures with no fix SLA T-kv KV Team
Projects
None yet
Development

No branches or pull requests