Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

RATIS-2235. Allow only one thread to perform appendLog #1206

Merged
merged 3 commits into from
Jan 6, 2025

Conversation

SzyWilliam
Copy link
Member

@SzyWilliam SzyWilliam marked this pull request as draft January 2, 2025 15:57
Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@SzyWilliam , thanks for working this! Please see a comment inlined.

Comment on lines 1648 to 1652
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);
}));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should retain the ref first, i.e.

@@ -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();

@SzyWilliam SzyWilliam marked this pull request as ready for review January 3, 2025 14:54
@SzyWilliam
Copy link
Member Author

@szetszwo Thanks for reviewing this pr and help with the code. Now I learned how to use zero-copy in async programming, thank you!

@szetszwo
Copy link
Contributor

szetszwo commented Jan 3, 2025

The test failures seem related. Not yet sure what is the problem.

@@ -1641,9 +1645,9 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde
state.updateConfiguration(entries);
}
future.join();
final CompletableFuture<Void> appendLog = entries.isEmpty()? appendLogFuture.get()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@SzyWilliam , I see the problem now -- when it is empty, it should not wait for the previous append. Otherwise, it blocks the heartbeats.

    final CompletableFuture<Void> appendLog = entries.isEmpty()? CompletableFuture.completedFuture(null)

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That is a tricky one. Thanks for spotting and troubleshooting this!

@SzyWilliam
Copy link
Member Author

@szetszwo the tests are all passed now, PTAL ~

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 the change looks good.

@szetszwo szetszwo merged commit 9b74401 into apache:master Jan 6, 2025
13 checks passed
@SzyWilliam
Copy link
Member Author

@szetszwo Thanks for reviewing and merging this PR!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants