1717 */
1818package org .apache .ratis .server .impl ;
1919
20- import java .util .concurrent .CountDownLatch ;
2120import org .apache .ratis .client .impl .ClientProtoUtils ;
2221import org .apache .ratis .conf .RaftProperties ;
2322import org .apache .ratis .metrics .Timekeeper ;
2423import org .apache .ratis .proto .RaftProtos .AppendEntriesReplyProto ;
2524import org .apache .ratis .proto .RaftProtos .AppendEntriesReplyProto .AppendResult ;
2625import org .apache .ratis .proto .RaftProtos .AppendEntriesRequestProto ;
2726import org .apache .ratis .proto .RaftProtos .CommitInfoProto ;
28- import org .apache .ratis .proto .RaftProtos .LogInfoProto ;
2927import org .apache .ratis .proto .RaftProtos .InstallSnapshotReplyProto ;
3028import org .apache .ratis .proto .RaftProtos .InstallSnapshotRequestProto ;
3129import org .apache .ratis .proto .RaftProtos .InstallSnapshotResult ;
3230import org .apache .ratis .proto .RaftProtos .LogEntryProto ;
31+ import org .apache .ratis .proto .RaftProtos .LogInfoProto ;
3332import org .apache .ratis .proto .RaftProtos .RaftClientRequestProto ;
3433import org .apache .ratis .proto .RaftProtos .RaftClientRequestProto .TypeCase ;
3534import org .apache .ratis .proto .RaftProtos .RaftConfigurationProto ;
8281import org .apache .ratis .server .RaftServerRpc ;
8382import org .apache .ratis .server .impl .LeaderElection .Phase ;
8483import org .apache .ratis .server .impl .RetryCacheImpl .CacheEntry ;
84+ import org .apache .ratis .server .impl .ServerImplUtils .ConsecutiveIndices ;
85+ import org .apache .ratis .server .impl .ServerImplUtils .NavigableIndices ;
8586import org .apache .ratis .server .leader .LeaderState ;
8687import org .apache .ratis .server .metrics .LeaderElectionMetrics ;
8788import org .apache .ratis .server .metrics .RaftServerMetricsImpl ;
112113import org .apache .ratis .util .ProtoUtils ;
113114import org .apache .ratis .util .TimeDuration ;
114115import org .apache .ratis .util .function .CheckedSupplier ;
116+ import org .apache .ratis .util .function .UncheckedAutoCloseableSupplier ;
115117
116118import java .io .File ;
117119import java .io .IOException ;
126128import java .util .Set ;
127129import java .util .concurrent .CompletableFuture ;
128130import java .util .concurrent .CompletionException ;
131+ import java .util .concurrent .CountDownLatch ;
129132import java .util .concurrent .ExecutionException ;
130133import java .util .concurrent .ExecutorService ;
131134import java .util .concurrent .ThreadLocalRandom ;
@@ -250,6 +253,7 @@ public long[] getFollowerNextIndices() {
250253 private final ThreadGroup threadGroup ;
251254
252255 private final AtomicReference <CompletableFuture <Void >> appendLogFuture ;
256+ private final NavigableIndices appendLogTermIndices = new NavigableIndices ();
253257
254258 RaftServerImpl (RaftGroup group , StateMachine stateMachine , RaftServerProxy proxy , RaftStorage .StartupOption option )
255259 throws IOException {
@@ -1621,9 +1625,15 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde
16211625 return reply ;
16221626 });
16231627 }
1628+
16241629 private CompletableFuture <Void > appendLog (List <LogEntryProto > entries ) {
1630+ final List <ConsecutiveIndices > entriesTermIndices = ConsecutiveIndices .convert (entries );
1631+ appendLogTermIndices .append (entriesTermIndices );
16251632 return appendLogFuture .updateAndGet (f -> f .thenCompose (
1626- ignored -> JavaUtils .allOf (state .getLog ().append (entries ))));
1633+ ignored -> JavaUtils .allOf (state .getLog ().append (entries ))))
1634+ .whenComplete ((v , e ) -> {
1635+ appendLogTermIndices .removeExisting (entriesTermIndices );
1636+ });
16271637 }
16281638
16291639 private long checkInconsistentAppendEntries (TermIndex previous , List <LogEntryProto > entries ) {
@@ -1650,7 +1660,7 @@ private long checkInconsistentAppendEntries(TermIndex previous, List<LogEntryPro
16501660 }
16511661
16521662 // Check if "previous" is contained in current state.
1653- if (previous != null && !state .containsTermIndex (previous )) {
1663+ if (previous != null && !( appendLogTermIndices . contains ( previous ) || state .containsTermIndex (previous ) )) {
16541664 final long replyNextIndex = Math .min (state .getNextIndex (), previous .getIndex ());
16551665 LOG .info ("{}: Failed appendEntries as previous log entry ({}) is not found" , getMemberId (), previous );
16561666 return replyNextIndex ;
0 commit comments