JAkutenshi commented on code in PR #7395:
URL: https://github.com/apache/ignite-3/pull/7395#discussion_r2693392970


##########
modules/transactions/src/integrationTest/java/org/apache/ignite/internal/disaster/ItDisasterRecoveryReconfigurationTest.java:
##########
@@ -1490,13 +1502,21 @@ void testAssignmentsChainUpdatedOnAutomaticReset() 
throws Exception {
 
         stopNodesInParallel(3, 4, 5, 6);
 
-        int firstPhaseReset = getNodeForFirstPhaseReset(partId, 0, 1, 2);
-
-        Assignments link2FirstPhaseReset = Assignments.of(Set.of(
-                Assignment.forPeer(node(firstPhaseReset).name())
-        ), timestamp);
+        // Wait for first phase of reset to complete.
+        // The reset selects the node with the highest raft log index (or 
lexicographically first on tie).
+        Set<String> aliveNodes = Set.of(node(0).name(), node(1).name(), 
node(2).name());
+        await().atMost(60, SECONDS)
+                .until(() -> {
+                    Assignments stable = getStableAssignments(node0, partId);
+                    return stable != null
+                            && stable.nodes().size() == 1
+                            && 
aliveNodes.contains(stable.nodes().iterator().next().consistentId());
+                });
 
-        assertStableAssignments(node0, partId, link2FirstPhaseReset, 60_000);
+        // Read the actual stable assignments - this is what the system 
selected.
+        Assignments link2FirstPhaseReset = getStableAssignments(node0, partId);
+        String selectedNode = 
link2FirstPhaseReset.nodes().iterator().next().consistentId();

Review Comment:
   May selected node there and the one from await differ? I guess no because we 
already have to had 1 node assignments but still.



