This is an automated email from the ASF dual-hosted git repository.

szetszwo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ratis.git


The following commit(s) were added to refs/heads/master by this push:
     new 09f6b392b RATIS-2166. Update nextChunkIndex when snapshot is 
ALREADY_INSTALLED (#1159)
09f6b392b is described below

commit 09f6b392b3fb5738a1a8a8efa771b05bc1a543dd
Author: 133tosakarin <[email protected]>
AuthorDate: Wed Oct 9 12:22:15 2024 +0800

    RATIS-2166. Update nextChunkIndex when snapshot is ALREADY_INSTALLED (#1159)
---
 .../org/apache/ratis/server/impl/SnapshotInstallationHandler.java | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)

diff --git 
a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java
 
b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java
index 9b84e8f1d..537b384c6 100644
--- 
a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java
+++ 
b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java
@@ -185,6 +185,7 @@ class SnapshotInstallationHandler {
         // considering a follower peer requiring a snapshot installation does 
not
         // have a lot of requests
         if (state.getLog().getLastCommittedIndex() >= lastIncludedIndex) {
+          nextChunkIndex.set(snapshotChunkRequest.getRequestIndex() + 1);
           return toInstallSnapshotReplyProto(leaderId, getMemberId(),
               currentTerm, snapshotChunkRequest.getRequestIndex(), 
InstallSnapshotResult.ALREADY_INSTALLED);
         }
@@ -192,8 +193,11 @@ class SnapshotInstallationHandler {
         //TODO: We should only update State with installed snapshot once the 
request is done.
         state.installSnapshot(request);
 
-        int idx = nextChunkIndex.getAndIncrement();
-        Preconditions.assertEquals(snapshotChunkRequest.getRequestIndex(), 
idx, "nextChunkIndex");
+        final int expectedChunkIndex = nextChunkIndex.getAndIncrement();
+        if (expectedChunkIndex != snapshotChunkRequest.getRequestIndex()) {
+          throw new IOException("Unexpected request chunk index: " + 
snapshotChunkRequest.getRequestIndex()
+              + " (the expected index is " + expectedChunkIndex + ")");
+        }
         // update the committed index
         // re-load the state machine if this is the last chunk
         if (snapshotChunkRequest.getDone()) {

Reply via email to