-
Notifications
You must be signed in to change notification settings - Fork 1k
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
Leadership transfer #306
Leadership transfer #306
Conversation
Exciting! I know this is a WIP so you'd probably get to it later, but as a TODO I'd add clear documentation on all the exported methods at least, if not the internal ones too. 😄 |
@mitchellh Absolutely, I added the todos! |
@tylertreat FYI. I know you were interested in this! |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This looks awesome Hans,
I think we have a few correctness/timing/race issues to look at as noted in the comments but I think the tweaks to fix those aren't huge - just subtle!
The "how long do we wait for replication" one is the most subtle I think.
raft.go
Outdated
}() | ||
|
||
// Step 4: send TimeoutNow message to target server. Technically the | ||
// leadership transfer is done now from the point of view of the leader. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think this comment is a bit confusing given that the goroutine above is executing in parallel - the leader has done what it can, but it still needs to take responsibility if the transfer times out. Maybe just "transfer is done" is confusing because it's not really done until the election happens and new leader takes over...
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Paul covered most of it. Just adding my 2¢ in a few places.
I addressed the fact, that the replication needs to be included in the timeout as well. I think it is much better now, because not only timeouts, but also loss of leadership can stop the transfer at any point in time: Lines 540 to 558 in ded6ce1
|
I also defer resetting the flag for leadershiptransfer in progress now: Lines 735 to 736 in ded6ce1
|
ok, I am at a point where I addressed the feedback except these two points:
|
In a call with @banks we decided, that we don't cut a new version. I will make it clear in the docs that you have to run the latest code in order to use leadership transfer - that v3 is not sufficient. With regards to blocking actions coming through |
If a leader attempts to transfer leadership to a follower that doesn't have that feature, it will fail gracefully on the follower and the leader. In that case the old leader keeps being the leader:
and on the leader as well:
If a node is asked to vote for another node that is the target of a leadership transfer, it will just vote for it as usual. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks good to me now.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Phew! I think this looks good. There are a couple of minor things mentioned inline but not important enough to need another round of reviews!
I should also ask - the failing CI tests, are those same tests failing in master and/or passing locally? Sad as it is, I know the test suite for this is not very robust in CI so we probably can't hold up the PR waiting for that to happen but would be good to just confirm that you verified no failures could be relevant to these changes - even if they are intermittent.
raft.go
Outdated
replState map[ServerID]*followerReplication | ||
notify map[*verifyFuture]struct{} | ||
stepDown chan struct{} | ||
lease <-chan time.Time |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we need to move this any more? I think you did it because originally you were waiting on it in the transfer until I pointed out that wasn't necessary.
I don't think it matters much so happy to leave it if that's easier but not sure it's necessary to pull it out of the leader loop method any more?
raft.go
Outdated
@@ -341,6 +356,8 @@ func (r *Raft) runLeader() { | |||
r.leaderState.replState = make(map[ServerID]*followerReplication) | |||
r.leaderState.notify = make(map[*verifyFuture]struct{}) | |||
r.leaderState.stepDown = make(chan struct{}, 1) | |||
r.leaderState.lease = time.After(r.conf.LeaderLeaseTimeout) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hmm is this right? It seems to be e remnant of the original lease stuff that turned out not to be relevant?
raft_test.go
Outdated
|
||
func TestRaft_LeadershipTransferResetsLeaderLease(t *testing.T) { | ||
t.Skip("How do I test this?") | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Shouldn't even be a test any more with lease stuff removed?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
agreed!
raft_test.go
Outdated
|
||
func TestRaft_LeadershipTransferToUnresponsiveServer(t *testing.T) { | ||
t.Skip("How do I test this?") | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should we just remove these if we have no way to test them?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
agreed!
raft.go
Outdated
// return early if this server is up to date | ||
if state.nextIndex > target { | ||
return &server | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is this necessary?
The loop isn't long and would always find the most up-to-date anyway so I don't think it's a big performance win. In some cases it might even work out to be less-than-optimal for transfer time because there may have been 10 new logs recorded since our getLastIndex
above and the first server is all 10 behind while another server in the list might be only 1 behind.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Agreed. Fixed in 8deca55.
if server.ID == r.localID { | ||
continue | ||
} | ||
state, ok := r.leaderState.replState[server.ID] |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is this OK? In general leaderState is expected to only be accessed by the single goroutine running the leader loop. Does this always run there? I think it does but worth checking. If so I suggest we update the comments on these new methods that assume they are part of the leader loop to make that explicit?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Great catch, it wasn't called from the leaderloop! Plus I found another place where leaderstate
is accessed outside of the leaderloop
. Fixed both things.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@banks, there are a bunch of minor fixes (by freeekanayaka) in PR form already that improve the CI stability of master. might want to consider merging those and rebasing on top. |
8deca55
to
50f6abc
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Super minor Q which may be wrong but could be a race. SOOOO close!
raft.go
Outdated
return nil | ||
} | ||
|
||
// pickServer returns the follower that is most up to date. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should we comment this is only to be called in leader loop?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
done!
raft.go
Outdated
r.setLeadershipTransferInProgress(true) | ||
defer func() { r.setLeadershipTransferInProgress(false) }() | ||
|
||
for repl.nextIndex <= r.getLastIndex() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This access of the followerReplication.nextIndex
made me wonder about thread safety. Presumably this is racey since the replication thread is still running and could be writing to the nextIndex
concurrently?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Using atomic
now to protect from the race. Great catch!
Co-Authored-By: Paul Banks <banks@banksco.de>
This PR is implementing the leadership transfer extension described in the thesis chap 3.10.
Background:
Consul is performing some setup after acquiring leadership. It is possible that the setup fails, but there is no good way to step down as a leader. It is possible to use
DemoteVoter
as show in hashicorp/consul#5247, but this is suboptimal because it relies on Consul's autopilot to promote the old leader to a voter again.Since there is a perfectly fine way described in the thesis: leadership transfer extension, we decided to implement that instead. Doing it this way also helps other teams, since it is more generic.
The necessary steps to perform are:
6a. If the election is successful, a message with the new term will make the old leader step down
6b. if after electiontimeout the leadership transfer did not complete, the old leader resumes operation
Todo:
RequestVote
as described in chap 4.2.3 Disruptive serversLeaderLeaseTimeout
before leadership transfer as described in chap 6.4.1 Using clocks to reduce messaging for read-only queriesResources: