[
https://issues.apache.org/jira/browse/RATIS-2523?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Luca Garulli updated RATIS-2523:
--------------------------------
Description:
A leader's `GrpcLogAppender` gets permanently stuck in an `INCONSISTENCY`
heartbeat loop after a follower restarts with an empty Raft log on a quiet
cluster. The leader emits
```
received INCONSISTENCY reply with nextIndex 0, errorCount=N,
request=AppendEntriesRequest:cid=M,entriesCount=0
```
every ~5 s indefinitely, `errorCount` keeps incrementing, and
`matchIndex`/`nextIndex` for the affected follower never converge. The follower
never catches up. The loop only clears once an unrelated user transaction
generates a real `AppendEntries` (entries non-empty); on an idle cluster
nothing breaks the cycle.
Reproduced reliably on Ratis **3.2.2** with `ratis-server` + `ratis-grpc`.
## Root cause
`LogAppenderBase.getNextIndexForInconsistency`
(ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java
in 3.2.2):
```java
protected long getNextIndexForInconsistency(long requestFirstIndex, long
replyNextIndex) {
long next = replyNextIndex;
final long i = getFollower().getMatchIndex() + 1;
if (i > next && i != requestFirstIndex) {
// Ideally, we should set nextIndex to a value greater than matchIndex.
// However, we must not resend the same first entry due to some special
cases (e.g. the log is empty).
// Otherwise, the follower will reply INCONSISTENCY again.
next = i;
}
if (next == requestFirstIndex && next > RaftLog.LEAST_VALID_LOG_INDEX) {
// Avoid resending the same first entry.
next--;
}
return next;
}
```
Trace of the failure:
1. A follower disconnects and is later restarted with no Raft storage on disk
(lastLogIndex = -1).
2. `FollowerInfo.matchIndex` on the leader still holds the pre-restart value
(it is only mutated on SUCCESS replies, never on disconnect).
3. The leader's `GrpcLogAppender` keeps issuing periodic heartbeats. A
heartbeat carries no entries, so `request.getFirstIndex()` returns
`RaftLog.INVALID_LOG_INDEX = -1`.
4. The follower replies `INCONSISTENCY` with `nextIndex = lastLogIndex + 1 = 0`
(its actual hint).
5. `getNextIndexForInconsistency(-1, 0)` runs:
- `next = 0`
- `i = matchIndex + 1` (stale, e.g. `1`)
- `i > 0 && i != -1` → true → `next = 1`
- `next == -1` → false
- returns `1` — **the follower's hint is discarded in favour of the stale
`matchIndex + 1`**.
6. The next heartbeat is identical. The follower replies `INCONSISTENCY` with
`nextIndex = 0` again. Loop forever.
The guard `i != requestFirstIndex` is the comment's "must not resend the same
first entry" rule. It works correctly for **real** AppendEntries (where
`requestFirstIndex == nextIndex`, so the `i != requestFirstIndex` test is false
and the follower's hint is honoured), and that is exactly why the issue is
invisible on busy clusters: any user-driven log entry breaks the loop. On an
idle cluster, only heartbeats flow and the loop is permanent.
## Reproducer
We reproduced this from ArcadeDB's HA layer (Apache Ratis-based) and have a
deterministic JUnit test:
- 4-node Ratis cluster, all on `localhost`, fresh storage on every node.
- No client traffic; the only Raft log entries are the configuration ones
produced at cluster bootstrap.
- Stop one follower (the "replica") with `RaftServer.close()` and **delete its
storage directory** (so the restart comes back with `lastLogIndex == -1`).
- Sleep 30 s — long enough for the leader to settle into its retry cadence.
- Restart the follower. It rejoins the group cleanly and Ratis acknowledges the
leader.
- After waiting for the follower's `lastAppliedIndex` to allegedly catch up to
the leader's, observe the leader's logs for 15 s.
Result: the leader emits the `INCONSISTENCY reply with nextIndex 0 ...
entriesCount=0` warning 3+ times in that 15 s window — i.e. the cadence keeps
going. With a 1-min observation it is still firing every ~5 s with
monotonically growing `errorCount`.
The same machine, same code, same cluster running an active write workload
never triggers the stuck loop because every user commit produces a real
AppendEntries that bypasses the bug.
Full test source:
https://github.com/ArcadeData/arcadedb/blob/main/ha-raft/src/test/java/com/arcadedb/server/ha/raft/RaftIdleReplicaRestartIT.java
Snippet of the leader's log during the stuck state:
```
WARNI [GrpcLogAppender] L@G->R-AppendLogResponseHandler: received INCONSISTENCY
reply with nextIndex 0, errorCount=39,
request=AppendEntriesRequest:cid=3104,entriesCount=0
WARNI [GrpcLogAppender] L@G->R-AppendLogResponseHandler: received INCONSISTENCY
reply with nextIndex 0, errorCount=40,
request=AppendEntriesRequest:cid=3106,entriesCount=0
WARNI [GrpcLogAppender] L@G->R-AppendLogResponseHandler: received INCONSISTENCY
reply with nextIndex 0, errorCount=41,
request=AppendEntriesRequest:cid=3108,entriesCount=0
WARNI [GrpcLogAppender] L@G->R-AppendLogResponseHandler: received INCONSISTENCY
reply with nextIndex 0, errorCount=42,
request=AppendEntriesRequest:cid=3110,entriesCount=0
WARNI [GrpcLogAppender] L@G->R-AppendLogResponseHandler: received INCONSISTENCY
reply with nextIndex 0, errorCount=43,
request=AppendEntriesRequest:cid=3112,entriesCount=0
```
## Suggested fix direction
Two options worth considering, both in
`LogAppenderBase.getNextIndexForInconsistency`:
1. **Trust `replyNextIndex` when it is strictly less than `matchIndex` for a
heartbeat.** A reply nextIndex below matchIndex is impossible under normal
protocol invariants — it is a strong signal that the follower has lost state
and matchIndex is stale. In that case `matchIndex` should be reset to
`replyNextIndex - 1` and the next heartbeat should carry `replyNextIndex` as
its target.
2. **Reset `FollowerInfo.matchIndex` on detected disconnect/reconnect.**
Whenever the leader's `GrpcLogAppender` enters `RUNNING` after
`RECONNECT`/`COMPLETE`/`ERROR`, conservatively snap matchIndex back to
`commitIndex` (or 0). The next AppendEntries cycle then converges naturally.
Either change would preserve the existing "do not resend the same first entry"
intent while handling the empty-follower-log case.
Happy to contribute a PR if there is interest in the approach. A workaround on
our side is feasible (force a no-op log entry or transfer leadership when the
loop is detected), but it is a one-line bug in Ratis that affects any quiet
cluster after a follower restart on ephemeral storage.
## Affected versions
Confirmed on `3.2.2` (the latest released as of 2026-05-06). The same code path
exists in 3.2.0/3.2.1.
## Environment
- Java 21 / OpenJDK 25
- macOS / Linux (mdre's production scenario is 4 Debian VMs on Proxmox; the
in-process JUnit reproducer is identical behaviour on macOS).
- `RaftServerConfigKeys.Log.Appender.installSnapshotEnabled = false`
(notification mode), but the symptom occurs irrespective of that flag because
the leader never decides a snapshot is needed (the log on the leader is short
and not compacted).
> A leader's `GrpcLogAppender` gets permanently stuck in an `INCONSISTENCY`
> heartbeat loop after a follower restarts with an empty Raft log on a quiet
> cluster
> ------------------------------------------------------------------------------------------------------------------------------------------------------------
>
> Key: RATIS-2523
> URL: https://issues.apache.org/jira/browse/RATIS-2523
> Project: Ratis
> Issue Type: Bug
> Reporter: Luca Garulli
> Priority: Major
>
> A leader's `GrpcLogAppender` gets permanently stuck in an `INCONSISTENCY`
> heartbeat loop after a follower restarts with an empty Raft log on a quiet
> cluster. The leader emits
> ```
> received INCONSISTENCY reply with nextIndex 0, errorCount=N,
> request=AppendEntriesRequest:cid=M,entriesCount=0
> ```
> every ~5 s indefinitely, `errorCount` keeps incrementing, and
> `matchIndex`/`nextIndex` for the affected follower never converge. The
> follower never catches up. The loop only clears once an unrelated user
> transaction generates a real `AppendEntries` (entries non-empty); on an idle
> cluster nothing breaks the cycle.
> Reproduced reliably on Ratis **3.2.2** with `ratis-server` + `ratis-grpc`.
> ## Root cause
> `LogAppenderBase.getNextIndexForInconsistency`
> (ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java
> in 3.2.2):
> ```java
> protected long getNextIndexForInconsistency(long requestFirstIndex, long
> replyNextIndex) {
> long next = replyNextIndex;
> final long i = getFollower().getMatchIndex() + 1;
> if (i > next && i != requestFirstIndex) {
> // Ideally, we should set nextIndex to a value greater than matchIndex.
> // However, we must not resend the same first entry due to some special
> cases (e.g. the log is empty).
> // Otherwise, the follower will reply INCONSISTENCY again.
> next = i;
> }
> if (next == requestFirstIndex && next > RaftLog.LEAST_VALID_LOG_INDEX) {
> // Avoid resending the same first entry.
> next--;
> }
> return next;
> }
> ```
> Trace of the failure:
> 1. A follower disconnects and is later restarted with no Raft storage on disk
> (lastLogIndex = -1).
> 2. `FollowerInfo.matchIndex` on the leader still holds the pre-restart value
> (it is only mutated on SUCCESS replies, never on disconnect).
> 3. The leader's `GrpcLogAppender` keeps issuing periodic heartbeats. A
> heartbeat carries no entries, so `request.getFirstIndex()` returns
> `RaftLog.INVALID_LOG_INDEX = -1`.
> 4. The follower replies `INCONSISTENCY` with `nextIndex = lastLogIndex + 1 =
> 0` (its actual hint).
> 5. `getNextIndexForInconsistency(-1, 0)` runs:
> - `next = 0`
> - `i = matchIndex + 1` (stale, e.g. `1`)
> - `i > 0 && i != -1` → true → `next = 1`
> - `next == -1` → false
> - returns `1` — **the follower's hint is discarded in favour of the stale
> `matchIndex + 1`**.
> 6. The next heartbeat is identical. The follower replies `INCONSISTENCY` with
> `nextIndex = 0` again. Loop forever.
> The guard `i != requestFirstIndex` is the comment's "must not resend the same
> first entry" rule. It works correctly for **real** AppendEntries (where
> `requestFirstIndex == nextIndex`, so the `i != requestFirstIndex` test is
> false and the follower's hint is honoured), and that is exactly why the issue
> is invisible on busy clusters: any user-driven log entry breaks the loop. On
> an idle cluster, only heartbeats flow and the loop is permanent.
> ## Reproducer
> We reproduced this from ArcadeDB's HA layer (Apache Ratis-based) and have a
> deterministic JUnit test:
> - 4-node Ratis cluster, all on `localhost`, fresh storage on every node.
> - No client traffic; the only Raft log entries are the configuration ones
> produced at cluster bootstrap.
> - Stop one follower (the "replica") with `RaftServer.close()` and **delete
> its storage directory** (so the restart comes back with `lastLogIndex == -1`).
> - Sleep 30 s — long enough for the leader to settle into its retry cadence.
> - Restart the follower. It rejoins the group cleanly and Ratis acknowledges
> the leader.
> - After waiting for the follower's `lastAppliedIndex` to allegedly catch up
> to the leader's, observe the leader's logs for 15 s.
> Result: the leader emits the `INCONSISTENCY reply with nextIndex 0 ...
> entriesCount=0` warning 3+ times in that 15 s window — i.e. the cadence keeps
> going. With a 1-min observation it is still firing every ~5 s with
> monotonically growing `errorCount`.
> The same machine, same code, same cluster running an active write workload
> never triggers the stuck loop because every user commit produces a real
> AppendEntries that bypasses the bug.
> Full test source:
> https://github.com/ArcadeData/arcadedb/blob/main/ha-raft/src/test/java/com/arcadedb/server/ha/raft/RaftIdleReplicaRestartIT.java
> Snippet of the leader's log during the stuck state:
> ```
> WARNI [GrpcLogAppender] L@G->R-AppendLogResponseHandler: received
> INCONSISTENCY reply with nextIndex 0, errorCount=39,
> request=AppendEntriesRequest:cid=3104,entriesCount=0
> WARNI [GrpcLogAppender] L@G->R-AppendLogResponseHandler: received
> INCONSISTENCY reply with nextIndex 0, errorCount=40,
> request=AppendEntriesRequest:cid=3106,entriesCount=0
> WARNI [GrpcLogAppender] L@G->R-AppendLogResponseHandler: received
> INCONSISTENCY reply with nextIndex 0, errorCount=41,
> request=AppendEntriesRequest:cid=3108,entriesCount=0
> WARNI [GrpcLogAppender] L@G->R-AppendLogResponseHandler: received
> INCONSISTENCY reply with nextIndex 0, errorCount=42,
> request=AppendEntriesRequest:cid=3110,entriesCount=0
> WARNI [GrpcLogAppender] L@G->R-AppendLogResponseHandler: received
> INCONSISTENCY reply with nextIndex 0, errorCount=43,
> request=AppendEntriesRequest:cid=3112,entriesCount=0
> ```
> ## Suggested fix direction
> Two options worth considering, both in
> `LogAppenderBase.getNextIndexForInconsistency`:
> 1. **Trust `replyNextIndex` when it is strictly less than `matchIndex` for a
> heartbeat.** A reply nextIndex below matchIndex is impossible under normal
> protocol invariants — it is a strong signal that the follower has lost state
> and matchIndex is stale. In that case `matchIndex` should be reset to
> `replyNextIndex - 1` and the next heartbeat should carry `replyNextIndex` as
> its target.
> 2. **Reset `FollowerInfo.matchIndex` on detected disconnect/reconnect.**
> Whenever the leader's `GrpcLogAppender` enters `RUNNING` after
> `RECONNECT`/`COMPLETE`/`ERROR`, conservatively snap matchIndex back to
> `commitIndex` (or 0). The next AppendEntries cycle then converges naturally.
> Either change would preserve the existing "do not resend the same first
> entry" intent while handling the empty-follower-log case.
> Happy to contribute a PR if there is interest in the approach. A workaround
> on our side is feasible (force a no-op log entry or transfer leadership when
> the loop is detected), but it is a one-line bug in Ratis that affects any
> quiet cluster after a follower restart on ephemeral storage.
> ## Affected versions
> Confirmed on `3.2.2` (the latest released as of 2026-05-06). The same code
> path exists in 3.2.0/3.2.1.
> ## Environment
> - Java 21 / OpenJDK 25
> - macOS / Linux (mdre's production scenario is 4 Debian VMs on Proxmox; the
> in-process JUnit reproducer is identical behaviour on macOS).
> - `RaftServerConfigKeys.Log.Appender.installSnapshotEnabled = false`
> (notification mode), but the symptom occurs irrespective of that flag because
> the leader never decides a snapshot is needed (the log on the leader is short
> and not compacted).
--
This message was sent by Atlassian Jira
(v8.20.10#820010)