##########
modules/transactions/src/integrationTest/java/org/apache/ignite/internal/disaster/ItDisasterRecoveryReconfigurationTest.java:
##########
@@ -1490,13 +1502,21 @@ void testAssignmentsChainUpdatedOnAutomaticReset() 
throws Exception {
 
         stopNodesInParallel(3, 4, 5, 6);
 
-        int firstPhaseReset = getNodeForFirstPhaseReset(partId, 0, 1, 2);
-
-        Assignments link2FirstPhaseReset = Assignments.of(Set.of(
-                Assignment.forPeer(node(firstPhaseReset).name())
-        ), timestamp);
+        // Wait for first phase of reset to complete.
+        // The reset selects the node with the highest raft log index (or 
lexicographically first on tie).
+        Set<String> aliveNodes = Set.of(node(0).name(), node(1).name(), 
node(2).name());
+        await().atMost(60, SECONDS)
+                .until(() -> {
+                    Assignments stable = getStableAssignments(node0, partId);
+                    return stable != null
+                            && stable.nodes().size() == 1
+                            && 
aliveNodes.contains(stable.nodes().iterator().next().consistentId());

Review Comment:
   Is there any case why non stopped node may be in stable assignments? I mean 
isn't `stable != null && stable.nodes().size() == 1` enough?



##########
modules/raft-api/src/main/java/org/apache/ignite/internal/raft/rebalance/RaftConfigurationException.java:
##########
@@ -15,26 +15,19 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.pagememory;
+package org.apache.ignite.internal.raft.rebalance;
+
+import org.apache.ignite.internal.lang.IgniteInternalCheckedException;
 
 /**
- * A test implementation of the {@link DataRegion} interface that returns the 
associated {@link PageMemory} instance.
- * The region size is fixed at {@code 0} for testing purposes.
+ * Exception thrown when a RAFT configuration change fails due to 
non-recoverable errors
+ * such as the node being in an invalid state (EPERM) or invalid arguments 
(EINVAL).
+ *
+ * <p>This exception is non-recoverable and should not be retried.
  */
-public class TestDataRegion<T extends PageMemory> implements DataRegion<T> {
-    private final T pageMemory;
-
-    public TestDataRegion(T pageMemory) {
-        this.pageMemory = pageMemory;
-    }
-
-    @Override
-    public T pageMemory() {
-        return pageMemory;
-    }
+public class RaftConfigurationException extends IgniteInternalCheckedException 
{

Review Comment:
   Mostly a question: should we some how differ RAFT Configuration from 
`org.apache.ignite.internal.raft.configuration` as `ConfigurationProperty` and 
`RaftPeerConfiguration`?



##########
modules/transactions/src/integrationTest/java/org/apache/ignite/internal/disaster/ItDisasterRecoveryReconfigurationTest.java:
##########
@@ -1490,13 +1502,21 @@ void testAssignmentsChainUpdatedOnAutomaticReset() 
throws Exception {
 
         stopNodesInParallel(3, 4, 5, 6);
 
-        int firstPhaseReset = getNodeForFirstPhaseReset(partId, 0, 1, 2);
-
-        Assignments link2FirstPhaseReset = Assignments.of(Set.of(
-                Assignment.forPeer(node(firstPhaseReset).name())
-        ), timestamp);
+        // Wait for first phase of reset to complete.
+        // The reset selects the node with the highest raft log index (or 
lexicographically first on tie).
+        Set<String> aliveNodes = Set.of(node(0).name(), node(1).name(), 
node(2).name());
+        await().atMost(60, SECONDS)
+                .until(() -> {
+                    Assignments stable = getStableAssignments(node0, partId);
+                    return stable != null
+                            && stable.nodes().size() == 1
+                            && 
aliveNodes.contains(stable.nodes().iterator().next().consistentId());
+                });
 
-        assertStableAssignments(node0, partId, link2FirstPhaseReset, 60_000);
+        // Read the actual stable assignments - this is what the system 
selected.
+        Assignments link2FirstPhaseReset = getStableAssignments(node0, partId);

Review Comment:
   And also I'd to collapse to the single statement without `To -> 2` kind of 
reduction.
   `getStableAssignments(node0, 
partId).nodes().iterator().next().consistentId()`



##########
modules/transactions/src/integrationTest/java/org/apache/ignite/internal/disaster/ItDisasterRecoveryReconfigurationTest.java:
##########
@@ -1490,13 +1502,21 @@ void testAssignmentsChainUpdatedOnAutomaticReset() 
throws Exception {
 
         stopNodesInParallel(3, 4, 5, 6);
 
-        int firstPhaseReset = getNodeForFirstPhaseReset(partId, 0, 1, 2);
-
-        Assignments link2FirstPhaseReset = Assignments.of(Set.of(
-                Assignment.forPeer(node(firstPhaseReset).name())
-        ), timestamp);
+        // Wait for first phase of reset to complete.
+        // The reset selects the node with the highest raft log index (or 
lexicographically first on tie).
+        Set<String> aliveNodes = Set.of(node(0).name(), node(1).name(), 
node(2).name());
+        await().atMost(60, SECONDS)
+                .until(() -> {
+                    Assignments stable = getStableAssignments(node0, partId);
+                    return stable != null
+                            && stable.nodes().size() == 1
+                            && 
aliveNodes.contains(stable.nodes().iterator().next().consistentId());

Review Comment:
   The same I'd like to exteract it to this class static method to reduce code 
duplicity.



##########
modules/transactions/src/integrationTest/java/org/apache/ignite/internal/disaster/ItDisasterRecoveryReconfigurationTest.java:
##########
@@ -1490,13 +1502,21 @@ void testAssignmentsChainUpdatedOnAutomaticReset() 
throws Exception {
 
         stopNodesInParallel(3, 4, 5, 6);
 
-        int firstPhaseReset = getNodeForFirstPhaseReset(partId, 0, 1, 2);
-
-        Assignments link2FirstPhaseReset = Assignments.of(Set.of(
-                Assignment.forPeer(node(firstPhaseReset).name())
-        ), timestamp);
+        // Wait for first phase of reset to complete.
+        // The reset selects the node with the highest raft log index (or 
lexicographically first on tie).
+        Set<String> aliveNodes = Set.of(node(0).name(), node(1).name(), 
node(2).name());

Review Comment:
   Extract `nodeNamesFromIndices(int... nodeIndices)` ? There is the same code 
as below and potentially useful for a range of tests, so may be placed into the 
base class.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to