Skip to content

Commit eba8343

Browse files
Leadership transfer (hashicorp#306)
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: 1. Leader picks target to transition to 2. Leader stops accepting client requests 3. Leader makes sure to replicate logs to the target 4. Leader sends TimeoutNow RPC request 5. Target receives TimeoutNow request, which triggers an election 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 Resources: https://github.com/etcd-io/etcd/tree/master/raft
1 parent 2c55169 commit eba8343

12 files changed

+835
-79
lines changed

api.go

+61-11
Original file line numberDiff line numberDiff line change
@@ -49,6 +49,10 @@ var (
4949
// ErrCantBootstrap is returned when attempt is made to bootstrap a
5050
// cluster that already has state present.
5151
ErrCantBootstrap = errors.New("bootstrap only works on new clusters")
52+
53+
// ErrLeadershipTransferInProgress is returned when the leader is rejecting
54+
// client requests because it is attempting to transfer leadership.
55+
ErrLeadershipTransferInProgress = errors.New("leadership transfer in progress")
5256
)
5357

5458
// Raft implements a Raft node.
@@ -97,6 +101,12 @@ type Raft struct {
97101
// leaderState used only while state is leader
98102
leaderState leaderState
99103

104+
// candidateFromLeadershipTransfer is used to indicate that this server became
105+
// candidate because the leader tries to transfer leadership. This flag is
106+
// used in RequestVoteRequest to express that a leadership transfer is going
107+
// on.
108+
candidateFromLeadershipTransfer bool
109+
100110
// Stores our local server ID, used to avoid sending RPCs to ourself
101111
localID ServerID
102112

@@ -157,6 +167,10 @@ type Raft struct {
157167
// is indexed by an artificial ID which is used for deregistration.
158168
observersLock sync.RWMutex
159169
observers map[uint64]*Observer
170+
171+
// leadershipTransferCh is used to start a leadership transfer from outside of
172+
// the main thread.
173+
leadershipTransferCh chan *leadershipTransferFuture
160174
}
161175

162176
// BootstrapCluster initializes a server's storage with the given cluster
@@ -443,17 +457,17 @@ func NewRaft(conf *Config, fsm FSM, logs LogStore, stable StableStore, snaps Sna
443457

444458
// Create Raft struct.
445459
r := &Raft{
446-
protocolVersion: protocolVersion,
447-
applyCh: make(chan *logFuture),
448-
conf: *conf,
449-
fsm: fsm,
450-
fsmMutateCh: make(chan interface{}, 128),
451-
fsmSnapshotCh: make(chan *reqSnapshotFuture),
452-
leaderCh: make(chan bool),
453-
localID: localID,
454-
localAddr: localAddr,
455-
logger: logger,
456-
logs: logs,
460+
protocolVersion: protocolVersion,
461+
applyCh: make(chan *logFuture),
462+
conf: *conf,
463+
fsm: fsm,
464+
fsmMutateCh: make(chan interface{}, 128),
465+
fsmSnapshotCh: make(chan *reqSnapshotFuture),
466+
leaderCh: make(chan bool),
467+
localID: localID,
468+
localAddr: localAddr,
469+
logger: logger,
470+
logs: logs,
457471
configurationChangeCh: make(chan *configurationChangeFuture),
458472
configurations: configurations{},
459473
rpcCh: trans.Consumer(),
@@ -467,6 +481,7 @@ func NewRaft(conf *Config, fsm FSM, logs LogStore, stable StableStore, snaps Sna
467481
configurationsCh: make(chan *configurationsFuture, 8),
468482
bootstrapCh: make(chan *bootstrapFuture),
469483
observers: make(map[uint64]*Observer),
484+
leadershipTransferCh: make(chan *leadershipTransferFuture, 1),
470485
}
471486

472487
// Initialize as a follower.
@@ -1011,3 +1026,38 @@ func (r *Raft) LastIndex() uint64 {
10111026
func (r *Raft) AppliedIndex() uint64 {
10121027
return r.getLastApplied()
10131028
}
1029+
1030+
// LeadershipTransfer will transfer leadership to a server in the cluster.
1031+
// This can only be called from the leader, or it will fail. The leader will
1032+
// stop accepting client requests, make sure the target server is up to date
1033+
// and starts the transfer with a TimeoutNow message. This message has the same
1034+
// effect as if the election timeout on the on the target server fires. Since
1035+
// it is unlikely that another server is starting an election, it is very
1036+
// likely that the target server is able to win the election. Note that raft
1037+
// protocol version 3 is not sufficient to use LeadershipTransfer. A recent
1038+
// version of that library has to be used that includes this feature. Using
1039+
// transfer leadership is safe however in a cluster where not every node has
1040+
// the latest version. If a follower cannot be promoted, it will fail
1041+
// gracefully.
1042+
func (r *Raft) LeadershipTransfer() Future {
1043+
if r.protocolVersion < 3 {
1044+
return errorFuture{ErrUnsupportedProtocol}
1045+
}
1046+
1047+
return r.initiateLeadershipTransfer(nil, nil)
1048+
}
1049+
1050+
// LeadershipTransferToServer does the same as LeadershipTransfer but takes a
1051+
// server in the arguments in case a leadership should be transitioned to a
1052+
// specific server in the cluster. Note that raft protocol version 3 is not
1053+
// sufficient to use LeadershipTransfer. A recent version of that library has
1054+
// to be used that includes this feature. Using transfer leadership is safe
1055+
// however in a cluster where not every node has the latest version. If a
1056+
// follower cannot be promoted, it will fail gracefully.
1057+
func (r *Raft) LeadershipTransferToServer(id ServerID, address ServerAddress) Future {
1058+
if r.protocolVersion < 3 {
1059+
return errorFuture{ErrUnsupportedProtocol}
1060+
}
1061+
1062+
return r.initiateLeadershipTransfer(&id, &address)
1063+
}

commands.go

+26
Original file line numberDiff line numberDiff line change
@@ -76,6 +76,11 @@ type RequestVoteRequest struct {
7676
// Used to ensure safety
7777
LastLogIndex uint64
7878
LastLogTerm uint64
79+
80+
// Used to indicate to peers if this vote was triggered by a leadership
81+
// transfer. It is required for leadership transfer to work, because servers
82+
// wouldn't vote otherwise if they are aware of an existing leader.
83+
LeadershipTransfer bool
7984
}
8085

8186
// See WithRPCHeader.
@@ -149,3 +154,24 @@ type InstallSnapshotResponse struct {
149154
func (r *InstallSnapshotResponse) GetRPCHeader() RPCHeader {
150155
return r.RPCHeader
151156
}
157+
158+
// TimeoutNowRequest is the command used by a leader to signal another server to
159+
// start an election.
160+
type TimeoutNowRequest struct {
161+
RPCHeader
162+
}
163+
164+
// See WithRPCHeader.
165+
func (r *TimeoutNowRequest) GetRPCHeader() RPCHeader {
166+
return r.RPCHeader
167+
}
168+
169+
// TimeoutNowResponse is the response to TimeoutNowRequest.
170+
type TimeoutNowResponse struct {
171+
RPCHeader
172+
}
173+
174+
// See WithRPCHeader.
175+
func (r *TimeoutNowResponse) GetRPCHeader() RPCHeader {
176+
return r.RPCHeader
177+
}

future.go

+15
Original file line numberDiff line numberDiff line change
@@ -58,6 +58,12 @@ type SnapshotFuture interface {
5858
Open() (*SnapshotMeta, io.ReadCloser, error)
5959
}
6060

61+
// LeadershipTransferFuture is used for waiting on a user-triggered leadership
62+
// transfer to complete.
63+
type LeadershipTransferFuture interface {
64+
Future
65+
}
66+
6167
// errorFuture is used to return a static error.
6268
type errorFuture struct {
6369
err error
@@ -227,6 +233,15 @@ type verifyFuture struct {
227233
voteLock sync.Mutex
228234
}
229235

236+
// leadershipTransferFuture is used to track the progress of a leadership
237+
// transfer internally.
238+
type leadershipTransferFuture struct {
239+
deferError
240+
241+
ID *ServerID
242+
Address *ServerAddress
243+
}
244+
230245
// configurationsFuture is used to retrieve the current configurations. This is
231246
// used to allow safe access to this information outside of the main thread.
232247
type configurationsFuture struct {

fuzzy/cluster.go

+37-17
Original file line numberDiff line numberDiff line change
@@ -205,34 +205,54 @@ func (c *cluster) appliedIndexes() map[string]uint64 {
205205
return r
206206
}
207207

208-
func (c *cluster) ApplyN(t *testing.T, leaderTimeout time.Duration, s *applySource, n uint) uint64 {
209-
f := make([]raft.ApplyFuture, n)
208+
func (c *cluster) generateNApplies(s *applySource, n uint) [][]byte {
210209
data := make([][]byte, n)
211-
startTime := time.Now()
212-
endTime := startTime.Add(leaderTimeout)
213210
for i := uint(0); i < n; i++ {
214-
ldr := c.Leader(endTime.Sub(time.Now()))
215-
if ldr != nil {
216-
data[i] = s.nextEntry()
217-
f[i] = ldr.raft.Apply(data[i], time.Second)
218-
}
211+
data[i] = s.nextEntry()
212+
}
213+
return data
214+
}
215+
216+
func (c *cluster) leadershipTransfer(leaderTimeout time.Duration) raft.Future {
217+
ldr := c.Leader(leaderTimeout)
218+
return ldr.raft.LeadershipTransfer()
219+
}
220+
221+
type applyFutureWithData struct {
222+
future raft.ApplyFuture
223+
data []byte
224+
}
225+
226+
func (c *cluster) sendNApplies(leaderTimeout time.Duration, data [][]byte) []applyFutureWithData {
227+
f := []applyFutureWithData{}
228+
229+
ldr := c.Leader(leaderTimeout)
230+
for _, d := range data {
231+
f = append(f, applyFutureWithData{future: ldr.raft.Apply(d, time.Second), data: d})
219232
}
233+
return f
234+
}
235+
236+
func (c *cluster) checkApplyFutures(futures []applyFutureWithData) uint64 {
220237
success := uint64(0)
221-
for i := uint(0); i < n; i++ {
222-
if f[i] == nil {
223-
continue
224-
}
225-
if err := f[i].Error(); err == nil {
238+
for _, a := range futures {
239+
if err := a.future.Error(); err == nil {
226240
success++
227-
c.lastApplySuccess = f[i]
228-
c.applied = append(c.applied, appliedItem{f[i].Index(), data[i]})
241+
c.lastApplySuccess = a.future
242+
c.applied = append(c.applied, appliedItem{a.future.Index(), a.data})
229243
} else {
230-
c.lastApplyFailure = f[i]
244+
c.lastApplyFailure = a.future
231245
}
232246
}
233247
return success
234248
}
235249

250+
func (c *cluster) ApplyN(t *testing.T, leaderTimeout time.Duration, s *applySource, n uint) uint64 {
251+
data := c.generateNApplies(s, n)
252+
futures := c.sendNApplies(leaderTimeout, data)
253+
return c.checkApplyFutures(futures)
254+
}
255+
236256
func (c *cluster) VerifyFSM(t *testing.T) {
237257
exp := c.nodes[0].fsm
238258
expName := c.nodes[0].name

fuzzy/leadershiptransfer_test.go

+73
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,73 @@
1+
package fuzzy
2+
3+
import (
4+
"math/rand"
5+
"testing"
6+
"time"
7+
8+
"github.com/hashicorp/raft"
9+
)
10+
11+
// 5 node cluster
12+
func TestRaft_FuzzyLeadershipTransfer(t *testing.T) {
13+
cluster := newRaftCluster(t, testLogWriter, "lt", 5, nil)
14+
r := rand.New(rand.NewSource(time.Now().UnixNano()))
15+
16+
s := newApplySource("LeadershipTransfer")
17+
data := cluster.generateNApplies(s, uint(r.Intn(10000)))
18+
futures := cluster.sendNApplies(time.Minute, data)
19+
cluster.leadershipTransfer(time.Minute)
20+
21+
data = cluster.generateNApplies(s, uint(r.Intn(10000)))
22+
futures = append(futures, cluster.sendNApplies(time.Minute, data)...)
23+
cluster.leadershipTransfer(time.Minute)
24+
25+
data = cluster.generateNApplies(s, uint(r.Intn(10000)))
26+
futures = append(futures, cluster.sendNApplies(time.Minute, data)...)
27+
cluster.leadershipTransfer(time.Minute)
28+
29+
data = cluster.generateNApplies(s, uint(r.Intn(10000)))
30+
futures = append(futures, cluster.sendNApplies(time.Minute, data)...)
31+
32+
ac := cluster.checkApplyFutures(futures)
33+
34+
cluster.Stop(t, time.Minute)
35+
cluster.VerifyLog(t, ac)
36+
cluster.VerifyFSM(t)
37+
}
38+
39+
type LeadershipTransferMode int
40+
41+
type LeadershipTransfer struct {
42+
verifier appendEntriesVerifier
43+
slowNodes map[string]bool
44+
delayMin time.Duration
45+
delayMax time.Duration
46+
mode LeadershipTransferMode
47+
}
48+
49+
func (lt *LeadershipTransfer) Report(t *testing.T) {
50+
lt.verifier.Report(t)
51+
}
52+
53+
func (lt *LeadershipTransfer) PreRPC(s, t string, r *raft.RPC) error {
54+
return nil
55+
}
56+
57+
func (lt *LeadershipTransfer) nap() {
58+
d := lt.delayMin + time.Duration(rand.Int63n((lt.delayMax - lt.delayMin).Nanoseconds()))
59+
time.Sleep(d)
60+
}
61+
62+
func (lt *LeadershipTransfer) PostRPC(src, target string, r *raft.RPC, res *raft.RPCResponse) error {
63+
return nil
64+
}
65+
66+
func (lt *LeadershipTransfer) PreRequestVote(src, target string, v *raft.RequestVoteRequest) (*raft.RequestVoteResponse, error) {
67+
return nil, nil
68+
}
69+
70+
func (lt *LeadershipTransfer) PreAppendEntries(src, target string, v *raft.AppendEntriesRequest) (*raft.AppendEntriesResponse, error) {
71+
lt.verifier.PreAppendEntries(src, target, v)
72+
return nil, nil
73+
}

fuzzy/transport.go

+11
Original file line numberDiff line numberDiff line change
@@ -112,9 +112,15 @@ func (t *transport) sendRPC(target string, req interface{}, resp interface{}) er
112112
}
113113
rpc := raft.RPC{RespChan: rc}
114114
var reqVote raft.RequestVoteRequest
115+
var timeoutNow raft.TimeoutNowRequest
115116
var appEnt raft.AppendEntriesRequest
116117
dec := codec.NewDecoderBytes(buff.Bytes(), &codecHandle)
117118
switch req.(type) {
119+
case *raft.TimeoutNowRequest:
120+
if err := dec.Decode(&timeoutNow); err != nil {
121+
return err
122+
}
123+
rpc.Command = &timeoutNow
118124
case *raft.RequestVoteRequest:
119125
if err := dec.Decode(&reqVote); err != nil {
120126
return err
@@ -166,6 +172,11 @@ func (t *transport) sendRPC(target string, req interface{}, resp interface{}) er
166172
return result.Error
167173
}
168174

175+
// TimeoutNow implements the Transport interface.
176+
func (t *transport) TimeoutNow(id raft.ServerID, target raft.ServerAddress, args *raft.TimeoutNowRequest, resp *raft.TimeoutNowResponse) error {
177+
return t.sendRPC(string(target), args, resp)
178+
}
179+
169180
// AppendEntries sends the appropriate RPC to the target node.
170181
func (t *transport) AppendEntries(id raft.ServerID, target raft.ServerAddress, args *raft.AppendEntriesRequest, resp *raft.AppendEntriesResponse) error {
171182
ae := appendEntries{

inmem_transport.go

+13
Original file line numberDiff line numberDiff line change
@@ -135,6 +135,19 @@ func (i *InmemTransport) InstallSnapshot(id ServerID, target ServerAddress, args
135135
return nil
136136
}
137137

138+
// TimeoutNow implements the Transport interface.
139+
func (i *InmemTransport) TimeoutNow(id ServerID, target ServerAddress, args *TimeoutNowRequest, resp *TimeoutNowResponse) error {
140+
rpcResp, err := i.makeRPC(target, args, nil, 10*i.timeout)
141+
if err != nil {
142+
return err
143+
}
144+
145+
// Copy the result back
146+
out := rpcResp.Response.(*TimeoutNowResponse)
147+
*resp = *out
148+
return nil
149+
}
150+
138151
func (i *InmemTransport) makeRPC(target ServerAddress, args interface{}, r io.Reader, timeout time.Duration) (rpcResp RPCResponse, err error) {
139152
i.RLock()
140153
peer, ok := i.peers[target]

0 commit comments

Comments
 (0)