raft: Use TransferLeadership to make leader demotion safer#1939
raft: Use TransferLeadership to make leader demotion safer#1939LK4D4 merged 1 commit intomoby:masterfrom
Conversation
Codecov Report
@@ Coverage Diff @@
## master #1939 +/- ##
==========================================
- Coverage 54.29% 53.99% -0.31%
==========================================
Files 108 108
Lines 18586 18588 +2
==========================================
- Hits 10092 10036 -56
- Misses 7257 7324 +67
+ Partials 1237 1228 -9Continue to review full report at Codecov.
|
|
Note to self: need to confirm this won't cause any problems in mixed version swarms. I don't think it will but it's important to be sure. |
manager/state/raft/raft.go
Outdated
| defer ticker.Stop() | ||
| for { | ||
| leader := n.leader() | ||
| if leader != 0 && leader != n.Config.ID { |
There was a problem hiding this comment.
replace 0 with raft.None.
Also shouldn't we wait for transferee to became leader? Or it's possible that other node became leader?
There was a problem hiding this comment.
I was thinking that if a different node starts an election at the same time, that one might win.
manager/state/raft/raft.go
Outdated
| } | ||
| select { | ||
| case <-ctx.Done(): | ||
| return errors.Wrap(err, "timed out waiting for leadership change") |
There was a problem hiding this comment.
it might be not timeout I think, maybe better just errors.Wrap(err, "failed to transfer leadership")
manager/state/raft/raft.go
Outdated
| start := time.Now() | ||
| log.G(ctx).Infof("raft: transfer leadership %x -> %x", n.Config.ID, transferee) | ||
| n.raftNode.TransferLeadership(ctx, n.Config.ID, transferee) | ||
| ticker := time.NewTicker(100 * time.Millisecond) |
There was a problem hiding this comment.
I see that etcd uses TickInterval here. Do you think it make sense in this case?
There was a problem hiding this comment.
Changed to TickInterval / 10 so we don't have to wait a full tick to recognize the leader has changed.
manager/state/raft/raft.go
Outdated
| } | ||
|
|
||
| func (n *Node) removeSelfGracefully(ctx context.Context) error { | ||
| transferCtx, cancelTransfer := context.WithTimeout(ctx, 10*time.Second) |
There was a problem hiding this comment.
It looks little too hardcody. Maybe we should inherit it from ElectionTick as etcd does?
There was a problem hiding this comment.
etcd seems to use this:
// ReqTimeout returns timeout for request to finish.
func (c *ServerConfig) ReqTimeout() time.Duration {
// 5s for queue waiting, computation and disk IO delay
// + 2 * election timeout for possible leader election
return 5*time.Second + 2*time.Duration(c.ElectionTicks)*time.Duration(c.TickMs)*time.Millisecond
}That seems reasonable, but I'm not sure it's that much better than a simple fixed timeout (it hardcodes 5 seconds, for example). I guess I'll switch to it anyway.
manager/state/raft/raft.go
Outdated
| } | ||
|
|
||
| return ErrCannotRemoveMember | ||
| // transferLeadership attempts to transfer leadership to a different node, |
There was a problem hiding this comment.
Non-blocking: would it make sense to duplicate the stopMu part of this comment up in removeSelfGracefully as well, since removeSelfGracefully doesn't hold stopMu either?
manager/state/raft/raft.go
Outdated
| case <-ticker.C: | ||
| } | ||
| } | ||
| log.G(ctx).Infof("raft: transfer leadership %x -> %x finished in %v", n.Config.ID, transferee, time.Since(start)) |
There was a problem hiding this comment.
Should this be leader instead of transferee, in case some other node started leader election and then won?
f7b7644 to
4e0b605
Compare
|
Addressed comments, PTAL |
|
@aaronlehmann I've tried TestDemoteToSingleManager and not a single transfer finished with success: |
manager/state/raft/raft.go
Outdated
| return errors.Wrap(err, "failed to get longest-active member") | ||
| } | ||
| start := time.Now() | ||
| log.G(ctx).Infof("raft: transfer leadership %x -> %x", n.Config.ID, transferee) |
There was a problem hiding this comment.
This message duplicates in etcd/raft, not sure if we care.
|
|
|
Tests is passing okay, but transfer leadership doesn't work. |
|
@LK4D4 Ah sorry, yes, thanks for clarifying. I'm seeing the same. |
4e0b605 to
8ba1a53
Compare
|
Thanks for pointing out that The first problem here was that the context would be cancelled when the old leader lost the leadership. This caused the function to return an error. I fixed this, but afterwards it turned out that the node would get removed twice: once by the call to So I'm trying a new, simpler, approach. PTAL |
|
...of course the tests consistently pass on my machine :/ |
8ba1a53 to
d667824
Compare
| case <-ticker.C: | ||
| } | ||
| } | ||
| return nil |
There was a problem hiding this comment.
Transfer finish is not logged by raft library :(
There was a problem hiding this comment.
It's not, but raft does log when a new leader is elected, so I thought that would be enough.
manager/state/raft/raft.go
Outdated
| ErrMemberUnknown = errors.New("raft: member unknown") | ||
| // ErrCantRemoveSelf is returned if RemoveMember is called with the | ||
| // local node as the argument. | ||
| ErrCantRemoveSelf = errors.New("raft: can't remove self") |
There was a problem hiding this comment.
Error above is using word Cannot, maybe better to keep it consistent.
manager/role_manager.go
Outdated
| if err == raft.ErrCantRemoveSelf { | ||
| // Don't use rmCtx, because we expect to lose | ||
| // leadership, which will cancel this context. | ||
| log.L.Info("demoted; ceding leadership") |
There was a problem hiding this comment.
I live in the USA for almost three years, and only vague know the meaning of the word ceding :) Let's use transferring or something more common.
|
have minor comments. |
d667824 to
392bdc3
Compare
|
I've made the cosmetic changes. I'm happy to add back the "transfered leadership" log message if you think it makes sense. I had some test failures in CI with an earlier version of this, but I can't reproduce them locally. Can you try running the tests a few times to see if they are stable? |
|
@aaronlehmann sure will run tests now. |
392bdc3 to
e05da5a
Compare
|
Added back the transfer timing. |
|
Adding back the |
|
Tests are quite stable for me, no leaks, no races. |
|
I'm rebuilding it a few more times in CI. |
|
I can't get this to fail in CI anymore. Either the initial failures were a fluke, or adding back |
manager/role_manager.go
Outdated
| } | ||
|
|
||
| rmCtx, rmCancel := context.WithTimeout(rm.ctx, 5*time.Second) | ||
| rmCtx, rmCancel := context.WithTimeout(context.Background(), 5*time.Second) |
There was a problem hiding this comment.
Non-blocking question: Does this need to be context.Background() if rm.raft.TransferLeadership uses a new context?
| assert.Len(t, nodes[4].GetMemberlist(), 4) | ||
| } | ||
|
|
||
| func TestRaftLeaderLeave(t *testing.T) { |
There was a problem hiding this comment.
Should this test be modified to just fail with the expected error rather than removed entirely?
There was a problem hiding this comment.
Actually even dumber question: does anything besides the tests call Leave on a raft member anymore? (I assume even if not, we'd have to leave it in for backwards compatibility, but it might be nice to comment)
There was a problem hiding this comment.
Actually even dumber question: does anything besides the tests call
Leaveon a raft member anymore?
Nope. Maybe we should remove it.
|
Nice, I'm not getting any intermittent context exceeded errors in other integration tests anymore. Just have a few questions above, but other than that LGTM. |
e05da5a to
ca85e3a
Compare
When we demote the leader, we currently wait for all queued messages to be sent, as a best-effort approach to making sure the other nodes find out that the node removal has been committed, and stop treating the current leader as a cluster member. This doesn't work perfectly. To make this more robust, use TransferLeadership when the leader is trying to remove itself. The new leader's reconcilation loop will kick in and remove the old leader. Signed-off-by: Aaron Lehmann <aaron.lehmann@docker.com>
ca85e3a to
5470e07
Compare
|
I've updated this to work slightly differently. If the leadership transfer times out, we fall back to self-demoting the old way. I've tested this with a 1.14-dev leader in a cluster with a 1.12.6 follower, and demoting the leader works properly (after the timeout elapses). PTAL. Hoping to merge this soon if it looks good. |
|
LGTM! Thank you for tracking down a different unrelated error! |
When we demote the leader, we currently wait for all queued messages to be sent, as a best-effort approach to making sure the other nodes find out that the node removal has been committed, and stop treating the current leader as a cluster member. This doesn't work perfectly.
To make this more robust, use
TransferLeadershipwhen the leader is trying to remove itself. The new leader's reconcilation loop will kick in and remove the old leader.cc @LK4D4 @cyli