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

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

commit e20a2e83bed8c5685a63ea60e9b6d34d8d8007d7
Author: Xiangpeng Hu <[email protected]>
AuthorDate: Fri Mar 17 08:17:07 2023 +0800

    RATIS-1816. appendEntryTimer is not accurate due to the return of 
writeFuture (#855)
    
    (cherry picked from commit 9a3f7e333e87b0107d9a1eb860a41a8d882d67a1)
---
 .../org/apache/ratis/server/metrics/SegmentedRaftLogMetrics.java     | 4 ++--
 .../org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java  | 5 ++---
 2 files changed, 4 insertions(+), 5 deletions(-)

diff --git 
a/ratis-server/src/main/java/org/apache/ratis/server/metrics/SegmentedRaftLogMetrics.java
 
b/ratis-server/src/main/java/org/apache/ratis/server/metrics/SegmentedRaftLogMetrics.java
index 810fcb003..b84726041 100644
--- 
a/ratis-server/src/main/java/org/apache/ratis/server/metrics/SegmentedRaftLogMetrics.java
+++ 
b/ratis-server/src/main/java/org/apache/ratis/server/metrics/SegmentedRaftLogMetrics.java
@@ -142,8 +142,8 @@ public class SegmentedRaftLogMetrics extends 
RaftLogMetricsBase {
     registry.counter(RAFT_LOG_APPEND_ENTRY_COUNT).inc();
   }
 
-  public Timer getRaftLogAppendEntryTimer() {
-    return getTimer(RAFT_LOG_APPEND_ENTRY_LATENCY);
+  public Timer.Context startAppendEntryTimer() {
+    return getTimer(RAFT_LOG_APPEND_ENTRY_LATENCY).time();
   }
 
   public Timer getRaftLogQueueTimer() {
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 0cb613ac0..5913f4adf 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
@@ -373,12 +373,12 @@ public class SegmentedRaftLog extends RaftLogBase {
 
   @Override
   protected CompletableFuture<Long> appendEntryImpl(LogEntryProto entry) {
-    final Timer.Context context = 
getRaftLogMetrics().getRaftLogAppendEntryTimer().time();
     checkLogState();
     if (LOG.isTraceEnabled()) {
       LOG.trace("{}: appendEntry {}", getName(), 
LogProtoUtils.toLogEntryString(entry));
     }
     try(AutoCloseableLock writeLock = writeLock()) {
+      final Timer.Context appendEntryTimerContext = 
getRaftLogMetrics().startAppendEntryTimer();
       validateLogEntry(entry);
       final LogSegment currentOpenSegment = cache.getOpenSegment();
       if (currentOpenSegment == null) {
@@ -413,12 +413,11 @@ public class SegmentedRaftLog extends RaftLogBase {
       } else {
         cache.appendEntry(entry, 
LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE);
       }
+      writeFuture.whenComplete((clientReply, exception) -> 
appendEntryTimerContext.stop());
       return writeFuture;
     } catch (Exception e) {
       LOG.error("{}: Failed to append {}", getName(), 
LogProtoUtils.toLogEntryString(entry), e);
       throw e;
-    } finally {
-      context.stop();
     }
   }
 

Reply via email to