[ 
https://issues.apache.org/jira/browse/RATIS-2487?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ivan Andika updated RATIS-2487:
-------------------------------
    Description: 
Found the following logs in tests
{code:java}
java.lang.NullPointerException: omNode-2@group-523986131536->omNode-1: Previous 
TermIndex not found for firstIndex = 93 at 
java.base/java.util.Objects.requireNonNull(Objects.java:360) at 
org.apache.ratis.server.leader.LogAppenderBase.assertProtos(LogAppenderBase.java:270)
        at 
org.apache.ratis.server.leader.LogAppenderBase.newAppendEntriesRequest(LogAppenderBase.java:255)
     at 
org.apache.ratis.grpc.server.GrpcLogAppender.appendLog(GrpcLogAppender.java:387)
     at 
org.apache.ratis.grpc.server.GrpcLogAppender.run(GrpcLogAppender.java:262)   at 
org.apache.ratis.server.leader.LogAppenderDaemon.run(LogAppenderDaemon.java:80) 
     at java.base/java.lang.Thread.run(Thread.java:1583) {code}
Seems to be related to this Ratis NPE when there is no previous log (when the 
logs are purged). After RATIS-2427, NPE will not cause LogAppender to be 
restarted. This can cause the leader to not send anything anymore and can cause 
the Raft group to be stuck.

 

AI Analysis

Is “previous log already purged” a valid explanation?

  Yes. The failure is thrown when the leader builds an AppendEntries request 
with non-empty entries and previous is null.
  getPrevious(nextIndex) does this:

   
ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java 
lines 165-185

    private TermIndex getPrevious(long nextIndex) {
     
{code:java}
if (nextIndex == RaftLog.LEAST_VALID_LOG_INDEX)
{         return null;       }
      final long previousIndex = nextIndex - 1;
      final TermIndex previous = getRaftLog().getTermIndex(previousIndex);
      if (previous != null) {         return previous;       }
      final SnapshotInfo snapshot = 
server.getStateMachine().getLatestSnapshot();
      if (snapshot != null) {
        final TermIndex snapshotTermIndex = snapshot.getTermIndex();
        if (snapshotTermIndex.getIndex() == previousIndex) {           return 
snapshotTermIndex;         }
      }
      return null;
    }
{code}
  So previous is null only if:
  1. Index nextIndex - 1 is not in the leader’s raft log (e.g. purged / before 
getStartIndex()), and
  2. The leader’s latest snapshot term index is not exactly nextIndex - 1.

  Then assertProtos requires a non-null previous (unless a specific follower 
snapshot boundary applies):

   
ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java 
lines 256-273

 
{code:java}
private void assertProtos(List<LogEntryProto> protos, long nextIndex, TermIndex 
previous, long snapshotIndex) {
      if (protos.isEmpty()) {
        return;
      }
      final long firstIndex = protos.get(0).getIndex();
      Preconditions.assertTrue(firstIndex == nextIndex,
          () -> follower.getName() + ": firstIndex = " + firstIndex + " != 
nextIndex = " + nextIndex);
      if (firstIndex > RaftLog.LEAST_VALID_LOG_INDEX) {
        // Check if nextIndex is 1 greater than the snapshotIndex. If yes, then
        // we do not have to check for the existence of previous.
        if (nextIndex != snapshotIndex + 1) {
          Objects.requireNonNull(previous,
              () -> follower.getName() + ": Previous TermIndex not found for 
firstIndex = " + firstIndex);
          Preconditions.assertTrue(previous.getIndex() == firstIndex - 1,
              () -> follower.getName() + ": Previous = " + previous + " but 
firstIndex = " + firstIndex);
        }
      }
    }{code}
 

  So for firstIndex = 93, the leader could not resolve index 92 from log or 
from “snapshot at 92,” and the follower’s snapshotIndex did not satisfy 
nextIndex == snapshotIndex + 1 (line 266), so requireNonNull fired.
  That matches purged / missing previous entry (or an inconsistent snapshot vs 
purge story on the leader).

  Why `installSnapshot()` might not run first

  GrpcLogAppender tries snapshot install/notify before append:

   
{code:java}
ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java 
lines 256-259
        if (shouldSendAppendEntries() || 
isFollowerCommitBehindLastCommitIndex()) {
          final boolean installingSnapshot = installSnapshot();
          appendLog(installingSnapshot || haveTooManyPendingRequests());
        }
{code}
 

  Notify path only treats “leader doesn’t have logs from the follower’s 
position” as `followerNextIndex < leaderStartIndex`:

   
{code:java}
ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java 
lines 839-849
      final long followerNextIndex = follower.getNextIndex();
      if (followerNextIndex >= leaderNextIndex) {         return null;       }
      if (followerNextIndex < leaderStartIndex) {
        // The Leader does not have the logs from the Follower's last log
        // index onwards. And install snapshot is disabled. So the Follower
        // should be notified to install the latest snapshot through its
        // State Machine.
        return firstAvailable;{code}
 

  If `followerNextIndex == leaderStartIndex` (e.g. both 93) but `previousIndex` 
(92) is not in the log and the latest snapshot is not at 92, you get exactly 
your stack trace: append proceeds, getPrevious(93) is null, assertProtos throws.
  So the issue is not only “purge” in the abstract; it is leader log/snapshot 
invariants and/or gaps in the notify/install conditions relative to getPrevious.

  Suggested improvements

  1. Operations / configuration (most important)
    • Ensure purge/truncation never removes entries that are still needed 
unless a snapshot exists at the last purged index (so leaderStartIndex - 1 is 
either still in the log or equals the latest snapshot index).
    • Review retention, purge, segment policies, and snapshot frequency so the 
latest snapshot index tracks the log start after purge.
  2. Hardening in Ratis (product/code)
    • Before building non-empty append batches, if 
getPrevious(follower.getNextIndex()) == null and you are not in the nextIndex 
== followerSnapshotIndex + 1 case, treat it like “follower must catch up via 
snapshot”: call the same path
      as shouldNotifyToInstallSnapshot / installSnapshot instead of failing 
assertProtos.
    • Optionally extend shouldNotifyToInstallSnapshot() to cover the case 
`followerNextIndex >= leaderStartIndex` but previous term index cannot be 
resolved (gap between snapshot and log start, or stale follower metadata).
  3. Clearer diagnostics
    • Replace or augment the Objects.requireNonNull message with leader 
getStartIndex(), latest snapshot index, and follower nextIndex / snapshotIndex 
so logs show purge vs snapshot mismatch immediately.

 

I prefer to support approach 2 to trigger installSnapshot.

  was:
Found the following logs in tests
{code:java}
java.lang.NullPointerException: omNode-2@group-523986131536->omNode-1: Previous 
TermIndex not found for firstIndex = 93 at 
java.base/java.util.Objects.requireNonNull(Objects.java:360) at 
org.apache.ratis.server.leader.LogAppenderBase.assertProtos(LogAppenderBase.java:270)
        at 
org.apache.ratis.server.leader.LogAppenderBase.newAppendEntriesRequest(LogAppenderBase.java:255)
     at 
org.apache.ratis.grpc.server.GrpcLogAppender.appendLog(GrpcLogAppender.java:387)
     at 
org.apache.ratis.grpc.server.GrpcLogAppender.run(GrpcLogAppender.java:262)   at 
org.apache.ratis.server.leader.LogAppenderDaemon.run(LogAppenderDaemon.java:80) 
     at java.base/java.lang.Thread.run(Thread.java:1583) {code}
Seems to be related to this Ratis NPE when there is no previous log (when the 
logs are purged). After RATIS-2427, NPE will not cause LogAppender to be 
restarted. This can cause the leader to not send anything anymore and can cause 
the Raft group to be stuck.

 

AI Analysis

Is “previous log already purged” a valid explanation?

  Yes. The failure is thrown when the leader builds an AppendEntries request 
with non-empty entries and previous is null.
  getPrevious(nextIndex) does this:

   
ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java 
lines 165-185

    private TermIndex getPrevious(long nextIndex) {
     
{code:java}
if (nextIndex == RaftLog.LEAST_VALID_LOG_INDEX)
{         return null;       }
      final long previousIndex = nextIndex - 1;
      final TermIndex previous = getRaftLog().getTermIndex(previousIndex);
      if (previous != null) {         return previous;       }
      final SnapshotInfo snapshot = 
server.getStateMachine().getLatestSnapshot();
      if (snapshot != null) {
        final TermIndex snapshotTermIndex = snapshot.getTermIndex();
        if (snapshotTermIndex.getIndex() == previousIndex) {           return 
snapshotTermIndex;         }
      }
      return null;
    }
{code}
  So previous is null only if:
  1. Index nextIndex - 1 is not in the leader’s raft log (e.g. purged / before 
getStartIndex()), and
  2. The leader’s latest snapshot term index is not exactly nextIndex - 1.

  Then assertProtos requires a non-null previous (unless a specific follower 
snapshot boundary applies):

   
ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java 
lines 256-273

 
{code:java}
private void assertProtos(List<LogEntryProto> protos, long nextIndex, TermIndex 
previous, long snapshotIndex) {
      if (protos.isEmpty()) {
        return;
      }
      final long firstIndex = protos.get(0).getIndex();
      Preconditions.assertTrue(firstIndex == nextIndex,
          () -> follower.getName() + ": firstIndex = " + firstIndex + " != 
nextIndex = " + nextIndex);
      if (firstIndex > RaftLog.LEAST_VALID_LOG_INDEX) {
        // Check if nextIndex is 1 greater than the snapshotIndex. If yes, then
        // we do not have to check for the existence of previous.
        if (nextIndex != snapshotIndex + 1) {
          Objects.requireNonNull(previous,
              () -> follower.getName() + ": Previous TermIndex not found for 
firstIndex = " + firstIndex);
          Preconditions.assertTrue(previous.getIndex() == firstIndex - 1,
              () -> follower.getName() + ": Previous = " + previous + " but 
firstIndex = " + firstIndex);
        }
      }
    }{code}

 

  So for firstIndex = 93, the leader could not resolve index 92 from log or 
from “snapshot at 92,” and the follower’s snapshotIndex did not satisfy 
nextIndex == snapshotIndex + 1 (line 266), so requireNonNull fired.
  That matches purged / missing previous entry (or an inconsistent snapshot vs 
purge story on the leader).


  Why `installSnapshot()` might not run first

  GrpcLogAppender tries snapshot install/notify before append:

   
{code:java}
ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java 
lines 256-259
        if (shouldSendAppendEntries() || 
isFollowerCommitBehindLastCommitIndex()) {
          final boolean installingSnapshot = installSnapshot();
          appendLog(installingSnapshot || haveTooManyPendingRequests());
        }
{code}

 

  Notify path only treats “leader doesn’t have logs from the follower’s 
position” as `followerNextIndex < leaderStartIndex`:

   
{code:java}
 {code}

ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java 
lines 839-849
      final long followerNextIndex = follower.getNextIndex();
      if (followerNextIndex >= leaderNextIndex) \{         return null;       }

      if (followerNextIndex < leaderStartIndex) {
        // The Leader does not have the logs from the Follower's last log
        // index onwards. And install snapshot is disabled. So the Follower
        // should be notified to install the latest snapshot through its
        // State Machine.
        return firstAvailable;
{code:java}
 {code}
 

  If `followerNextIndex == leaderStartIndex` (e.g. both 93) but `previousIndex` 
(92) is not in the log and the latest snapshot is not at 92, you get exactly 
your stack trace: append proceeds, getPrevious(93) is null, assertProtos throws.
  So the issue is not only “purge” in the abstract; it is leader log/snapshot 
invariants and/or gaps in the notify/install conditions relative to getPrevious.

  Suggested improvements

  1. Operations / configuration (most important)
    • Ensure purge/truncation never removes entries that are still needed 
unless a snapshot exists at the last purged index (so leaderStartIndex - 1 is 
either still in the log or equals the latest snapshot index).
    • Review retention, purge, segment policies, and snapshot frequency so the 
latest snapshot index tracks the log start after purge.
  2. Hardening in Ratis (product/code)
    • Before building non-empty append batches, if 
getPrevious(follower.getNextIndex()) == null and you are not in the nextIndex 
== followerSnapshotIndex + 1 case, treat it like “follower must catch up via 
snapshot”: call the same path
      as shouldNotifyToInstallSnapshot / installSnapshot instead of failing 
assertProtos.
    • Optionally extend shouldNotifyToInstallSnapshot() to cover the case 
`followerNextIndex >= leaderStartIndex` but previous term index cannot be 
resolved (gap between snapshot and log start, or stale follower metadata).
  3. Clearer diagnostics
    • Replace or augment the Objects.requireNonNull message with leader 
getStartIndex(), latest snapshot index, and follower nextIndex / snapshotIndex 
so logs show purge vs snapshot mismatch immediately.

 

I prefer to support approach 2 to trigger installSnapshot.


> NPE when there is no previous log
> ---------------------------------
>
>                 Key: RATIS-2487
>                 URL: https://issues.apache.org/jira/browse/RATIS-2487
>             Project: Ratis
>          Issue Type: Bug
>            Reporter: Ivan Andika
>            Assignee: Ivan Andika
>            Priority: Major
>
> Found the following logs in tests
> {code:java}
> java.lang.NullPointerException: omNode-2@group-523986131536->omNode-1: 
> Previous TermIndex not found for firstIndex = 93       at 
> java.base/java.util.Objects.requireNonNull(Objects.java:360) at 
> org.apache.ratis.server.leader.LogAppenderBase.assertProtos(LogAppenderBase.java:270)
>         at 
> org.apache.ratis.server.leader.LogAppenderBase.newAppendEntriesRequest(LogAppenderBase.java:255)
>      at 
> org.apache.ratis.grpc.server.GrpcLogAppender.appendLog(GrpcLogAppender.java:387)
>      at 
> org.apache.ratis.grpc.server.GrpcLogAppender.run(GrpcLogAppender.java:262)   
> at 
> org.apache.ratis.server.leader.LogAppenderDaemon.run(LogAppenderDaemon.java:80)
>       at java.base/java.lang.Thread.run(Thread.java:1583) {code}
> Seems to be related to this Ratis NPE when there is no previous log (when the 
> logs are purged). After RATIS-2427, NPE will not cause LogAppender to be 
> restarted. This can cause the leader to not send anything anymore and can 
> cause the Raft group to be stuck.
>  
> AI Analysis
> Is “previous log already purged” a valid explanation?
>   Yes. The failure is thrown when the leader builds an AppendEntries request 
> with non-empty entries and previous is null.
>   getPrevious(nextIndex) does this:
>    
> ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java
>  lines 165-185
>     private TermIndex getPrevious(long nextIndex) {
>      
> {code:java}
> if (nextIndex == RaftLog.LEAST_VALID_LOG_INDEX)
> {         return null;       }
>       final long previousIndex = nextIndex - 1;
>       final TermIndex previous = getRaftLog().getTermIndex(previousIndex);
>       if (previous != null) {         return previous;       }
>       final SnapshotInfo snapshot = 
> server.getStateMachine().getLatestSnapshot();
>       if (snapshot != null) {
>         final TermIndex snapshotTermIndex = snapshot.getTermIndex();
>         if (snapshotTermIndex.getIndex() == previousIndex) {           return 
> snapshotTermIndex;         }
>       }
>       return null;
>     }
> {code}
>   So previous is null only if:
>   1. Index nextIndex - 1 is not in the leader’s raft log (e.g. purged / 
> before getStartIndex()), and
>   2. The leader’s latest snapshot term index is not exactly nextIndex - 1.
>   Then assertProtos requires a non-null previous (unless a specific follower 
> snapshot boundary applies):
>    
> ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java
>  lines 256-273
>  
> {code:java}
> private void assertProtos(List<LogEntryProto> protos, long nextIndex, 
> TermIndex previous, long snapshotIndex) {
>       if (protos.isEmpty()) {
>         return;
>       }
>       final long firstIndex = protos.get(0).getIndex();
>       Preconditions.assertTrue(firstIndex == nextIndex,
>           () -> follower.getName() + ": firstIndex = " + firstIndex + " != 
> nextIndex = " + nextIndex);
>       if (firstIndex > RaftLog.LEAST_VALID_LOG_INDEX) {
>         // Check if nextIndex is 1 greater than the snapshotIndex. If yes, 
> then
>         // we do not have to check for the existence of previous.
>         if (nextIndex != snapshotIndex + 1) {
>           Objects.requireNonNull(previous,
>               () -> follower.getName() + ": Previous TermIndex not found for 
> firstIndex = " + firstIndex);
>           Preconditions.assertTrue(previous.getIndex() == firstIndex - 1,
>               () -> follower.getName() + ": Previous = " + previous + " but 
> firstIndex = " + firstIndex);
>         }
>       }
>     }{code}
>  
>   So for firstIndex = 93, the leader could not resolve index 92 from log or 
> from “snapshot at 92,” and the follower’s snapshotIndex did not satisfy 
> nextIndex == snapshotIndex + 1 (line 266), so requireNonNull fired.
>   That matches purged / missing previous entry (or an inconsistent snapshot 
> vs purge story on the leader).
>   Why `installSnapshot()` might not run first
>   GrpcLogAppender tries snapshot install/notify before append:
>    
> {code:java}
> ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java 
> lines 256-259
>         if (shouldSendAppendEntries() || 
> isFollowerCommitBehindLastCommitIndex()) {
>           final boolean installingSnapshot = installSnapshot();
>           appendLog(installingSnapshot || haveTooManyPendingRequests());
>         }
> {code}
>  
>   Notify path only treats “leader doesn’t have logs from the follower’s 
> position” as `followerNextIndex < leaderStartIndex`:
>    
> {code:java}
> ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java 
> lines 839-849
>       final long followerNextIndex = follower.getNextIndex();
>       if (followerNextIndex >= leaderNextIndex) {         return null;       }
>       if (followerNextIndex < leaderStartIndex) {
>         // The Leader does not have the logs from the Follower's last log
>         // index onwards. And install snapshot is disabled. So the Follower
>         // should be notified to install the latest snapshot through its
>         // State Machine.
>         return firstAvailable;{code}
>  
>   If `followerNextIndex == leaderStartIndex` (e.g. both 93) but 
> `previousIndex` (92) is not in the log and the latest snapshot is not at 92, 
> you get exactly your stack trace: append proceeds, getPrevious(93) is null, 
> assertProtos throws.
>   So the issue is not only “purge” in the abstract; it is leader log/snapshot 
> invariants and/or gaps in the notify/install conditions relative to 
> getPrevious.
>   Suggested improvements
>   1. Operations / configuration (most important)
>     • Ensure purge/truncation never removes entries that are still needed 
> unless a snapshot exists at the last purged index (so leaderStartIndex - 1 is 
> either still in the log or equals the latest snapshot index).
>     • Review retention, purge, segment policies, and snapshot frequency so 
> the latest snapshot index tracks the log start after purge.
>   2. Hardening in Ratis (product/code)
>     • Before building non-empty append batches, if 
> getPrevious(follower.getNextIndex()) == null and you are not in the nextIndex 
> == followerSnapshotIndex + 1 case, treat it like “follower must catch up via 
> snapshot”: call the same path
>       as shouldNotifyToInstallSnapshot / installSnapshot instead of failing 
> assertProtos.
>     • Optionally extend shouldNotifyToInstallSnapshot() to cover the case 
> `followerNextIndex >= leaderStartIndex` but previous term index cannot be 
> resolved (gap between snapshot and log start, or stale follower metadata).
>   3. Clearer diagnostics
>     • Replace or augment the Objects.requireNonNull message with leader 
> getStartIndex(), latest snapshot index, and follower nextIndex / 
> snapshotIndex so logs show purge vs snapshot mismatch immediately.
>  
> I prefer to support approach 2 to trigger installSnapshot.



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

Reply via email to