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

Raft pre-vote extension implementation #530

Open
wants to merge 89 commits into
base: main
Choose a base branch
from
Open

Conversation

dhiaayachi
Copy link
Contributor

@dhiaayachi dhiaayachi commented Nov 9, 2022

This is an idea to implement pre-vote without breaking backward compatibility I'm exploring.

Pre-vote, based on the original raft thesis, is an extension to raft that help reduce term inflation and cluster instability when a node is partitioned from the cluster for a while and then is back online.

In normal election when a node have no leader, it transition into candidate state. In that state the node will try to run an election by incrementing its term and sending a vote-request to the other nodes in the cluster.

If the node win the election, the node become a leader and the cluster is stable again. But in the event that this node don't win the election, let's say because it's not connected to a quorum of nodes in the cluster, it will retry by incrementing it's term again and running another round of election. That could happen multiple times, which create a term inflation.

Once the node is connected back to quorum of nodes it's almost certain that its term will be higher than the other nodes in the cluster, as it was running rounds of elections non stop, while most likely the remaining of the cluster was stable. This will lead to the node making the current raft leader to step-down and another round of election is run. That's not a desired behaviour because the cluster was already stable and that node most likely don't have the latest log, so it won't be able to win the election anyway.

In this PR a new API call, RequestPrevote, is introduced. This API call is performed by a candidate node to ask another node to grant a pre-vote for a specific term by sending a RequestPrevoteRequest. The receiving node will evaluate if it will grant the pre-vote based on the same set of rules used to grant a normal vote, except that it will not alter its internal state (store the vote, increment internal term) in all cases, especially if the pre-vote is granted.

The candidate node will count the pre-vote and if it have quorum, will proceed to a normal election by using RequestVote as usual.

If the candidate node discover a higher term while it is performing pre-vote, it will transition to follower and join the cluster without disturbing the current leader.

If the candidate node is denied a pre-vote by a majority of nodes, it will wait for election timeout and restart the process, this will give the chance to the cluster (current leader) to send newer data/heartbeat to the candidate node, which will make it transition to follower, without impacting the cluster stability.

To preserve retro-compatibility, this PR treat rpcUnexpectedCommandError as a "special" error, when this error is returned as a result of RequestPrevote the candidate learn that the node returning this error don't have pre-vote capability and the prevote is considered as granted, so the candidate can transition to a real vote to avoid a situation where a cluster with mixed (pre-vote capable and incapable) nodes is stuck in a pre-election loop.

The first iteration of this PR considered implementing the pre-vote extension as an extension of the RequestVote API call, we decided to not keep this implementation because in a mixed cluster situation (pre-vote capable and incapable), the pre-vote would alter the state of the pre-vote incapable nodes. This was deemed to be too complex to reason about. As the pre-vote capable node will need to make assumption based on the response and transition to leader in some situation. For perennity, the details of this implementation is below:

In this PR, a new bool called preVote is introduced to both the RequestVoteRequest and RequestVoteResponse. When a node transition to a candidate state and pre-vote is activated in the config, it will run a round of pre-election by sending RequestVoteRequest with preVote set to true and a term equal its current term + 1 but without incrementing its current term.

If the node that receive the request support pre-vote and have pre-vote activated, it will respond by sending a RequestVoteResponse with preVote set to true and grant its vote as if it's a normal election, but without incrementing its vote.

Otherwise the node will respond by sending a RequestVoteResponse with preVote set to false and grant its vote as it does for normal election, including incrementing its term.

The candidate node will count all the votes with preVote set to true, grantedPrevotes and all the votes with preVote set to false, grantedVotes.

If grantedPrevotes is bigger than the needed votes the candidate will run a normal election.

If grantedVotes is bigger than the needed votes the candidate will consider it self winning the election and transition to leader.

@dhiaayachi dhiaayachi marked this pull request as draft November 9, 2022 21:35
@banks
Copy link
Member

banks commented Nov 10, 2022

Nice! So glad you are looking into this Dhia.

Not looked at the code just yet but first thinking through the algorithm you described.

At first read, it seems problematic to me that old nodes that don't know about preVote could treat this as an actual election and update their persistent state to a term that no correct candidate has actually proposed yet. Concretely, what if the partitioned candidate crashes after it sends out it's preVote but before it gets a quorum of responses? Now one or more (maybe a quorum) of old nodes have "voted" for a candidate in their persistent state who will restart without knowing they ever entered that term that they've been voted for in. On the face of it I don't see an immediate way this leads to incorrectness, but it's certainly hard to reason about and breaks a core assumption of the correctness proofs of Raft's leader election since a node might win an election in a term which it doesn't know it's participated in!

If it is incorrect, I guess it would be due to potentially duelling candidates both holding elections at the same time. I thought and didn't come up with a specific scenario where a bad thing happened... but I did come across enough complicated details that make me 😬 . Working out which cases we must persist the CurrentTerm and which we don't becomes much more subtle - right now we assume any message or nack with a larger term means we should perist that term and never participate in a lower numbered election again... but that's not true for preVotes... but it is true for requests that were sent as a preVote but interpretted by an old server as an actual vote!

All that said I think it could be correct provided we are especially careful about how we manage the self-voting part. Today when we start the election in electSelf we increment term and then we deliver a self-vote as if it was coming from a peer to the same chan and we persist the lastVotedTerm.

For a preVote, we'd need to not persist the CurrentTerm increase, not the LastVotedTerm but send ourselves a RequestVoteResponse with preVote=true. OK.

If grantedVotes is bigger than the needed votes the candidate will consider it self winning the election and transition to leader.

This is the subtle bit: it's technically safe to do so if you receive a quorum - 1 of these because we can just change our vote to us (provided we didn't see a higher term in the mean time). But you have to be careful to persist our own LastVotedTerm before we transition to leader. If we play it safe and only transition if a quorum not including ourselves granted the vote for real, we should probably still be careful about persisting our vote term because so far it's an invariant that if you won an election you updated your LastVotedTerm in the process and so would be confusing to break that.

The bit that makes me most unsure is that trying to reason through all the possibilities is pretty hard! If preVote is always completely separate from real elections I think it's significantly easier to be sure we've not compromised the correctness of the election algorithm. Imagine for example reasoning through all possibilities with 5 servers going through a rolling upgrade. You even have to account for things like one server participating in a preVote as an old server and then being upgraded before the vote concludes so now being a new server who understands preVoting.

Would it be simpler (possible?) for example to take a different approach where we add preVote as a totally new RPC but make it optional. Our network transport will return an error response if we call an unknown RPC so we could send out the preVotes and then wait for a quorum of responses. We can count the errors and if we either get a respone back from everyone or timeout without a quorum of good responses, we can just abandon the preVote and switch to an actual vote.

This wouldn't need an explicit protocol migration I don't think - preVote would silently degrade and then start working as soon as a quorum of new servers is available, but we wouldn't have to reason about the correctness of preVotes and real votes being handled differently by different peers (or even the same peer at different times!).

If that doesn't work for some reason, I think your proposal is possible, it just seems like it will be harder work to convince ourselves it's correct in all possible cases!

Copy link
Member

@banks banks left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A whole bunch of thoughts for now.

There is a lot here - mostly because I'm trying to reason about it for myself rather than because this is necessarily a bad approach. I could also be wrong in any of my thought processes so have a think and we can maybe discuss some more which overall approach we think will be simplest to reason about and implement correctly!

raft.go Outdated

r.setCurrentTerm(term)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is the part that I think we should probably also persistVote for ourselves even though in this code you didn't count our own preVote in the quorum. I don't think this is necessarily incorrect as it is, but I wonder if we'll violate assumptions later that we are leader in a term but we never voted in that term? I think it's safe in all cases to decide to "upgrade" our own preVote to a real vote here anyway and avoids ever needing to think about that case.

Technically if we keep this approach we could get to this point even faster by checking for grantedVotes >= (votesNeeded-1) since we can always upgrade our own vote to make up the quorum if enough other peers voted for real.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually I came up with a scenario where the cluster could get stuck with one node running PreVote campaigns that never get enough support to complete but prevent other nodes getting elected despite a quorum of healthy nodes with this logic. If we change it to only need quorum - 1 and upgrade our own vote then those cases can't happen...

raft.go Outdated
if preVoteGrantedVotes >= votesNeeded {
r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes)
preVoteGrantedVotes = 0
grantedVotes = 0
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need to reset this? If some of the peers already voted "for real" then we already have their vote as far as they are concerned. If we really wanted to, we could modify electSelf to skip sending a requests to any nodes that already granted a "real" vote! (Not saying we should though, just an observation). As it is this would trigger our "info" log about duplicate vote requests on those old nodes, but that's not a huge deal.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we really wanted to, we could modify electSelf to skip sending a requests to any nodes that already granted a "real" vote! (Not saying we should though, just an observation).

I think it's safer if we don't, at least from code complexity perspective. Also I think we should recommend to only have mixed clusters as a transient state, so I don't think the logs are a big issue in that case.

Also about not resetting this, if we don't implement skipping the nodes that already voted, it's incorrect as there is a possibility we have the same node vote double.

raft.go Outdated
Comment on lines 330 to 336
if vote.Term > term && vote.PreVote {
r.logger.Debug("newer term discovered on pre-vote, fallback to follower", "term", vote.Term)
r.setState(Follower)
r.setCurrentTerm(vote.Term)
return
}

Copy link
Member

@banks banks Nov 10, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need different logic for PreVote as well as this?

For example if the peer we sent it to denied us because there is a stable leader, we should treat that the same way and switch to Follower to try and wait for the leader to catch us up rather than just running pre vote over and over.

Actually I feel like that's not special to PreVote 🤔. Maybe we should do that for real elections too when we are rejected without a higher election term being present 🤔 .

Ideally, as a candidate, especially one having prevotes rejected, we need a path to transition back to follower so we can continue without disrupting the cluster. The only way out of candidate state right now I think is to either win an election in a new term ourselves, or observe another election happeneing - either way we can't become a healthy follower for the stable cluster leader without the cluster being disrupted by an election unless there is some other path to follower that doesn't involve observing a term higher that our prevote one!

It doesn't quite make PreVote useless if we don't because it at least stops the partitioned server from disrupting the cluster when it comes back, but it does leave it stuck PreVoting forever until the next real election completes unless there is a path back to Follower that doesn't require a higher term to be observed!

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think this is needed in the election code, if we are part of the cluster already. The leader will eventually send an AppendEntry that will transition us to followers. Doing so in here have the potential, for example, of making a node not part of the cluster think it's a follower.

raft.go Outdated
r.logger.Info("pre election won", "term", vote.Term, "tally", preVoteGrantedVotes)
preVoteGrantedVotes = 0
grantedVotes = 0
voteCh = r.electSelf(false)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we also need to reset the electionTimer? I'm not sure. If we don't it means that we have to get PreVote and Vote rounds within one election timeout. If we are close to timing out when we get PreVotes through then it seems wasteful to start a real election that's really likely to fail due to timing out before responses are gathered and then restart prevoting all over again!

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes that's probably needed and explain some of the timing issue I was having in tests.

raft.go Outdated
Comment on lines 287 to 293
var voteCh <-chan *voteResult
var prevoteCh <-chan *voteResult
if r.preVote {
prevoteCh = r.electSelf(true)
} else {
voteCh = r.electSelf(false)
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

At first I felt like we need to somehow keep the state around so that if we timeout a PreVote campaign we fall back to a real one next time... but then I realise that would defeat the purpose of a prevote because a partitioned server would only prevote for one round before it resumed incrementing it's term...

I do wonder a bit about whether it's possible to get "stuck" never quite winning a pre vote in time but also not making progress.

Here's the scenario I came up with.

  • Three servers A, B, C
  • Only A is upgraded to have PreVote
  • B is currently leader
  • C is down for a bit
  • A disconnects temporarily
  • B looses leadership and becomes a follower waiting for an election (or to start one)
  • A starts election first and use PreVote
  • B treats it like RealVote and so grants it an resets its contact time (which prevents it from timing out and becoming a candidate
  • A sees the request granted by B but it's not enough for a quorum (using the code here) so keeps waiting
  • A's prevote campaign times out and it starts again... electionTimeout is always less than or equal to heartbeat timeout so this could continue forever even though there are a quorum of healthy servers.

Note that this scenario is "fixed" if we use the suggestion I made below about only requiring quorum - 1 real vote grants and then upgrading our own. But it's an example of something non-obvious that could cause the cluster to get stuck in a whole new way that isn't possible today which makes me 🤔.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You just solved a bug I was chasing with one of the tests, when I set the number of nodes to 2 prevotes,1 no prevote 😄

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

While going through this use case to fix it I think it's part of a more general use case. Where grantedVotes + grantedPreVotes >= quorum but grantedVotes < quorum and grantedPreVotes < quorum.
I think the best way to solve this would be to transition to election as soon as grantedVotes + grantedPreVotes >= quorum.

@dhiaayachi
Copy link
Contributor Author

I think that the different RPC idea is simpler, the main reason I steered away from it for the following reasons:

  • pre-vote need to always stay aligned with real vote (except for state persistence) for it to work but I guess we can make sure that underneath both RPCs call the same code.
  • from optimization perspective the current solution will stabilize in 1 round in mixed server scenario while the different RPC will need 2 rounds

That said, it's easier to reason about special cases like the bug you just found out because we avoid changing the state of the no pre-vote nodes while doing pre-vote.

@dhiaayachi
Copy link
Contributor Author

Thinking more about implementing this using a different RPC command, there is few use cases here, but the most interesting use case is when some nodes support prevote, some don't and a minority of nodes is down.

In that case if neither the nodes which support prevote or the nodes which don't can form a quorum we will be stuck in pre-election forever. One way of solving this would be to say if we get enough answers (error + granted-prevote) to form a quorum we transition to full election.

The issue with that solution is that it create another gap where, we have this condition true but without receiving all the nodes answer which can make the transition to full election premature.

I don't think this is a deal breaker as with the solution in here we will be able to progress even if the pre-vote is not optimal, but it show that the different RPC alternative will have a similar complexity.

rename test and pin submodule to version 1.5.0

rename test file

leaderOld := c.Leader()
c.Followers()
c.Partition([]ServerAddress{leaderOld.localAddr})
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@mkeeler notice that we partition the leader here to force leader election and then expect after that a new leader got elected.

@dhiaayachi
Copy link
Contributor Author

👋 @mkeeler @banks
after a discussion I had with @banks yesterday I changed the implementation to use a different RPC call for pre-vote and rely on this call to error when a node is running an older version (which don't know about pre-vote). The reasoning behind that, is that using a different RPC would ensure that an old node state is not altered when receiving a pre-vote, which could present a whole class of risks and deviation from the raft thesis.I also added/modified tests to account for @mkeeler comments.

Sorry for the messed up github PR, I rebased on main and github got confused about it 😞

@dhiaayachi dhiaayachi requested a review from mkeeler March 26, 2024 13:04
@dhiaayachi dhiaayachi self-assigned this Mar 26, 2024
commands.go Outdated
// Used to indicate to peers if this vote was triggered by a leadership
// transfer. It is required for leadership transfer to work, because servers
// wouldn't vote otherwise if they are aware of an existing leader.
LeadershipTransfer bool
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do leadership transfers utilize pre-vote? I think they should probably avoid pre-voting.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes it does. I traced leadership transfer and it work as follow:

  • When a node receive a leadership transfer command, it sends a TimeoutNow RPC to the chosen new leader and step down.
  • Upon receiving the TimeoutNow RPC, the chosen leader will set it's state to leadershipTransfer and transition to Candidate

From this point normal leadership election will happen (including pre-vote) the only exception is that we send the LeadershipTransfer as part of the vote (and pre-vote) to avoid not granting the vote/pre-vote when the voting node think it still have a leader.

I'm mitigated about if we should keep it as is or completely remove pre-vote from the equation when performing leadership transfer I think it won't change anything about the behaviour in leadership transfer as in all cases we will end up in leader-election.

I would be interested in hearing your thoughts and @banks about this.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

🤔 My initial thought here is that we should probably skip pre-vote for a leadership transfer to keep it easier to reason about:

  • Leaders only try to transfer to nodes that are healthy and connected anyway (should check that) so by definition it won't have an elevated term
  • Leadershp transfer is like an explicit permission to hold an election so it seems strange that we'd then spend time issuing prevote requests to see if we should hold an election
  • Other than maybe taking a bit longer, the extra redundant round of messages adds complication which makes it more likely that transfer doesn't succeed (there are already many complex ways that transfer might not go to plan and result in actual timeouts and disruptive elections, it's not immune to that because it can't be, but it's an optimization to reduce the time it takes to get a new healthy leader in the common case. Adding more steps to it seems to remove some optimization and increase the chance it won't succeed in my mind.

I've not looked to see how hard that is in the code though. But I'd be tempted rather than adding this flag here to instead just avoid sending prevote round at all when responding to a TimeoutNow.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Leaders only try to transfer to nodes that are healthy and connected anyway (should check that) so by definition it won't have an elevated term

This is not necessarily true as we allow providing the server we would like to transfer leadership to and it's a best effort transfer, so if the server we specify is not able to become a leader the transfer won't happen.

That said, I agree on the fact that leadership transfer is probably not something that we will benefit from optimizing the election for and removing it from the pre-vote flow remove some of the complexity.

integ_test.go Outdated
@@ -73,7 +73,8 @@ func (r *RaftEnv) Restart(t *testing.T) {
func MakeRaft(tb testing.TB, conf *Config, bootstrap bool) *RaftEnv {
// Set the config
if conf == nil {
conf = inmemConfig(tb)
t := tb.(*testing.T)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Would it be better to modify the inmemConfig method to consume a testing.TB instead of a *testing.T?

raft-compat/prevote_test.go Show resolved Hide resolved
a, _ := getLeader.GetRaft().(*raftprevious.Raft).LeaderWithID()
require.Equal(t, a, leader)

//
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks like maybe after rebasing and such this stray comment lives on.

a, _ := getLeader.GetRaft().(*raft.Raft).LeaderWithID()
require.Equal(t, a, leader)

//
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Stray/empty comment

raft.go Outdated
// This could happen when a node, previously voter, is converted to non-voter
// The reason we need to step in is to permit to the cluster to make progress in such a scenario
// More details about that in https://github.com/hashicorp/raft/pull/526
if len(req.ID) > 0 {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This check like the one further up should be unnecessary.

raft.go Outdated
candidateID := ServerID(req.ID)
if len(r.configurations.latest.Servers) > 0 && !hasVote(r.configurations.latest, candidateID) {
r.logger.Warn("rejecting pre-vote request since node is not a voter", "from", candidate)
return
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we could still end up accidentally granting the vote here.

Line 1793 will set Granted in the response to true whenever the pre-vote request has a newer term. Here when we detect that the requester is a non-voter we are returning but are not modifying the response.

I think this function would be easier to follower and clearer if the logic to decide whether a vote is granted is moved into a separate method that could have a simple bool return. Then this function could translate that bool into the response without having to use the deferred func to send it.

Alternatively, we could just not set Granted on line 1793 however I think the code would be improved by breaking out the logic of deciding to grant the vote from the rest which will craft the response and send it back.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree on the fact that it would be easier to follow if refactored, but I think it's better if we do this in a second phase and refactor both vote and prevote. I would like at least for the time being to keep pre-vote similar to vote.

raft.go Outdated
Comment on lines 1809 to 1825
// Reject if their term is older
lastIdx, lastTerm := r.getLastEntry()
if lastTerm > req.LastLogTerm {
r.logger.Warn("rejecting vote request since our last term is greater",
"candidate", candidate,
"last-term", lastTerm,
"last-candidate-term", req.LastLogTerm)
return
}

if lastTerm == req.LastLogTerm && lastIdx > req.LastLogIndex {
r.logger.Warn("rejecting vote request since our last index is greater",
"candidate", candidate,
"last-index", lastIdx,
"last-candidate-index", req.LastLogIndex)
return
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Similar to the previous comment, these returns will inadvertently send a response that grants the vote instead of rejecting it.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed this by avoiding to set granted to true above and keep processing further.

raft_test.go Outdated
c.FullyConnect()
time.Sleep(3 * c.propagateTimeout)

// Check that the number of followers increase and
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the rest of this comment would be useful to readers even though I think I can piece together what you would want to convey.

testing.go Outdated
@@ -21,13 +21,13 @@ import (
var userSnapshotErrorsOnNoData = true

// Return configurations optimized for in-memory
func inmemConfig(tb testing.TB) *Config {
func inmemConfig(t *testing.T) *Config {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Any particular reason for this change?

@mkeeler
Copy link
Member

mkeeler commented Mar 26, 2024

I accidentally hit the submit button on that review too soon. What I wanted to convey was that the main request blocking approval is the bug in processing the request pre-vote RPC that will inadvertently grant votes when it shouldn't. Most everything else is just stylistic or code simplification related.

@dhiaayachi dhiaayachi marked this pull request as ready for review March 26, 2024 15:56
@dhiaayachi dhiaayachi requested a review from a team as a code owner March 26, 2024 15:56
@dhiaayachi dhiaayachi requested a review from mkeeler March 26, 2024 16:42
Copy link
Member

@banks banks left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Amazing Dhia.

I'm much happier with this approach overall and the tests with mixed versions and upgrades are great. I have a bunch of things inline which are mostly minor but probably worthwhile.

The biggest thing I think is the leadership transfer which seems to me like it really shouldn't ever involve a pre-vote but maybe I'm not thinking about it right!

commands.go Outdated
// Used to indicate to peers if this vote was triggered by a leadership
// transfer. It is required for leadership transfer to work, because servers
// wouldn't vote otherwise if they are aware of an existing leader.
LeadershipTransfer bool
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

🤔 My initial thought here is that we should probably skip pre-vote for a leadership transfer to keep it easier to reason about:

  • Leaders only try to transfer to nodes that are healthy and connected anyway (should check that) so by definition it won't have an elevated term
  • Leadershp transfer is like an explicit permission to hold an election so it seems strange that we'd then spend time issuing prevote requests to see if we should hold an election
  • Other than maybe taking a bit longer, the extra redundant round of messages adds complication which makes it more likely that transfer doesn't succeed (there are already many complex ways that transfer might not go to plan and result in actual timeouts and disruptive elections, it's not immune to that because it can't be, but it's an optimization to reduce the time it takes to get a new healthy leader in the common case. Adding more steps to it seems to remove some optimization and increase the chance it won't succeed in my mind.

I've not looked to see how hard that is in the code though. But I'd be tempted rather than adding this flag here to instead just avoid sending prevote round at all when responding to a TimeoutNow.

commands.go Outdated
Comment on lines 156 to 159
// Peers is deprecated, but required by servers that only understand
// protocol version 0. This is not populated in protocol version 2
// and later.
Peers []byte
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'd maybe lean towards taking this out. I can understand that it might be simpler initially to have identical code in Vote and PreVote so I might be persuaded, but it seems odd that we'd add in legacy stuff that will never be used and will need to be cleaned up again in the future.

config.go Outdated
Comment on lines 237 to 239

// PreVote activate the pre-vote feature
PreVote bool
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thinking aloud: I wonder if it would be useful to make this Reloadable too 🤔 . It would be easy enough with an atomic flag I think, just thinking through that might make it possible say for Autopilot or something else to flip it on automatically once we've verified all voters are at the right version? I know this PR should mean that's not necessary but just thinking through why we'd ever disable it if it wasn't for compatibility and would we then want to be able to flip that at runtime?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh and a minor thing, but if we keep this, In my mind it should probably go above the private skipStartup since that's an internal-only detail and this is a public config param?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, I also wonder if this should be opt-out rather than opt-in? It's a pretty major defect of this implementation that it can be disrupted by network partitions so provided we're confident it will be correct through upgrades (which I think we need to be to merge it at all) then it would be better if everyone gets it by default. We should probably cut a SemVer minor release and note the change in behavior and that you can opt out.

Also see my comment on api.go about auto-disabling it if the transport doesn't actually support it.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree about making this an opt-out flag, I'm not sure about how useful will it be to be a dynamic flag. Especially considering the impact related to toggling it in the middle of operations or while the cluster is unstable 🤔

config.go Outdated
@@ -317,7 +320,7 @@ func DefaultConfig() *Config {
SnapshotInterval: 120 * time.Second,
SnapshotThreshold: 8192,
LeaderLeaseTimeout: 500 * time.Millisecond,
LogLevel: "DEBUG",
LogLevel: "TRACE",
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Did you mean to commit this? I'm not sure that changing default log level to TRACE makes sense for most users?

raft_test.go Show resolved Hide resolved
raft.go Outdated
r.logger.Debug("prevote granted", "from", preVote.voterID, "term", preVote.Term, "tally", preVoteGrantedVotes)
} else {
preVoteRefusedVotes++
r.logger.Debug("prevote refused", "from", preVote.voterID, "term", preVote.Term, "tally", preVoteGrantedVotes)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
r.logger.Debug("prevote refused", "from", preVote.voterID, "term", preVote.Term, "tally", preVoteGrantedVotes)
r.logger.Debug("pre-vote denied", "from", preVote.voterID, "term", preVote.Term, "tally", preVoteGrantedVotes)

raft.go Outdated
// Check if the preVote is granted
if preVote.Granted {
preVoteGrantedVotes++
r.logger.Debug("prevote granted", "from", preVote.voterID, "term", preVote.Term, "tally", preVoteGrantedVotes)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
r.logger.Debug("prevote granted", "from", preVote.voterID, "term", preVote.Term, "tally", preVoteGrantedVotes)
r.logger.Debug("pre-vote granted", "from", preVote.voterID, "term", preVote.Term, "tally", preVoteGrantedVotes)

raft.go Outdated

// Check if we've won the pre-vote and proceed to election if so
if preVoteGrantedVotes >= votesNeeded {
r.logger.Info("pre election won", "term", preVote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
r.logger.Info("pre election won", "term", preVote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1)
r.logger.Info("pre-vote successful, starting election", "term", preVote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1)

raft.go Outdated
// Check if we've lost the pre-vote and wait for the election to timeout so we can do another time of
// prevote.
if preVoteRefusedVotes >= votesNeeded {
r.logger.Info("pre election lost, wait for election to timeout", "term", preVote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
r.logger.Info("pre election lost, wait for election to timeout", "term", preVote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1)
r.logger.Info("pre-vote campaign failed, waiting for election timeout", "term", preVote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1)

api.go Outdated
@@ -560,6 +563,7 @@ func NewRaft(conf *Config, fsm FSM, logs LogStore, stable StableStore, snaps Sna
leaderNotifyCh: make(chan struct{}, 1),
followerNotifyCh: make(chan struct{}, 1),
mainThreadSaturation: newSaturationMetric([]string{"raft", "thread", "main", "saturation"}, 1*time.Second),
preVote: conf.PreVote,
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we should check that transport actually supports prevote here and auto-disable it if not.

For example if a product like Consul has a custom transport that wraps NetTransport and they don't realise they need to implement the optional interface, then when they upgrade they will end up silently trying to prevote but not actually sending messages and then timing out pointlessly.

@dhiaayachi
Copy link
Contributor Author

@banks @mkeeler I updated based on all the review, I think this is getting close now 😅

@dhiaayachi dhiaayachi requested a review from banks March 28, 2024 19:51
Copy link
Member

@mkeeler mkeeler left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Two last minor requests to remove the deprecated Candidate field from the RequestPreVoteRequest

commands.go Outdated
Comment on lines 130 to 132

// Deprecated: use RPCHeader.Addr instead
Candidate []byte
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
// Deprecated: use RPCHeader.Addr instead
Candidate []byte

Could we remove this field. Seeing as its been deprecated for the regular RequestVoteRequest it seems reasonable for pre-vote to just never support this way of specifying the candidate server. Also in processing pre vote requests we never use this field.

raft.go Outdated
Comment on lines 2064 to 2065
// this is needed for retro compatibility, before RPCHeader.Addr was added
Candidate: r.trans.EncodePeer(r.localID, r.localAddr),
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
// this is needed for retro compatibility, before RPCHeader.Addr was added
Candidate: r.trans.EncodePeer(r.localID, r.localAddr),

This change goes in tandem with my other proposal to remove the Candidate field from the struct definition.

@dhiaayachi dhiaayachi requested a review from mkeeler April 2, 2024 15:28
Copy link
Member

@banks banks left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Amazing @dhiaayachi. 🎉

One minor thing. The PR description still says things like:

In this PR, a new bool called preVote is introduced to both the RequestVoteRequest and RequestVoteResponse. When a node transition to a candidate state and pre-vote is activated in the config, it will run a round of pre-election by sending RequestVoteRequest with preVote set to true and a term equal its current term + 1 but without incrementing its current term.

Which was the original implementation we moved away from. Since this is a significant change, what do you think about editing the PR description with the same great detail but about the actual implementation that merges (and why we took this path vs the original one)? We don't have an RFC for this but the detail we've thought through in the PR and offline would be great to capture somewhere coherent for future reference and I think this PR is the most natural home for that?

Overall I think this is ready to ship. There are a couple of minor suggestions inline but all are non-blocking potential code clarity improvements that I'll leave you to choose if they are worthwhile!

api.go Show resolved Hide resolved

// Check if we've won the pre-vote and proceed to election if so
if preVoteGrantedVotes >= votesNeeded {
r.logger.Info("pre-vote successful, starting election", "term", preVote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

🤔 I'm trying to work out why we -1 the votesNeeded here. I think it's because we auto-vote for ourselves in votesNeeded and so it might be confusing to operators whether votesNeeded includes that self-vote or not.

I.e. in a cluster of 3 votesNeeded would actually be 2, but we effectively only need one vote from another server in addition to our own so you've chosen to show it as 1 instead here.

I initially thought this was potentially more surprising because for those who know about consensus, they'd maybe expect this number to match the quorum number of the cluster. I changed my mind when I considered the UX and assumed we'd only see "vote granted" logs for the remote peers so having this number correlate to that might be more intuitive. But then I went and looked at preElectSelf again and we do Debug log "voting for self".

I don't think ultimately it's a big deal but on balance it seems slightly less confusing to not subtract 1 here both to match any intuitions about quorum size, and to remain consistent with debug log output where we make explicit that we vote for ourselves too?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

At this level our own vote is completely transparent to us. It come through the preVoteCh like any other vote. see:

raft/raft.go

Line 2105 in 1912201

respCh <- &preVoteResult{

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

huh... then that makes me wonder even more why we have votesNeeded-1 in the log here?

raft.go Outdated
Comment on lines 1750 to 1751
// check the LeadershipTransfer flag is set. Usually votes are rejected if
// there is a known leader. But if the leader initiated a leadership transfer,
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the comments about LeadershipTransfer are out of sync with the code now right?

raft.go Outdated
// Construct a function to ask for a vote
askPeer := func(peer Server) {
r.goFunc(func() {
defer metrics.MeasureSince([]string{"raft", "candidate", "electSelf"}, time.Now())
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we make this a separate metric preElectSelf?

raft.go Outdated
r.goFunc(func() {
defer metrics.MeasureSince([]string{"raft", "candidate", "electSelf"}, time.Now())
resp := &preVoteResult{voterID: peer.ID}
if prevoteTrans, ok := r.trans.(WithPreVote); ok {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: would it be cleaner to do this type assertion right at the top of preElectSelf and bail out immediately if the transport doesn't support it (which shouldn't be possible due to the check in NewRaft but defensive against future change...).

Main rationale is that it allows un-indenting and simplifying the body of this callback which makes it easier to follow the important parts!

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I changed that and added a defensive panic. Please check if this is what you have in mind.

@dhiaayachi
Copy link
Contributor Author

Thank you @banks!
I updated based on your comments, I also rewrote the PR description based on the current implementation. Can you please give it another 👀 ?

@dhiaayachi dhiaayachi requested a review from banks April 8, 2024 14:29
Copy link
Member

@banks banks left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks so awesome Dhia.

Some super minor things I spotted inline but see what you think. I don't think they are blocking.

We did discuss offline verifying this change end-to-end in Consul and Vault before we actually merge out of an abundance of caution. Lets report back here when that testing is done!


// Check if we've won the pre-vote and proceed to election if so
if preVoteGrantedVotes >= votesNeeded {
r.logger.Info("pre-vote successful, starting election", "term", preVote.Term, "tally", preVoteGrantedVotes, "votesNeeded", votesNeeded-1)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

huh... then that makes me wonder even more why we have votesNeeded-1 in the log here?

}

resp.Granted = true
r.setLastContact()
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmmm I didn't spot this before... but should we be setting this? I think lastContact is supposed to indicate when we last heard from the leader of the cluster. (Yep, I checked the doc comments for it) In a real vote, it seems reasonable to expect that the server we just gave our vote to is about to become the leader. In this case, that's not quite as clear (thought still likely I suppose).

Did you consider whether it's more like to be better or worse to set this? I think I'd lean towards not calling this because it changes state that at least some other part of raft relies on and so makes it just a tiny bit more risky than a pre-vote campaign could cause new behaviour in the cluster. I think semantically, the node that spoke to us is not the leader right now either so it seems like we shouldn't treat it like it's already won an election. I guess that logic also applies to a real election though.

I don't think it is likely to be a problem either way. Just curious to think it through and see if we can imagine any impact or increased risk of impact either way. 🤔

OK so it is used in a follower when heartbeat timeout is called when in follower state:

raft/raft.go

Line 219 in cc2bb08

if time.Since(lastContact) < hbTimeout {

I think if we get here (granting the pre vote) then this node must already not have a leader (i.e. be in candidate state)... So I don't think it really is a problem, but I'm still not sure it makes sense to pretend we "heard from a leader" based on a prevote?

// Create a response channel
respCh := make(chan *preVoteResult, len(r.configurations.latest.Servers))

// Increment the term
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
// Increment the term
// Propose the next term without actually changing our state

// If the target server do not support Pre-vote RPC we count this as a granted vote to allow
// the cluster to progress.
if err != nil && strings.Contains(err.Error(), rpcUnexpectedCommandError) {
r.logger.Error("target do not support pre-vote RPC",
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
r.logger.Error("target do not support pre-vote RPC",
r.logger.Error("target does not support pre-vote RPC, treating as granted",

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

3 participants