szetszwo commented on code in PR #1206:
URL: https://github.com/apache/ratis/pull/1206#discussion_r1901220350


##########
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java:
##########
@@ -1641,9 +1645,11 @@ leaderId, getMemberId(), currentTerm, followerCommit, 
inconsistencyReplyNextInde
       state.updateConfiguration(entries);
     }
     future.join();
-
-    final List<CompletableFuture<Long>> futures = entries.isEmpty() ? 
Collections.emptyList()
+    final CompletableFuture<Void> appendLog = appendLogFuture.updateAndGet(f 
-> f.thenCompose(ignored -> {
+      final List<CompletableFuture<Long>> futures = entries.isEmpty() ? 
Collections.emptyList()
         : state.getLog().append(requestRef.delegate(entries));
+      return JavaUtils.allOf(futures);
+    }));

Review Comment:
   We should retain the ref first, i.e.
   ```java
   @@ -1641,9 +1645,8 @@ class RaftServerImpl implements RaftServer.Division,
          state.updateConfiguration(entries);
        }
        future.join();
   -
   -    final List<CompletableFuture<Long>> futures = entries.isEmpty() ? 
Collections.emptyList()
   -        : state.getLog().append(requestRef.delegate(entries));
   +    final CompletableFuture<Void> appendLog = entries.isEmpty()? 
appendLogFuture.get()
   +        : appendLog(requestRef.delegate(entries));
        proto.getCommitInfosList().forEach(commitInfoCache::update);
    
        CodeInjectionForTesting.execute(LOG_SYNC, getId(), null);
   @@ -1657,7 +1660,7 @@ class RaftServerImpl implements RaftServer.Division,
    
        final long commitIndex = effectiveCommitIndex(proto.getLeaderCommit(), 
previous, entries.size());
        final long matchIndex = isHeartbeat? RaftLog.INVALID_LOG_INDEX: 
entries.get(entries.size() - 1).getIndex();
   -    return JavaUtils.allOf(futures).whenCompleteAsync((r, t) -> {
   +    return appendLog.whenCompleteAsync((r, t) -> {
          followerState.ifPresent(fs -> 
fs.updateLastRpcTime(FollowerState.UpdateType.APPEND_COMPLETE));
          timer.stop();
        }, getServerExecutor()).thenApply(v -> {
   @@ -1675,6 +1678,13 @@ class RaftServerImpl implements RaftServer.Division,
        });
      }
    
   +  private CompletableFuture<Void> 
appendLog(ReferenceCountedObject<List<LogEntryProto>> entriesRef) {
   +    entriesRef.retain();
   +    return appendLogFuture.updateAndGet(f -> f.thenCompose(
   +            ignored -> JavaUtils.allOf(state.getLog().append(entriesRef))))
   +        .whenComplete((v, e) -> entriesRef.release());
   +  }
   +
      private long checkInconsistentAppendEntries(TermIndex previous, 
List<LogEntryProto> entries) {
        // Check if a snapshot installation through state machine is in 
progress.
        final long installSnapshot = 
snapshotInstallationHandler.getInProgressInstallSnapshotIndex();
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to