RATIS-2428. Allow installation of Snapshot from followers by spacemonkd · Pull Request #1370 · apache/ratis
What changes were proposed in this pull request?
Problem
In the current implementation, the leader provides the snapshot. However this causes tasks to get paused until the snapshot installation is completed and also puts unnecessary pressure on the leader.
Goal
Allow a lagging follower to install a snapshot from another follower without making that source follower act as the leader. This will let the lagging follower stay either in sync or "catch-up" to the point where it can append new entries without a complete snapshot.
High Level Flow Diagram
How do we select the follower source?
For selecting the follower source we consider the following metrics / conditions.
Inputs
- Target follower (
T) which needs snapshot - Leader log state - specifically
lastEntry,logStartIndexandfirstAvailableTermIndex - Current follower progresses - specifically
matchIndex,commitIndex,snapshotIndex,lastRespondedAppendEntriesSendTimeandlastRpcResponseTime.
Eligibility
A follower (F) is considered a source only if:
Fis recently responsive on the append path, for this we can uselastRespondedAppendEntriesSendTimeas a check and fallback tolastRpcResponseTime.F.matchIndex >= requiredSnapshotIndexwhererequiredSnapshotIndex = firstAvailableTermIndex.index - 1
This is because:
requiredSnapshotIndexis the minimum snapshot index that still lets the target resume normal AppendEntries from the leader after install.- If
F.matchIndex < requiredSnapshotIndex, that follower is too far behind to bridge the leader's log gap for this target, so the leader should not choose it.
Ranking
Rank eligible followers by this lexicographic order:
- Exact sync with leader i.e. it is fully caught up to the leader
- Highest
matchIndex - Highest
commitIndexin case match index is tied - Freshest
lastRespondedAppendEntriesSendTime
IMPORTANT: If no follower satisfies matchIndex >= requiredSnapshotIndex, do not attempt follower-sourced install. Fall back immediately to the existing leader path because otherwise the target follower will need to perform another snapshot install to catchup anyway.
What is the link to the Apache JIRA
https://issues.apache.org/jira/browse/RATIS-2428
How was this patch tested?
Patch was tested using the unit tests.
