This is an automated email from the ASF dual-hosted git repository.

szetszwo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ratis.git


The following commit(s) were added to refs/heads/master by this push:
     new ee09075c0 RATIS-2059. Missing reference count when putting log entries 
to cache on follower. (#1067)
ee09075c0 is described below

commit ee09075c0e930756e5ad390fc9250bd5d1ffea52
Author: Duong Nguyen <[email protected]>
AuthorDate: Wed Apr 17 10:10:13 2024 -0700

    RATIS-2059. Missing reference count when putting log entries to cache on 
follower. (#1067)
---
 .../org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

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 7f9ef3098..a3c18f01c 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 @@ public final class SegmentedRaftLog extends RaftLogBase {
       // 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));

Reply via email to