etcd-io / raft

Raft library for maintaining a replicated state machine
Apache License 2.0
630 stars 160 forks source link

Delay the next campaign if the node lost the vote #169

Open ahrtr opened 6 months ago

ahrtr commented 6 months ago

Delay the next campaign if the node lost the vote. It's highly likely it will also lose next campaign, so it makes more sense to prioritize campaigns by other nodes within the current term.

Alternative to resolve https://github.com/etcd-io/etcd/issues/17455. It has better readability than https://github.com/etcd-io/raft/pull/167.

boringhello commented 6 months ago

I think a better approach would be to verify if the rejection is due to outdated logs before delaying the next request.This approach may potentially increase the downtime of the cluster.

ahrtr commented 6 months ago

I think a better approach would be to verify if the rejection is due to outdated logs before delaying the next request.

Yes, I was thinking the same. But unfortunately, we don't have such info in the MsgVoteResp.

This approach may potentially increase the downtime of the cluster.

It should be fine. It just delays current node's next campaign. It doesn't change other nodes' campaign timings. Also the randomizedElectionTimeout will be reset to the normal value in next term.

ahrtr commented 6 months ago

@Cjen1 I think this PR can also resolve what your PR https://github.com/etcd-io/raft/pull/86 is trying to fix, and with minimum change. Please take a look and let me know if you have any comment. thx.

@erikgrinaker @pav-kv PTAL

erikgrinaker commented 6 months ago

@ahrtr Hey, sorry for the review lag here, it's been a crazy week. I'll be on PTO for the next week, but maybe @pav-kv can take a look.

pav-kv commented 6 months ago

@ahrtr Are you able to reproduce the problem in https://github.com/etcd-io/etcd/issues/17455 in a datadriven test (the tests in testdata, for TestInteraction)? What kind of race are we trying to fix?

pav-kv commented 6 months ago

This change seems to be concerned with the situation when a candidate loses because of having an outdated log. However, the fix may affect the situation of a split vote. When two candidates duel and both lose campaign because of a split vote, both will bump the election timeout, and the election may stall for a bit. It appears that there is no silver bullet against all these situations.

Enabling PreVote would address the outdated log problem more directly.

ahrtr commented 6 months ago

Enabling PreVote would address the outdated log problem more directly.

Yes. This PR is fixing the use cases where PreVote isn't enabled at all.

The idea is to skip the next electionTimeout to give other nodes a chance to win in the term+1 rather than current term?

The idea is to increase the current node's randomizedElectionTimeout when it lost the vote. Normally it's in range [electionTimeout, 2*electionTimeout). If a node lost the vote, we intentionally set the range to [2*electionTimeout, 3*electionTimeout), so that other nodes will always campaign before the node, which already lost the vote, within current term.

Let's work with an example. Assuming a cluster has 3 nodes, and one of the node (say node A) is lag behind other nodes. When they start to campaign, the node A has a possibility of 33% to start the campaign before other nodes. Its vote will definitely be rejected by other nodes. Then all the 3 nodes will conduct the next round of election fairly. The node A still has a possibility of 33% to start the campaign before other nodes. Eventually it may lead to a long time for a real leader to get elected. It's exactly this PR tries to fix.

Note that randomizedElectionTimeout will automatically be reset in next term.

When two candidates duel and both lose campaign because of a split vote, both will bump the election timeout, and the election may stall for a bit.

Yes, it's true. But I imagine that the possibility of running into such split vote should be far less than 33% mentioned above. Usually we recommend odd number of members in a cluster. But it isn't verified yet. Ideally we should a tool to measure the average recovery time, something like what https://github.com/etcd-io/raft/pull/86 does.

Are you able to reproduce the problem in https://github.com/etcd-io/etcd/issues/17455 in a datadriven test (the tests in testdata, for TestInteraction)?

The datadriven test might not be a good choice to reproduce & verify the fix. The PR doesn't change the logic, so it hasn't any impact on correctness.

ahrtr commented 6 months ago

When two candidates duel and both lose campaign because of a split vote, both will bump the election timeout, and the election may stall for a bit.

Even in split vote, we do not set all nodes' randomizedElectionTimeout to [2*electionTimeout, 3*electionTimeout). Instead we only update the randomizedElectionTimeout for the nodes which have lost the vote.

pav-kv commented 6 months ago

The datadriven test might not be a good choice to reproduce & verify the fix. The PR doesn't change the logic, so it hasn't any impact on correctness.

The datadriven test is a good way, IMO. It clearly shows the behaviour. Yes, we're not changing correctness, but we're changing the behaviour. The datadriven harness gives ways to emulate lagging and duelling campaigns, you can cause specific message delivery ordering that repro the problem. There is also a command for setting specific election timeouts which might be useful here.

To demonstrate the effect of this PR, it would be good to 1) in the first commit add a test with the election rejection scenario you're targetting, 2) fix it in the second commit - the diff in the testdata file will demonstrate the effect.

The idea is to increase the current node's randomizedElectionTimeout when it lost the vote. Normally it's in range [electionTimeout, 2electionTimeout). If a node lost the vote, we intentionally set the range to [2electionTimeout, 3*electionTimeout), so that other nodes will always campaign before the node, which already lost the vote, within current term.

I'm still not getting the problem. Say, all the nodes campaign in the [electionTimeout, 2*electionTimeout) interval: at t1, t2, t3. Yes, the first node A can lose the election at t1. If the reason for rejected votes was a short log, all the nodes will vote in this same term for another, longer log, at t2 or t3. Upon losing the election, node A already moves its election time by at least electionTimeout, so next time it will campaign t4 is > t2 and > t3.

ahrtr commented 6 months ago

If the reason for rejected votes was a short log, all the nodes will vote in this same term for another, longer log, at t2 or t3. Upon losing the election, node A already moves its election time by at least electionTimeout, so next time it will campaign t4 is > t2 and > t3.

Note this PR is fixing the cases where preVote isn't enabled at all. Each time when any node campaign, it will increase its term by 1. Other nodes will be converted to a follower when receiving the MsgVote. It means even a node lost its vote, all nodes' randomizedElectionTimeout and electionElapsed will be reset.

https://github.com/etcd-io/raft/blob/d475d7e4aa8ce5c2b2a42765e9bdeb9689966663/raft.go#L1094

https://github.com/etcd-io/raft/blob/d475d7e4aa8ce5c2b2a42765e9bdeb9689966663/raft.go#L774-L777

Cjen1 commented 6 months ago

@ahrtr Just checking my intuition here. Given there is a lagging node that calls and fails to be elected, you want to bias the retried election to the other replicas?

I think some issues arise when there is asynchrony or the election timeout is roughly the same order of magnitude as the round-trip-time. Specifically this means that before receiving the next msgVote the nodes have already sent their own (This causes duelling leaders), and hence cause each retry to be (2->3) * electionTimeout rather than (1->2) * electionTimeout).

For context I've been running some tests in cloudlab in a 3 node cluster with ~50ms latency between nodes. With 500ms electionTimeout, I'm seeing 2-5 retries before a leader is elected. (I'm currently calibrating the reckon test setup, so I am not 100% satisified with the reliability of these numbers, or that they weren't statistical flukes)

pav-kv commented 6 months ago

@ahrtr I see. You're saying that the nodes who rejected the vote, will become followers and reset the timeout. So my assumption that t4 > t2 and > t3, is incorrect? Then it's possible that the short log A campaigns first again.

I wonder though if, alternatively to the fix in this PR, we should just not reset the election timeout in the nodes who reject a vote and becomeFollower(leader=None). Then they will campaign earlier than the rejected node A tries again.

ahrtr commented 6 months ago

I think some issues arise when there is asynchrony or the election timeout is roughly the same order of magnitude as the round-trip-time.

Yes, it's true. It's a real issue even without the change in this PR. It's a misconfiguration of the election timeout. Usually we recommend the heartbeat interval to be roughly the round-trip time between members, and the election timeout to be the 10 times of heartbeat interval/timeout.

Please refer to https://etcd.io/docs/v3.5/tuning/#time-parameters

So my assumption that t4 > t2 and > t3, is incorrect? Then it's possible that the short log A campaigns first again.

Correct.

I wonder though if, alternatively to the fix in this PR, we should just not reset the election timeout in the nodes who reject a vote and becomeFollower(leader=None). Then they will campaign earlier than the rejected node A tries again.

it's exactly my original thought. Please see https://github.com/etcd-io/raft/pull/167.

ahrtr commented 6 months ago

Just checking my intuition here. Given there is a lagging node that calls and fails to be elected, you want to bias the retried election to the other replicas?

Yes. Normally the randomized election timeout is in range [electionTimeout, 2*electionTimeout). If a node lost the vote, we intentionally set the range to [2*electionTimeout, 3*electionTimeout).

But it's only within one term. All the time parameters will be reset in next term.

boringhello commented 6 months ago

Yes, I was thinking the same. But unfortunately, we don't have such info in the MsgVoteResp.

Is it possible to add whether it was rejected because of the log? It feels like there are very few things to add to MsgVoteResp.And it can stop in this raft activity, because it cannot become a leader unless it receives the appendMsg and gets the log. This may also be an optimization point. I feel that the original raft’s expression of reject is not perfect.Because rejected by high terms and logs, there is a little difference between them.

ahrtr commented 6 months ago

