[ 
https://issues.apache.org/jira/browse/RATIS-1979?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17827620#comment-17827620
 ] 

Duong edited comment on RATIS-1979 at 3/28/24 8:50 PM:
-------------------------------------------------------

This includes the following changes:
 * For stateMachineCaching, the log entry (with stateMachine data trimmed) is 
copied and independent from the original buffer. The zero-copy buffer can be 
released independently from the LogSegment cache.
 * However, because the stateMachine data is cached by StateMachine, the 
StateMachine will hold the reference count to the stateMachine data entry. This 
is done already via StateMachine.write and StateMachine.applyTransaction being 
supplied by {_}ReferenceCountedObject{_}. _StateMachine.read_ now has to 
provide the data via a _ReferenceCountedObject._


was (Author: JIRAUSER290990):
This includes the following changes:
 * For stateMachineCaching, the log entry (with stateMachine data trimmed) is 
copied and independent from the original buffer. The zero-copy buffer can be 
released independently from the LogSegment cache.
 * However, the stateMachine data may be cached by StateMachine. StateMachine 
will hold the reference count to the stateMachine data entry if it caches the 
data, this is done already via StateMachine.write and 
StateMachine.applyTransaction being supplied by {_}ReferenceCountedObject{_}. 
_StateMachine.read_ now has to provide the data via a _ReferenceCountedObject._

> Correct LogEntryProto cache logic stateMachineCache
> ---------------------------------------------------
>
>                 Key: RATIS-1979
>                 URL: https://issues.apache.org/jira/browse/RATIS-1979
>             Project: Ratis
>          Issue Type: Sub-task
>            Reporter: Duong
>            Priority: Major
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> With zero-copy (https://github.com/apache/ratis/pull/990), we rely on RaftLog 
> cache eviction to release the zero-copy input streams.
> There's a problems, with stateMachineCachingEnabled, the cache-size of 
> LogEntryproto is calculated with StateMachineData trimmed. That leave the 
> actual cached size if a log entry is too small comparing to the amount of 
> direct memory backed for the StateMachineData. 
>  
> {code:java}
> static long getEntrySize(LogEntryProto entry, Op op) {
>   LogEntryProto e = entry;
>   if (op == Op.CHECK_SEGMENT_FILE_FULL) {
>     e = LogProtoUtils.removeStateMachineData(entry);
>   } else if (op == Op.LOAD_SEGMENT_FILE || op == 
> Op.WRITE_CACHE_WITH_STATE_MACHINE_CACHE) {
>     Preconditions.assertTrue(entry == 
> LogProtoUtils.removeStateMachineData(entry),
>         () -> "Unexpected LogEntryProto with StateMachine data: op=" + op + 
> ", entry=" + entry);
>   } else {
>     Preconditions.assertTrue(op == Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE 
> || op == Op.REMOVE_CACHE,
>         () -> "Unexpected op " + op + ", entry=" + entry);
>   }
>   final int serialized = e.getSerializedSize();
>   return serialized + CodedOutputStream.computeUInt32SizeNoTag(serialized) + 
> 4L;
> } {code}
>  
> With the default 200MB limit for raft log cache, cache eviction is likely 
> never happens until the server run out of direct memory for zero-copy.
>  
> Maybe for cache calculation, we should take the StateMachineData size into 
> account.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to