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
Merged
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -133,6 +133,7 @@
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import java.util.function.Supplier;
import java.util.stream.Collectors;
Expand Down Expand Up @@ -250,6 +251,8 @@ public long[] getFollowerNextIndices() {
private final AtomicBoolean firstElectionSinceStartup = new AtomicBoolean(true);
private final ThreadGroup threadGroup;

private final AtomicReference<CompletableFuture<Void>> appendLogFuture;

RaftServerImpl(RaftGroup group, StateMachine stateMachine, RaftServerProxy proxy, RaftStorage.StartupOption option)
throws IOException {
final RaftPeerId id = proxy.getId();
Expand Down Expand Up @@ -283,6 +286,7 @@ public long[] getFollowerNextIndices() {
this.transferLeadership = new TransferLeadership(this, properties);
this.snapshotRequestHandler = new SnapshotManagementRequestHandler(this);
this.snapshotInstallationHandler = new SnapshotInstallationHandler(this, properties);
this.appendLogFuture = new AtomicReference<>(CompletableFuture.completedFuture(null));

this.serverExecutor = ConcurrentUtils.newThreadPoolWithMax(
RaftServerConfigKeys.ThreadPool.serverCached(properties),
Expand Down Expand Up @@ -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);
}));
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();

proto.getCommitInfosList().forEach(commitInfoCache::update);

CodeInjectionForTesting.execute(LOG_SYNC, getId(), null);
Expand All @@ -1657,7 +1663,7 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde

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 -> {
Expand Down
Loading