Skip to content

Commit

Permalink
raft,tracker: track commit index of each follower
Browse files Browse the repository at this point in the history
This commit closes a gap in commit index tracking.

Previously, the leader did not precisely know what commit index the
follower is at, and always had to send an empty MsgApp to brind it up to
date if it's not.

With this commit, followers now send the commit index of their logs back
to the leader, and the leader tracks each follower's commit index. This
will allow the leader (see other commits) to send an empty MsgApp with a
commit index update only if the tracked index is behind, which will
reduce the number of unnecessary message in the system.

Signed-off-by: Pavel Kalinnikov <[email protected]>
  • Loading branch information
pav-kv committed Jan 25, 2024
1 parent 0a540c9 commit a4d4ec5
Show file tree
Hide file tree
Showing 22 changed files with 331 additions and 317 deletions.
12 changes: 8 additions & 4 deletions raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -815,7 +815,7 @@ func (r *raft) appendEntry(es ...pb.Entry) (accepted bool) {
// if r.maybeCommit() {
// r.bcastAppend()
// }
r.send(pb.Message{To: r.id, Type: pb.MsgAppResp, Index: li})
r.send(pb.Message{To: r.id, Type: pb.MsgAppResp, Index: li, Commit: r.raftLog.committed})
return true
}

Expand Down Expand Up @@ -1465,6 +1465,7 @@ func stepLeader(r *raft, m pb.Message) error {
r.sendAppend(m.From)
}
} else {
pr.UpdateCommit(m.Commit)
oldPaused := pr.IsPaused()
// We want to update our tracking if the response updates our
// matched index or if the response can move a probing peer back
Expand Down Expand Up @@ -1505,6 +1506,8 @@ func stepLeader(r *raft, m pb.Message) error {
} else if oldPaused {
// If we were paused before, this node may be missing the
// latest commit index, so send it.
// TODO(pav-kv): remove this branch, and decide on sending the commit
// index update based on pr.Commit.
r.sendAppend(m.From)
}
// We've updated flow control information above, which may
Expand Down Expand Up @@ -1730,12 +1733,13 @@ func stepFollower(r *raft, m pb.Message) error {
}

func (r *raft) handleAppendEntries(m pb.Message) {
if m.Index < r.raftLog.committed {
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
if committed := r.raftLog.committed; m.Index < committed {
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: committed, Commit: committed})
return
}
if mlastIndex, ok := r.raftLog.maybeAppend(m.Index, m.LogTerm, m.Commit, m.Entries...); ok {
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: mlastIndex})
commit := r.raftLog.committed
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: mlastIndex, Commit: commit})
return
}
r.logger.Debugf("%x [logterm: %d, index: %d] rejected MsgApp [logterm: %d, index: %d] from %x",
Expand Down
142 changes: 71 additions & 71 deletions testdata/async_storage_writes.txt

Large diffs are not rendered by default.

18 changes: 9 additions & 9 deletions testdata/async_storage_writes_append_aba_race.txt
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ Messages:
2->5 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"]
2->6 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"]
2->7 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "init_prop"]
2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "init_prop"] Responses:[2->2 MsgAppResp Term:1 Log:0/12, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/12]
2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "init_prop"] Responses:[2->2 MsgAppResp Term:1 Log:0/12 Commit:11, AppendThread->2 MsgStorageAppendResp Term:1 Log:1/12]

deliver-msgs 1 drop=(3,4,5,6,7)
----
Expand All @@ -61,7 +61,7 @@ Ready MustSync=true:
Entries:
1/12 EntryNormal "init_prop"
Messages:
1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "init_prop"] Responses:[1->2 MsgAppResp Term:1 Log:0/12, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12]
1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "init_prop"] Responses:[1->2 MsgAppResp Term:1 Log:0/12 Commit:11, AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12]

# Step 4: node 3 becomes the leader after getting a vote from nodes 4, 5, and 6.

Expand Down Expand Up @@ -183,7 +183,7 @@ Messages:
3->5 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""]
3->6 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""]
3->7 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""]
3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[2/12 EntryNormal ""] Responses:[3->3 MsgAppResp Term:2 Log:0/12, AppendThread->3 MsgStorageAppendResp Term:2 Log:2/12]
3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[2/12 EntryNormal ""] Responses:[3->3 MsgAppResp Term:2 Log:0/12 Commit:11, AppendThread->3 MsgStorageAppendResp Term:2 Log:2/12]

