Skip to content

Commit ace424e

Browse files
authored
Add HeartbeatTimeout and ElectionTimeout to reloadable config. (#496)
1 parent f2fdbd6 commit ace424e

File tree

6 files changed

+228
-4
lines changed

6 files changed

+228
-4
lines changed

api.go

+16
Original file line numberDiff line numberDiff line change
@@ -201,6 +201,12 @@ type Raft struct {
201201
// leadershipTransferCh is used to start a leadership transfer from outside of
202202
// the main thread.
203203
leadershipTransferCh chan *leadershipTransferFuture
204+
205+
// leaderNotifyCh is used to tell leader that config has changed
206+
leaderNotifyCh chan struct{}
207+
208+
// followerNotifyCh is used to tell followers that config has changed
209+
followerNotifyCh chan struct{}
204210
}
205211

206212
// BootstrapCluster initializes a server's storage with the given cluster
@@ -545,6 +551,8 @@ func NewRaft(conf *Config, fsm FSM, logs LogStore, stable StableStore, snaps Sna
545551
bootstrapCh: make(chan *bootstrapFuture),
546552
observers: make(map[uint64]*Observer),
547553
leadershipTransferCh: make(chan *leadershipTransferFuture, 1),
554+
leaderNotifyCh: make(chan struct{}, 1),
555+
followerNotifyCh: make(chan struct{}, 1),
548556
}
549557

550558
r.conf.Store(*conf)
@@ -696,6 +704,14 @@ func (r *Raft) ReloadConfig(rc ReloadableConfig) error {
696704
return err
697705
}
698706
r.conf.Store(newCfg)
707+
708+
if rc.HeartbeatTimeout < oldCfg.HeartbeatTimeout {
709+
// On leader, ensure replication loops running with a longer
710+
// timeout than what we want now discover the change.
711+
asyncNotifyCh(r.leaderNotifyCh)
712+
// On follower, update current timer to use the shorter new value.
713+
asyncNotifyCh(r.followerNotifyCh)
714+
}
699715
return nil
700716
}
701717

config.go

+14-2
Original file line numberDiff line numberDiff line change
@@ -260,6 +260,14 @@ type ReloadableConfig struct {
260260
// we perform a snapshot. This is to prevent excessive snapshots when we can
261261
// just replay a small set of logs.
262262
SnapshotThreshold uint64
263+
264+
// HeartbeatTimeout specifies the time in follower state without
265+
// a leader before we attempt an election.
266+
HeartbeatTimeout time.Duration
267+
268+
// ElectionTimeout specifies the time in candidate state without
269+
// a leader before we attempt an election.
270+
ElectionTimeout time.Duration
263271
}
264272

265273
// apply sets the reloadable fields on the passed Config to the values in
@@ -269,6 +277,8 @@ func (rc *ReloadableConfig) apply(to Config) Config {
269277
to.TrailingLogs = rc.TrailingLogs
270278
to.SnapshotInterval = rc.SnapshotInterval
271279
to.SnapshotThreshold = rc.SnapshotThreshold
280+
to.HeartbeatTimeout = rc.HeartbeatTimeout
281+
to.ElectionTimeout = rc.ElectionTimeout
272282
return to
273283
}
274284

@@ -277,6 +287,8 @@ func (rc *ReloadableConfig) fromConfig(from Config) {
277287
rc.TrailingLogs = from.TrailingLogs
278288
rc.SnapshotInterval = from.SnapshotInterval
279289
rc.SnapshotThreshold = from.SnapshotThreshold
290+
rc.HeartbeatTimeout = from.HeartbeatTimeout
291+
rc.ElectionTimeout = from.ElectionTimeout
280292
}
281293

282294
// DefaultConfig returns a Config with usable defaults.
@@ -334,10 +346,10 @@ func ValidateConfig(config *Config) error {
334346
return fmt.Errorf("LeaderLeaseTimeout is too low")
335347
}
336348
if config.LeaderLeaseTimeout > config.HeartbeatTimeout {
337-
return fmt.Errorf("LeaderLeaseTimeout cannot be larger than heartbeat timeout")
349+
return fmt.Errorf("LeaderLeaseTimeout (%s) cannot be larger than heartbeat timeout (%s)", config.LeaderLeaseTimeout, config.HeartbeatTimeout)
338350
}
339351
if config.ElectionTimeout < config.HeartbeatTimeout {
340-
return fmt.Errorf("ElectionTimeout must be equal or greater than Heartbeat Timeout")
352+
return fmt.Errorf("ElectionTimeout (%s) must be equal or greater than Heartbeat Timeout (%s)", config.ElectionTimeout, config.HeartbeatTimeout)
341353
}
342354
return nil
343355
}

integ_test.go

+133
Original file line numberDiff line numberDiff line change
@@ -5,10 +5,12 @@ import (
55
"fmt"
66
"io/ioutil"
77
"os"
8+
"sync/atomic"
89
"testing"
910
"time"
1011

1112
"github.com/hashicorp/go-hclog"
13+
"github.com/stretchr/testify/require"
1214
)
1315

1416
// CheckInteg will skip a test if integration testing is not enabled.
@@ -355,3 +357,134 @@ func TestRaft_Integ(t *testing.T) {
355357
e.Release()
356358
}
357359
}
360+
361+
func TestRaft_RestartFollower_LongInitialHeartbeat(t *testing.T) {
362+
CheckInteg(t)
363+
tests := []struct {
364+
name string
365+
restartInitialTimeouts time.Duration
366+
expectNewLeader bool
367+
}{
368+
{"Default", 0, true},
369+
{"InitialHigher", time.Second, false},
370+
}
371+
for _, tt := range tests {
372+
t.Run(tt.name, func(t *testing.T) {
373+
conf := DefaultConfig()
374+
conf.LocalID = ServerID("first")
375+
conf.HeartbeatTimeout = 50 * time.Millisecond
376+
conf.ElectionTimeout = 50 * time.Millisecond
377+
conf.LeaderLeaseTimeout = 50 * time.Millisecond
378+
conf.CommitTimeout = 5 * time.Millisecond
379+
conf.SnapshotThreshold = 100
380+
conf.TrailingLogs = 10
381+
382+
// Create a single node
383+
env1 := MakeRaft(t, conf, true)
384+
NoErr(WaitFor(env1, Leader), t)
385+
386+
// Join a few nodes!
387+
var envs []*RaftEnv
388+
for i := 0; i < 2; i++ {
389+
conf.LocalID = ServerID(fmt.Sprintf("next-batch-%d", i))
390+
env := MakeRaft(t, conf, false)
391+
addr := env.trans.LocalAddr()
392+
NoErr(WaitFuture(env1.raft.AddVoter(conf.LocalID, addr, 0, 0), t), t)
393+
envs = append(envs, env)
394+
}
395+
allEnvs := append([]*RaftEnv{env1}, envs...)
396+
397+
// Wait for a leader
398+
_, err := WaitForAny(Leader, append([]*RaftEnv{env1}, envs...))
399+
NoErr(err, t)
400+
401+
CheckConsistent(append([]*RaftEnv{env1}, envs...), t)
402+
// TODO without this sleep, the restarted follower doesn't have any stored config
403+
// and aborts the election because it doesn't know of any peers. Shouldn't
404+
// CheckConsistent prevent that?
405+
time.Sleep(time.Second)
406+
407+
// shutdown a follower
408+
disconnected := envs[len(envs)-1]
409+
disconnected.logger.Info("stopping follower")
410+
disconnected.Shutdown()
411+
412+
seeNewLeader := func(o *Observation) bool { _, ok := o.Data.(LeaderObservation); return ok }
413+
leaderCh := make(chan Observation)
414+
// TODO Closing this channel results in panics, even though we're calling Release.
415+
//defer close(leaderCh)
416+
leaderChanges := new(uint32)
417+
go func() {
418+
for range leaderCh {
419+
atomic.AddUint32(leaderChanges, 1)
420+
}
421+
}()
422+
423+
requestVoteCh := make(chan Observation)
424+
seeRequestVote := func(o *Observation) bool { _, ok := o.Data.(RequestVoteRequest); return ok }
425+
requestVotes := new(uint32)
426+
go func() {
427+
for range requestVoteCh {
428+
atomic.AddUint32(requestVotes, 1)
429+
}
430+
}()
431+
432+
for _, env := range allEnvs {
433+
env.raft.RegisterObserver(NewObserver(leaderCh, false, seeNewLeader))
434+
}
435+
436+
// Unfortunately we need to wait for the leader to start backing off RPCs to the down follower
437+
// such that when the follower comes back up it'll run an election before it gets an rpc from
438+
// the leader
439+
time.Sleep(time.Second * 5)
440+
441+
if tt.restartInitialTimeouts != 0 {
442+
disconnected.conf.HeartbeatTimeout = tt.restartInitialTimeouts
443+
disconnected.conf.ElectionTimeout = tt.restartInitialTimeouts
444+
}
445+
disconnected.logger.Info("restarting follower")
446+
disconnected.Restart(t)
447+
448+
time.Sleep(time.Second * 2)
449+
450+
if tt.expectNewLeader {
451+
require.NotEqual(t, 0, atomic.LoadUint32(leaderChanges))
452+
} else {
453+
require.Equal(t, uint32(0), atomic.LoadUint32(leaderChanges))
454+
}
455+
456+
if tt.restartInitialTimeouts != 0 {
457+
for _, env := range envs {
458+
env.raft.RegisterObserver(NewObserver(requestVoteCh, false, seeRequestVote))
459+
NoErr(env.raft.ReloadConfig(ReloadableConfig{
460+
TrailingLogs: conf.TrailingLogs,
461+
SnapshotInterval: conf.SnapshotInterval,
462+
SnapshotThreshold: conf.SnapshotThreshold,
463+
HeartbeatTimeout: 250 * time.Millisecond,
464+
ElectionTimeout: 250 * time.Millisecond,
465+
}), t)
466+
}
467+
// Make sure that reload by itself doesn't trigger a vote
468+
time.Sleep(300 * time.Millisecond)
469+
require.Equal(t, uint32(0), atomic.LoadUint32(requestVotes))
470+
471+
// Stop the leader, ensure that we don't see a request vote within the first 50ms
472+
// (original config of the non-restarted follower), but that we do see one within
473+
// the 250ms both followers should now be using for heartbeat timeout. Well, not
474+
// quite: we wait for two heartbeat intervals (plus a fudge factor), because the
475+
// first time around, last contact will have been recent enough that no vote will
476+
// be triggered.
477+
env1.logger.Info("stopping leader")
478+
env1.Shutdown()
479+
time.Sleep(50 * time.Millisecond)
480+
require.Equal(t, uint32(0), atomic.LoadUint32(requestVotes))
481+
time.Sleep(600 * time.Millisecond)
482+
require.NotEqual(t, uint32(0), atomic.LoadUint32(requestVotes))
483+
}
484+
485+
for _, e := range allEnvs {
486+
e.Release()
487+
}
488+
})
489+
}
490+
}

observer.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -10,7 +10,7 @@ type Observation struct {
1010
// Raft holds the Raft instance generating the observation.
1111
Raft *Raft
1212
// Data holds observation-specific data. Possible types are
13-
// *RequestVoteRequest
13+
// RequestVoteRequest
1414
// RaftState
1515
// PeerObservation
1616
// LeaderObservation

raft.go

+25-1
Original file line numberDiff line numberDiff line change
@@ -190,6 +190,12 @@ func (r *Raft) runFollower() {
190190
case b := <-r.bootstrapCh:
191191
b.respond(r.liveBootstrap(b.configuration))
192192

193+
case <-r.leaderNotifyCh:
194+
// Ignore since we are not the leader
195+
196+
case <-r.followerNotifyCh:
197+
heartbeatTimer = time.After(0)
198+
193199
case <-heartbeatTimer:
194200
// Restart the heartbeat timer
195201
hbTimeout := r.config().HeartbeatTimeout
@@ -275,7 +281,8 @@ func (r *Raft) runCandidate() {
275281
// otherwise.
276282
defer func() { r.candidateFromLeadershipTransfer = false }()
277283

278-
electionTimer := randomTimeout(r.config().ElectionTimeout)
284+
electionTimeout := r.config().ElectionTimeout
285+
electionTimer := randomTimeout(electionTimeout)
279286

280287
// Tally the votes, need a simple majority
281288
grantedVotes := 0
@@ -337,6 +344,15 @@ func (r *Raft) runCandidate() {
337344
case b := <-r.bootstrapCh:
338345
b.respond(ErrCantBootstrap)
339346

347+
case <-r.leaderNotifyCh:
348+
// Ignore since we are not the leader
349+
350+
case <-r.followerNotifyCh:
351+
if electionTimeout != r.config().ElectionTimeout {
352+
electionTimeout = r.config().ElectionTimeout
353+
electionTimer = randomTimeout(electionTimeout)
354+
}
355+
340356
case <-electionTimer:
341357
// Election failed! Restart the election. We simply return,
342358
// which will kick us back into runCandidate
@@ -826,6 +842,14 @@ func (r *Raft) leaderLoop() {
826842
// Renew the lease timer
827843
lease = time.After(checkInterval)
828844

845+
case <-r.leaderNotifyCh:
846+
for _, repl := range r.leaderState.replState {
847+
asyncNotifyCh(repl.notifyCh)
848+
}
849+
850+
case <-r.followerNotifyCh:
851+
// Ignore since we are not a follower
852+
829853
case <-r.shutdownCh:
830854
return
831855
}

raft_test.go

+39
Original file line numberDiff line numberDiff line change
@@ -2466,6 +2466,7 @@ func TestRaft_CacheLogWithStoreError(t *testing.T) {
24662466

24672467
func TestRaft_ReloadConfig(t *testing.T) {
24682468
conf := inmemConfig(t)
2469+
conf.LeaderLeaseTimeout = 40 * time.Millisecond
24692470
c := MakeCluster(1, t, conf)
24702471
defer c.Close()
24712472
raft := c.rafts[0]
@@ -2480,6 +2481,8 @@ func TestRaft_ReloadConfig(t *testing.T) {
24802481
TrailingLogs: 12345,
24812482
SnapshotInterval: 234 * time.Second,
24822483
SnapshotThreshold: 6789,
2484+
HeartbeatTimeout: 45 * time.Millisecond,
2485+
ElectionTimeout: 46 * time.Millisecond,
24832486
}
24842487

24852488
require.NoError(t, raft.ReloadConfig(newCfg))
@@ -2488,6 +2491,8 @@ func TestRaft_ReloadConfig(t *testing.T) {
24882491
require.Equal(t, newCfg.TrailingLogs, raft.config().TrailingLogs)
24892492
require.Equal(t, newCfg.SnapshotInterval, raft.config().SnapshotInterval)
24902493
require.Equal(t, newCfg.SnapshotThreshold, raft.config().SnapshotThreshold)
2494+
require.Equal(t, newCfg.HeartbeatTimeout, raft.config().HeartbeatTimeout)
2495+
require.Equal(t, newCfg.ElectionTimeout, raft.config().ElectionTimeout)
24912496
}
24922497

24932498
func TestRaft_ReloadConfigValidates(t *testing.T) {
@@ -2776,3 +2781,37 @@ func TestRaft_runFollower_State_Transition(t *testing.T) {
27762781
})
27772782
}
27782783
}
2784+
2785+
func TestRaft_runFollower_ReloadTimeoutConfigs(t *testing.T) {
2786+
conf := DefaultConfig()
2787+
conf.LocalID = ServerID("first")
2788+
conf.HeartbeatTimeout = 500 * time.Millisecond
2789+
conf.ElectionTimeout = 500 * time.Millisecond
2790+
conf.LeaderLeaseTimeout = 50 * time.Millisecond
2791+
conf.CommitTimeout = 5 * time.Millisecond
2792+
conf.SnapshotThreshold = 100
2793+
conf.TrailingLogs = 10
2794+
conf.skipStartup = true
2795+
2796+
env := MakeRaft(t, conf, false)
2797+
servers := []Server{{Voter, "first", ""}}
2798+
env.raft.setLatestConfiguration(Configuration{Servers: servers}, 1)
2799+
env.raft.setState(Follower)
2800+
2801+
// run the follower loop exclusively
2802+
go env.raft.runFollower()
2803+
2804+
newCfg := ReloadableConfig{
2805+
TrailingLogs: conf.TrailingLogs,
2806+
SnapshotInterval: conf.SnapshotInterval,
2807+
SnapshotThreshold: conf.SnapshotThreshold,
2808+
HeartbeatTimeout: 50 * time.Millisecond,
2809+
ElectionTimeout: 50 * time.Millisecond,
2810+
}
2811+
require.NoError(t, env.raft.ReloadConfig(newCfg))
2812+
// wait enough time to have HeartbeatTimeout
2813+
time.Sleep(3 * newCfg.HeartbeatTimeout)
2814+
2815+
// Check the follower loop set the right state
2816+
require.Equal(t, Candidate, env.raft.getState())
2817+
}

0 commit comments

Comments
 (0)