RATIS-2059. Missing reference count when putting log entries to cache on follower. (#1067)
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java
index 7f9ef30..a3c18f0 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java
@@ -457,7 +457,7 @@
// to statemachine first and then to the cache. Not following the order
// will leave a spurious entry in the cache.
final Task write = fileLogWorker.writeLogEntry(entryRef, removedStateMachineData, context);
- if (stateMachineCachingEnabled) {
+ if (stateMachineCachingEnabled && (removedStateMachineData != entry)) {
// The stateMachineData will be cached inside the StateMachine itself.
cache.appendEntry(LogSegment.Op.WRITE_CACHE_WITH_STATE_MACHINE_CACHE,
ReferenceCountedObject.wrap(removedStateMachineData));