Skip to content

Commit eb6bfc6

Browse files
committed
raft: advance commit index safely
This change makes the commit index advancement in handleHeartbeat safe. Previously, a follower would attempt to update the commit index to whichever was sent in the MsgHeartbeat message. Out-of-bound indices would crash the node. It is always safe to advance a commit index if the follower's log is "in sync" with the leader, i.e. when its log is guaranteed to be a prefix of the leader's log. This becomes true when the first MsgApp append message succeeds. At the moment, the leader will never send a commit index that exceeds the follower's log size. However, this may change in future. This change is a defence-in-depth. Signed-off-by: Pavel Kalinnikov <[email protected]>
1 parent 026484c commit eb6bfc6

File tree

3 files changed

+27
-5
lines changed

3 files changed

+27
-5
lines changed

raft.go

+13-1
Original file line numberDiff line numberDiff line change
@@ -375,6 +375,11 @@ type raft struct {
375375

376376
// the leader id
377377
lead uint64
378+
// logSynced is true if this node's log is guaranteed to be a prefix of the
379+
// leader's log at this term. Always true for the leader. Always false for a
380+
// candidate. For a follower, becomes true the first time a MsgApp append to
381+
// the log succeeds.
382+
logSynced bool
378383
// leadTransferee is id of the leader transfer target when its value is not zero.
379384
// Follow the procedure defined in raft thesis 3.10.
380385
leadTransferee uint64
@@ -763,6 +768,7 @@ func (r *raft) reset(term uint64) {
763768
r.Vote = None
764769
}
765770
r.lead = None
771+
r.logSynced = false
766772

767773
r.electionElapsed = 0
768774
r.heartbeatElapsed = 0
@@ -908,6 +914,7 @@ func (r *raft) becomeLeader() {
908914
r.reset(r.Term)
909915
r.tick = r.tickHeartbeat
910916
r.lead = r.id
917+
r.logSynced = true // the leader's log is in sync with itself
911918
r.state = StateLeader
912919
// Followers enter replicate mode when they've been successfully probed
913920
// (perhaps after having received a snapshot as a result). The leader is
@@ -1735,6 +1742,7 @@ func (r *raft) handleAppendEntries(m pb.Message) {
17351742
return
17361743
}
17371744
if mlastIndex, ok := r.raftLog.maybeAppend(m.Index, m.LogTerm, m.Commit, m.Entries...); ok {
1745+
r.logSynced = true // from now on, the log is a prefix of the leader's log
17381746
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: mlastIndex})
17391747
return
17401748
}
@@ -1770,7 +1778,11 @@ func (r *raft) handleAppendEntries(m pb.Message) {
17701778
}
17711779

17721780
func (r *raft) handleHeartbeat(m pb.Message) {
1773-
r.raftLog.commitTo(m.Commit)
1781+
// If our log is not a prefix of the leader's log, it is unsafe to advance the
1782+
// commit index, because the entries at this index may mismatch.
1783+
if r.logSynced {
1784+
r.raftLog.commitTo(min(m.Commit, r.raftLog.lastIndex()))
1785+
}
17741786
r.send(pb.Message{To: m.From, Type: pb.MsgHeartbeatResp, Context: m.Context})
17751787
}
17761788

raft_test.go

+13-4
Original file line numberDiff line numberDiff line change
@@ -1332,11 +1332,18 @@ func TestHandleMsgApp(t *testing.T) {
13321332
func TestHandleHeartbeat(t *testing.T) {
13331333
commit := uint64(2)
13341334
tests := []struct {
1335-
m pb.Message
1336-
wCommit uint64
1335+
m pb.Message
1336+
logSynced bool
1337+
wCommit uint64
13371338
}{
1338-
{pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeat, Term: 2, Commit: commit + 1}, commit + 1},
1339-
{pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeat, Term: 2, Commit: commit - 1}, commit}, // do not decrease commit
1339+
{pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeat, Term: 2, Commit: commit + 1}, true, commit + 1},
1340+
{pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeat, Term: 2, Commit: commit - 1}, true, commit}, // do not decrease commit
1341+
{pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeat, Term: 2, Commit: commit - 1}, false, commit},
1342+
1343+
// Increase the commit index only if the log is in sync with the leader.
1344+
{pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeat, Term: 2, Commit: commit + 1}, false, commit},
1345+
// Do not increase the commit index beyond our log size.
1346+
{pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeat, Term: 2, Commit: commit + 10}, true, commit + 1},
13401347
}
13411348

13421349
for i, tt := range tests {
@@ -1345,6 +1352,8 @@ func TestHandleHeartbeat(t *testing.T) {
13451352
sm := newTestRaft(1, 5, 1, storage)
13461353
sm.becomeFollower(2, 2)
13471354
sm.raftLog.commitTo(commit)
1355+
sm.logSynced = tt.logSynced
1356+
13481357
sm.handleHeartbeat(tt.m)
13491358
if sm.raftLog.committed != tt.wCommit {
13501359
t.Errorf("#%d: committed = %d, want %d", i, sm.raftLog.committed, tt.wCommit)

rawnode_test.go

+1
Original file line numberDiff line numberDiff line change
@@ -952,6 +952,7 @@ func TestRawNodeCommitPaginationAfterRestart(t *testing.T) {
952952
if err != nil {
953953
t.Fatal(err)
954954
}
955+
rawNode.raft.logSynced = true // needed to be able to advance the commit index
955956

956957
for highestApplied := uint64(0); highestApplied != 11; {
957958
rd := rawNode.Ready()

0 commit comments

Comments
 (0)