From 09f6b392b3fb5738a1a8a8efa771b05bc1a543dd Mon Sep 17 00:00:00 2001 From: 133tosakarin <97331129+133tosakarin@users.noreply.github.com> Date: Wed, 9 Oct 2024 12:22:15 +0800 Subject: [PATCH] RATIS-2166. Update nextChunkIndex when snapshot is ALREADY_INSTALLED (#1159) --- .../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 9b84e8f1d1..537b384c64 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 @@ private InstallSnapshotReplyProto checkAndInstallSnapshot(InstallSnapshotRequest // 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 @@ private InstallSnapshotReplyProto checkAndInstallSnapshot(InstallSnapshotRequest //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()) {