8181import org .apache .ratis .server .RaftServerRpc ;
8282import org .apache .ratis .server .impl .LeaderElection .Phase ;
8383import org .apache .ratis .server .impl .RetryCacheImpl .CacheEntry ;
84- import org .apache .ratis .server .impl .ServerImplUtils .ConsecutiveIndices ;
8584import org .apache .ratis .server .impl .ServerImplUtils .NavigableIndices ;
8685import org .apache .ratis .server .leader .LeaderState .StepDownReason ;
8786import org .apache .ratis .server .metrics .LeaderElectionMetrics ;
133132import java .util .concurrent .ThreadLocalRandom ;
134133import java .util .concurrent .TimeUnit ;
135134import java .util .concurrent .atomic .AtomicBoolean ;
136- import java .util .concurrent .atomic .AtomicReference ;
137135import java .util .function .Function ;
138136import java .util .function .Supplier ;
139137import java .util .stream .Collectors ;
@@ -260,8 +258,7 @@ public long[] getFollowerMatchIndices() {
260258 private final AtomicBoolean firstElectionSinceStartup = new AtomicBoolean (true );
261259 private final ThreadGroup threadGroup ;
262260
263- private final AtomicReference <CompletableFuture <Void >> appendLogFuture ;
264- private final NavigableIndices appendLogTermIndices = new NavigableIndices ();
261+ private final NavigableIndices appendLogTermIndices ;
265262
266263 RaftServerImpl (RaftGroup group , StateMachine stateMachine , RaftServerProxy proxy , RaftStorage .StartupOption option )
267264 throws IOException {
@@ -296,7 +293,8 @@ public long[] getFollowerMatchIndices() {
296293 this .transferLeadership = new TransferLeadership (this , properties );
297294 this .snapshotRequestHandler = new SnapshotManagementRequestHandler (this );
298295 this .snapshotInstallationHandler = new SnapshotInstallationHandler (this , properties );
299- this .appendLogFuture = new AtomicReference <>(CompletableFuture .completedFuture (null ));
296+ this .appendLogTermIndices = RaftServerConfigKeys .Log .appendEntriesComposeEnabled (properties ) ?
297+ new NavigableIndices () : null ;
300298
301299 this .serverExecutor = ConcurrentUtils .newThreadPoolWithMax (
302300 RaftServerConfigKeys .ThreadPool .serverCached (properties ),
@@ -1620,7 +1618,8 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde
16201618 state .updateConfiguration (entries );
16211619 }
16221620 future .join ();
1623- final CompletableFuture <Void > appendLog = entries .isEmpty ()? CompletableFuture .completedFuture (null )
1621+ final CompletableFuture <Void > appendFuture = entries .isEmpty ()? CompletableFuture .completedFuture (null )
1622+ : appendLogTermIndices != null ? appendLogTermIndices .append (entries , this ::appendLog )
16241623 : appendLog (entries );
16251624
16261625 proto .getCommitInfosList ().forEach (commitInfoCache ::update );
@@ -1636,7 +1635,7 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde
16361635
16371636 final long commitIndex = effectiveCommitIndex (proto .getLeaderCommit (), previous , entries .size ());
16381637 final long matchIndex = isHeartbeat ? RaftLog .INVALID_LOG_INDEX : entries .get (entries .size () - 1 ).getIndex ();
1639- return appendLog .whenCompleteAsync ((r , t ) -> {
1638+ return appendFuture .whenCompleteAsync ((r , t ) -> {
16401639 followerState .ifPresent (fs -> fs .updateLastRpcTime (FollowerState .UpdateType .APPEND_COMPLETE ));
16411640 timer .stop ();
16421641 }, getServerExecutor ()).thenApply (v -> {
@@ -1654,16 +1653,8 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde
16541653 });
16551654 }
16561655 private CompletableFuture <Void > appendLog (List <LogEntryProto > entries ) {
1657- final List <ConsecutiveIndices > entriesTermIndices = ConsecutiveIndices .convert (entries );
1658- if (!appendLogTermIndices .append (entriesTermIndices )) {
1659- // index already exists, return the last future
1660- return appendLogFuture .get ();
1661- }
1662-
1663-
1664- return appendLogFuture .updateAndGet (f -> f .thenComposeAsync (
1665- ignored -> JavaUtils .allOf (state .getLog ().append (entries )), serverExecutor ))
1666- .whenComplete ((v , e ) -> appendLogTermIndices .removeExisting (entriesTermIndices ));
1656+ return CompletableFuture .completedFuture (null )
1657+ .thenComposeAsync (dummy -> JavaUtils .allOf (state .getLog ().append (entries )), serverExecutor );
16671658 }
16681659
16691660 private long checkInconsistentAppendEntries (TermIndex previous , List <LogEntryProto > entries ) {
@@ -1690,7 +1681,9 @@ private long checkInconsistentAppendEntries(TermIndex previous, List<LogEntryPro
16901681 }
16911682
16921683 // Check if "previous" is contained in current state.
1693- if (previous != null && !(appendLogTermIndices .contains (previous ) || state .containsTermIndex (previous ))) {
1684+ if (previous != null
1685+ && !(appendLogTermIndices != null && appendLogTermIndices .contains (previous ))
1686+ && !state .containsTermIndex (previous )) {
16941687 final long replyNextIndex = Math .min (state .getNextIndex (), previous .getIndex ());
16951688 LOG .info ("{}: Failed appendEntries as previous log entry ({}) is not found" , getMemberId (), previous );
16961689 return replyNextIndex ;
0 commit comments