Skip to content

Commit 42d3446

Browse files
author
Dhia Ayachi
authored
Do not update leader last-contact when granting a pre-vote request (#609)
* do not update leader last-contact when granting a pre-vote request * fix data race and remove extra logs
1 parent 185ae2e commit 42d3446

File tree

2 files changed

+96
-1
lines changed

2 files changed

+96
-1
lines changed

integ_test.go

Lines changed: 96 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -5,6 +5,7 @@ package raft
55

66
import (
77
"bytes"
8+
"context"
89
"fmt"
910
"os"
1011
"sync/atomic"
@@ -490,3 +491,98 @@ func TestRaft_RestartFollower_LongInitialHeartbeat(t *testing.T) {
490491
})
491492
}
492493
}
494+
495+
// TestRaft_PreVote_LeaderSpam test that when a leader spam the followers
496+
// with pre-vote requests they can still transition to candidate.
497+
// The reason this test need to live in here is that we need the transport heartbeat fast-path
498+
// to use as a trick to avoid heartbeat keeping the cluster stable.
499+
// That fast-path only exists in the net transport.
500+
func TestRaft_PreVote_LeaderSpam(t *testing.T) {
501+
CheckInteg(t)
502+
conf := DefaultConfig()
503+
conf.LocalID = ServerID("first")
504+
conf.HeartbeatTimeout = 50 * time.Millisecond
505+
conf.ElectionTimeout = 50 * time.Millisecond
506+
conf.LeaderLeaseTimeout = 50 * time.Millisecond
507+
conf.CommitTimeout = 5 * time.Second
508+
conf.SnapshotThreshold = 100
509+
conf.TrailingLogs = 10
510+
511+
// Create a single node
512+
leader := MakeRaft(t, conf, true)
513+
NoErr(WaitFor(leader, Leader), t)
514+
515+
// Join a few nodes!
516+
var followers []*RaftEnv
517+
for i := 0; i < 2; i++ {
518+
conf.LocalID = ServerID(fmt.Sprintf("next-batch-%d", i))
519+
env := MakeRaft(t, conf, false)
520+
addr := env.trans.LocalAddr()
521+
NoErr(WaitFuture(leader.raft.AddVoter(conf.LocalID, addr, 0, 0)), t)
522+
followers = append(followers, env)
523+
}
524+
525+
// Wait for a leader
526+
_, err := WaitForAny(Leader, append([]*RaftEnv{leader}, followers...))
527+
NoErr(err, t)
528+
529+
CheckConsistent(append([]*RaftEnv{leader}, followers...), t)
530+
531+
leaderT := leader.raft.trans
532+
533+
// spam all the followers with pre-vote requests from the leader
534+
// those requests should be granted as long as the leader haven't changed.
535+
ctx, cancel := context.WithCancel(context.Background())
536+
defer cancel()
537+
go func() {
538+
for {
539+
ticker := time.NewTicker(conf.HeartbeatTimeout / 2)
540+
for _, f := range followers {
541+
rsp := RequestPreVoteResponse{}
542+
reqPreVote := RequestPreVoteRequest{
543+
RPCHeader: leader.raft.getRPCHeader(),
544+
Term: leader.raft.getCurrentTerm() + 1,
545+
LastLogIndex: leader.raft.getLastIndex(),
546+
LastLogTerm: leader.raft.getCurrentTerm(),
547+
}
548+
// We don't need to check the error here because when leader change
549+
// it will start failing with "rejecting pre-vote request since we have a leader"
550+
_ = leaderT.(WithPreVote).RequestPreVote(f.raft.localID, f.raft.localAddr, &reqPreVote, &rsp)
551+
}
552+
select {
553+
case <-ticker.C:
554+
case <-ctx.Done():
555+
return
556+
}
557+
}
558+
}()
559+
time.Sleep(time.Second)
560+
561+
// for all followers ignore heartbeat from current leader, so we can transition to candidate state.
562+
// the purpose of this test is to verify that spamming nodes with pre-votes don't cause them to never
563+
// transition to Candidates.
564+
for _, f := range followers {
565+
//copy f to avoid data race
566+
f1 := f
567+
f1.trans.SetHeartbeatHandler(func(rpc RPC) {
568+
if a, ok := rpc.Command.(*AppendEntriesRequest); ok {
569+
if ServerID(a.GetRPCHeader().ID) == leader.raft.localID {
570+
resp := &AppendEntriesResponse{
571+
RPCHeader: f1.raft.getRPCHeader(),
572+
Term: f1.raft.getCurrentTerm(),
573+
LastLog: f1.raft.getLastIndex(),
574+
Success: false,
575+
NoRetryBackoff: false,
576+
}
577+
rpc.Respond(resp, nil)
578+
} else {
579+
f.raft.processHeartbeat(rpc)
580+
}
581+
}
582+
})
583+
}
584+
time.Sleep(1 * time.Second)
585+
// New leader should be one of the former followers.
586+
_, err = WaitForAny(Leader, followers)
587+
NoErr(err, t)
588+
}

raft.go

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1805,7 +1805,6 @@ func (r *Raft) requestPreVote(rpc RPC, req *RequestPreVoteRequest) {
18051805
}
18061806

18071807
resp.Granted = true
1808-
r.setLastContact()
18091808
}
18101809

18111810
// installSnapshot is invoked when we get a InstallSnapshot RPC call.

0 commit comments

Comments
 (0)