Conversation
|
@es-chow Can you talk a little bit about your use case or potential use case? I know some one has already done this for etcd/raft. I can ask him to review the code too. |
|
The background for this is that Cockroachdb has an idea to coincide raft leader with range leader, https://github.com/cockroachdb/cockroach/blob/master/docs/design.md#relationship-to-raft-leadership. This coincide may be achieved by leader transfer. |
|
@bdarnell Can you confirm cockroach need this feature? |
raft/raft.go
Outdated
| r.Step(pb.Message{From: r.id, Type: pb.MsgBeat}) | ||
| } | ||
|
|
||
| // If current leader cannot transfer leader in election timeout, become leader again. |
There was a problem hiding this comment.
Shouldn't this be inside the r.electionElapsed >= r.electionTimeout block above at line 440?
|
Yes, we could use some sort of orderly leadership handoff to allow us to choose the best leader. I don't like doing this by introducing a new |
OK. I will take a closer look at this pr. |
raft/raft.go
Outdated
|
|
||
| logger Logger | ||
| // Leader transfer target. | ||
| transferee uint64 |
|
@bdarnell, as transferee will be reset to zero after sending out a MsgTimeoutNow to prevent multiple MsgTimeoutNow to send out when pipelining MsgAppResp been received. So if we judge if a MsgProp can be honored by check the value of transferee, it may make a MsgProp being honored just after a MsgTimeoutNow been sent out. Or we have to introduce one more flag to distinguish this. |
|
I believe @dterei has implemented this feature for etcd/raft. So we would love to hear his opinion! |
raft/node_test.go
Outdated
| } | ||
| } else { | ||
| if msgt == raftpb.MsgBeat || msgt == raftpb.MsgHup || msgt == raftpb.MsgUnreachable || msgt == raftpb.MsgSnapStatus || msgt == raftpb.MsgCheckQuorum { | ||
| if msgt == raftpb.MsgBeat || msgt == raftpb.MsgHup || msgt == raftpb.MsgUnreachable || msgt == raftpb.MsgSnapStatus || msgt == raftpb.MsgCheckQuorum || msgt == raftpb.MsgTransferLeader { |
There was a problem hiding this comment.
probably it is time for us to make this multiple lines.
e465718 to
e17ed54
Compare
raft/raft.go
Outdated
| lead uint64 | ||
|
|
||
| // Leader transfer target. | ||
| transferee uint64 |
There was a problem hiding this comment.
// leadTransfree is id of the leader transfer target when its value is not zero.
leadTransfree?
raft/raft_test.go
Outdated
| nt := newNetwork(nil, nil, nil) | ||
| nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) | ||
|
|
||
| for j := 0; j <= 5; j++ { |
There was a problem hiding this comment.
why do we send 5 proposals in this test case?
|
@es-chow Looks good overall. Please fix the comments. Add LeaderTransfere prefix to all tests. Thanks! |
0768bc4 to
24e2dbf
Compare
raft/raft.go
Outdated
| case len(r.votes) - gr: | ||
| r.becomeFollower(r.Term, None) | ||
| } | ||
| case pb.MsgTransferLeader: |
There was a problem hiding this comment.
can you move these to line 555. (same as how we handle m.Type == pb.MsgHup )
|
LGTM. Defer to @bdarnell or anyone from cockroach team. |
|
@xiang90 I'd like to see this happen too. And i will port the feature to TiKV later. |
| } | ||
| // Transfer leadership is in progress. | ||
| if m.From == r.leadTransferee && pr.Match == r.raftLog.lastIndex() && !r.timeoutNowSent { | ||
| r.logger.Infof("%x sent MsgTimeoutNow to %x after received MsgAppResp", r.id, m.From) |
There was a problem hiding this comment.
If we assume that the transport (usually) preserves order, we could send the MsgTimeoutNow as soon as we have sent a MsgApp containing the last index, without waiting for the MsgAppResp. This is similar to the way we optimistically increase Progress.Next when sending MsgApp.
There was a problem hiding this comment.
@bdarnell, @xiang90, I feel we cannot send MsgTimeoutNow as soon as we have sent a MsgApp containing the last index in all cases, if we add that code and remove the code to send MsgTimeoutNow when a MsgAppResp is received, the test case TestLeaderTransferToSlowFollower cannot be passed as node 3 will reject that MsgApp just before the MsgTimeoutNow triggered by pb.MsgTransferLeader as its log is not up-to-date, but the following MsgTimeoutNow will trigger node3 to start an election which also will fail.
- Maybe one way is we keep the code which send
MsgTimeoutNowas receivingMsgAppResp, but add one optimistic sendingMsgTimeoutNowonly when we optimistically update thepr.Next, but this may also have the risk that peer may reject thatMsgAppandMsgTimeoutNowwill make the group unavailable for one election timeout. And also we have to keep thetimeoutNowSentflag. - We keep the current code as we can also remove the
timeoutNowSentflag. This is the updated commit.
There was a problem hiding this comment.
I feel this is an optimization. We can think this more and do it in another pr. @es-chow Can you set up another issue to track this potential optimization?
There was a problem hiding this comment.
Yeah, this optimization can wait. One thing we could do to avoid the scenario in which the follower rejects the MsgApp and then starts an election that fails is to include the leader's last log index in the MsgTimeoutNow. The follower would only start campaigning if it matches the index; if it doesn't then it knows it can't win.
|
LGTM. Thanks @es-chow! |
|
@es-chow Can you fix the last few comments? We are getting really close to get this merged! Thanks! |
| if lastLeadTransferee == None { | ||
| r.logger.Debugf("%x is already leader. Ignored transfer leadership to %x", r.id, r.id) | ||
| } else { | ||
| r.logger.Debugf("%x abort transfer leadership to %x, transfer to current leader %x.", r.id, lastLeadTransferee, r.id) |
There was a problem hiding this comment.
it seems we never run to this line, otherwise r.abortLeaderTransfer() should be add.
Try to implement transfer leader as per raft thesis
3.10 Leadership transfer extension.