Added a data-driven test case. @pav-kv PTAL, suggest to review it commit by commit. @Cjen1 Please also take a look if you have time. Thanks both.

Also responding to your previous comment on split vote, actually the vote result in split vote is VotePending. This PR only updates the randomized election timeout when the vote result is VoteLost.

Is it possible to add whether it was rejected because of the log?

No such plan in this PR for now. Also note that one peer's rejection doesn't mean current node will have to necessarily lose the campaign.

boringhello commented 6 months ago

Also note that one peer's rejection doesn't mean current node will have to necessarily lose

at the majority of nodes

ahrtr commented 6 months ago

Also responding to your previous comment on split vote, actually the vote result in split vote is VotePending. This PR only updates the randomized election timeout when the vote result is VoteLost.

This isn't correct. For split vote, the vote result is VoteLost instead of VotePending. But my previous comment https://github.com/etcd-io/raft/pull/169#issuecomment-1964584497 still stands.

ahrtr commented 6 months ago

@erikgrinaker @pav-kv PTAL when you have time.

@Cjen1 do you have any immediate concern?

Thanks.

pav-kv commented 6 months ago

I'm concerned with a duelling leaders situation. Can we have a second datadriven test (similar to the one you added) to test this scenario? Something like:

  1. Both nodes 1 and 2 campaign.
  2. Both lose (because the peer has already voted for itself).
    • As a result, both bump the election time by an extra electionTimeout?
  3. So now there is a temporary election stall? If we tick both nodes electionTimeout times, nobody will campaign. Someone will only campaign if we tick another electionTimeout times.

Maybe there is a 2* multiplier in this reasoning. But the idea is: wouldn't this fix introduce an extra electionTimeout delay in this case?

Cjen1 commented 6 months ago

@erikgrinaker @pav-kv PTAL when you have time.

@Cjen1 do you have any immediate concern?

Thanks.

I am still concerned about this impacting the minimal viable recovery time, since if leaders duel it will double the length of each iteration (afaict).

I fully understand that this isn't a massive concern since the recommendation is to minimize any erroneous elections.

Is there an easy way for me to build a binary to test it in reckon? Otherwise if you swap out the etcd binary in reckon that will also work.

(Additionally I'm on holiday until Monday so have limited time to test this until then)

ahrtr commented 6 months ago

I'm concerned with a duelling leaders situation.

3. So now there is a temporary election stall? If we tick both nodes electionTimeout times, nobody will campaign.

It's true if there are only two nodes in the cluster and the two nodes are campaigning at exactly the same time.

Is there an easy way for me to build a binary to test it in reckon? Otherwise if you swap out the etcd binary in reckon that will also work.

Is it enough to provide an etcd binary on top of this PR? I am happy to help on the test.

Cjen1 commented 6 months ago

I've run this PR through reckon, using 100 leader loss events for each config, varying the number of nodes (3 and 5) and the election timeout.

fault-aggregate

I think the result is that this PR has no significant effect on the average time to recovery of etcd. There are some subtle changes in the tail for example I think there is evidence that it reduces the number of repeated elections, but also prolongs each of these rounds, but I think it would require some analytical techniques to properly pull that apart.

erikgrinaker commented 6 months ago

I share @pav-kv's concerns in https://github.com/etcd-io/raft/pull/169#issuecomment-1983994248, and also the concerns voiced in #167. Spurious election disruption is a known weakness when not using pre-vote and checkquorum, and it sounds like enabling those would adequately address https://github.com/etcd-io/etcd/issues/17455. I think we should recommend employing those as a first measure, and only tweak election timeout policies when we see significant problems with those enabled.

ahrtr commented 6 months ago

I think the result is that this PR has no significant effect on the average time to recovery of etcd. There are some subtle changes in the tail for example I think there is evidence that it reduces the number of repeated elections, but also prolongs each of these rounds, but I think it would require some analytical techniques to properly pull that apart.

Thanks for the test result. My dev branch is based on etcd release-3.5, while your baseline is based on release-3.4. It might be another reason for the subtle change.

Spurious election disruption is a known weakness when not using pre-vote and checkquorum, and it sounds like enabling those would adequately address etcd-io/etcd#17455.

The PR is trying to mitigate such kind of known weakness. Not all applications enable pre-vote. Note etcd release-3.4 is a stable release, it has never enabled pre-vote.

As mentioned above https://github.com/etcd-io/raft/pull/169#issuecomment-1964573546, a lag node always has 33% possibility (in a 3-node cluster) to start the campaign before other nodes without this PR. The possibility is 50% if there are only two available nodes.

But the possibility of split vote in a 3 node cluster is below based on raft paper 9.2:

Obviously 33% vs 3-10%, or 50% vs 18-35%, this PR improves the situation.