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: Raft prevote does not prevent election despite active leader #92088

Closed
erikgrinaker opened this issue Nov 17, 2022 · 31 comments · Fixed by #104042
Closed

kvserver: Raft prevote does not prevent election despite active leader #92088

erikgrinaker opened this issue Nov 17, 2022 · 31 comments · Fixed by #104042
Assignees
Labels
A-kv-replication Relating to Raft, consensus, and coordination. C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior.

Comments

@erikgrinaker
Copy link
Contributor

erikgrinaker commented Nov 17, 2022

In https://github.com/cockroachlabs/support/issues/1875, we saw indications that range quiescence may interfere with Raft prevote, which in turn can cause range unavailability with asymmetric/partial network partitions where the Raft leader is unreachable from other replicas. This can be reproduced in the following scenario:

# Start a five-node cluster.
roachprod create --gce-zones europe-west1-b -n 5 grinaker-partition
roachprod stage grinaker-partition release v22.2.0-rc.2
roachprod stage grinaker-partition workload
roachprod start grinaker-partition --racks 5

# Set lease preferences to avoid n5.
ALTER RANGE default CONFIGURE ZONE USING constraints = '[]', lease_preferences = '[[-rack=4]]';
# ...same for all other existing zone configs.

# Write some data, then start a read-only workload, against n1-n4.
./workload run kv --init --splits 1000 --concurrency 256 --read-percent 0 {pgurl:1-4}
./workload run kv --splits 1000 --concurrency 256 --read-percent 100 --max-rate 1024 {pgurl:1-4}

# On n5, block inbound connections from all nodes except n4. Keep established connections.
sudo iptables -A INPUT -m state --state NEW -p tcp --dport 26257 -s n4 -j ACCEPT
sudo iptables -A INPUT -m state --state NEW -p tcp --dport 26257 -j REJECT
# To undo: sudo iptables -F INPUT

# Stop, and after some time restart, n5 to sever existing connections.
roachprod stop grinaker-partition:5
roachprod start grinaker-partition:5 --racks 5

We can now see n5 pick up Raft leaderships, even though prevote should have prevented it from doing so. We also see that range leases are mostly not transferred across to the Raft leader, respecting the lease preferences (although not always). Leaseholders on other nodes wouldn't be able to propose a lease transfer or Raft leadership transfer anyway (#92089). Once the partition heals (at 21:10), the Raft leaderships are immediately transferred to the appropriate leaseholder.

Screenshot 2022-11-17 at 22 14 57

This does not happen when the workload is changed to a write-only workload, where the ranges are not quiesced -- the restarted node is not able to acquire any Raft leaderships at all.

It thus appears that range quiescence somehow interferes with Raft prevote, allowing a partitioned replica to acquire Raft leadership. If the restarted node is then not reachable from some of the other nodes, the existing leaseholder may not be able to send proposals to the new Raft leader, which prevents it from reclaiming Raft leadership. I haven't mapped out all the details here yet.

We should ensure the prevote protection still works as expected with quiesced ranges.

Jira issue: CRDB-21567

Epic CRDB-25199

@erikgrinaker erikgrinaker added C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior. A-kv-replication Relating to Raft, consensus, and coordination. T-kv-replication labels Nov 17, 2022
@blathers-crl
Copy link

blathers-crl bot commented Nov 17, 2022

cc @cockroachdb/replication

@tbg
Copy link
Member

tbg commented Nov 18, 2022

Maybe I'm missing something, but I don't understand this part:

even though prevote should have prevented it from doing so.

PreVote is like Vote, except that it doesn't actually crown the new leader. The raft leaders you're seeing here must've actually won the vote. So they would've also won the corresponding PreVote. And they probably did: you have a group of three, a candidate, and another voter who voted for that candidate. I'm not sure what this has to do with range quiescence. I think the real ingredient that quiescent ranges provide is that the newly restarted replica is up-to-date on the log. If it were even just an entry behind the other voters, it wouldn't win the PreVote phase.

So to me this "just" looks like one of the ways in which we don't handle asymmetric network partitions well: we can end up electing a raft leader that is not reachable from the leaseholder, because there's no requirement that the leaseholder has to be a part of the quorum that decides the election.

@tbg
Copy link
Member

tbg commented Nov 18, 2022

It also doesn't seem like CheckQuorum would've helped here, as the new raft leader did get heartbeat responses from a quorum (itself and nX) while the leaseholder was partitioned off.

@erikgrinaker
Copy link
Contributor Author

PreVote is like Vote, except that it doesn't actually crown the new leader. The raft leaders you're seeing here must've actually won the vote. So they would've also won the corresponding PreVote. And they probably did: you have a group of three, a candidate, and another voter who voted for that candidate.

Wait, what? Isn't the whole idea of prevote to avoid disrupting an established leader? Why would the followers then choose to vote for a different leader during prevote? Wouldn't that give a 50% chance of disruption?

I think the real ingredient that quiescent ranges provide is that the newly restarted replica is up-to-date on the log. If it were even just an entry behind the other voters, it wouldn't win the PreVote phase.

Yeah, this is clearly it. I was staring at this stuff for way too long. Still doesn't seem helpful that it can win a prevote when it's up to date on its log.

It also doesn't seem like CheckQuorum would've helped here

Nah, CheckQuorum is no good here.

@erikgrinaker
Copy link
Contributor Author

erikgrinaker commented Nov 18, 2022

Yeah, prevotes will only be successful if the follower hasn't received a heartbeat from the existing leader for the past election timeout interval:

A server only pre-votes for a potential candidate during the PreVote phase if it would vote for it during a normal election, and importantly, if it has not received an AppendEntries RPC from the leader with in the election timeout.

https://decentralizedthoughts.github.io/2020-12-12-raft-liveness-full-omission/

So I'm guessing that quiescence messes this up somehow. A quiesced follower may not have seen a heartbeat from the leader in a very long time (in wall time), but quiesced replicas likely track time in ticks, which I don't expect to progress while quiesced. Maybe it's because unquiescing the follower doesn't also unquiesce the leader or something, but I thought the leader was responsible for unquiescing. I'm not familiar with the details here, but seems worth looking into. I see there are a few conditions that cause the range to hold an election when unquiescing too:

func shouldCampaignOnWake(
leaseStatus kvserverpb.LeaseStatus,
storeID roachpb.StoreID,
raftStatus raft.BasicStatus,
livenessMap liveness.IsLiveMap,
desc *roachpb.RangeDescriptor,
requiresExpiringLease bool,
) bool {
// When waking up a range, campaign unless we know that another
// node holds a valid lease (this is most important after a split,
// when all replicas create their raft groups at about the same
// time, with a lease pre-assigned to one of them). Note that
// thanks to PreVote, unnecessary campaigns are not disruptive so
// we should err on the side of campaigining here.
if leaseStatus.IsValid() && !leaseStatus.OwnedBy(storeID) {
return false
}
// If we're already campaigning don't start a new term.
if raftStatus.RaftState != raft.StateFollower {
return false
}
// If we don't know who the leader is, then campaign.
if raftStatus.Lead == raft.None {
return true
}
// Avoid a circular dependency on liveness and skip the is leader alive
// check for ranges that always use expiration based leases.
if requiresExpiringLease {
return false
}
// Determine if we think the leader is alive, if we don't have the leader
// in the descriptor we assume it is, since it could be an indication that this
// replica is behind.
replDesc, ok := desc.GetReplicaDescriptorByID(roachpb.ReplicaID(raftStatus.Lead))
if !ok {
return false
}
// If we don't know about the leader in our liveness map, then we err on the side
// of caution and dont campaign.
livenessEntry, ok := livenessMap[replDesc.NodeID]
if !ok {
return false
}
return !livenessEntry.IsLive
}

@erikgrinaker
Copy link
Contributor Author