deliver-msgs 1 drop=(2,4,5,6,7)
----
Expand All @@ -210,7 +210,7 @@ HardState Term:2 Commit:11
Entries:
2/12 EntryNormal ""
Messages:
1->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Entries:[2/12 EntryNormal ""] Responses:[1->3 MsgVoteResp Term:2 Log:0/0 Rejected (Hint: 0), 1->3 MsgAppResp Term:2 Log:0/12, AppendThread->1 MsgStorageAppendResp Term:2 Log:2/12]
1->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Entries:[2/12 EntryNormal ""] Responses:[1->3 MsgVoteResp Term:2 Log:0/0 Rejected (Hint: 0), 1->3 MsgAppResp Term:2 Log:0/12 Commit:11, AppendThread->1 MsgStorageAppendResp Term:2 Log:2/12]

# Step 6: node 3 crashes and node 4 becomes leader getting the vote from 5, 6, and 7.

Expand Down Expand Up @@ -325,7 +325,7 @@ Messages:
4->5 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""]
4->6 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""]
4->7 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""]
4->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[3/12 EntryNormal ""] Responses:[4->4 MsgAppResp Term:3 Log:0/12, AppendThread->4 MsgStorageAppendResp Term:3 Log:3/12]
4->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[3/12 EntryNormal ""] Responses:[4->4 MsgAppResp Term:3 Log:0/12 Commit:11, AppendThread->4 MsgStorageAppendResp Term:3 Log:3/12]

# Step 7: before the new entries reach node 1, it hears of the term change
# through a heartbeat and persists the new term. Node 1 then receives these
Expand Down Expand Up @@ -390,7 +390,7 @@ Ready MustSync=true:
Entries:
3/12 EntryNormal ""
Messages:
1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[3/12 EntryNormal ""] Responses:[1->4 MsgAppResp Term:3 Log:0/12, AppendThread->1 MsgStorageAppendResp Term:3 Log:3/12]
1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[3/12 EntryNormal ""] Responses:[1->4 MsgAppResp Term:3 Log:0/12 Commit:11, AppendThread->1 MsgStorageAppendResp Term:3 Log:3/12]

# Step 8: The asynchronous log appends from the first Ready complete and the
# MsgStorageAppendResp is returned to the raft node state machine. A decision
Expand All @@ -405,7 +405,7 @@ process-append-thread 1
Processing:
1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/12 EntryNormal "init_prop"]
Responses:
1->2 MsgAppResp Term:1 Log:0/12
1->2 MsgAppResp Term:1 Log:0/12 Commit:11
AppendThread->1 MsgStorageAppendResp Term:1 Log:1/12

raft-log 1
Expand All @@ -432,7 +432,7 @@ Processing:
1->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Entries:[2/12 EntryNormal ""]
Responses:
1->3 MsgVoteResp Term:2 Log:0/0 Rejected (Hint: 0)
1->3 MsgAppResp Term:2 Log:0/12
1->3 MsgAppResp Term:2 Log:0/12 Commit:11
AppendThread->1 MsgStorageAppendResp Term:2 Log:2/12

raft-log 1
Expand Down Expand Up @@ -467,7 +467,7 @@ process-append-thread 1
Processing:
1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[3/12 EntryNormal ""]
Responses:
1->4 MsgAppResp Term:3 Log:0/12
1->4 MsgAppResp Term:3 Log:0/12 Commit:11
AppendThread->1 MsgStorageAppendResp Term:3 Log:3/12

raft-log 1
Expand Down
16 changes: 8 additions & 8 deletions testdata/campaign.txt
Original file line number Diff line number Diff line change
Expand Up @@ -76,17 +76,17 @@ stabilize
Entries:
1/3 EntryNormal ""
Messages:
2->1 MsgAppResp Term:1 Log:0/3
2->1 MsgAppResp Term:1 Log:0/3 Commit:2
> 3 handling Ready
Ready MustSync=true:
Lead:1 State:StateFollower
Entries:
1/3 EntryNormal ""
Messages:
3->1 MsgAppResp Term:1 Log:0/3
3->1 MsgAppResp Term:1 Log:0/3 Commit:2
> 1 receiving messages
2->1 MsgAppResp Term:1 Log:0/3
3->1 MsgAppResp Term:1 Log:0/3
2->1 MsgAppResp Term:1 Log:0/3 Commit:2
3->1 MsgAppResp Term:1 Log:0/3 Commit:2
> 1 handling Ready
Ready MustSync=false:
HardState Term:1 Vote:1 Commit:3
Expand All @@ -105,14 +105,14 @@ stabilize
CommittedEntries:
1/3 EntryNormal ""
Messages:
2->1 MsgAppResp Term:1 Log:0/3
2->1 MsgAppResp Term:1 Log:0/3 Commit:3
> 3 handling Ready
Ready MustSync=false:
HardState Term:1 Vote:1 Commit:3
CommittedEntries:
1/3 EntryNormal ""
Messages:
3->1 MsgAppResp Term:1 Log:0/3
3->1 MsgAppResp Term:1 Log:0/3 Commit:3
> 1 receiving messages
2->1 MsgAppResp Term:1 Log:0/3
3->1 MsgAppResp Term:1 Log:0/3
2->1 MsgAppResp Term:1 Log:0/3 Commit:3
3->1 MsgAppResp Term:1 Log:0/3 Commit:3
8 changes: 4 additions & 4 deletions testdata/campaign_learner_must_vote.txt
Original file line number Diff line number Diff line change
Expand Up @@ -129,10 +129,10 @@ stabilize 2 3
CommittedEntries:
1/4 EntryConfChangeV2 v3
Messages:
3->2 MsgAppResp Term:2 Log:0/5
3->2 MsgAppResp Term:2 Log:0/5 Commit:4
INFO 3 switched to configuration voters=(1 2 3)
> 2 receiving messages
3->2 MsgAppResp Term:2 Log:0/5
3->2 MsgAppResp Term:2 Log:0/5 Commit:4
> 2 handling Ready
Ready MustSync=false:
HardState Term:2 Vote:2 Commit:5
Expand All @@ -148,6 +148,6 @@ stabilize 2 3
CommittedEntries:
2/5 EntryNormal ""
Messages:
3->2 MsgAppResp Term:2 Log:0/5
3->2 MsgAppResp Term:2 Log:0/5 Commit:5
> 2 receiving messages
3->2 MsgAppResp Term:2 Log:0/5
3->2 MsgAppResp Term:2 Log:0/5 Commit:5
16 changes: 8 additions & 8 deletions testdata/checkquorum.txt
Original file line number Diff line number Diff line change
Expand Up @@ -193,18 +193,18 @@ stabilize
Entries:
3/12 EntryNormal ""
Messages:
1->2 MsgAppResp Term:3 Log:0/12
1->2 MsgAppResp Term:3 Log:0/12 Commit:11
> 3 handling Ready
Ready MustSync=true:
Lead:2 State:StateFollower
HardState Term:3 Commit:11
Entries:
3/12 EntryNormal ""
Messages:
3->2 MsgAppResp Term:3 Log:0/12
3->2 MsgAppResp Term:3 Log:0/12 Commit:11
> 2 receiving messages
1->2 MsgAppResp Term:3 Log:0/12
3->2 MsgAppResp Term:3 Log:0/12
1->2 MsgAppResp Term:3 Log:0/12 Commit:11
3->2 MsgAppResp Term:3 Log:0/12 Commit:11
> 2 handling Ready
Ready MustSync=false:
HardState Term:3 Vote:2 Commit:12
Expand All @@ -223,14 +223,14 @@ stabilize
CommittedEntries:
3/12 EntryNormal ""
Messages:
1->2 MsgAppResp Term:3 Log:0/12
1->2 MsgAppResp Term:3 Log:0/12 Commit:12
> 3 handling Ready
Ready MustSync=false:
HardState Term:3 Commit:12
CommittedEntries:
3/12 EntryNormal ""
Messages:
3->2 MsgAppResp Term:3 Log:0/12
3->2 MsgAppResp Term:3 Log:0/12 Commit:12
> 2 receiving messages
1->2 MsgAppResp Term:3 Log:0/12
3->2 MsgAppResp Term:3 Log:0/12
1->2 MsgAppResp Term:3 Log:0/12 Commit:12
3->2 MsgAppResp Term:3 Log:0/12 Commit:12
4 changes: 2 additions & 2 deletions testdata/confchange_v1_add_single.txt
Original file line number Diff line number Diff line change
Expand Up @@ -101,6 +101,6 @@ stabilize
> 2 handling Ready
Ready MustSync=false:
Messages:
2->1 MsgAppResp Term:1 Log:0/4
2->1 MsgAppResp Term:1 Log:0/4 Commit:4
> 1 receiving messages
2->1 MsgAppResp Term:1 Log:0/4
2->1 MsgAppResp Term:1 Log:0/4 Commit:4
48 changes: 24 additions & 24 deletions testdata/confchange_v1_remove_leader.txt
Original file line number Diff line number Diff line change
Expand Up @@ -70,8 +70,8 @@ stabilize 2
1/4 EntryConfChange r1
1/5 EntryNormal "foo"
Messages:
2->1 MsgAppResp Term:1 Log:0/4
2->1 MsgAppResp Term:1 Log:0/5
2->1 MsgAppResp Term:1 Log:0/4 Commit:3
2->1 MsgAppResp Term:1 Log:0/5 Commit:3

# Put another entry in n1's log.
propose 1 bar
Expand All @@ -92,8 +92,8 @@ stabilize 1
1->2 MsgApp Term:1 Log:1/5 Commit:3 Entries:[1/6 EntryNormal "bar"]
1->3 MsgApp Term:1 Log:1/5 Commit:3 Entries:[1/6 EntryNormal "bar"]
> 1 receiving messages
2->1 MsgAppResp Term:1 Log:0/4
2->1 MsgAppResp Term:1 Log:0/5
2->1 MsgAppResp Term:1 Log:0/4 Commit:3
2->1 MsgAppResp Term:1 Log:0/5 Commit:3
> 1 handling Ready
Ready MustSync=false:
HardState Term:1 Vote:1 Commit:5
Expand Down Expand Up @@ -129,18 +129,18 @@ stabilize 2
1/4 EntryConfChange r1
1/5 EntryNormal "foo"
Messages:
2->1 MsgAppResp Term:1 Log:0/6
2->1 MsgAppResp Term:1 Log:0/6
2->1 MsgAppResp Term:1 Log:0/6
2->1 MsgAppResp Term:1 Log:0/6 Commit:3
2->1 MsgAppResp Term:1 Log:0/6 Commit:4
2->1 MsgAppResp Term:1 Log:0/6 Commit:5
INFO 2 switched to configuration voters=(2 3)

# ... which thankfully is what we see on the leader.
stabilize 1
----
> 1 receiving messages
2->1 MsgAppResp Term:1 Log:0/6
2->1 MsgAppResp Term:1 Log:0/6
2->1 MsgAppResp Term:1 Log:0/6
2->1 MsgAppResp Term:1 Log:0/6 Commit:3
2->1 MsgAppResp Term:1 Log:0/6 Commit:4
2->1 MsgAppResp Term:1 Log:0/6 Commit:5

# When n3 responds, quorum is reached and everything falls into place.
stabilize
Expand All @@ -162,18 +162,18 @@ stabilize
1/4 EntryConfChange r1
1/5 EntryNormal "foo"
Messages:
3->1 MsgAppResp Term:1 Log:0/4
3->1 MsgAppResp Term:1 Log:0/5
3->1 MsgAppResp Term:1 Log:0/6
3->1 MsgAppResp Term:1 Log:0/6
3->1 MsgAppResp Term:1 Log:0/6
3->1 MsgAppResp Term:1 Log:0/4 Commit:3
3->1 MsgAppResp Term:1 Log:0/5 Commit:3
3->1 MsgAppResp Term:1 Log:0/6 Commit:3
3->1 MsgAppResp Term:1 Log:0/6 Commit:4
3->1 MsgAppResp Term:1 Log:0/6 Commit:5
INFO 3 switched to configuration voters=(2 3)
> 1 receiving messages
3->1 MsgAppResp Term:1 Log:0/4
3->1 MsgAppResp Term:1 Log:0/5
3->1 MsgAppResp Term:1 Log:0/6
3->1 MsgAppResp Term:1 Log:0/6
3->1 MsgAppResp Term:1 Log:0/6
3->1 MsgAppResp Term:1 Log:0/4 Commit:3
3->1 MsgAppResp Term:1 Log:0/5 Commit:3
3->1 MsgAppResp Term:1 Log:0/6 Commit:3
3->1 MsgAppResp Term:1 Log:0/6 Commit:4
3->1 MsgAppResp Term:1 Log:0/6 Commit:5
> 1 handling Ready
Ready MustSync=false:
HardState Term:1 Vote:1 Commit:6
Expand All @@ -192,17 +192,17 @@ stabilize
CommittedEntries:
1/6 EntryNormal "bar"
Messages:
2->1 MsgAppResp Term:1 Log:0/6
2->1 MsgAppResp Term:1 Log:0/6 Commit:6
> 3 handling Ready
Ready MustSync=false:
HardState Term:1 Vote:1 Commit:6
CommittedEntries:
1/6 EntryNormal "bar"
Messages:
3->1 MsgAppResp Term:1 Log:0/6
3->1 MsgAppResp Term:1 Log:0/6 Commit:6
> 1 receiving messages
2->1 MsgAppResp Term:1 Log:0/6
3->1 MsgAppResp Term:1 Log:0/6
2->1 MsgAppResp Term:1 Log:0/6 Commit:6
3->1 MsgAppResp Term:1 Log:0/6 Commit:6

# However not all is well. n1 is still leader but unconditionally drops all
# proposals on the floor, so we're effectively stuck if it still heartbeats
Expand Down
Loading

0 comments on commit a4d4ec5

Please sign in to comment.