Ivan Andika created RATIS-2500:
----------------------------------

             Summary: Infinite notify InstallsSnapshot
                 Key: RATIS-2500
                 URL: https://issues.apache.org/jira/browse/RATIS-2500
             Project: Ratis
          Issue Type: Bug
            Reporter: Ivan Andika


Found in a test where cluster is stuck in infinite snapshot loop where the 
leader keeps sending install snapshot notifications, and the follower keeps 
replying ALREADY_INSTALLED

Problematic code in 
SnapshotInstallationHandler#notifyStateMachineToInstallSnapshot

 
{code:java}
if (snapshotIndex != INVALID_LOG_INDEX && snapshotIndex + 1 >= 
firstAvailableLogIndex &&
    firstAvailableLogIndex > INVALID_LOG_INDEX) {
  // State Machine has already installed the snapshot. Return the
  // latest snapshot index to the Leader.

  inProgressInstallSnapshotIndex.compareAndSet(firstAvailableLogIndex, 
INVALID_LOG_INDEX);
  LOG.info("{}: InstallSnapshot notification result: {}, current snapshot 
index: {}", getMemberId(),
      InstallSnapshotResult.ALREADY_INSTALLED, snapshotIndex);
  final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, 
getMemberId(), currentTerm,
      InstallSnapshotResult.ALREADY_INSTALLED, snapshotIndex);
  return future.thenApply(dummy -> reply);
} {code}


The flow

 
 # Initial state: omNode-1 is leader, processing transactions. 
SNAPSHOT_THRESHOLD = 50 and LOG_PURGE_GAP = 50.
 # `omNode-1` shuts down: Before shutdown, takeSnapshot() is called. The 
snapshotInfo is updated with the current lastAppliedTermIndex. The 
TransactionInfo is written to RocksDB.
 # `omNode-3` becomes leader: The cluster continues processing transactions. 
After 50 more transactions, omNode-3 takes a snapshot and purges old log 
entries. The leader's firstAvailableLogIndex advances.
 # omNode-1` restarts: During initialization:
    • loadSnapshotInfoFromDB() reads TransactionInfo from RocksDB and sets 
snapshotInfo to the last applied index before shutdown (e.g., index 0 or some 
small value from term 1).
    • getLatestSnapshot() returns this snapshotInfo.
    • The RaftLogBase constructor calls getSnapshotIndexFromStateMachine which 
returns snapshotInfo.getIndex() (e.g., 0).
    • Both commitIndex and snapshotIndex in the Raft log are initialized to 
this value (0).
 # Leader sends install snapshot notification: omNode-3 detects omNode-1 is 
behind and sends installSnapshot with firstAvailableLogIndex = 1 (the first 
available log entry in the leader's log after purging).
 # In notifyStateMachineToInstallSnapshot if condition returns true 
(snapshotIndex (0) != INVALID_LOG_INDEX (-1)  → true, snapshotIndex + 1 (1) >= 
firstAvailableLogIndex (1)  → true, firstAvailableLogIndex (1) > 
INVALID_LOG_INDEX (-1)  → true
 # Follower returns already ALREADY_INSTALLED with snapshotIndex = 0. This is 
incorrect because omNode-1's snapshot at index 0 is from term 1, while the 
leader is in term 2 with entries starting at index 1. The follower has NOT 
actually applied the entries that the leader has, it just happens that its 
local snapshot index (0) satisfies the arithmetic condition 0 + 1 >= 1

The Core Problem

  The ALREADY_INSTALLED check only compares indices without considering terms. 
The follower's snapshotIndex of 0 (from term 1) is treated as equivalent to the 
leader's firstAvailableLogIndex of 1 (from term 2), even though they
  represent completely different states. The condition snapshotIndex + 1 >= 
firstAvailableLogIndex is a heuristic that assumes if the follower's snapshot 
is at or past the leader's first available log entry, the follower doesn't need
  the snapshot. But this breaks when:
 * The follower's snapshot is from a previous term
 * The leader's log has been purged to a very low index (like 1) in a new term
 * The follower missed all entries in the new term

The fix should be to also take into account the snapshot term when comparing 
snapshot, instead of only snapshot index.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to