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


##########
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java:
##########
@@ -260,8 +261,10 @@ public long[] getFollowerMatchIndices() {
   private final AtomicBoolean firstElectionSinceStartup = new 
AtomicBoolean(true);
   private final ThreadGroup threadGroup;
 
+  // Conditional fields for appendLog synchronization (RATIS-2235)
   private final AtomicReference<CompletableFuture<Void>> appendLogFuture;
-  private final NavigableIndices appendLogTermIndices = new NavigableIndices();
+  private final NavigableIndices appendLogTermIndices;
+  private final boolean appendEntriesSynchronized;

Review Comment:
   Let's simply check if appendLogTermIndices == null instead of adding a new 
boolean.



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java:
##########
@@ -296,7 +299,21 @@ public long[] getFollowerMatchIndices() {
     this.transferLeadership = new TransferLeadership(this, properties);
     this.snapshotRequestHandler = new SnapshotManagementRequestHandler(this);
     this.snapshotInstallationHandler = new SnapshotInstallationHandler(this, 
properties);
-    this.appendLogFuture = new 
AtomicReference<>(CompletableFuture.completedFuture(null));
+
+    // Initialize appendLog synchronization components conditionally 
(RATIS-2235)
+    // Use system property for single-file configuration (avoids updating 
ratis-server-api jar)
+    this.appendEntriesSynchronized = Boolean.parseBoolean(
+        System.getProperty("raft.server.log.append.entries.synchronized", 
"true"));
+    LOG.info("{}: appendLog synchronization mode: {}", getMemberId(),
+        appendEntriesSynchronized ? "synchronized" : "parallel");

Review Comment:
   >  Use system property for single-file configuration (avoids updating 
ratis-server-api jar)
   
   This conf should be added to `RaftServerConfigKeys` but not adding a new way 
for setting it.
   
   For the property name, let's name it 
"raft.server.log.append-entries.compose.enabled".  BTW, appendEntries is always 
synchronized.   The change of RATIS-2235 was to compose the previous future. 



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java:
##########
@@ -1620,8 +1637,16 @@ leaderId, getMemberId(), currentTerm, followerCommit, 
inconsistencyReplyNextInde
       state.updateConfiguration(entries);
     }
     future.join();
-    final CompletableFuture<Void> appendLog = entries.isEmpty()? 
CompletableFuture.completedFuture(null)
-        : appendLog(entries);
+
+    // Conditional appendLog based on configuration (RATIS-2235)
+    final CompletableFuture<Void> appendOperation;
+    if (appendEntriesSynchronized && !entries.isEmpty()) {
+      appendOperation = appendLogSynchronized(entries);
+    } else {
+      final List<CompletableFuture<Long>> futures = entries.isEmpty() ? 
Collections.emptyList()
+          : state.getLog().append(entries);

Review Comment:
   We should run it with `serverExecutor`.



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java:
##########
@@ -1653,7 +1678,12 @@ leaderId, getMemberId(), currentTerm, followerCommit, 
inconsistencyReplyNextInde
       return reply;
     });
   }
-  private CompletableFuture<Void> appendLog(List<LogEntryProto> entries) {
+
+  /**
+   * Synchronized appendLog operation to ensure only one thread performs 
appendLog at a time.
+   * This is the RATIS-2235 implementation that can be enabled via 
configuration.
+   */
+  private CompletableFuture<Void> appendLogSynchronized(List<LogEntryProto> 
entries) {

Review Comment:
   After this change, this method and `appendLogFuture` are only used when it 
is enabled.  Let's move them to `NavigableIndices`.



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java:
##########
@@ -1620,8 +1637,16 @@ leaderId, getMemberId(), currentTerm, followerCommit, 
inconsistencyReplyNextInde
       state.updateConfiguration(entries);
     }
     future.join();
-    final CompletableFuture<Void> appendLog = entries.isEmpty()? 
CompletableFuture.completedFuture(null)
-        : appendLog(entries);
+
+    // Conditional appendLog based on configuration (RATIS-2235)
+    final CompletableFuture<Void> appendOperation;

Review Comment:
   Let's call it `appendFuture`.



-- 
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