You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ib...@apache.org on 2023/01/25 15:03:42 UTC

[ignite-3] branch main updated: IGNITE-18465 Use Cluster in AbstractClusterIntegrationTest (#1580)

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

ibessonov 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 54f7e17b66 IGNITE-18465 Use Cluster in AbstractClusterIntegrationTest (#1580)
54f7e17b66 is described below

commit 54f7e17b66e54c2e98a363f9da6fec305a40edad
Author: Roman Puchkovskiy <ro...@gmail.com>
AuthorDate: Wed Jan 25 19:03:35 2023 +0400

    IGNITE-18465 Use Cluster in AbstractClusterIntegrationTest (#1580)
---
 .../internal/AbstractClusterIntegrationTest.java   | 160 +++++++--------------
 .../java/org/apache/ignite/internal/Cluster.java   |  10 +-
 .../internal/compute/ItLogicalTopologyTest.java    |  17 ++-
 .../inmemory/ItRaftStorageVolatilityTest.java      |   6 +-
 .../raftsnapshot/ItTableRaftSnapshotsTest.java     |  14 +-
 5 files changed, 75 insertions(+), 132 deletions(-)

diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/AbstractClusterIntegrationTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/AbstractClusterIntegrationTest.java
index f00ea7e66b..338972f435 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/AbstractClusterIntegrationTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/AbstractClusterIntegrationTest.java
@@ -17,35 +17,21 @@
 
 package org.apache.ignite.internal;
 
-import static java.util.stream.Collectors.toList;
 import static org.apache.ignite.internal.sql.engine.util.CursorUtils.getAllFromCursor;
-import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName;
-import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willSucceedIn;
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.hamcrest.Matchers.greaterThan;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.notNullValue;
 
 import java.nio.file.Path;
-import java.util.ArrayList;
 import java.util.List;
-import java.util.Objects;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.IntStream;
 import org.apache.ignite.Ignite;
-import org.apache.ignite.IgnitionManager;
 import org.apache.ignite.internal.app.IgniteImpl;
 import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.logger.Loggers;
 import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.lang.IgniteStringFormatter;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.api.Timeout;
 import org.junit.jupiter.api.extension.ExtendWith;
 
 /**
@@ -53,7 +39,6 @@ import org.junit.jupiter.api.extension.ExtendWith;
  */
 @SuppressWarnings("ALL")
 @ExtendWith(WorkDirectoryExtension.class)
-// TODO: IGNITE-18465- use Cluster class
 public abstract class AbstractClusterIntegrationTest extends BaseIgniteAbstractTest {
     private static final IgniteLogger LOG = Loggers.forClass(AbstractClusterIntegrationTest.class);
 
@@ -61,7 +46,7 @@ public abstract class AbstractClusterIntegrationTest extends BaseIgniteAbstractT
     private static final int BASE_PORT = 3344;
 
     /** Nodes bootstrap configuration pattern. */
-    private static final String NODE_BOOTSTRAP_CFG = "{\n"
+    private static final String NODE_BOOTSTRAP_CFG_TEMPLATE = "{\n"
             + "  \"network\": {\n"
             + "    \"port\":{},\n"
             + "    \"nodeFinder\":{\n"
@@ -70,69 +55,45 @@ public abstract class AbstractClusterIntegrationTest extends BaseIgniteAbstractT
             + "  }\n"
             + "}";
 
-    /** Cluster nodes. */
-    private final List<Ignite> clusterNodes = new ArrayList<>();
+    protected Cluster cluster;
 
     /** Work directory. */
     @WorkDirectory
-    protected static Path WORK_DIR;
+    protected Path workDir;
 
     /**
-     * Starts nodes before each test.
+     * Invoked before each test starts.
      *
-     * @param testInfo Test information object.
+     * @param testInfo Test information oject.
+     * @throws Exception If failed.
      */
     @BeforeEach
-    void startNodes(TestInfo testInfo) {
-        List<CompletableFuture<Ignite>> futures = IntStream.range(0, initialNodes())
-                .mapToObj(i -> startNode0(i, testInfo))
-                .collect(toList());
-
-        String metaStorageNodeName = testNodeName(testInfo, initialNodes() - 1);
-
-        IgnitionManager.init(metaStorageNodeName, List.of(metaStorageNodeName), "cluster");
-
-        for (CompletableFuture<Ignite> future : futures) {
-            assertThat(future, willSucceedIn(10, TimeUnit.SECONDS));
-
-            clusterNodes.add(future.join());
-        }
-    }
-
-    private static CompletableFuture<Ignite> startNode0(int nodeIndex, TestInfo testInfo) {
-        String connectNodeAddr = "\"localhost:" + BASE_PORT + '\"';
-
-        String nodeName = testNodeName(testInfo, nodeIndex);
-
-        String config = IgniteStringFormatter.format(NODE_BOOTSTRAP_CFG, BASE_PORT + nodeIndex, connectNodeAddr);
-
-        return IgnitionManager.start(nodeName, config, WORK_DIR.resolve(nodeName));
+    public void setup(TestInfo testInfo) throws Exception {
+        setupBase(testInfo, workDir);
     }
 
     /**
-     * Starts an Ignite node with the given index.
+     * Invoked after each test has finished.
      *
-     * @param nodeIndex Zero-based index (used to build node name).
-     * @param testInfo Test info (used to build node name).
-     * @return Started Ignite node.
+     * @param testInfo Test information oject.
+     * @throws Exception If failed.
      */
-    protected IgniteImpl startNode(int nodeIndex, TestInfo testInfo) {
-        CompletableFuture<Ignite> future = startNode0(nodeIndex, testInfo);
-
-        assertThat(future, willSucceedIn(10, TimeUnit.SECONDS));
+    @AfterEach
+    public void tearDown(TestInfo testInfo) throws Exception {
+        tearDownBase(testInfo);
+    }
 
-        Ignite ignite = future.join();
+    @BeforeEach
+    void startAndInitCluster(TestInfo testInfo) {
+        cluster = new Cluster(testInfo, workDir, NODE_BOOTSTRAP_CFG_TEMPLATE);
 
-        if (nodeIndex < clusterNodes.size()) {
-            clusterNodes.set(nodeIndex, ignite);
-        } else if (nodeIndex == clusterNodes.size()) {
-            clusterNodes.add(ignite);
-        } else {
-            throw new IllegalArgumentException("Cannot start node with index " + nodeIndex + " because we only have "
-                    + clusterNodes.size() + " nodes");
-        }
+        cluster.startAndInit(initialNodes());
+    }
 
-        return (IgniteImpl) ignite;
+    @AfterEach
+    @Timeout(60)
+    void shutdownCluster() {
+        cluster.shutdown();
     }
 
     /**
@@ -145,79 +106,54 @@ public abstract class AbstractClusterIntegrationTest extends BaseIgniteAbstractT
     }
 
     /**
-     * Stops all nodes that are not stopped yet.
+     * Returns node bootstrap config template.
+     *
+     * @return Node bootstrap config template.
      */
-    @AfterEach
-    void stopNodes(TestInfo testInfo) throws Exception {
-        LOG.info("Start tearDown()");
-
-        List<AutoCloseable> closeables = clusterNodes.stream()
-                .filter(Objects::nonNull)
-                .map(node -> (AutoCloseable) () -> IgnitionManager.stop(node.name()))
-                .collect(toList());
-
-        IgniteUtils.closeAll(closeables);
-
-        clusterNodes.clear();
-
-        LOG.info("End tearDown()");
+    protected String getNodeBootstrapConfigTemplate() {
+        return NODE_BOOTSTRAP_CFG_TEMPLATE;
     }
 
     /**
-     * Stops a node by index.
+     * Starts an Ignite node with the given index.
      *
-     * @param nodeIndex Node index.
-     * @param testInfo Test info (used to construct node name).
+     * @param nodeIndex Zero-based index (used to build node name).
+     * @return Started Ignite node.
      */
-    protected final void stopNode(int nodeIndex, TestInfo testInfo) {
-        assertThat(clusterNodes.size(), is(greaterThan(nodeIndex)));
-        assertThat(clusterNodes.get(nodeIndex), is(notNullValue()));
-
-        IgnitionManager.stop(testNodeName(testInfo, nodeIndex));
-
-        clusterNodes.set(nodeIndex, null);
+    protected final IgniteImpl startNode(int nodeIndex) {
+        return cluster.startNode(nodeIndex);
     }
 
     /**
-     * Restarts a node by index.
+     * Stops a node by index.
      *
      * @param nodeIndex Node index.
-     * @param testInfo Test info (used to construct node name).
-     * @return New node instance.
      */
-    protected final Ignite restartNode(int nodeIndex, TestInfo testInfo) {
-        stopNode(nodeIndex, testInfo);
-
-        return startNode(nodeIndex, testInfo);
+    protected final void stopNode(int nodeIndex) {
+        cluster.stopNode(nodeIndex);
     }
 
     /**
-     * Invokes before the test will start.
+     * Restarts a node by index.
      *
-     * @param testInfo Test information oject.
-     * @throws Exception If failed.
+     * @param nodeIndex Node index.
+     * @return New node instance.
      */
-    @BeforeEach
-    public void setup(TestInfo testInfo) throws Exception {
-        setupBase(testInfo, WORK_DIR);
+    protected final Ignite restartNode(int nodeIndex) {
+        return cluster.restartNode(nodeIndex);
     }
 
     /**
-     * Invokes after the test has finished.
+     * Gets a node by index.
      *
-     * @param testInfo Test information oject.
-     * @throws Exception If failed.
+     * @param index Node index.
+     * @return Node by index.
      */
-    @AfterEach
-    public void tearDown(TestInfo testInfo) throws Exception {
-        tearDownBase(testInfo);
-    }
-
     protected final IgniteImpl node(int index) {
-        return (IgniteImpl) clusterNodes.get(index);
+        return cluster.node(index);
     }
 
-    protected List<List<Object>> executeSql(String sql, Object... args) {
+    protected final List<List<Object>> executeSql(String sql, Object... args) {
         return getAllFromCursor(
                 node(0).queryEngine().queryAsync("PUBLIC", sql, args).get(0).join()
         );
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/Cluster.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/Cluster.java
index 03ccb9ba40..0d12ea192c 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/Cluster.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/Cluster.java
@@ -87,7 +87,7 @@ public class Cluster {
 
     private final Path workDir;
 
-    private final String nodeBootstrapConfig;
+    private final String nodeBootstrapConfigTemplate;
 
     /** Cluster nodes. */
     private final List<IgniteImpl> nodes = new CopyOnWriteArrayList<>();
@@ -109,10 +109,10 @@ public class Cluster {
     /**
      * Creates a new cluster with the given bootstrap config.
      */
-    public Cluster(TestInfo testInfo, Path workDir, String nodeBootstrapConfig) {
+    public Cluster(TestInfo testInfo, Path workDir, String nodeBootstrapConfigTemplate) {
         this.testInfo = testInfo;
         this.workDir = workDir;
-        this.nodeBootstrapConfig = nodeBootstrapConfig;
+        this.nodeBootstrapConfigTemplate = nodeBootstrapConfigTemplate;
     }
 
     /**
@@ -149,7 +149,7 @@ public class Cluster {
     public CompletableFuture<IgniteImpl> startClusterNode(int nodeIndex) {
         String nodeName = testNodeName(testInfo, nodeIndex);
 
-        String config = IgniteStringFormatter.format(nodeBootstrapConfig, BASE_PORT + nodeIndex, CONNECT_NODE_ADDR);
+        String config = IgniteStringFormatter.format(nodeBootstrapConfigTemplate, BASE_PORT + nodeIndex, CONNECT_NODE_ADDR);
 
         return IgnitionManager.start(nodeName, config, workDir.resolve(nodeName))
                 .thenApply(IgniteImpl.class::cast)
@@ -203,8 +203,6 @@ public class Cluster {
      *     is not initialized, the node is returned in a state in which it is ready to join the cluster).
      */
     public IgniteImpl startNode(int index) {
-        checkNodeIndex(index);
-
         IgniteImpl newIgniteNode;
 
         try {
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/compute/ItLogicalTopologyTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/compute/ItLogicalTopologyTest.java
index 8de3186259..0964baa022 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/compute/ItLogicalTopologyTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/compute/ItLogicalTopologyTest.java
@@ -37,7 +37,6 @@ import org.apache.ignite.internal.network.message.ScaleCubeMessage;
 import org.apache.ignite.internal.tostring.S;
 import org.apache.ignite.network.ClusterNode;
 import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.TestInfo;
 
 /**
  * Integration tests for functionality of logical topology events subscription.
@@ -64,13 +63,13 @@ class ItLogicalTopologyTest extends AbstractClusterIntegrationTest {
     }
 
     @Test
-    void receivesLogicalTopologyEvents(TestInfo testInfo) throws Exception {
+    void receivesLogicalTopologyEvents() throws Exception {
         IgniteImpl entryNode = node(0);
 
         entryNode.logicalTopologyService().addEventListener(listener);
 
         // Checking that onAppeared() is received.
-        Ignite secondIgnite = startNode(1, testInfo);
+        Ignite secondIgnite = startNode(1);
 
         assertTrue(waitForCondition(() -> !events.isEmpty(), 10_000));
 
@@ -83,7 +82,7 @@ class ItLogicalTopologyTest extends AbstractClusterIntegrationTest {
         assertThat(firstEvent.topologyVersion, is(2L));
 
         // Checking that onDisappeared() is received.
-        stopNode(1, testInfo);
+        stopNode(1);
 
         assertTrue(waitForCondition(() -> events.size() > 1, 10_000));
 
@@ -97,14 +96,14 @@ class ItLogicalTopologyTest extends AbstractClusterIntegrationTest {
     }
 
     @Test
-    void receivesLogicalTopologyEventsCausedByNodeRestart(TestInfo testInfo) throws Exception {
+    void receivesLogicalTopologyEventsCausedByNodeRestart() throws Exception {
         IgniteImpl entryNode = node(0);
 
-        Ignite secondIgnite = startNode(1, testInfo);
+        Ignite secondIgnite = startNode(1);
 
         entryNode.logicalTopologyService().addEventListener(listener);
 
-        restartNode(1, testInfo);
+        restartNode(1);
 
         waitForCondition(() -> events.size() >= 2, 10_000);
 
@@ -124,10 +123,10 @@ class ItLogicalTopologyTest extends AbstractClusterIntegrationTest {
     }
 
     @Test
-    void nodeReturnedToPhysicalTopologyReturnsToLogicalTopology(TestInfo testInfo) throws Exception {
+    void nodeReturnedToPhysicalTopologyReturnsToLogicalTopology() throws Exception {
         IgniteImpl entryNode = node(0);
 
-        IgniteImpl secondIgnite = startNode(1, testInfo);
+        IgniteImpl secondIgnite = startNode(1);
 
         makeSecondNodeDisappearForFirstNode(entryNode, secondIgnite);
 
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/inmemory/ItRaftStorageVolatilityTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/inmemory/ItRaftStorageVolatilityTest.java
index bde357dd6f..df352db8b6 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/inmemory/ItRaftStorageVolatilityTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/inmemory/ItRaftStorageVolatilityTest.java
@@ -86,7 +86,7 @@ class ItRaftStorageVolatilityTest extends AbstractClusterIntegrationTest {
      * @return Paths for 'meta' directories corresponding to Raft meta storages for partitions of the test table.
      */
     private List<Path> partitionRaftMetaPaths(IgniteImpl ignite) {
-        try (Stream<Path> paths = Files.list(WORK_DIR.resolve(ignite.name()))) {
+        try (Stream<Path> paths = Files.list(workDir.resolve(ignite.name()))) {
             return paths
                     .filter(path -> isPartitionDir(path, ignite))
                     .map(path -> path.resolve("meta"))
@@ -119,7 +119,7 @@ class ItRaftStorageVolatilityTest extends AbstractClusterIntegrationTest {
 
         node(0).close();
 
-        Path logRocksDbDir = WORK_DIR.resolve(nodeName).resolve("log");
+        Path logRocksDbDir = workDir.resolve(nodeName).resolve("log");
 
         List<ColumnFamilyDescriptor> cfDescriptors = List.of(
                 // Column family to store configuration log entry.
@@ -173,7 +173,7 @@ class ItRaftStorageVolatilityTest extends AbstractClusterIntegrationTest {
 
         node(0).close();
 
-        Path logRocksDbDir = WORK_DIR.resolve(nodeName).resolve("log");
+        Path logRocksDbDir = workDir.resolve(nodeName).resolve("log");
 
         List<ColumnFamilyDescriptor> cfDescriptors = List.of(
                 // Column family to store configuration log entry.
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 77d24e6007..015941c00e 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
@@ -60,6 +60,7 @@ import org.apache.ignite.internal.storage.pagememory.VolatilePageMemoryStorageEn
 import org.apache.ignite.internal.storage.rocksdb.RocksDbStorageEngine;
 import org.apache.ignite.internal.table.distributed.raft.snapshot.message.SnapshotMetaResponse;
 import org.apache.ignite.internal.table.distributed.replicator.TablePartitionId;
+import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
 import org.apache.ignite.internal.testframework.IgniteTestUtils;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
@@ -95,8 +96,7 @@ import org.junit.jupiter.params.provider.ValueSource;
 @SuppressWarnings("resource")
 @ExtendWith(WorkDirectoryExtension.class)
 @Timeout(90)
-// TODO: IGNITE-18465- extend AbstractClusterIntegrationTest
-class ItTableRaftSnapshotsTest {
+class ItTableRaftSnapshotsTest extends BaseIgniteAbstractTest {
     private static final IgniteLogger LOG = Loggers.forClass(ItTableRaftSnapshotsTest.class);
 
     /**
@@ -142,6 +142,16 @@ class ItTableRaftSnapshotsTest {
         cluster.shutdown();
     }
 
+    @BeforeEach
+    public void setup(TestInfo testInfo) throws Exception {
+        setupBase(testInfo, workDir);
+    }
+
+    @AfterEach
+    public void tearDown(TestInfo testInfo) throws Exception {
+        tearDownBase(testInfo);
+    }
+
     /**
      * Executes the given action, retrying it up to a few times if a transient failure occurs (like node unavailability).
      */