Skip to content

Commit

Permalink
RATIS-2093. Decouple metadata and configuration entries from appendEn…
Browse files Browse the repository at this point in the history
…tries buffer for stateMachineCache' (#1096)
  • Loading branch information
duongkame committed May 24, 2024
1 parent 26385f3 commit e002587
Showing 1 changed file with 9 additions and 5 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -456,13 +456,17 @@ protected CompletableFuture<Long> appendEntryImpl(ReferenceCountedObject<LogEntr
// 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) {
Expand Down

0 comments on commit e002587

Please sign in to comment.