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 ;
113114import org .apache .ratis .util .ReferenceCountedObject ;
114115import org .apache .ratis .util .TimeDuration ;
115116import org .apache .ratis .util .function .CheckedSupplier ;
117+ import org .apache .ratis .util .function .UncheckedAutoCloseableSupplier ;
116118
117119import java .io .File ;
118120import java .io .IOException ;
128130import java .util .Set ;
129131import java .util .concurrent .CompletableFuture ;
130132import java .util .concurrent .CompletionException ;
133+ import java .util .concurrent .CountDownLatch ;
131134import java .util .concurrent .ExecutionException ;
132135import java .util .concurrent .ExecutorService ;
133136import java .util .concurrent .ThreadLocalRandom ;
@@ -260,6 +263,7 @@ public long[] getFollowerMatchIndices() {
260263 private final ThreadGroup threadGroup ;
261264
262265 private final AtomicReference <CompletableFuture <Void >> appendLogFuture ;
266+ private final NavigableIndices appendLogTermIndices = new NavigableIndices ();
263267
264268 RaftServerImpl (RaftGroup group , StateMachine stateMachine , RaftServerProxy proxy , RaftStorage .StartupOption option )
265269 throws IOException {
@@ -1687,10 +1691,19 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde
16871691 });
16881692 }
16891693 private CompletableFuture <Void > appendLog (ReferenceCountedObject <List <LogEntryProto >> entriesRef ) {
1694+ final List <ConsecutiveIndices > entriesTermIndices ;
1695+ try (UncheckedAutoCloseableSupplier <List <LogEntryProto >> entries = entriesRef .retainAndReleaseOnClose ()) {
1696+ entriesTermIndices = ConsecutiveIndices .convert (entries .get ());
1697+ appendLogTermIndices .append (entriesTermIndices );
1698+ }
1699+
16901700 entriesRef .retain ();
16911701 return appendLogFuture .updateAndGet (f -> f .thenCompose (
16921702 ignored -> JavaUtils .allOf (state .getLog ().append (entriesRef ))))
1693- .whenComplete ((v , e ) -> entriesRef .release ());
1703+ .whenComplete ((v , e ) -> {
1704+ entriesRef .release ();
1705+ appendLogTermIndices .removeExisting (entriesTermIndices );
1706+ });
16941707 }
16951708
16961709 private long checkInconsistentAppendEntries (TermIndex previous , List <LogEntryProto > entries ) {
@@ -1717,7 +1730,7 @@ private long checkInconsistentAppendEntries(TermIndex previous, List<LogEntryPro
17171730 }
17181731
17191732 // Check if "previous" is contained in current state.
1720- if (previous != null && !state .containsTermIndex (previous )) {
1733+ if (previous != null && !( appendLogTermIndices . contains ( previous ) || state .containsTermIndex (previous ) )) {
17211734 final long replyNextIndex = Math .min (state .getNextIndex (), previous .getIndex ());
17221735 LOG .info ("{}: Failed appendEntries as previous log entry ({}) is not found" , getMemberId (), previous );
17231736 return replyNextIndex ;
0 commit comments