Whatever's causing this, it ain't quiescence. :( I ran another experiment with COCKROACH_DISABLE_QUIESCENCE=true, and I'm still seeing the restarted node pick up Raft leaders. I'm going to keep this issue open until I get all the way to the bottom of it.

@tbg
Copy link
Member

tbg commented Nov 18, 2022

Hmm, good point. Here's the code in raft:

https://github.com/etcd-io/etcd/blob/07a3cfe22eaa96c34898922ddd27a2354bf9459a/raft/raft.go#L970-L978

I thought that we'd maybe reinit the *RawNode when unquiescing, but doesn't seem to be the case.

Could it just be "natural" that there isn't a leader at this point in the incident triggering this issue? The experiment makes me think that no - since there is a lease on these ranges, they must've elected a leader "at some point". The only node that went down is n5 (the then half-partitioned node). So maybe that node was the leader before for the affected ranges? In that case, the other members would PreVote for it. I don't know if this is plausible in this experiment, as the lease being on a range should typically pull the leader to it as well, but it's the only explanation I have.

@erikgrinaker
Copy link
Contributor Author

The node doesn't have any leaders before the restart. The lease preferences pin leases on other nodes, and the Raft leader follows the lease. This is confirmed by metrics.

The interesting bit is that it's picking the leaderships up at a regular rate of about 1 every 10 seconds.

@erikgrinaker
Copy link
Contributor Author

erikgrinaker commented Nov 18, 2022

Looks like it is in fact receiving prevotes from n4. Here's the r789 layout:

                "internalReplicas": [
                  {
                    "nodeId": 1,
                    "storeId": 1,
                    "replicaId": 1,
                    "type": 0
                  },
                  {
                    "nodeId": 5,
                    "storeId": 5,
                    "replicaId": 2,
                    "type": 0
                  },
                  {
                    "nodeId": 4,
                    "storeId": 4,
                    "replicaId": 3,
                    "type": 0
                  }
                ],

And we can see that n5 is receiving a prevote from 3 (i.e. n4):

I221118 12:52:26.766015 2192 go.etcd.io/etcd/raft/v3/raft.go:1651 ⋮ [n5,s5,r789/2:‹/Table/106/1/-75{2796…-0953…}›] 934  2 switched to configuration ‹voters=(1 2 3)›
I221118 12:52:26.766068 2192 go.etcd.io/etcd/raft/v3/raft.go:680 ⋮ [n5,s5,r789/2:‹/Table/106/1/-75{2796…-0953…}›] 935  2 became follower at term 7
I221118 12:52:26.766108 2192 go.etcd.io/etcd/raft/v3/raft.go:361 ⋮ [n5,s5,r789/2:‹/Table/106/1/-75{2796…-0953…}›] 936  newRaft 2 [peers: [‹1,2,3›], term: 7, commit: 1778, applied: 1778, lastindex: 1778, lastterm: 7]
I221118 12:52:26.766129 2192 go.etcd.io/etcd/raft/v3/raft.go:1421 ⋮ [n5,s5,r789/2:‹/Table/106/1/-75{2796…-0953…}›] 937  2 no leader at term 7; dropping proposal
I221118 12:52:31.649315 175 go.etcd.io/etcd/raft/v3/raft.go:767 ⋮ [n5,s5,r789/2:‹/Table/106/1/-75{2796…-0953…}›] 1123  2 is starting a new election at term 7
I221118 12:52:31.649619 175 go.etcd.io/etcd/raft/v3/raft.go:709 ⋮ [n5,s5,r789/2:‹/Table/106/1/-75{2796…-0953…}›] 1131  2 became pre-candidate at term 7
I221118 12:52:31.649635 175 go.etcd.io/etcd/raft/v3/raft.go:827 ⋮ [n5,s5,r789/2:‹/Table/106/1/-75{2796…-0953…}›] 1132  2 received ‹MsgPreVoteResp› from 2 at term 7
I221118 12:52:31.649654 175 go.etcd.io/etcd/raft/v3/raft.go:814 ⋮ [n5,s5,r789/2:‹/Table/106/1/-75{2796…-0953…}›] 1133  2 [logterm: 7, index: 1778] sent ‹MsgPreVote› request to 1 at term 7
I221118 12:52:31.649670 175 go.etcd.io/etcd/raft/v3/raft.go:814 ⋮ [n5,s5,r789/2:‹/Table/106/1/-75{2796…-0953…}›] 1134  2 [logterm: 7, index: 1778] sent ‹MsgPreVote› request to 3 at term 7
I221118 12:52:31.650008 218 go.etcd.io/etcd/raft/v3/raft.go:827 ⋮ [n5,s5,r789/2:‹/Table/106/1/-75{2796…-0953…}›] 1137  2 received ‹MsgPreVoteResp› from 3 at term 7
I221118 12:52:31.650034 218 go.etcd.io/etcd/raft/v3/raft.go:1397 ⋮ [n5,s5,r789/2:‹/Table/106/1/-75{2796…-0953…}›] 1138  2 has received 2 ‹MsgPreVoteResp› votes and 0 vote rejections
I221118 12:52:31.650055 218 go.etcd.io/etcd/raft/v3/raft.go:693 ⋮ [n5,s5,r789/2:‹/Table/106/1/-75{2796…-0953…}›] 1139  2 became candidate at term 8
I221118 12:52:31.650075 218 go.etcd.io/etcd/raft/v3/raft.go:827 ⋮ [n5,s5,r789/2:‹/Table/106/1/-75{2796…-0953…}›] 1140  2 received ‹MsgVoteResp› from 2 at term 8
I221118 12:52:31.650094 218 go.etcd.io/etcd/raft/v3/raft.go:814 ⋮ [n5,s5,r789/2:‹/Table/106/1/-75{2796…-0953…}›] 1141  2 [logterm: 7, index: 1778] sent ‹MsgVote› request to 1 at term 8
I221118 12:52:31.650110 218 go.etcd.io/etcd/raft/v3/raft.go:814 ⋮ [n5,s5,r789/2:‹/Table/106/1/-75{2796…-0953…}›] 1142  2 [logterm: 7, index: 1778] sent ‹MsgVote› request to 3 at term 8
I221118 12:52:31.651336 173 go.etcd.io/etcd/raft/v3/raft.go:827 ⋮ [n5,s5,r789/2:‹/Table/106/1/-75{2796…-0953…}›] 1143  2 received ‹MsgVoteResp› from 3 at term 8
I221118 12:52:31.651358 173 go.etcd.io/etcd/raft/v3/raft.go:1397 ⋮ [n5,s5,r789/2:‹/Table/106/1/-75{2796…-0953…}›] 1144  2 has received 2 ‹MsgVoteResp› votes and 0 vote rejections
I221118 12:52:31.651374 173 go.etcd.io/etcd/raft/v3/raft.go:745 ⋮ [n5,s5,r789/2:‹/Table/106/1/-75{2796…-0953…}›] 1145  2 became leader at term 8

@erikgrinaker
Copy link
Contributor Author

Did another run with Raft logging enabled across all nodes. Range r561 with replicas 2:n4 5:n5 6:n3.

We first see that n3 becomes the leader at 13:04:12:

I221118 13:04:12.752036 3500 raft.go:1651 ⋮ [n3,s3,r561/6] 1316  6 switched to configuration ‹voters=(2 5 6)›
I221118 13:04:12.752104 3500 raft.go:680 ⋮ [n3,s3,r561/6] 1317  6 became follower at term 10
I221118 13:04:12.752150 3500 raft.go:361 ⋮ [n3,s3,r561/6] 1318  newRaft 6 [peers: [‹2,5,6›], term: 10, commit: 1769, applied: 1769, lastindex: 1769, lastterm: 10]
I221118 13:04:12.752179 3500 raft.go:767 ⋮ [n3,s3,r561/6] 1319  6 is starting a new election at term 10
I221118 13:04:12.752194 3500 raft.go:709 ⋮ [n3,s3,r561/6] 1320  6 became pre-candidate at term 10
I221118 13:04:12.752208 3500 raft.go:827 ⋮ [n3,s3,r561/6] 1321  6 received ‹MsgPreVoteResp› from 6 at term 10
I221118 13:04:12.752228 3500 raft.go:814 ⋮ [n3,s3,r561/6] 1322  6 [logterm: 10, index: 1769] sent ‹MsgPreVote› request to 2 at term 10
I221118 13:04:12.752240 3500 raft.go:814 ⋮ [n3,s3,r561/6] 1323  6 [logterm: 10, index: 1769] sent ‹MsgPreVote› request to 5 at term 10
I221118 13:04:12.752269 3500 raft.go:1384 ⋮ [n3,s3,r561/6] 1324  6 no leader at term 10; dropping proposal
I221118 13:04:12.753292 237 raft.go:827 ⋮ [n3,s3,r561/6] 1325  6 received ‹MsgPreVoteResp› from 5 at term 10
I221118 13:04:12.753333 237 raft.go:1397 ⋮ [n3,s3,r561/6] 1326  6 has received 2 ‹MsgPreVoteResp› votes and 0 vote rejections
I221118 13:04:12.753358 237 raft.go:693 ⋮ [n3,s3,r561/6] 1327  6 became candidate at term 11
I221118 13:04:12.753372 237 raft.go:827 ⋮ [n3,s3,r561/6] 1328  6 received ‹MsgVoteResp› from 6 at term 11
I221118 13:04:12.753385 237 raft.go:814 ⋮ [n3,s3,r561/6] 1329  6 [logterm: 10, index: 1769] sent ‹MsgVote› request to 2 at term 11
I221118 13:04:12.753396 237 raft.go:814 ⋮ [n3,s3,r561/6] 1330  6 [logterm: 10, index: 1769] sent ‹MsgVote› request to 5 at term 11
I221118 13:04:12.755320 229 raft.go:827 ⋮ [n3,s3,r561/6] 1331  6 received ‹MsgVoteResp› from 5 at term 11
I221118 13:04:12.755352 229 raft.go:1397 ⋮ [n3,s3,r561/6] 1332  6 has received 2 ‹MsgVoteResp› votes and 0 vote rejections
I221118 13:04:12.755372 229 raft.go:745 ⋮ [n3,s3,r561/6] 1333  6 became leader at term 11

n4 confirms this:

I221118 13:04:12.752936 215 raft.go:1651 ⋮ [n4,s4,r561/2] 1000  2 switched to configuration ‹voters=(2 5 6)›
I221118 13:04:12.753012 215 raft.go:680 ⋮ [n4,s4,r561/2] 1001  2 became follower at term 10
I221118 13:04:12.753082 215 raft.go:361 ⋮ [n4,s4,r561/2] 1002  newRaft 2 [peers: [‹2,5,6›], term: 10, commit: 1769, applied: 1769, lastindex: 1769, lastterm: 10]
I221118 13:04:12.753112 215 raft.go:945 ⋮ [n4,s4,r561/2] 1003  2 [logterm: 10, index: 1769, vote: 6] cast ‹MsgPreVote› for 6 [logterm: 10, index: 1769] at term 10
I221118 13:04:12.754293 217 raft.go:862 ⋮ [n4,s4,r561/2] 1004  2 [term: 10] received a ‹MsgVote› message with higher term from 6 [term: 11]
I221118 13:04:12.754331 217 raft.go:680 ⋮ [n4,s4,r561/2] 1005  2 became follower at term 11
I221118 13:04:12.754346 217 raft.go:945 ⋮ [n4,s4,r561/2] 1006  2 [logterm: 10, index: 1769, vote: 0] cast ‹MsgVote› for 6 [logterm: 10, index: 1769] at term 11

When n5 is restarted, it solicits and receives prevotes and then votes from n4, even though n4 already has n3 as leader:

I221118 13:14:10.328731 2272 raft.go:1651 ⋮ [n5,s5,r561/5] 1022  5 switched to configuration ‹voters=(2 5 6)›
I221118 13:14:10.328783 2272 raft.go:680 ⋮ [n5,s5,r561/5] 1023  5 became follower at term 11
I221118 13:14:10.328822 2272 raft.go:361 ⋮ [n5,s5,r561/5] 1024  newRaft 5 [peers: [‹2,5,6›], term: 11, commit: 1805, applied: 1805, lastindex: 1805, lastterm: 11]
I221118 13:14:10.328845 2272 raft.go:1421 ⋮ [n5,s5,r561/5] 1025  5 no leader at term 11; dropping proposal
I221118 13:14:15.757321 200 raft.go:767 ⋮ [n5,s5,r561/5] 1269  5 is starting a new election at term 11
I221118 13:14:15.757365 200 raft.go:709 ⋮ [n5,s5,r561/5] 1270  5 became pre-candidate at term 11
I221118 13:14:15.757380 200 raft.go:827 ⋮ [n5,s5,r561/5] 1271  5 received ‹MsgPreVoteResp› from 5 at term 11
I221118 13:14:15.757407 200 raft.go:814 ⋮ [n5,s5,r561/5] 1272  5 [logterm: 11, index: 1805] sent ‹MsgPreVote› request to 2 at term 11
I221118 13:14:15.757425 200 raft.go:814 ⋮ [n5,s5,r561/5] 1273  5 [logterm: 11, index: 1805] sent ‹MsgPreVote› request to 6 at term 11
I221118 13:14:15.758119 209 raft.go:827 ⋮ [n5,s5,r561/5] 1274  5 received ‹MsgPreVoteResp› from 2 at term 11
I221118 13:14:15.758150 209 raft.go:1397 ⋮ [n5,s5,r561/5] 1275  5 has received 2 ‹MsgPreVoteResp› votes and 0 vote rejections
I221118 13:14:15.758170 209 raft.go:693 ⋮ [n5,s5,r561/5] 1276  5 became candidate at term 12
I221118 13:14:15.758179 209 raft.go:827 ⋮ [n5,s5,r561/5] 1277  5 received ‹MsgVoteResp› from 5 at term 12
I221118 13:14:15.758193 209 raft.go:814 ⋮ [n5,s5,r561/5] 1278  5 [logterm: 11, index: 1805] sent ‹MsgVote› request to 2 at term 12
I221118 13:14:15.758205 209 raft.go:814 ⋮ [n5,s5,r561/5] 1279  5 [logterm: 11, index: 1805] sent ‹MsgVote› request to 6 at term 12
I221118 13:14:15.759910 179 raft.go:827 ⋮ [n5,s5,r561/5] 1280  5 received ‹MsgVoteResp› from 2 at term 12
I221118 13:14:15.759932 179 raft.go:1397 ⋮ [n5,s5,r561/5] 1281  5 has received 2 ‹MsgVoteResp› votes and 0 vote rejections
I221118 13:14:15.759950 179 raft.go:745 ⋮ [n5,s5,r561/5] 1282  5 became leader at term 12

n4 doesn't really say anything about why:

I221118 13:14:15.757846 202 raft.go:945 ⋮ [n4,s4,r561/2] 9610  2 [logterm: 11, index: 1805, vote: 6] cast ‹MsgPreVote› for 5 [logterm: 11, index: 1805] at term 11
I221118 13:14:15.758946 207 raft.go:862 ⋮ [n4,s4,r561/2] 9611  2 [term: 11] received a ‹MsgVote› message with higher term from 5 [term: 12]
I221118 13:14:15.758985 207 raft.go:680 ⋮ [n4,s4,r561/2] 9612  2 became follower at term 12
I221118 13:14:15.759001 207 raft.go:945 ⋮ [n4,s4,r561/2] 9613  2 [logterm: 11, index: 1805, vote: 0] cast ‹MsgVote› for 5 [logterm: 11, index: 1805] at term 12

And n3 then happily obeys the new leader:

I221118 13:14:15.757877 223 raft.go:945 ⋮ [n3,s3,r561/6] 11971  6 [logterm: 11, index: 1805, vote: 6] cast ‹MsgPreVote› for 5 [logterm: 11, index: 1805] at term 11
I221118 13:14:15.758948 175 raft.go:862 ⋮ [n3,s3,r561/6] 11972  6 [term: 11] received a ‹MsgVote› message with higher term from 5 [term: 12]
I221118 13:14:15.758987 175 raft.go:680 ⋮ [n3,s3,r561/6] 11973  6 became follower at term 12
I221118 13:14:15.758998 175 raft.go:945 ⋮ [n3,s3,r561/6] 11974  6 [logterm: 11, index: 1805, vote: 0] cast ‹MsgVote› for 5 [logterm: 11, index: 1805] at term 12
I221118 13:14:15.822173 177 raft.go:904 ⋮ [n3,s3,r561/6] 11975  6 [term: 12] ignored a ‹MsgHeartbeatResp› message with lower term from 2 [term: 11]

It's certainly interesting that even n3 casts a prevote for n5 though, even though n3 is supposedly the current leader (this prevote never reaches n5 due to the partition). The Raft logs above are complete, so there's no indication that n3 lost its leadership. I'll have to go read some Raft code, I think.

@erikgrinaker
Copy link
Contributor Author

erikgrinaker commented Nov 18, 2022

It seems like this is a bug in the Raft library. When n5 sends a PreVote, it sends it for the next term (in this case 12):

https://github.com/etcd-io/etcd/blob/87258efd90224bc8b59e000f75fe07fdeab68e2d/raft/raft.go#L784-L785

But a node will always accept a PreVote for a future term:

https://github.com/etcd-io/etcd/blob/87258efd90224bc8b59e000f75fe07fdeab68e2d/raft/raft.go#L924

@erikgrinaker erikgrinaker changed the title kvserver: range quiescence may interfere with Raft prevote kvserver: Raft prevote does not prevent election despite active leader Nov 18, 2022
@erikgrinaker erikgrinaker self-assigned this Nov 18, 2022
@erikgrinaker
Copy link
Contributor Author

Tried out a trivial patch to disable that condition, and the node no longer picks up Raft leaderships. Will submit a proper patch upstream.

@nvanbenschoten
Copy link
Member

and importantly, if it has not received an AppendEntries RPC from the leader with in the election timeout.

I don't recall seeing code that implements this logic in etcd/raft. Do you mind pointing me to it?

Wait, what? Isn't the whole idea of prevote to avoid disrupting an established leader? Why would the followers then choose to vote for a different leader during prevote? Wouldn't that give a 50% chance of disruption?

My understanding of prevote is that it's intended to avoid disrupting an established leader when the candidate wouldn't otherwise be able to win the election. I didn't think it said anything about cases where the candidate could win the election. So I didn't think it would provide any protection in cases where the follower is caught up on its log.

I can see how the election timeout check you cited would skew pre-vote elections towards established leaders in cases of ties. Is that missing? Is that the patch you plan to submit upstream?

@nvanbenschoten
Copy link
Member

I didn't think it said anything about cases where the candidate could win the election.

And yet, mention of the election timeout is right there in section 9.6 of the Raft thesis.

This is a nice find! @bdarnell do you recall whether this was intentionally omitted in etcd-io/etcd#6624?

I wouldn't be surprised if fixing this causes other fallout. Specifically, there are places where we call Raft elections to safely request a Raft leadership transfer. For instance, that's what we did in #87244. If campaigning doesn't break ties and we need buy-in from the leader then we'll need to rethink the mechanism that we use in cases like these.

@erikgrinaker
Copy link
Contributor Author

erikgrinaker commented Nov 18, 2022

My understanding of prevote is that it's intended to avoid disrupting an established leader when the candidate wouldn't otherwise be able to win the election.

No, the motivation is that a node rejoining a cluster following e.g. a restart or network partition shouldn't immediately trigger an election, instead respecting an already established quorum. This is particularly important in cases with asymmetric or partial network partitions, where a node may otherwise end up continually triggering elections, destabilizing the quorum (one such scenario was described in the link above, https://decentralizedthoughts.github.io/2020-12-12-raft-liveness-full-omission/).

I don't recall seeing code that implements this logic in etcd/raft

I think this condition might be sufficient:

https://github.com/etcd-io/etcd/blob/87258efd90224bc8b59e000f75fe07fdeab68e2d/raft/raft.go#L921-L922

But I'm not very familiar with the etcd/raft codebase yet, so I'll have to do some reading. I'm hopeful it should be fairly straightforward though.

I wouldn't be surprised if fixing this causes other fallout. Specifically, there are places where we call Raft elections to safely request a Raft leadership transfer.

I believe we can request a cooperative leadership transfer:

https://github.com/etcd-io/etcd/blob/dad8208a4ddd46bcbab937d0c2cac5fedf5a3113/raft/node.go#L555-L562

But I expect there will be some fallout, yeah.

@bdarnell
Copy link
Contributor

And yet, mention of the election timeout is right there in section 9.6 of the Raft thesis.

This is a nice find! bdarnell do you recall whether this was intentionally omitted in etcd-io/etcd#6624?

I believe the relevant check is in https://github.com/etcd-io/etcd/blob/87258efd90224bc8b59e000f75fe07fdeab68e2d/raft/raft.go#L843, where the check for time since hearing from the leader is tied to the CheckQuorum setting, just like it is for regular votes. We deliberately decoupled these things because quiesced replicas would not have a notion of how long it had been since they heard from the leader.

When we originally implemented PreVote, the major problem we were trying to solve is that when a leader receives a MsgVote for a higher term, it immediately reverts to follower, even if the node calling for the vote is doomed to fail. PreVote addresses this by ensuring that there are no side effects until we have a reasonable expectation of winning the election.

Enabling CheckQuorum would prevent some additional categories of disruption, but it would make quiesced replicas much more expensive (and resetting the replica after unquiescing would I think make recoveries slower).

@erikgrinaker
Copy link
Contributor Author

Thanks for the context Ben. I'll need to look closer at this, but my sense is that as a first step we could simply consider time to be frozen while the replica is quiesced -- we don't really need a Raft leader until there are any state changes. As you point out, that would make recoveries slower for quiesced replicas -- this would be somewhat mitigated by #91947, and we can probably come up with other improvements to handle that case.

@bdarnell
Copy link
Contributor

You said in #92088 (comment) that quiescence is not the problem. And I think the talk of PreVote is kind of a distraction too. The problem is that in partial partition scenarios, a node that is reachable only by some of its peers can win an election. This is an especially noticeable problem if it ends up stealing leadership from a node that is more broadly reachable (and this is what we're talking about here), but it's also a problem if the partially-reachable node steps up in a leadership vacuum. CheckQuorum and similar rules based on time and the validity of a leader can help prevent the problem of stealing leadership, but they make it harder to recover from the latter case in which a bad leader was elected in the first place.

We've never made much of an investment in improving support for partial partitions because a sufficiently diabolical partition can always cause unavailability (see jepsen's "majority ring" nemesis: each node can see a majority of its peers, but each one sees a different subset). There's likely some low-hanging fruit to detect and mitigate some of the more common cases. For example, the one asymmetric partition that I know we've seen multiple times is when the network is actually fine, but certificate issues mean that connections from A to B work but B to A do not (and this is especially tricky since RPC responses flow back on the same connection as their request, but raft "responses" are modeled as separate one-way messages). It would be useful to detect this case in the RPC heartbeat system.

@erikgrinaker
Copy link
Contributor Author

You said in #92088 (comment) that quiescence is not the problem.

Yes, I meant that if we were to implement prevote and checkquorum as commonly understood (including the election timeout criterion) then quiescence could become a problem, e.g. by increasing recovery time.

And I think the talk of PreVote is kind of a distraction too. The problem is that in partial partition scenarios, a node that is reachable only by some of its peers can win an election. This is an especially noticeable problem if it ends up stealing leadership from a node that is more broadly reachable (and this is what we're talking about here), but it's also a problem if the partially-reachable node steps up in a leadership vacuum. CheckQuorum and similar rules based on time and the validity of a leader can help prevent the problem of stealing leadership, but they make it harder to recover from the latter case in which a bad leader was elected in the first place.

I mostly agree with what you're saying, but don't understand the last part: aren't PreVote and CheckQuorum complementary pieces of the solution? PreVote (as described in the thesis) will make it harder to unseat an elected leader as long as that leader is able to send heartbeats to a quorum of followers, but CheckQuorum will make the leader itself step down if a quorum of followers are unable to acknowledge those heartbeats. As far as I can tell, these two conditions ensure that a quorum of followers have bidirectional communication links with the leader and that it won't be disrupted by spurious elections.

Of course, that's insufficient in CockroachDB, because the leaseholder can still be partitioned away from the Raft leader, preventing it from proposing writes, and a SQL gateway can be partitioned away from a leaseholder, preventing it from making any RPC calls. These two problems will need to be addressed separately, and my sense is that we'll ultimately have to solve them with mesh routing (hopefully not inside CRDB itself).

I still think there's value in making the underlying Raft implementation more resistant to partial network partitions, but if we don't feel like it's worth the effort at the moment then there's no shortage of other work that needs doing. We may still want to file an issue about this in the upstream library in that case, since other library users may have different expectations of prevote.

There's likely some low-hanging fruit to detect and mitigate some of the more common cases. For example, the one asymmetric partition that I know we've seen multiple times is when the network is actually fine, but certificate issues mean that connections from A to B work but B to A do not (and this is especially tricky since RPC responses flow back on the same connection as their request, but raft "responses" are modeled as separate one-way messages). It would be useful to detect this case in the RPC heartbeat system.

Yes, we're pursuing this separately in #84289, and I agree that it should be the primary focus.

@bdarnell
Copy link
Contributor

I mostly agree with what you're saying, but don't understand the last part: aren't PreVote and CheckQuorum complementary pieces of the solution? PreVote (as described in the thesis) will make it harder to unseat an elected leader as long as that leader is able to send heartbeats to a quorum of followers, but CheckQuorum will make the leader itself step down if a quorum of followers are unable to acknowledge those heartbeats.

This is confusing because there are three pieces of functionality being described with two different names, and the etcd implementation names things differently from the thesis.

  1. A PreVote step which is non-binding and does not trigger side effects.
  2. Followers track time-since-heartbeat and refuse to vote to depose an active leader
  3. Leaders track time-since-heartbeat-response and step down to follower if they're partitioned away, instead of waiting for a former follower's MsgVote

The etcd implementation combines both time-tracking features under the CheckQuorum setting (this is due to cockroach's influence, since we were interested in the ability to quiesce replicas and turn off all time-tracking functionality), while the thesis describes the second feature as a part of the pre-vote.

Each of these features solves a partially-overlapping set of problems. In terms of improving behavior during partial partitions, PreVote and CheckQuorum are complementary. But in terms of preventing disruption from a restarted node in the absence of a partition, they're redundant, and this was the original focus when they were implemented, so we've historically seen them more as alternatives than as complements.

@erikgrinaker
Copy link
Contributor Author

I see, that clears things up -- thank you!

@erikgrinaker
Copy link
Contributor Author

I'm putting this on the backburner for now. We'll consider this as part of normal release planning, when we're ready to prioritize partial network partitions.

@erikgrinaker
Copy link
Contributor Author

erikgrinaker commented May 29, 2023

I'm working on enabling CheckQuorum.

This has the effect of setting Progress.RecentActive = false on leaders every election timeout, such that it can flip it back to true the next time it hears from a follower. Previously, this was always true since we did not enable CheckQuorum. We do use RecentActive in a few spots in CRDB, but only after replacing it with our own notion of activity via updateRaftProgressFromActivity(), so that's fine.

There is a lot of ancient history around CheckQuorum and quiescence that we'll need to revisit here:

For now, I'm submitting a PR in #104040 to enable CheckQuorum via envvar, so we can try it out.

@erikgrinaker
Copy link
Contributor Author

I ran the failover/chaos test suite with CheckQuorum enabled (except the deadlock and partial partition failure modes which we can't yet handle anyway), and the maximum unavailability didn't change, neither for the read-write workload nor the read-only workload. I also ran the entire CI suite, with no failures. So this isn't obviously broken, at least.

Will add some more targeted integration tests.

@erikgrinaker
Copy link
Contributor Author

I ran tpccbench/nodes=3/cpu=16 with this enabled and it passed 2075 warehouses, which is in line with the nightlies.

@erikgrinaker
Copy link
Contributor Author

erikgrinaker commented May 29, 2023

Added some integration tests for PreVote and CheckQuorum under various scenarios in #104057. They mostly work as expected, except for the case of a partial partition with a quiesced range, where we get a spurious election that prevote should have prevented. I haven't looked into it yet, but I think it's the same case that's discussed in e.g. #9372.

@erikgrinaker
Copy link
Contributor Author

This appears more straightforward than the situation in #9372: n3's prevote unquiesces n2 (who rejects it), but n2 doesn't wake the leader. Once n2's election timeout elapses, n2 and n3 can hold a new election. I'll try a patch where we wake the leader on receipt of a MsgPreVote.

The scenario is a partial partition between n1 (leader) and n3:

               n1 (leader)
              /  x
             /    x
(follower) n2 ---- n3 (partitioned)

Relevant logs:

    client_raft_test.go:6045: n1 has lease
08:00:37.806374 kv/kvserver/replica_raft_quiesce.go:43  [T1,n1,s1,r64/1:/{Table/Max-Max},raft] 802  quiescing 64
08:00:37.906519 kv/kvserver/replica_raft_quiesce.go:43  [T1,n3,s3,r64/3:/{Table/Max-Max},raft] 804  quiescing 64
08:00:37.906516 kv/kvserver/replica_raft_quiesce.go:43  [T1,n2,s2,r64/2:/{Table/Max-Max},raft] 803  quiescing 64
    client_raft_test.go:6050: n3 quiesced
    client_raft_test.go:6058: n3 partitioned
    client_raft_test.go:6063: n1 StateLeader at term=6 commit=22
08:00:37.935097 kv/kvserver/replica_raft_quiesce.go:94  [T1,n3,s3,r64/3:/{Table/Max-Max}] 805  unquiescing 64: waking leader
    client_raft_test.go:6068: n3 unquiesced
08:00:38.137838 go.etcd.io/raft/v3/raft.go:928  [T1,n3,s3,r64/3:/{Table/Max-Max}] 809  3 is starting a new election at term 6
08:00:38.137896 go.etcd.io/raft/v3/raft.go:867  [T1,n3,s3,r64/3:/{Table/Max-Max}] 810  3 became pre-candidate at term 6
08:00:38.137913 go.etcd.io/raft/v3/raft.go:971  [T1,n3,s3,r64/3:/{Table/Max-Max}] 811  3 [logterm: 6, index: 22] sent MsgPreVote request to 1 at term 6
08:00:38.137938 go.etcd.io/raft/v3/raft.go:971  [T1,n3,s3,r64/3:/{Table/Max-Max}] 812  3 [logterm: 6, index: 22] sent MsgPreVote request to 2 at term 6
08:00:38.138058 go.etcd.io/raft/v3/raft.go:984  [T1,n3,s3,r64/3:/{Table/Max-Max}] 813  3 received MsgPreVoteResp from 3 at term 6
08:00:38.138108 go.etcd.io/raft/v3/raft.go:1589  [T1,n3,s3,r64/3:/{Table/Max-Max}] 814  3 has received 1 MsgPreVoteResp votes and 0 vote rejections
08:00:38.138490 kv/kvserver/replica_raft_quiesce.go:71  [T1,n2,s2,r64/2:/{Table/Max-Max}] 816  unquiescing 64
08:00:38.138537 go.etcd.io/raft/v3/raft.go:1004  [T1,n2,s2,r64/2:/{Table/Max-Max}] 817  2 [logterm: 6, index: 22, vote: 0] ignored MsgPreVote from 3 [logterm: 6, index: 22] at term 6: lease is not expired (remaining ticks: 2)
    client_raft_test.go:6077: n3 StatePreCandidate at term=6 commit=22
    client_raft_test.go:6080: n3 became pre-candidate
08:00:38.937698 go.etcd.io/raft/v3/raft.go:928  [T1,n3,s3,r64/3:/{Table/Max-Max}] 820  3 is starting a new election at term 6
08:00:38.937778 go.etcd.io/raft/v3/raft.go:867  [T1,n3,s3,r64/3:/{Table/Max-Max}] 821  3 became pre-candidate at term 6
08:00:38.937797 go.etcd.io/raft/v3/raft.go:971  [T1,n3,s3,r64/3:/{Table/Max-Max}] 822  3 [logterm: 6, index: 22] sent MsgPreVote request to 1 at term 6
08:00:38.937826 go.etcd.io/raft/v3/raft.go:971  [T1,n3,s3,r64/3:/{Table/Max-Max}] 823  3 [logterm: 6, index: 22] sent MsgPreVote request to 2 at term 6
08:00:38.937901 go.etcd.io/raft/v3/raft.go:984  [T1,n3,s3,r64/3:/{Table/Max-Max}] 824  3 received MsgPreVoteResp from 3 at term 6
08:00:38.937973 go.etcd.io/raft/v3/raft.go:1589  [T1,n3,s3,r64/3:/{Table/Max-Max}] 825  3 has received 1 MsgPreVoteResp votes and 0 vote rejections
08:00:38.938063 go.etcd.io/raft/v3/raft.go:1132  [T1,n2,s2,r64/2:/{Table/Max-Max}] 827  2 [logterm: 6, index: 22, vote: 0] cast MsgPreVote for 3 [logterm: 6, index: 22] at term 6
08:00:38.938559 go.etcd.io/raft/v3/raft.go:984  [T1,n3,s3,r64/3:/{Table/Max-Max}] 828  3 received MsgPreVoteResp from 2 at term 6
08:00:38.938599 go.etcd.io/raft/v3/raft.go:1589  [T1,n3,s3,r64/3:/{Table/Max-Max}] 829  3 has received 2 MsgPreVoteResp votes and 0 vote rejections
08:00:38.938643 go.etcd.io/raft/v3/raft.go:851  [T1,n3,s3,r64/3:/{Table/Max-Max}] 830  3 became candidate at term 7
08:00:38.938658 go.etcd.io/raft/v3/raft.go:971  [T1,n3,s3,r64/3:/{Table/Max-Max}] 831  3 [logterm: 6, index: 22] sent MsgVote request to 1 at term 7
08:00:38.938678 go.etcd.io/raft/v3/raft.go:971  [T1,n3,s3,r64/3:/{Table/Max-Max}] 832  3 [logterm: 6, index: 22] sent MsgVote request to 2 at term 7
08:00:38.938760 go.etcd.io/raft/v3/raft.go:984  [T1,n3,s3,r64/3:/{Table/Max-Max}] 833  3 received MsgVoteResp from 3 at term 7
08:00:38.938819 go.etcd.io/raft/v3/raft.go:1589  [T1,n3,s3,r64/3:/{Table/Max-Max}] 834  3 has received 1 MsgVoteResp votes and 0 vote rejections
08:00:38.938857 go.etcd.io/raft/v3/raft.go:1019  [T1,n2,s2,r64/2:/{Table/Max-Max}] 836  2 [term: 6] received a MsgVote message with higher term from 3 [term: 7]
08:00:38.938896 go.etcd.io/raft/v3/raft.go:838  [T1,n2,s2,r64/2:/{Table/Max-Max}] 837  2 became follower at term 7
08:00:38.938910 go.etcd.io/raft/v3/raft.go:1132  [T1,n2,s2,r64/2:/{Table/Max-Max}] 838  2 [logterm: 6, index: 22, vote: 0] cast MsgVote for 3 [logterm: 6, index: 22] at term 7
08:00:38.939089 go.etcd.io/raft/v3/raft.go:984  [T1,n3,s3,r64/3:/{Table/Max-Max}] 839  3 received MsgVoteResp from 2 at term 7
08:00:38.939125 go.etcd.io/raft/v3/raft.go:1589  [T1,n3,s3,r64/3:/{Table/Max-Max}] 840  3 has received 2 MsgVoteResp votes and 0 vote rejections
08:00:38.939145 go.etcd.io/raft/v3/raft.go:906  [T1,n3,s3,r64/3:/{Table/Max-Max}] 841  3 became leader at term 7
    client_raft_test.go:6085: n3 StateLeader at term=7 commit=23

@erikgrinaker
Copy link
Contributor Author

Submitted a fix in #104057, where receipt of a pre-vote will wake the leader. I gated this on CheckQuorum for now, since this may cause mass unquiescence under partial network partitions, where the partitioned follower will continually keep the ranges unquiesced.

We'll need more testing and auditing here, but at least all the current test cases pass for now.

craig bot pushed a commit that referenced this issue May 30, 2023
103909: ui: add Created SQL Connections chart on SQL dashboard r=koorosh a=koorosh

Added Created SQL Connections chart that shows rate of created SQL connection over time.

Release note (ui change): added Created SQL Connections chart on
Metrics page, SQL Dashboard in DB Console.

Resolves: #93486

![Screenshot 2023-05-25 at 21 24 48](https://github.com/cockroachdb/cockroach/assets/3106437/9fbba32e-3bdc-4bce-ac51-4e57d141b926)


104025: kvserver: add a few Raft proposal trace events r=erikgrinaker a=erikgrinaker

Epic: none
Release note: None

104026: kvserver: don't apply quota pool to lease requests r=erikgrinaker a=erikgrinaker

Lease requests and transfers are latency sensitive and small, so we allow them to bypass the quota pool. This is particularly important for expiration lease extensions, which happen every 3 seconds.

During TPCC imports, the quota pool was often found to delay lease requests by as much as 3 seconds.

Resolves #98124.

Epic: none
Release note: None

104040: kvserver: add `COCKROACH_RAFT_ENABLE_CHECKQUORUM` r=erikgrinaker a=erikgrinaker

This patch adds `COCKROACH_RAFT_ENABLE_CHECKQUORUM`, which enables Raft CheckQuorum (and as a side-effect also fully enables PreVote). See code comment for details. It defaults to false, since we need to investigate liveness concerns from when CheckQuorum was previously enabled in the ancient past.

Raft uses the field `Progress.RecentActive` to track follower activity for CheckQuorum. When CheckQuorum is disabled, this is always `true`, but when enabled it will be reset to `false` regularly (with an interval that equals the election timeout). There are some uses of this field in CRDB, but we always replace this field with our own notion of activity via `updateRaftProgressFromActivity` (tracked in wall time with a timeout equaling the range lease interval).

Touches #92088.

Epic: none
Release note: None

104050: cli/zip: avoid including PII-laden details in debug zip r=postamar a=knz

Fixes #104049.
Epic: CRDB-27642.

Release note (bug fix): The details of errors pertaining to invalid descriptors are not included any more in redacted debug zips.

104053: kvserver: add DisableQuiescence testing knob r=erikgrinaker a=erikgrinaker

This patch turns `COCKROACH_DISABLE_QUIESCENCE` into a testing knob, for better control in tests.

Epic: none
Release note: None

Co-authored-by: Andrii Vorobiov <and.vorobiov@gmail.com>
Co-authored-by: Erik Grinaker <grinaker@cockroachlabs.com>
Co-authored-by: Raphael 'kena' Poss <knz@thaumogen.net>
@erikgrinaker
Copy link
Contributor Author

A fundamental tradeoff here is that we'll now delay explicit campaigning by up to 2 seconds (the election timeout). This typically comes up in two cases: when the current leader is dead according to liveness, and we're either initializing/unquiescing a replica or attempting to acquire a lease.

In these cases, we want to respect the full PreVote+CheckQuorum condition to not cause spurious elections or steal leadership away. This is particularly important under partial or asymmetric network partitions, since the replica can otherwise steal leadership even though the leaseholder can't reach it, causing permanent range unavailability (the case which motivated this issue). We can't trust the liveness or lease information for the leader in this case, because it's likely to be stale or inaccurate.

There is one case where we can still campaign immediately: when we removed the leader from the range.

See #104189.

@erikgrinaker
Copy link
Contributor Author

I read over the spurious election case in #9372 (comment), caused by CheckQuorum, and it's covered by PreVote. When both are enabled, the unquiesced node won't be able to call an election that unseats the leader (especially with #104057 which also tests this scenario).

@erikgrinaker
Copy link
Contributor Author

I read over the spurious election case in #9372 (comment), caused by CheckQuorum, and it's covered by PreVote.

This is also confirmed by this code comment:

https://github.com/etcd-io/raft/blob/e293d6b9d465bd77845b50d40079dbe44996e504/raft.go#L1051-L1073

craig bot pushed a commit that referenced this issue Jun 7, 2023
104057: kvserver: wake leader on Raft message from other replica r=erikgrinaker a=erikgrinaker

**kvserver: wake leader on Raft message from other replica**

Previously, in the case of a quiesced range, a follower who received a pre-vote request could unquiesce itself only to reject the pre-vote. Under a partial network partition, where the leader itself did not receive a corresponding pre-vote request, we would now have two unquiesced followers with a quiesced leader. Eventually, both of the followers' election timeouts would lapse and they would hold an election, upsetting the established leader. Furthermore, the leadership would often be immediately transferred back to the leaseholder, i.e. the old leader, and the cycle would repeat.

The same holds for any other message that a follower might send to a different follower, without also sending a message to the leader.

This patch makes a quiesced follower unquiesce both itself and the leader when it receives a message from anyone but the leader, allowing the leader to maintain leadership.

Note that both before and after this patch, such partial partitions often result in persistent mass unquiescence due to the continuous prevotes.

Touches #92088.

Epic: None.
Release note: None.

**kvserver: don't unquiesce during snapshot application**

Previously, snapshot application would explicitly unquiesce the newly initialized replica. `handleRaftReady` would also unquiesce the replica once the snapshot is applied, but the former did so without also waking the leader. This can lead to spurious elections if multiple replicas are left unquiesced without waking the leader, e.g. during a partial network partition.

This patch therefore removes the explicit unquiesce during snapshot application, and instead unquiesces normally (along with the leader) during `handleRaftReady`. This is also symmetric with other replica initialization, which does not explicitly unquiesce the replica either.

Touches #92088.

Epic: none
Release note: None

Co-authored-by: Erik Grinaker <grinaker@cockroachlabs.com>
craig bot pushed a commit that referenced this issue Jun 16, 2023
104969: kvserver: force-campaign leaseholder on leader removal r=erikgrinaker a=erikgrinaker

**kvserver: clarify Raft campaign behavior with PreVote+CheckQuorum**

This patch tweaks and clarifies explicit campaign behavior when Raft PreVote and CheckQuorum are enabled. In this case, followers will only grant prevotes if they haven't heard from a leader in the past election timeout. It also adds a test covering this.
  
**kvserver: add `Replica.forceCampaignLocked()`**

This patch adds `forceCampaignLocked()`, which can be used to force an election, transitioning the replica directly to candidate and bypassing PreVote+CheckQuorum.
  
**kvserver: force-campaign leaseholder on leader removal**

Previously, when the leader was removed from the range via a conf change, the first voter in the range descriptor would campaign to avoid waiting for an election timeout. This had a few drawbacks:

* If the first voter is unavailable or lags, noone will campaign.

* If the first voter isn't the leaseholder, it has to immediately transfer leadership to the leaseholder.

* It used Raft PreVote, so it wouldn't be able to win when CheckQuorum is enabled, since followers won't grant votes when they've recently heard from the leader.

This patch instead campaigns on the current leaseholder. We know there can only be one, avoiding election ties. The conf change is typically proposed by the leaseholder anyway so it's likely to be up-to-date. And we want it to be colocated with the leader. If there is no leaseholder then waiting out the election timeout is less problematic, since either we'll have to wait out the lease interval anyway, or the range is idle.

It also forces an election by transitioning directly to candidate, bypassing PreVote. This is ok, since we know the current leader is dead.


To avoid election ties in mixed 23.1/23.2 clusters, we retain the old voter designation until the upgrade is finalized, but always force an election instead of using pre-vote.

Resolves #104871.
Touches #92088.
Follows #104189.
Epic: none.

Release note: None

Co-authored-by: Erik Grinaker <grinaker@cockroachlabs.com>
craig bot pushed a commit that referenced this issue Jun 27, 2023
105126: kvserver: don't campaign when unquiescing for a Raft message r=erikgrinaker a=erikgrinaker

We're going to generalize this to all (pre)votes, see #105132. However, I'm merging this as-is to keep a record of the alternative, since we may want to change to this approach later, when e.g. 23.1 compatibility and epoch leases is less of a concern.

---

**DEPS: upgrade Raft to a10cd45**

Adds `ForgetLeader()`, which we need for CheckQuorum handling.

```
a10cd45 2023-06-26: Merge pull request 78 from erikgrinaker/forget-leader [Benjamin Wang]
1159466 2023-06-16: add `ForgetLeader` [Erik Grinaker]
09ea4c5 2023-06-16: rafttest: show term and leader for `raft-state` [Erik Grinaker]
26ce926 2023-06-20: rafttest: add `log-level` argument for `stabilize` [Erik Grinaker]
30e2fa4 2023-06-12: Merge pull request 70 from erikgrinaker/prevote-checkquorum-datadriven [Benjamin Wang]
a042ce3 2023-06-09: Merge pull request 75 from charles-chenzz/update-go-patch [Benjamin Wang]
dd2340f 2023-06-08: update go to patch release 1.19.10 [charles-chenzz]
27dd2c2 2023-06-02: add data-driven tests for PreVote and CheckQuorum [Erik Grinaker]
```

**kvserver: tweak updateLivenessMap comment**

**kvserver: add `Replica.forgetLeaderLocked()`**

**kvserver: don't campaign when unquiescing for a Raft message**

This patch forgets the leader when unquiescing in response to a Raft message and finding a dead leader (according to liveness). We don't campaign, because that could result in election ties, but forgetting the leader allows us to grant (pre)votes even though we've heard from the leader recently, avoiding having to wait out an election timeout.

Touches #92088.

Epic: none
Release note: None


Co-authored-by: Erik Grinaker <grinaker@cockroachlabs.com>
craig bot pushed a commit that referenced this issue Jun 27, 2023
105132: kvserver: maybe forget leader on (Pre)Vote requests  r=erikgrinaker a=erikgrinaker

This patch will forget the Raft leader when receiving a (Pre)Vote request and finding the leader to be dead (according to liveness) or removed. This allows a quorum of replicas to campaign despite the PreVote+CheckQuorum condition if they independently consider the leader to be dead.

A more targeted alternative was explored, where we forget the leader only when unquiescing to a dead leader. However, this had a number of drawbacks. It would have to use `TransferLeader` to steal leadership away from a Raft leader who can't heartbeat liveness during lease acquisitions, but this could cause unavailability due to races where multiple replicas request leader transfers and some are not up-to-date on the log. It would also need mixed-version logic since 23.1 nodes could otherwise be unable to obtain necessary prevotes.

We instead choose to be lenient for now, and we can consider tightening the conditions later when we're more confident in the PreVote+CheckQuorum handling and may no longer need epoch leases.

Touches #92088.

Epic: none
Release note: None

Co-authored-by: Erik Grinaker <grinaker@cockroachlabs.com>
@craig craig bot closed this as completed in 2689002 Jun 28, 2023
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-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior.
Projects
None yet
Development

Successfully merging a pull request may close this issue.

4 participants