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 e0025875e RATIS-2093. Decouple metadata and configuration entries from 
appendEntries buffer for stateMachineCache' (#1096)
e0025875e is described below

commit e0025875ecf81ce4b84986147b023822eb14a4ee
Author: Duong Nguyen <[email protected]>
AuthorDate: Fri May 24 13:08:55 2024 -0700

    RATIS-2093. Decouple metadata and configuration entries from appendEntries 
buffer for stateMachineCache' (#1096)
---
 .../ratis/server/raftlog/segmented/SegmentedRaftLog.java   | 14 +++++++++-----
 1 file changed, 9 insertions(+), 5 deletions(-)

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 7c33adf00..27b771ff1 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
@@ -456,13 +456,17 @@ 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 && (removedStateMachineData != entry)) {
+      if (stateMachineCachingEnabled) {
         // The stateMachineData will be cached inside the StateMachine itself.
-        cache.appendEntry(LogSegment.Op.WRITE_CACHE_WITH_STATE_MACHINE_CACHE,
-            ReferenceCountedObject.wrap(removedStateMachineData));
+        if (removedStateMachineData != entry) {
+          cache.appendEntry(LogSegment.Op.WRITE_CACHE_WITH_STATE_MACHINE_CACHE,
+              ReferenceCountedObject.wrap(removedStateMachineData));
+        } else {
+          cache.appendEntry(LogSegment.Op.WRITE_CACHE_WITH_STATE_MACHINE_CACHE,
+              
ReferenceCountedObject.wrap(LogProtoUtils.copy(removedStateMachineData)));
+        }
       } else {
-        
cache.appendEntry(LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, 
entryRef
-        );
+        
cache.appendEntry(LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, 
entryRef);
       }
       return write.getFuture().whenComplete((clientReply, exception) -> 
appendEntryTimerContext.stop());
     } catch (Exception e) {

Reply via email to