rpuch commented on code in PR #5083:
URL: https://github.com/apache/ignite-3/pull/5083#discussion_r1925125692


##########
modules/raft-api/src/test/java/org/apache/ignite/internal/raft/RaftGroupConfigurationSerializerTest.java:
##########
@@ -61,10 +65,12 @@ void serializationAndDeserializationWithNulls() {
 
     @Test
     void v1CanBeDeserialized() {

Review Comment:
   If this PR is not going to be included in the 3.0 release, then changes to 
this test need to be reverted and another test needs to be added 
(`v2CanBeDeserialized()`)



##########
modules/raft-api/src/main/java/org/apache/ignite/internal/raft/RaftGroupConfiguration.java:
##########
@@ -32,11 +32,16 @@
 public class RaftGroupConfiguration implements Serializable {
     private static final long serialVersionUID = 0;
 
+    private final long index;
+    private final long term;
+
     @IgniteToStringInclude
     private final List<String> peers;
     @IgniteToStringInclude
     private final List<String> learners;
 
+
+

Review Comment:
   Extra blank line



##########
modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/FSMCallerImpl.java:
##########
@@ -603,6 +603,8 @@ private void doSnapshotSave(final SaveSnapshotClosure done) 
{
         }
 
         SnapshotMetaBuilder metaBuilder = msgFactory.snapshotMeta()
+            .cfgIndex(confEntry.getId().getIndex())
+            .cfgTerm(confEntry.getId().getTerm())

Review Comment:
   These 2 fields are only needed for partitions, and for partitions we don't 
save snapshot files. Hence, a question: do we really need to fill them here?



##########
modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java:
##########
@@ -3506,6 +3507,102 @@ public void 
onRawConfigurationCommitted(ConfigurationEntry conf) {
         );
     }
 
+    @Test
+    public void testIndexAndTermOfCfgArePropagatedToSnapshotMeta() throws 
Exception {
+        TestPeer peer0 = new TestPeer(testInfo, TestUtils.INIT_PORT);
+
+        AtomicLong term = new AtomicLong(-1);
+        AtomicLong index = new AtomicLong(-1);
+
+        AtomicLong metaTerm = new AtomicLong(-1);
+        AtomicLong metaIndex = new AtomicLong(-1);
+
+        cluster = new TestCluster(
+                "testIndexAndTermOfCfgArePropagatedToSnapshotMeta",
+                dataPath,
+                Collections.singletonList(peer0),
+                new LinkedHashSet<>(),
+                ELECTION_TIMEOUT_MILLIS,
+                (peerId, opts) -> {
+                    opts.setFsm(new MockStateMachine(peerId) {
+                        @Override
+                        public boolean onSnapshotLoad(SnapshotReader reader) {
+                            SnapshotMeta meta = reader.load();
+
+                            metaTerm.set(meta.cfgTerm());
+                            metaIndex.set(meta.cfgIndex());
+
+                            return super.onSnapshotLoad(reader);
+                        }
+
+                        @Override
+                        public void 
onRawConfigurationCommitted(ConfigurationEntry conf) {
+                            term.set(conf.getId().getTerm());
+                            index.set(conf.getId().getIndex());
+                            System.out.println("Index + term = " + index + " " 
+ term);
+                            super.onRawConfigurationCommitted(conf);
+                        }
+                    });
+                },
+                testInfo
+        );
+
+        assertTrue(cluster.start(peer0));
+
+        Node leader = cluster.waitAndGetLeader();
+
+        assertEquals(1, term.get());
+        assertEquals(1, index.get());
+
+        TestPeer newPeer = new TestPeer(testInfo, TestUtils.INIT_PORT + 1);
+        TestPeer otherPeer = new TestPeer(testInfo, TestUtils.INIT_PORT + 2);
+
+        assertTrue(cluster.start(newPeer, false, 300));
+        assertTrue(cluster.start(otherPeer, false, 300));
+
+        // Wait until new node node sees every other node, otherwise

Review Comment:
   ```suggestion
           // Wait until new node sees every other node, otherwise
   ```



##########
modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java:
##########
@@ -3506,6 +3507,102 @@ public void 
onRawConfigurationCommitted(ConfigurationEntry conf) {
         );
     }
 
+    @Test
+    public void testIndexAndTermOfCfgArePropagatedToSnapshotMeta() throws 
Exception {
+        TestPeer peer0 = new TestPeer(testInfo, TestUtils.INIT_PORT);
+
+        AtomicLong term = new AtomicLong(-1);
+        AtomicLong index = new AtomicLong(-1);
+
+        AtomicLong metaTerm = new AtomicLong(-1);
+        AtomicLong metaIndex = new AtomicLong(-1);

Review Comment:
   ```suggestion
           AtomicLong metaConfigTerm = new AtomicLong(-1);
           AtomicLong metaConfigIndex = new AtomicLong(-1);
   ```



##########
modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java:
##########
@@ -3506,6 +3507,102 @@ public void 
onRawConfigurationCommitted(ConfigurationEntry conf) {
         );
     }
 
+    @Test
+    public void testIndexAndTermOfCfgArePropagatedToSnapshotMeta() throws 
Exception {
+        TestPeer peer0 = new TestPeer(testInfo, TestUtils.INIT_PORT);
+
+        AtomicLong term = new AtomicLong(-1);
+        AtomicLong index = new AtomicLong(-1);

Review Comment:
   ```suggestion
           AtomicLong configTerm = new AtomicLong(-1);
           AtomicLong configIndex = new AtomicLong(-1);
   ```



##########
modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java:
##########
@@ -3506,6 +3507,102 @@ public void 
onRawConfigurationCommitted(ConfigurationEntry conf) {
         );
     }
 
+    @Test
+    public void testIndexAndTermOfCfgArePropagatedToSnapshotMeta() throws 
Exception {
+        TestPeer peer0 = new TestPeer(testInfo, TestUtils.INIT_PORT);
+
+        AtomicLong term = new AtomicLong(-1);
+        AtomicLong index = new AtomicLong(-1);
+
+        AtomicLong metaTerm = new AtomicLong(-1);
+        AtomicLong metaIndex = new AtomicLong(-1);
+
+        cluster = new TestCluster(
+                "testIndexAndTermOfCfgArePropagatedToSnapshotMeta",
+                dataPath,
+                Collections.singletonList(peer0),
+                new LinkedHashSet<>(),
+                ELECTION_TIMEOUT_MILLIS,
+                (peerId, opts) -> {
+                    opts.setFsm(new MockStateMachine(peerId) {
+                        @Override
+                        public boolean onSnapshotLoad(SnapshotReader reader) {
+                            SnapshotMeta meta = reader.load();
+
+                            metaTerm.set(meta.cfgTerm());
+                            metaIndex.set(meta.cfgIndex());
+
+                            return super.onSnapshotLoad(reader);
+                        }
+
+                        @Override
+                        public void 
onRawConfigurationCommitted(ConfigurationEntry conf) {
+                            term.set(conf.getId().getTerm());
+                            index.set(conf.getId().getIndex());
+                            System.out.println("Index + term = " + index + " " 
+ term);
+                            super.onRawConfigurationCommitted(conf);
+                        }
+                    });
+                },
+                testInfo
+        );
+
+        assertTrue(cluster.start(peer0));
+
+        Node leader = cluster.waitAndGetLeader();
+
+        assertEquals(1, term.get());
+        assertEquals(1, index.get());
+
+        TestPeer newPeer = new TestPeer(testInfo, TestUtils.INIT_PORT + 1);
+        TestPeer otherPeer = new TestPeer(testInfo, TestUtils.INIT_PORT + 2);
+
+        assertTrue(cluster.start(newPeer, false, 300));
+        assertTrue(cluster.start(otherPeer, false, 300));
+
+        // Wait until new node node sees every other node, otherwise
+        // changePeersAndLearnersAsync can fail.
+        waitForTopologyOnEveryNode(1, cluster);

Review Comment:
   Here we seem to wait till every node sees at least 1 node in its physical 
topology. But physical topology includes the node itself, so how is it about 
seeing other nodes, let alone seeing *each* other node?



##########
modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java:
##########
@@ -3506,6 +3507,102 @@ public void 
onRawConfigurationCommitted(ConfigurationEntry conf) {
         );
     }
 
+    @Test
+    public void testIndexAndTermOfCfgArePropagatedToSnapshotMeta() throws 
Exception {
+        TestPeer peer0 = new TestPeer(testInfo, TestUtils.INIT_PORT);
+
+        AtomicLong term = new AtomicLong(-1);
+        AtomicLong index = new AtomicLong(-1);
+
+        AtomicLong metaTerm = new AtomicLong(-1);
+        AtomicLong metaIndex = new AtomicLong(-1);
+
+        cluster = new TestCluster(
+                "testIndexAndTermOfCfgArePropagatedToSnapshotMeta",
+                dataPath,
+                Collections.singletonList(peer0),
+                new LinkedHashSet<>(),
+                ELECTION_TIMEOUT_MILLIS,
+                (peerId, opts) -> {
+                    opts.setFsm(new MockStateMachine(peerId) {
+                        @Override
+                        public boolean onSnapshotLoad(SnapshotReader reader) {
+                            SnapshotMeta meta = reader.load();
+
+                            metaTerm.set(meta.cfgTerm());
+                            metaIndex.set(meta.cfgIndex());

Review Comment:
   If we only need indexes and terms from the leader, then why do we need other 
nodes? Why one of them is stopped?



##########
modules/raft-api/src/main/java/org/apache/ignite/internal/raft/RaftGroupConfigurationSerializer.java:
##########
@@ -34,6 +34,8 @@ public class RaftGroupConfigurationSerializer extends 
VersionedSerializer<RaftGr
 
     @Override
     protected void writeExternalData(RaftGroupConfiguration config, 
IgniteDataOutput out) throws IOException {
+        out.writeLong(config.index());

Review Comment:
   This PR is not going to go to release 3.0 and this is a breaking change. 
Let's create second version of the serializer. Another option is to agree on 
the mailing list to include this issue to the release.



##########
modules/raft-api/src/main/java/org/apache/ignite/internal/raft/RaftGroupConfiguration.java:
##########
@@ -32,11 +32,16 @@
 public class RaftGroupConfiguration implements Serializable {

Review Comment:
   This class now seems to be identical to `CommittedConfiguration`. Do we need 
to keep both?



##########
modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java:
##########
@@ -3506,6 +3507,102 @@ public void 
onRawConfigurationCommitted(ConfigurationEntry conf) {
         );
     }
 
+    @Test
+    public void testIndexAndTermOfCfgArePropagatedToSnapshotMeta() throws 
Exception {

Review Comment:
   The scenario is pretty lengthy and complex. Could you please elaborate on it 
in the javadoc? What it tests for, what the scenario is, what verifications are 
made...



##########
modules/raft/src/main/java/org/apache/ignite/raft/jraft/entity/RaftOutter.java:
##########
@@ -57,6 +57,10 @@ public interface SnapshotMeta extends Message {
 
         long lastIncludedTerm();
 
+        long cfgIndex();

Review Comment:
   Do we need these fields for all raft groups or just for partitions (or for 
partitions + metastorage)? If not for all of them, then probably they don't 
need to be added here as they are not part of the Raft protocol. If only 
partitions need them, `PartitionSnapshotMeta` seems to be a more suitable place 
to add these fields.



##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/raft/MetaStorageSnapshotStorageFactory.java:
##########
@@ -68,6 +68,8 @@ public MetaStorageSnapshotStorageFactory(KeyValueStorage 
storage) {
         assert configuration != null;
 
         return new RaftMessagesFactory().snapshotMeta()
+                .cfgIndex(configuration.index())

Review Comment:
   For Metastorage, these fields are not needed, so now it looks like we are 
just conforming to a ceremony. It would be nice if we could avoid using these 
new 2 fields here.



##########
modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java:
##########
@@ -3506,6 +3507,102 @@ public void 
onRawConfigurationCommitted(ConfigurationEntry conf) {
         );
     }
 
+    @Test
+    public void testIndexAndTermOfCfgArePropagatedToSnapshotMeta() throws 
Exception {
+        TestPeer peer0 = new TestPeer(testInfo, TestUtils.INIT_PORT);
+
+        AtomicLong term = new AtomicLong(-1);
+        AtomicLong index = new AtomicLong(-1);
+
+        AtomicLong metaTerm = new AtomicLong(-1);
+        AtomicLong metaIndex = new AtomicLong(-1);
+
+        cluster = new TestCluster(
+                "testIndexAndTermOfCfgArePropagatedToSnapshotMeta",
+                dataPath,
+                Collections.singletonList(peer0),
+                new LinkedHashSet<>(),
+                ELECTION_TIMEOUT_MILLIS,
+                (peerId, opts) -> {
+                    opts.setFsm(new MockStateMachine(peerId) {
+                        @Override
+                        public boolean onSnapshotLoad(SnapshotReader reader) {
+                            SnapshotMeta meta = reader.load();
+
+                            metaTerm.set(meta.cfgTerm());
+                            metaIndex.set(meta.cfgIndex());
+
+                            return super.onSnapshotLoad(reader);
+                        }
+
+                        @Override
+                        public void 
onRawConfigurationCommitted(ConfigurationEntry conf) {
+                            term.set(conf.getId().getTerm());
+                            index.set(conf.getId().getIndex());
+                            System.out.println("Index + term = " + index + " " 
+ term);
+                            super.onRawConfigurationCommitted(conf);
+                        }
+                    });
+                },
+                testInfo
+        );
+
+        assertTrue(cluster.start(peer0));
+
+        Node leader = cluster.waitAndGetLeader();
+
+        assertEquals(1, term.get());
+        assertEquals(1, index.get());
+
+        TestPeer newPeer = new TestPeer(testInfo, TestUtils.INIT_PORT + 1);
+        TestPeer otherPeer = new TestPeer(testInfo, TestUtils.INIT_PORT + 2);
+
+        assertTrue(cluster.start(newPeer, false, 300));
+        assertTrue(cluster.start(otherPeer, false, 300));
+
+        // Wait until new node node sees every other node, otherwise
+        // changePeersAndLearnersAsync can fail.
+        waitForTopologyOnEveryNode(1, cluster);
+
+        SynchronizedClosure done = new SynchronizedClosure();
+        leader.changePeersAndLearnersAsync(
+                new Configuration(List.of(peer0.getPeerId(), 
newPeer.getPeerId(), otherPeer.getPeerId()), List.of()), 
leader.getCurrentTerm(),
+                done
+        );
+
+        assertEquals(done.await(), Status.OK());
+
+        assertTrue(waitForCondition(() -> 
cluster.getLeader().listAlivePeers().contains(newPeer.getPeerId()), 10_000));
+
+        assertTrue(cluster.stop(newPeer.getPeerId()));

Review Comment:
   Why do we stop `newPeer`? Also, is there a guarantee it will not be a 
leader? Or that it will be a leader?



##########
modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java:
##########
@@ -3506,6 +3507,102 @@ public void 
onRawConfigurationCommitted(ConfigurationEntry conf) {
         );
     }
 
+    @Test
+    public void testIndexAndTermOfCfgArePropagatedToSnapshotMeta() throws 
Exception {
+        TestPeer peer0 = new TestPeer(testInfo, TestUtils.INIT_PORT);
+
+        AtomicLong term = new AtomicLong(-1);
+        AtomicLong index = new AtomicLong(-1);
+
+        AtomicLong metaTerm = new AtomicLong(-1);
+        AtomicLong metaIndex = new AtomicLong(-1);
+
+        cluster = new TestCluster(
+                "testIndexAndTermOfCfgArePropagatedToSnapshotMeta",
+                dataPath,
+                Collections.singletonList(peer0),
+                new LinkedHashSet<>(),
+                ELECTION_TIMEOUT_MILLIS,
+                (peerId, opts) -> {
+                    opts.setFsm(new MockStateMachine(peerId) {
+                        @Override
+                        public boolean onSnapshotLoad(SnapshotReader reader) {
+                            SnapshotMeta meta = reader.load();
+
+                            metaTerm.set(meta.cfgTerm());
+                            metaIndex.set(meta.cfgIndex());
+
+                            return super.onSnapshotLoad(reader);
+                        }
+
+                        @Override
+                        public void 
onRawConfigurationCommitted(ConfigurationEntry conf) {
+                            term.set(conf.getId().getTerm());
+                            index.set(conf.getId().getIndex());
+                            System.out.println("Index + term = " + index + " " 
+ term);
+                            super.onRawConfigurationCommitted(conf);
+                        }
+                    });
+                },
+                testInfo
+        );
+
+        assertTrue(cluster.start(peer0));
+
+        Node leader = cluster.waitAndGetLeader();
+
+        assertEquals(1, term.get());
+        assertEquals(1, index.get());
+
+        TestPeer newPeer = new TestPeer(testInfo, TestUtils.INIT_PORT + 1);
+        TestPeer otherPeer = new TestPeer(testInfo, TestUtils.INIT_PORT + 2);
+
+        assertTrue(cluster.start(newPeer, false, 300));
+        assertTrue(cluster.start(otherPeer, false, 300));
+
+        // Wait until new node node sees every other node, otherwise
+        // changePeersAndLearnersAsync can fail.
+        waitForTopologyOnEveryNode(1, cluster);
+
+        SynchronizedClosure done = new SynchronizedClosure();
+        leader.changePeersAndLearnersAsync(
+                new Configuration(List.of(peer0.getPeerId(), 
newPeer.getPeerId(), otherPeer.getPeerId()), List.of()), 
leader.getCurrentTerm(),
+                done
+        );
+
+        assertEquals(done.await(), Status.OK());
+
+        assertTrue(waitForCondition(() -> 
cluster.getLeader().listAlivePeers().contains(newPeer.getPeerId()), 10_000));
+
+        assertTrue(cluster.stop(newPeer.getPeerId()));
+
+        // apply something more
+        sendTestTaskAndWait(leader);
+        triggerLeaderSnapshot(cluster, leader);
+
+        sendTestTaskAndWait(leader, 10);
+
+        triggerLeaderSnapshot(cluster, leader, 2);
+
+        //restart follower.
+        cluster.clean(newPeer.getPeerId());
+        assertTrue(cluster.start(newPeer, false, 300));
+
+        cluster.ensureSame();
+
+        assertEquals(3, cluster.getFsms().size());
+        for (MockStateMachine fsm : cluster.getFsms())
+            assertEquals(20, fsm.getLogs().size(), fsm.getPeerId().toString());

Review Comment:
   Why are these assertions needed? They don't seem to relate to the thing the 
test is testing, so they just serve as an obstacle when trying to understand 
what happens here



##########
modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java:
##########
@@ -3506,6 +3507,102 @@ public void 
onRawConfigurationCommitted(ConfigurationEntry conf) {
         );
     }
 
+    @Test
+    public void testIndexAndTermOfCfgArePropagatedToSnapshotMeta() throws 
Exception {
+        TestPeer peer0 = new TestPeer(testInfo, TestUtils.INIT_PORT);
+
+        AtomicLong term = new AtomicLong(-1);
+        AtomicLong index = new AtomicLong(-1);
+
+        AtomicLong metaTerm = new AtomicLong(-1);
+        AtomicLong metaIndex = new AtomicLong(-1);
+
+        cluster = new TestCluster(
+                "testIndexAndTermOfCfgArePropagatedToSnapshotMeta",
+                dataPath,
+                Collections.singletonList(peer0),
+                new LinkedHashSet<>(),
+                ELECTION_TIMEOUT_MILLIS,
+                (peerId, opts) -> {
+                    opts.setFsm(new MockStateMachine(peerId) {
+                        @Override
+                        public boolean onSnapshotLoad(SnapshotReader reader) {
+                            SnapshotMeta meta = reader.load();
+
+                            metaTerm.set(meta.cfgTerm());
+                            metaIndex.set(meta.cfgIndex());

Review Comment:
   Can any of 3 nodes write its index+term to atomics or just the leader? How 
about restricting this explicitly by `peerId`?



##########
modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java:
##########
@@ -3506,6 +3507,102 @@ public void 
onRawConfigurationCommitted(ConfigurationEntry conf) {
         );
     }
 
+    @Test
+    public void testIndexAndTermOfCfgArePropagatedToSnapshotMeta() throws 
Exception {
+        TestPeer peer0 = new TestPeer(testInfo, TestUtils.INIT_PORT);
+
+        AtomicLong term = new AtomicLong(-1);
+        AtomicLong index = new AtomicLong(-1);
+
+        AtomicLong metaTerm = new AtomicLong(-1);
+        AtomicLong metaIndex = new AtomicLong(-1);
+
+        cluster = new TestCluster(
+                "testIndexAndTermOfCfgArePropagatedToSnapshotMeta",
+                dataPath,
+                Collections.singletonList(peer0),
+                new LinkedHashSet<>(),
+                ELECTION_TIMEOUT_MILLIS,
+                (peerId, opts) -> {
+                    opts.setFsm(new MockStateMachine(peerId) {
+                        @Override
+                        public boolean onSnapshotLoad(SnapshotReader reader) {
+                            SnapshotMeta meta = reader.load();
+
+                            metaTerm.set(meta.cfgTerm());
+                            metaIndex.set(meta.cfgIndex());
+
+                            return super.onSnapshotLoad(reader);
+                        }
+
+                        @Override
+                        public void 
onRawConfigurationCommitted(ConfigurationEntry conf) {
+                            term.set(conf.getId().getTerm());
+                            index.set(conf.getId().getIndex());
+                            System.out.println("Index + term = " + index + " " 
+ term);
+                            super.onRawConfigurationCommitted(conf);
+                        }
+                    });
+                },
+                testInfo
+        );
+
+        assertTrue(cluster.start(peer0));
+
+        Node leader = cluster.waitAndGetLeader();
+
+        assertEquals(1, term.get());
+        assertEquals(1, index.get());
+
+        TestPeer newPeer = new TestPeer(testInfo, TestUtils.INIT_PORT + 1);
+        TestPeer otherPeer = new TestPeer(testInfo, TestUtils.INIT_PORT + 2);
+
+        assertTrue(cluster.start(newPeer, false, 300));
+        assertTrue(cluster.start(otherPeer, false, 300));
+
+        // Wait until new node node sees every other node, otherwise
+        // changePeersAndLearnersAsync can fail.
+        waitForTopologyOnEveryNode(1, cluster);
+
+        SynchronizedClosure done = new SynchronizedClosure();
+        leader.changePeersAndLearnersAsync(
+                new Configuration(List.of(peer0.getPeerId(), 
newPeer.getPeerId(), otherPeer.getPeerId()), List.of()), 
leader.getCurrentTerm(),
+                done
+        );
+
+        assertEquals(done.await(), Status.OK());
+
+        assertTrue(waitForCondition(() -> 
cluster.getLeader().listAlivePeers().contains(newPeer.getPeerId()), 10_000));
+
+        assertTrue(cluster.stop(newPeer.getPeerId()));
+
+        // apply something more

Review Comment:
   ```suggestion
           // Apply something more.
   ```



##########
modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java:
##########
@@ -3506,6 +3507,102 @@ public void 
onRawConfigurationCommitted(ConfigurationEntry conf) {
         );
     }
 
+    @Test
+    public void testIndexAndTermOfCfgArePropagatedToSnapshotMeta() throws 
Exception {
+        TestPeer peer0 = new TestPeer(testInfo, TestUtils.INIT_PORT);
+
+        AtomicLong term = new AtomicLong(-1);
+        AtomicLong index = new AtomicLong(-1);
+
+        AtomicLong metaTerm = new AtomicLong(-1);
+        AtomicLong metaIndex = new AtomicLong(-1);
+
+        cluster = new TestCluster(
+                "testIndexAndTermOfCfgArePropagatedToSnapshotMeta",
+                dataPath,
+                Collections.singletonList(peer0),
+                new LinkedHashSet<>(),
+                ELECTION_TIMEOUT_MILLIS,
+                (peerId, opts) -> {
+                    opts.setFsm(new MockStateMachine(peerId) {
+                        @Override
+                        public boolean onSnapshotLoad(SnapshotReader reader) {
+                            SnapshotMeta meta = reader.load();
+
+                            metaTerm.set(meta.cfgTerm());
+                            metaIndex.set(meta.cfgIndex());
+
+                            return super.onSnapshotLoad(reader);
+                        }
+
+                        @Override
+                        public void 
onRawConfigurationCommitted(ConfigurationEntry conf) {
+                            term.set(conf.getId().getTerm());
+                            index.set(conf.getId().getIndex());
+                            System.out.println("Index + term = " + index + " " 
+ term);
+                            super.onRawConfigurationCommitted(conf);
+                        }
+                    });
+                },
+                testInfo
+        );
+
+        assertTrue(cluster.start(peer0));
+
+        Node leader = cluster.waitAndGetLeader();
+
+        assertEquals(1, term.get());
+        assertEquals(1, index.get());
+
+        TestPeer newPeer = new TestPeer(testInfo, TestUtils.INIT_PORT + 1);
+        TestPeer otherPeer = new TestPeer(testInfo, TestUtils.INIT_PORT + 2);
+
+        assertTrue(cluster.start(newPeer, false, 300));
+        assertTrue(cluster.start(otherPeer, false, 300));
+
+        // Wait until new node node sees every other node, otherwise
+        // changePeersAndLearnersAsync can fail.
+        waitForTopologyOnEveryNode(1, cluster);
+
+        SynchronizedClosure done = new SynchronizedClosure();
+        leader.changePeersAndLearnersAsync(
+                new Configuration(List.of(peer0.getPeerId(), 
newPeer.getPeerId(), otherPeer.getPeerId()), List.of()), 
leader.getCurrentTerm(),
+                done
+        );
+
+        assertEquals(done.await(), Status.OK());
+
+        assertTrue(waitForCondition(() -> 
cluster.getLeader().listAlivePeers().contains(newPeer.getPeerId()), 10_000));
+
+        assertTrue(cluster.stop(newPeer.getPeerId()));
+
+        // apply something more
+        sendTestTaskAndWait(leader);
+        triggerLeaderSnapshot(cluster, leader);
+
+        sendTestTaskAndWait(leader, 10);
+
+        triggerLeaderSnapshot(cluster, leader, 2);
+
+        //restart follower.
+        cluster.clean(newPeer.getPeerId());
+        assertTrue(cluster.start(newPeer, false, 300));
+
+        cluster.ensureSame();
+
+        assertEquals(3, cluster.getFsms().size());
+        for (MockStateMachine fsm : cluster.getFsms())
+            assertEquals(20, fsm.getLogs().size(), fsm.getPeerId().toString());
+
+        // Leader hasn't been changed, term must stay the same
+        assertEquals(1, term.get());
+        // idx_2 == joint consensus, idx_3 is expected final cfg

Review Comment:
   If I'm not mistaken, the scenario starts with having just [A] as voting set 
and then expands it to [ABC]. It looks like joint config *is* the final 
configuration. Does it really require 2 steps?



-- 
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: notifications-unsubscr...@ignite.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to