Append messages [MsgApp] that leader sends to a follower mainly serve 3 purposes:
Update the follower's log. The message contains a batch of entries extending the log.
Update the "commit index" for a follower whose commit index might be behind.
Learn the follower's log state.
Upon a successful MsgApp processing, the follower sends a MsgAppResp back to the leader, confirming the new size of its log. Note that it does not confirm the update of the commit index.
Problem
The follower can be completely up-to-date (both the log and the commit index), but the leader can still send empty MsgApp for purpose (2), to update the commit index (e.g. here). The conditions for sending these empty messages are brittle and probabilistic.
Symmetrically, sometimes the leader does not choose to update the follower's commit index even though the latter is behind. The leader does not have precise understanding of where the follower's commit index is.
It would be good to have a simpler condition for sending commit index update messages, such that it is hard to get wrong and doesn't require guessing the follower's state. This will lead to: a) fewer unnecessary MsgApp messages, b) faster convergence of the follower's commit index in some cases.
Solution
Observe that it is only necessary to send a commit index update to a follower if the follower's commit index is behind. This is because the commit index never regresses (it is stored in HardState).
TODO: need to double-check whether storing commit index is synced before we send a reply. If not, the solution here is not completely working, and still relies on heartbeats to fill the gaps. To make it work, we should be sending the HardState.Commit in the message, rather than raftLog.commit.
The conditions for sending an empty MsgApp with a commit index update will become simpler if the leader tracks the follower's commit index (similarly to how it tracks the follower's log size for deciding when to send new log entries).
To support this, the follower should notify the leader of the Commit index. It should be as simple as setting the Commit field of the MsgAppResp message that it sends to the leader.
Background
Append messages [MsgApp] that leader sends to a follower mainly serve 3 purposes:
Upon a successful
MsgApp
processing, the follower sends aMsgAppResp
back to the leader, confirming the new size of its log. Note that it does not confirm the update of the commit index.Problem
The follower can be completely up-to-date (both the log and the commit index), but the leader can still send empty
MsgApp
for purpose (2), to update the commit index (e.g. here). The conditions for sending these empty messages are brittle and probabilistic.Symmetrically, sometimes the leader does not choose to update the follower's commit index even though the latter is behind. The leader does not have precise understanding of where the follower's commit index is.
It would be good to have a simpler condition for sending commit index update messages, such that it is hard to get wrong and doesn't require guessing the follower's state. This will lead to: a) fewer unnecessary
MsgApp
messages, b) faster convergence of the follower's commit index in some cases.Solution
Observe that it is only necessary to send a commit index update to a follower if the follower's commit index is behind. This is because the commit index never regresses (it is stored in
HardState
).TODO: need to double-check whether storing commit index is synced before we send a reply. If not, the solution here is not completely working, and still relies on heartbeats to fill the gaps. To make it work, we should be sending the
HardState.Commit
in the message, rather thanraftLog.commit
.The conditions for sending an empty
MsgApp
with a commit index update will become simpler if the leader tracks the follower's commit index (similarly to how it tracks the follower's log size for deciding when to send new log entries).To support this, the follower should notify the leader of the
Commit
index. It should be as simple as setting theCommit
field of theMsgAppResp
message that it sends to the leader.