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

rpuch 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 ac40e77614 IGNITE-23581 Fix 
ItMetastorageGroupDisasterRecoveryTest#oldMgLeaderDoesNotHijackLeadership() 
(#4661)
ac40e77614 is described below

commit ac40e77614b7eb20030a812c502de987e14f2f91
Author: Roman Puchkovskiy <[email protected]>
AuthorDate: Thu Oct 31 17:26:09 2024 +0400

    IGNITE-23581 Fix 
ItMetastorageGroupDisasterRecoveryTest#oldMgLeaderDoesNotHijackLeadership() 
(#4661)
    
    * Make sure Metastorage does not diverge in this test (it's an unrelated 
matter)
    * Include stack trace info in a failed assertion description
---
 .../CompletableFutureExceptionMatcher.java         |  7 +++++-
 .../ItMetastorageGroupDisasterRecoveryTest.java    | 28 ++++++++++++----------
 2 files changed, 22 insertions(+), 13 deletions(-)

diff --git 
a/modules/core/src/testFixtures/java/org/apache/ignite/internal/testframework/matchers/CompletableFutureExceptionMatcher.java
 
b/modules/core/src/testFixtures/java/org/apache/ignite/internal/testframework/matchers/CompletableFutureExceptionMatcher.java
index e64aee13f3..2797bc69b0 100644
--- 
a/modules/core/src/testFixtures/java/org/apache/ignite/internal/testframework/matchers/CompletableFutureExceptionMatcher.java
+++ 
b/modules/core/src/testFixtures/java/org/apache/ignite/internal/testframework/matchers/CompletableFutureExceptionMatcher.java
@@ -26,6 +26,7 @@ import java.util.Arrays;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import org.apache.ignite.internal.util.ExceptionUtils;
 import org.hamcrest.Description;
 import org.hamcrest.Matcher;
 import org.hamcrest.TypeSafeMatcher;
@@ -106,7 +107,7 @@ public class CompletableFutureExceptionMatcher extends 
TypeSafeMatcher<Completab
             try {
                 item.join();
             } catch (Exception e) {
-                mismatchDescription.appendText("was completed exceptionally 
with ").appendValue(unwrapCause(e));
+                mismatchDescription.appendText("was completed exceptionally 
with ").appendValue(describeThrowable(unwrapCause(e)));
             }
         } else if (item.isDone()) {
             mismatchDescription.appendText("was completed successfully");
@@ -115,6 +116,10 @@ public class CompletableFutureExceptionMatcher extends 
TypeSafeMatcher<Completab
         }
     }
 
+    private static String describeThrowable(Throwable throwable) {
+        return throwable.toString() + ", stack trace " + 
ExceptionUtils.getFullStackTrace(throwable);
+    }
+
     private boolean matchesWithCause(Throwable e) {
         for (Throwable current = e; current != null; current = 
current.getCause()) {
             if (matchesException(current) || 
Arrays.stream(current.getSuppressed()).anyMatch(this::matchesWithCause)) {
diff --git 
a/modules/system-disaster-recovery/src/integrationTest/java/org/apache/ignite/internal/disaster/system/ItMetastorageGroupDisasterRecoveryTest.java
 
b/modules/system-disaster-recovery/src/integrationTest/java/org/apache/ignite/internal/disaster/system/ItMetastorageGroupDisasterRecoveryTest.java
index 3b965544c4..42bd6f55d9 100644
--- 
a/modules/system-disaster-recovery/src/integrationTest/java/org/apache/ignite/internal/disaster/system/ItMetastorageGroupDisasterRecoveryTest.java
+++ 
b/modules/system-disaster-recovery/src/integrationTest/java/org/apache/ignite/internal/disaster/system/ItMetastorageGroupDisasterRecoveryTest.java
@@ -209,13 +209,12 @@ class ItMetastorageGroupDisasterRecoveryTest extends 
ItSystemGroupDisasterRecove
         startAndInitCluster(2, new int[]{0}, new int[]{1});
         waitTillClusterStateIsSavedToVaultOnConductor(0);
 
-        // This makes the MG majority go away.
-        cluster.stopNode(1);
-
-        IgniteImpl igniteImpl0BeforeRestart = igniteImpl(0);
-
-        assertThatMgHasNoMajority(igniteImpl0BeforeRestart);
+        // Copy Metastorage state from old leader (1) to future leader (0) to 
make sure that 1 is not ahead of 0 and there will be
+        // no Metastorage divergence when we make 0 new leader and migrate 1 
to cluster again.
+        // This stops both nodes.
+        makeSure2NodesHaveSameMetastorageState(1, 0);
 
+        cluster.startEmbeddedNode(0);
         initiateMgRepairVia(0, 1, 0);
 
         IgniteImpl restartedIgniteImpl0 = waitTillNodeRestartsInternally(0);
@@ -257,14 +256,10 @@ class ItMetastorageGroupDisasterRecoveryTest extends 
ItSystemGroupDisasterRecove
         startAndInitCluster(2, new int[]{0}, new int[]{1});
         waitTillClusterStateIsSavedToVaultOnConductor(0);
 
-        ComponentWorkingDir msWorkDir0 = igniteImpl(0).metastorageWorkDir();
-        ComponentWorkingDir msWorkDir1 = igniteImpl(1).metastorageWorkDir();
-
-        IntStream.of(0, 1).parallel().forEach(cluster::stopNode);
-
         // Copy Metastorage state from old leader (1) to future leader (0) to 
make sure that 1 is not ahead of 0 and there will be
         // no Metastorage divergence when we make 0 new leader and migrate 1 
to cluster again.
-        copyMetastorageState(msWorkDir1, msWorkDir0);
+        // This stops both nodes.
+        makeSure2NodesHaveSameMetastorageState(1, 0);
 
         // Repair MG with just node 0 in CMG.
         cluster.startEmbeddedNode(0);
@@ -279,6 +274,15 @@ class ItMetastorageGroupDisasterRecoveryTest extends 
ItSystemGroupDisasterRecove
         assertTopologyContainsNode(1, topologySnapshot);
     }
 
+    private void makeSure2NodesHaveSameMetastorageState(int leaderIndex, int 
followerIndex) throws IOException {
+        ComponentWorkingDir followerMsWorkDir = 
igniteImpl(followerIndex).metastorageWorkDir();
+        ComponentWorkingDir leaderMsWorkDir = 
igniteImpl(leaderIndex).metastorageWorkDir();
+
+        IntStream.of(followerIndex, 
leaderIndex).parallel().forEach(cluster::stopNode);
+
+        copyMetastorageState(leaderMsWorkDir, followerMsWorkDir);
+    }
+
     private static void copyMetastorageState(ComponentWorkingDir source, 
ComponentWorkingDir dest) throws IOException {
         replaceDir(source.dbPath(), dest.dbPath());
         replaceDir(source.raftLogPath(), dest.raftLogPath());

Reply via email to