This is an automated email from the ASF dual-hosted git repository.
sk0x50 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 0f4112df9f8 IGNITE-26957 fix a bug in
testChangePeersAndLearnersAddMultiNodes test (#6999)
0f4112df9f8 is described below
commit 0f4112df9f8ee578c62365c7663648a98cf87786
Author: Mirza Aliev <[email protected]>
AuthorDate: Wed Nov 19 11:31:57 2025 +0400
IGNITE-26957 fix a bug in testChangePeersAndLearnersAddMultiNodes test
(#6999)
---
.../apache/ignite/raft/jraft/core/ItNodeTest.java | 3 +
.../ignite/raft/jraft/core/MockStateMachine.java | 18 +++++
.../apache/ignite/raft/jraft/core/TestCluster.java | 78 ++++++++++++++++++++--
3 files changed, 93 insertions(+), 6 deletions(-)
diff --git
a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java
b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java
index abe8d105f7e..4c39b671853 100644
---
a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java
+++
b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java
@@ -4005,6 +4005,9 @@ public class ItNodeTest extends BaseIgniteAbstractTest {
assertTrue(await.isOk(), await.getErrorMsg());
cluster.ensureSame();
+
+ cluster.ensureSameConf();
+
assertEquals(3, cluster.getFsms().size());
for (MockStateMachine fsm : cluster.getFsms())
diff --git
a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/MockStateMachine.java
b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/MockStateMachine.java
index 891864df40e..1de8279352e 100644
---
a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/MockStateMachine.java
+++
b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/MockStateMachine.java
@@ -34,6 +34,7 @@ import org.apache.ignite.internal.logger.Loggers;
import org.apache.ignite.raft.jraft.Closure;
import org.apache.ignite.raft.jraft.Iterator;
import org.apache.ignite.raft.jraft.Status;
+import org.apache.ignite.raft.jraft.conf.Configuration;
import org.apache.ignite.raft.jraft.entity.LeaderChangeContext;
import org.apache.ignite.raft.jraft.entity.PeerId;
import org.apache.ignite.raft.jraft.entity.RaftOutter.SnapshotMeta;
@@ -50,6 +51,7 @@ public class MockStateMachine extends StateMachineAdapter {
private volatile int onStopFollowingTimes = 0;
private volatile long appliedIndex = -1;
private volatile long snapshotIndex = -1L;
+ private volatile Configuration conf = new Configuration();
private final List<ByteBuffer> logs = new ArrayList<>();
private final PeerId peerId;
private final AtomicInteger saveSnapshotTimes = new AtomicInteger(0);
@@ -106,6 +108,16 @@ public class MockStateMachine extends StateMachineAdapter {
}
}
+ public Configuration getConf() {
+ this.lock.lock();
+ try {
+ return conf;
+ }
+ finally {
+ this.lock.unlock();
+ }
+ }
+
private final AtomicLong lastAppliedIndex = new AtomicLong(-1);
@Override
@@ -223,6 +235,12 @@ public class MockStateMachine extends StateMachineAdapter {
this.leaderTerm = -1;
}
+ @Override
+ public void onConfigurationCommitted(Configuration conf) {
+ super.onConfigurationCommitted(conf);
+ this.conf = conf;
+ }
+
@Override
public void onStopFollowing(final LeaderChangeContext ctx) {
super.onStopFollowing(ctx);
diff --git
a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/TestCluster.java
b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/TestCluster.java
index 919e47c3b34..853fcf29714 100644
---
a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/TestCluster.java
+++
b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/TestCluster.java
@@ -16,10 +16,12 @@
*/
package org.apache.ignite.raft.jraft.core;
+import static java.util.concurrent.TimeUnit.SECONDS;
import static java.util.stream.Collectors.toList;
import static
org.apache.ignite.internal.network.utils.ClusterServiceTestUtils.clusterService;
import static
org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
import static
org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.awaitility.Awaitility.await;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertSame;
@@ -534,16 +536,15 @@ public class TestCluster {
return ret;
}
- public void ensureSame() throws InterruptedException {
+ public void ensureSame() {
ensureSame(addr -> false);
}
/**
+ * Asserts that all FSM state are the same.
* @param filter The node to exclude filter.
- * @return {@code True} if all FSM state are the same.
- * @throws InterruptedException
*/
- public void ensureSame(Predicate<PeerId> filter) throws
InterruptedException {
+ void ensureSame(Predicate<PeerId> filter) {
this.lock.lock();
List<MockStateMachine> fsmList = new ArrayList<>(this.fsms.values());
@@ -563,7 +564,7 @@ public class TestCluster {
LOG.info("Start ensureSame, leader={}", leader);
try {
- assertTrue(TestUtils.waitForCondition(() -> {
+ await().atMost(20, SECONDS).until(() -> {
first.lock();
try {
@@ -600,7 +601,7 @@ public class TestCluster {
}
return true;
- }, 20_000));
+ });
}
finally {
this.lock.unlock();
@@ -613,6 +614,71 @@ public class TestCluster {
}
}
+ /**
+ * Asserts that all configurations that was applied to FSM are the same.
+ */
+ void ensureSameConf() {
+ this.lock.lock();
+
+ List<MockStateMachine> fsmList = new ArrayList<>(this.fsms.values());
+
+ if (fsmList.size() <= 1) {
+ LOG.warn("ensureSame is skipped because only one node in the
group");
+ this.lock.unlock();
+ return;
+ }
+
+ Node leader = getLeader();
+
+ assertNotNull(leader);
+
+ MockStateMachine first = fsms.get(leader.getNodeId().getPeerId());
+
+ LOG.info("Start ensureSameConf, leader={}", leader);
+
+ try {
+ await().atMost(20, SECONDS).until(() -> {
+ first.lock();
+
+ try {
+ for (int i = 0; i < fsmList.size(); i++) {
+ MockStateMachine fsm = fsmList.get(i);
+
+ if (fsm == first)
+ continue;
+
+ fsm.lock();
+
+ try {
+ Configuration conf0 = first.getConf();
+ Configuration conf1 = fsm.getConf();
+
+ if (!conf0.equals(conf1))
+ return false;
+ }
+ finally {
+ fsm.unlock();
+ }
+ }
+ }
+ finally {
+ first.unlock();
+ }
+
+ return true;
+ });
+ }
+ finally {
+ this.lock.unlock();
+
+ Node leader1 = getLeader();
+
+ LOG.info("End ensureSameConf, leader={}", leader1);
+
+ assertSame(leader, leader1, "Leader shouldn't change while
comparing fsms configurations");
+ }
+ }
+
public void setNodeOptionsCustomizer(BiConsumer<PeerId, NodeOptions>
customizer) {
this.optsClo = customizer;
}