Skip to content

RATIS-2428. Allow installation of Snapshot from followers#1370

Open
spacemonkd wants to merge 5 commits intoapache:masterfrom
spacemonkd:RATIS-2428
Open

RATIS-2428. Allow installation of Snapshot from followers#1370
spacemonkd wants to merge 5 commits intoapache:masterfrom
spacemonkd:RATIS-2428

Conversation

@spacemonkd
Copy link
Copy Markdown
Contributor

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

image

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, logStartIndex and firstAvailableTermIndex
  • Current follower progresses - specifically matchIndex, commitIndex, snapshotIndex, lastRespondedAppendEntriesSendTime and lastRpcResponseTime.

Eligibility

A follower (F) is considered a source only if:

  • F is recently responsive on the append path, for this we can use lastRespondedAppendEntriesSendTime as a check and fallback to lastRpcResponseTime.
  • F.matchIndex >= requiredSnapshotIndex where requiredSnapshotIndex = firstAvailableTermIndex.index - 1

This is because:

  • requiredSnapshotIndex is 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 commitIndex in 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.

@spacemonkd spacemonkd marked this pull request as draft March 9, 2026 15:33
@spacemonkd spacemonkd marked this pull request as ready for review March 9, 2026 15:59
@spacemonkd
Copy link
Copy Markdown
Contributor Author

Hi @szetszwo, could you take a look at the proposal and code changes?
Also do you think it might be a good idea to update snapshot.md and put this behind a config?

I wasn't sure what config category to put this under

@szetszwo
Copy link
Copy Markdown
Contributor

szetszwo commented Apr 2, 2026

@spacemonkd , thanks for submitting this PR. It is a great idea!

Before making code change, we need to think about:

  • Compatibility: what happens if new leader talk to an old follower, or the other way around.
  • Conf raft.server.log.appender.install.snapshot.enabled: what should happen if it is enabled/disabled ?

BTW, this change is too big. Let's break it down to subtasks.

@jojochuang
Copy link
Copy Markdown
Contributor

Ironically, we want to disallow installing snapshots from follower: apache/ozone#9873

@spacemonkd
Copy link
Copy Markdown
Contributor Author

@jojochuang, we will probably be keeping this behind a configuration.
My idea is that followers can be used to fetch snapshot from in order to put less pressure on leader.

This means that one follower can catch up to another follower which would allow it to get delta updates from the leader while letting the leader keep working

However if my understanding is wrong I would be really grateful for more inputs

@spacemonkd
Copy link
Copy Markdown
Contributor Author

@szetszwo what are your thoughts on this?
Should I continue with this? It might be good to understand why we are removing this in Ozone

@szetszwo
Copy link
Copy Markdown
Contributor

szetszwo commented Apr 7, 2026

@spacemonkd , It actually is more useful to allow appendEntries from followers, especially for listeners to listen from followers. Similar to this change, it will not be a small change. If you have time, we could start that.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants