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

tkalkirill pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new 94dc641d0e IGNITE-18864 Add a test for a leader change during full 
rebalance to a leader with enough log (#1750)
94dc641d0e is described below

commit 94dc641d0e8897ff763ee43b2c68afe5abc73bcb
Author: Roman Puchkovskiy <[email protected]>
AuthorDate: Fri Mar 3 20:53:04 2023 +0400

    IGNITE-18864 Add a test for a leader change during full rebalance to a 
leader with enough log (#1750)
---
 .../raftsnapshot/ItTableRaftSnapshotsTest.java     | 52 ++++++++++++++++++++++
 1 file changed, 52 insertions(+)

diff --git 
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/raftsnapshot/ItTableRaftSnapshotsTest.java
 
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/raftsnapshot/ItTableRaftSnapshotsTest.java
index 31b005d23a..414eb9cdb1 100644
--- 
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/raftsnapshot/ItTableRaftSnapshotsTest.java
+++ 
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/raftsnapshot/ItTableRaftSnapshotsTest.java
@@ -25,6 +25,7 @@ import static 
org.apache.ignite.internal.testframework.matchers.CompletableFutur
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.is;
+import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
 
@@ -812,4 +813,55 @@ class ItTableRaftSnapshotsTest extends 
IgniteIntegrationTest {
 
         replicatorLogger.addHandler(replicaLoggerHandler);
     }
+
+    /**
+     * This tests the following schenario.
+     *
+     * <ol>
+     *     <li>
+     *         A snapshot installation is started from Node A that is a leader 
because A does not have enough RAFT log to feed a follower
+     *         with AppendEntries
+     *     </li>
+     *     <li>It is cancelled in the middle</li>
+     *     <li>Node B is elected as a leader; B has enough log to feed the 
follower with AppendEntries</li>
+     *     <li>The follower gets data from the leader using AppendEntries, not 
using InstallSnapshot</li>>
+     * </ol>
+     */
+    @Test
+    void testChangeLeaderDuringSnapshotInstallationToLeaderWithEnoughLog() 
throws Exception {
+        CompletableFuture<Void> sentSnapshotMetaResponseFormNode0Future = new 
CompletableFuture<>();
+
+        
prepareClusterForInstallingSnapshotToNode2(NodeKnockout.PARTITION_NETWORK, 
DEFAULT_STORAGE_ENGINE, cluster -> {
+            // Let's hang the InstallSnapshot in the "middle" from the leader 
with index 0.
+            
cluster.node(0).dropMessages(dropSnapshotMetaResponse(sentSnapshotMetaResponseFormNode0Future));
+        });
+
+        CompletableFuture<Void> installSnapshotSuccessfulFuture = new 
CompletableFuture<>();
+
+        listenForSnapshotInstalledSuccessFromLogger(1, 2, 
installSnapshotSuccessfulFuture);
+
+        // Return node 2.
+        cluster.reanimateNode(2, NodeKnockout.PARTITION_NETWORK);
+
+        // Waiting for the InstallSnapshot from node 2 to hang in the "middle".
+        assertThat(sentSnapshotMetaResponseFormNode0Future, willSucceedIn(1, 
TimeUnit.MINUTES));
+
+        // Change the leader to node 1.
+        transferLeadershipOnSolePartitionTo(1);
+
+        boolean replicated = waitForCondition(() -> {
+            List<IgniteBiTuple<Integer, String>> rows = queryWithRetry(2, 
"select * from test", ItTableRaftSnapshotsTest::readRows);
+            return rows.size() == 1;
+        }, 20_000);
+
+        assertTrue(replicated, "Data has not been replicated to node 2 in 
time");
+
+        // No snapshot must be installed.
+        assertFalse(installSnapshotSuccessfulFuture.isDone());
+
+        // Make sure the rebalancing is complete.
+        List<IgniteBiTuple<Integer, String>> rows = queryWithRetry(2, "select 
* from test", ItTableRaftSnapshotsTest::readRows);
+
+        assertThat(rows, is(List.of(new IgniteBiTuple<>(1, "one"))));
+    }
 }

Reply via email to