From b40025be5f0a4a44c4e8f77306c422891ac78523 Mon Sep 17 00:00:00 2001 From: Alexander Polovtcev Date: Wed, 29 Sep 2021 12:55:53 +0300 Subject: [PATCH] IGNITE-15579 Use test class and test method names as a part of cluster node names (#359) --- .../ignite/cli/ITConfigCommandTest.java | 21 +- .../testframework/IgniteTestUtils.java | 15 ++ .../client/ITMetaStorageServiceTest.java | 5 +- .../network/scalecube/ITNodeRestartsTest.java | 14 +- .../ITScaleCubeNetworkMessagingTest.java | 46 ++-- .../ignite/utils/ClusterServiceTestUtils.java | 9 +- .../scalecube/ScaleCubeTopologyService.java | 22 +- .../ITAbstractListenerSnapshotTest.java | 27 +- .../raft/jraft/core/ITCliServiceTest.java | 4 +- .../ignite/raft/jraft/core/ITNodeTest.java | 172 ++++++------- .../raft/server/ITJRaftCounterServerTest.java | 12 +- .../server/ITSimpleCounterServerTest.java | 14 +- .../raft/server/RaftServerAbstractTest.java | 16 +- .../ignite/raft/jraft/core/TestCluster.java | 130 ++++++---- .../ignite/raft/jraft/rpc/IgniteRpcTest.java | 13 +- ...istributedConfigurationPropertiesTest.java | 24 +- ...ITDistributedConfigurationStorageTest.java | 23 +- .../runner/app/AbstractSchemaChangeTest.java | 83 ++++--- .../app/ITDynamicTableCreationTest.java | 79 +++--- .../runner/app/ITIgniteNodeRestartTest.java | 62 ++--- .../internal/runner/app/ITIgnitionTest.java | 84 ++++--- .../runner/app/ITTableCreationTest.java | 230 ++++++++++-------- .../app/ITThinClientConnectionTest.java | 57 +++-- .../runner/app/PlatformTestNodeRunner.java | 7 +- .../runner/app/jdbc/AbstractJdbcSelfTest.java | 23 +- .../distributed/ITDistributedTableTest.java | 13 +- 26 files changed, 694 insertions(+), 511 deletions(-) diff --git a/modules/cli/src/integrationTest/java/org/apache/ignite/cli/ITConfigCommandTest.java b/modules/cli/src/integrationTest/java/org/apache/ignite/cli/ITConfigCommandTest.java index 768acf2d342..0d23c7a1ffb 100644 --- a/modules/cli/src/integrationTest/java/org/apache/ignite/cli/ITConfigCommandTest.java +++ b/modules/cli/src/integrationTest/java/org/apache/ignite/cli/ITConfigCommandTest.java @@ -29,6 +29,7 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.io.TempDir; import picocli.CommandLine; @@ -58,7 +59,7 @@ public class ITConfigCommandTest extends AbstractCliTest { /** */ @BeforeEach - private void setup(@TempDir Path workDir) throws IOException { + void setup(@TempDir Path workDir, TestInfo testInfo) throws IOException { // TODO: IGNITE-15131 Must be replaced by receiving the actual port configs from the started node. // This approach still can produce the port, which will be unavailable at the moment of node start. restPort = getAvailablePort(); @@ -69,7 +70,7 @@ private void setup(@TempDir Path workDir) throws IOException { "rest.port=" + restPort + "\n" + "rest.portRange=0" + "\n" + "clientConnector.port=" + clientPort + "\n" + "clientConnector.portRange=0"; - IgnitionManager.start("node1", configStr, workDir); + IgnitionManager.start(testNodeName(testInfo, networkPort), configStr, workDir); ctx = ApplicationContext.run(Environment.TEST); @@ -77,12 +78,24 @@ private void setup(@TempDir Path workDir) throws IOException { out = new ByteArrayOutputStream(); } + /** */ @AfterEach - private void tearDown() { - IgnitionManager.stop("node1"); + void tearDown(TestInfo testInfo) { + IgnitionManager.stop(testNodeName(testInfo, networkPort)); ctx.stop(); } + /** + * Creates a unique Ignite node name for the given test. + */ + private static String testNodeName(TestInfo testInfo, int port) { + return testInfo.getTestClass() + .map(Class::getCanonicalName) + .flatMap(clsName -> testInfo.getTestMethod().map(method -> clsName + '#' + method.getName())) + .map(name -> name + ':' + port) + .orElseThrow(); + } + @Test public void setAndGetWithManualHost() { int exitCode = cmd(ctx).execute( diff --git a/modules/core/src/test/java/org/apache/ignite/internal/testframework/IgniteTestUtils.java b/modules/core/src/test/java/org/apache/ignite/internal/testframework/IgniteTestUtils.java index 572efed86c4..3315ca5f970 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/testframework/IgniteTestUtils.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/testframework/IgniteTestUtils.java @@ -25,6 +25,7 @@ import org.apache.ignite.lang.IgniteInternalException; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import org.junit.jupiter.api.TestInfo; import static java.lang.Thread.sleep; import static org.junit.jupiter.api.Assertions.fail; @@ -257,4 +258,18 @@ public static String randomString(Random rnd, int len) { return sb.toString(); } + + /** + * Creates a unique Ignite node name for the given test. + */ + public static String testNodeName(TestInfo testInfo, int port) { + return testInfo.getTestClass() + .map(Class::getCanonicalName) + .map(name -> testInfo.getTestMethod() + .map(method -> name + '#' + method.getName()) + .orElse(name) + ) + .map(name -> name + ':' + port) + .orElseThrow(); + } } diff --git a/modules/metastorage-client/src/integrationTest/java/org/apache/ignite/internal/metastorage/client/ITMetaStorageServiceTest.java b/modules/metastorage-client/src/integrationTest/java/org/apache/ignite/internal/metastorage/client/ITMetaStorageServiceTest.java index 98e22377af2..5a18733df19 100644 --- a/modules/metastorage-client/src/integrationTest/java/org/apache/ignite/internal/metastorage/client/ITMetaStorageServiceTest.java +++ b/modules/metastorage-client/src/integrationTest/java/org/apache/ignite/internal/metastorage/client/ITMetaStorageServiceTest.java @@ -58,6 +58,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.ArgumentCaptor; import org.mockito.Mock; @@ -194,13 +195,13 @@ public class ITMetaStorageServiceTest { * Run {@code NODES} cluster nodes. */ @BeforeEach - public void beforeTest() throws Exception { + public void beforeTest(TestInfo testInfo) throws Exception { var nodeFinder = new LocalPortRangeNodeFinder(NODE_PORT_BASE, NODE_PORT_BASE + NODES); nodeFinder.findNodes().stream() .map( addr -> ClusterServiceTestUtils.clusterService( - addr.toString(), + testInfo, addr.port(), nodeFinder, SERIALIZATION_REGISTRY, diff --git a/modules/network/src/integrationTest/java/org/apache/ignite/network/scalecube/ITNodeRestartsTest.java b/modules/network/src/integrationTest/java/org/apache/ignite/network/scalecube/ITNodeRestartsTest.java index 91449895e8b..3f1ca158148 100644 --- a/modules/network/src/integrationTest/java/org/apache/ignite/network/scalecube/ITNodeRestartsTest.java +++ b/modules/network/src/integrationTest/java/org/apache/ignite/network/scalecube/ITNodeRestartsTest.java @@ -30,6 +30,7 @@ import org.apache.ignite.utils.ClusterServiceTestUtils; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -60,13 +61,13 @@ void tearDown() { * Tests that restarting nodes get discovered in an established topology. */ @Test - public void testRestarts() { + public void testRestarts(TestInfo testInfo) { final int initPort = 3344; var nodeFinder = new LocalPortRangeNodeFinder(initPort, initPort + 5); services = nodeFinder.findNodes().stream() - .map(addr -> startNetwork(addr, nodeFinder)) + .map(addr -> startNetwork(testInfo, addr, nodeFinder)) .collect(Collectors.toCollection(ArrayList::new)); // ensure mutability for (ClusterService service : services) { @@ -86,11 +87,11 @@ public void testRestarts() { services.get(idx1).stop(); LOG.info("Starting {}", addresses.get(idx0)); - ClusterService svc0 = startNetwork(addresses.get(idx0), nodeFinder); + ClusterService svc0 = startNetwork(testInfo, addresses.get(idx0), nodeFinder); services.set(idx0, svc0); LOG.info("Starting {}", addresses.get(idx1)); - ClusterService svc2 = startNetwork(addresses.get(idx1), nodeFinder); + ClusterService svc2 = startNetwork(testInfo, addresses.get(idx1), nodeFinder); services.set(idx1, svc2); for (ClusterService service : services) { @@ -104,13 +105,14 @@ public void testRestarts() { /** * Creates a {@link ClusterService} using the given local address and the node finder. * + * @param testInfo Test info. * @param addr Node address. * @param nodeFinder Node finder. * @return Created Cluster Service. */ - private ClusterService startNetwork(NetworkAddress addr, NodeFinder nodeFinder) { + private ClusterService startNetwork(TestInfo testInfo, NetworkAddress addr, NodeFinder nodeFinder) { ClusterService clusterService = ClusterServiceTestUtils.clusterService( - addr.toString(), + testInfo, addr.port(), nodeFinder, serializationRegistry, diff --git a/modules/network/src/integrationTest/java/org/apache/ignite/network/scalecube/ITScaleCubeNetworkMessagingTest.java b/modules/network/src/integrationTest/java/org/apache/ignite/network/scalecube/ITScaleCubeNetworkMessagingTest.java index 37345780aae..0cb4c108be5 100644 --- a/modules/network/src/integrationTest/java/org/apache/ignite/network/scalecube/ITScaleCubeNetworkMessagingTest.java +++ b/modules/network/src/integrationTest/java/org/apache/ignite/network/scalecube/ITScaleCubeNetworkMessagingTest.java @@ -50,6 +50,7 @@ import org.apache.ignite.utils.ClusterServiceTestUtils; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; import reactor.core.publisher.Mono; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe; @@ -87,12 +88,12 @@ public void tearDown() { * @throws Exception in case of errors. */ @Test - public void messageWasSentToAllMembersSuccessfully() throws Exception { + public void messageWasSentToAllMembersSuccessfully(TestInfo testInfo) throws Exception { Map messageStorage = new ConcurrentHashMap<>(); var messageReceivedLatch = new CountDownLatch(3); - testCluster = new Cluster(3); + testCluster = new Cluster(3, testInfo); for (ClusterService member : testCluster.members) { member.messagingService().addMessageHandler( @@ -128,8 +129,8 @@ public void messageWasSentToAllMembersSuccessfully() throws Exception { * @throws Exception If failed. */ @Test - public void testShutdown() throws Exception { - testShutdown0(false); + public void testShutdown(TestInfo testInfo) throws Exception { + testShutdown0(testInfo, false); } /** @@ -138,8 +139,8 @@ public void testShutdown() throws Exception { * @throws Exception If failed. */ @Test - public void testForcefulShutdown() throws Exception { - testShutdown0(true); + public void testForcefulShutdown(TestInfo testInfo) throws Exception { + testShutdown0(testInfo, true); } /** @@ -148,8 +149,8 @@ public void testForcefulShutdown() throws Exception { * @throws Exception in case of errors. */ @Test - public void testSendMessageToSelf() throws Exception { - testCluster = new Cluster(1); + public void testSendMessageToSelf(TestInfo testInfo) throws Exception { + testCluster = new Cluster(1, testInfo); testCluster.startAwait(); ClusterService member = testCluster.members.get(0); @@ -196,8 +197,8 @@ private Data(TestMessage message, NetworkAddress sender, String correlationId) { * @throws Exception in case of errors. */ @Test - public void testInvokeMessageToSelf() throws Exception { - testCluster = new Cluster(1); + public void testInvokeMessageToSelf(TestInfo testInfo) throws Exception { + testCluster = new Cluster(1, testInfo); testCluster.startAwait(); ClusterService member = testCluster.members.get(0); @@ -228,8 +229,8 @@ public void testInvokeMessageToSelf() throws Exception { * the corresponding future completes exceptionally. */ @Test - public void testInvokeDuringStop() throws InterruptedException { - testCluster = new Cluster(2); + public void testInvokeDuringStop(TestInfo testInfo) throws InterruptedException { + testCluster = new Cluster(2, testInfo); testCluster.startAwait(); ClusterService member0 = testCluster.members.get(0); @@ -287,8 +288,8 @@ private static class MockNetworkMessage implements NetworkMessage, Serializable * @throws Exception in case of errors. */ @Test - public void testMessageGroupsHandlers() throws Exception { - testCluster = new Cluster(2); + public void testMessageGroupsHandlers(TestInfo testInfo) throws Exception { + testCluster = new Cluster(2, testInfo); testCluster.startAwait(); ClusterService node1 = testCluster.members.get(0); @@ -337,11 +338,12 @@ public void testMessageGroupsHandlers() throws Exception { /** * Tests shutdown. * + * @param testInfo Test info. * @param forceful Whether shutdown should be forceful. * @throws Exception If failed. */ - private void testShutdown0(boolean forceful) throws Exception { - testCluster = new Cluster(2); + private void testShutdown0(TestInfo testInfo, boolean forceful) throws Exception { + testCluster = new Cluster(2, testInfo); testCluster.startAwait(); ClusterService alice = testCluster.members.get(0); @@ -423,8 +425,9 @@ private static final class Cluster { * Creates a test cluster with the given amount of members. * * @param numOfNodes Amount of cluster members. + * @param testInfo Test info. */ - Cluster(int numOfNodes) { + Cluster(int numOfNodes, TestInfo testInfo) { startupLatch = new CountDownLatch(numOfNodes - 1); int initialPort = 3344; @@ -434,21 +437,24 @@ private static final class Cluster { var isInitial = new AtomicBoolean(true); members = nodeFinder.findNodes().stream() - .map(addr -> startNode(addr, nodeFinder, isInitial.getAndSet(false))) + .map(addr -> startNode(testInfo, addr, nodeFinder, isInitial.getAndSet(false))) .collect(Collectors.toUnmodifiableList()); } /** * Start cluster node. * + * @param testInfo Test info. * @param addr Node address. * @param nodeFinder Node finder. * @param initial Whether this node is the first one. * @return Started cluster node. */ - private ClusterService startNode(NetworkAddress addr, NodeFinder nodeFinder, boolean initial) { + private ClusterService startNode( + TestInfo testInfo, NetworkAddress addr, NodeFinder nodeFinder, boolean initial + ) { ClusterService clusterSvc = ClusterServiceTestUtils.clusterService( - addr.toString(), + testInfo, addr.port(), nodeFinder, serializationRegistry, diff --git a/modules/network/src/integrationTest/java/org/apache/ignite/utils/ClusterServiceTestUtils.java b/modules/network/src/integrationTest/java/org/apache/ignite/utils/ClusterServiceTestUtils.java index 93ca0e390d3..4a5078217fa 100644 --- a/modules/network/src/integrationTest/java/org/apache/ignite/utils/ClusterServiceTestUtils.java +++ b/modules/network/src/integrationTest/java/org/apache/ignite/utils/ClusterServiceTestUtils.java @@ -31,6 +31,9 @@ import org.apache.ignite.network.NodeFinder; import org.apache.ignite.network.TopologyService; import org.apache.ignite.network.serialization.MessageSerializationRegistry; +import org.junit.jupiter.api.TestInfo; + +import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName; /** * Test utils that provide sort of cluster service mock that manages required node configuration internally. @@ -42,20 +45,20 @@ public class ClusterServiceTestUtils { * Manages configuration manager lifecycle: on cluster service start starts node configuration manager, * on cluster service stop - stops node configuration manager. * - * @param nodeName Local name. + * @param testInfo Test info. * @param port Local port. * @param nodeFinder Node finder for discovering the initial cluster members. * @param msgSerializationRegistry Message serialization registry. * @param clusterSvcFactory Cluster service factory. */ public static ClusterService clusterService( - String nodeName, + TestInfo testInfo, int port, NodeFinder nodeFinder, MessageSerializationRegistry msgSerializationRegistry, ClusterServiceFactory clusterSvcFactory ) { - var ctx = new ClusterLocalConfiguration(nodeName, msgSerializationRegistry); + var ctx = new ClusterLocalConfiguration(testNodeName(testInfo, port), msgSerializationRegistry); ConfigurationManager nodeConfigurationMgr = new ConfigurationManager( Collections.singleton(NetworkConfiguration.KEY), diff --git a/modules/network/src/main/java/org/apache/ignite/network/scalecube/ScaleCubeTopologyService.java b/modules/network/src/main/java/org/apache/ignite/network/scalecube/ScaleCubeTopologyService.java index 68097071fad..cf9b2dd97c4 100644 --- a/modules/network/src/main/java/org/apache/ignite/network/scalecube/ScaleCubeTopologyService.java +++ b/modules/network/src/main/java/org/apache/ignite/network/scalecube/ScaleCubeTopologyService.java @@ -70,21 +70,29 @@ void onMembershipEvent(MembershipEvent event) { fireAppearedEvent(member); } else if (event.isRemoved()) { - members.compute(member.address(), // Ignore stale remove event. - (k, v) -> v.id().equals(member.id()) ? null : v); + members.compute(member.address(), (addr, node) -> { + // Ignore stale remove event. + if (node == null || node.id().equals(member.id())) + return null; + else + return node; + }); LOG.info("Node left: " + member); fireDisappearedEvent(member); } - StringBuilder snapshotMsg = new StringBuilder("Topology snapshot [nodes=").append(members.size()).append("]\n"); + if (LOG.isInfoEnabled()) { + StringBuilder snapshotMsg = new StringBuilder("Topology snapshot [nodes=") + .append(members.size()) + .append("]\n"); - for (ClusterNode node : members.values()) { - snapshotMsg.append(" ^-- ").append(node).append('\n'); - } + for (ClusterNode node : members.values()) + snapshotMsg.append(" ^-- ").append(node).append('\n'); - LOG.info(snapshotMsg.toString().trim()); + LOG.info(snapshotMsg.toString().trim()); + } } /** diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/client/service/ITAbstractListenerSnapshotTest.java b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/client/service/ITAbstractListenerSnapshotTest.java index be8c05602a3..ae4f6a083bc 100644 --- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/client/service/ITAbstractListenerSnapshotTest.java +++ b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/client/service/ITAbstractListenerSnapshotTest.java @@ -42,6 +42,7 @@ import org.apache.ignite.raft.jraft.rpc.impl.RaftGroupServiceImpl; import org.apache.ignite.utils.ClusterServiceTestUtils; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; @@ -153,13 +154,14 @@ private static List testSnapshotData() { * Tests that a joining raft node successfully restores a snapshot. * * @param testData Test parameters. + * @param testInfo Test info. * @throws Exception If failed. */ @ParameterizedTest @MethodSource("testSnapshotData") - public void testSnapshot(TestData testData) throws Exception { + public void testSnapshot(TestData testData, TestInfo testInfo) throws Exception { // Set up a raft group service - RaftGroupService service = prepareRaftGroup(); + RaftGroupService service = prepareRaftGroup(testInfo); beforeFollowerStop(service); @@ -205,7 +207,7 @@ public void testSnapshot(TestData testData) throws Exception { } // Restart the node - JRaftServerImpl restarted = startServer(stopIdx); + JRaftServerImpl restarted = startServer(testInfo, stopIdx); assertTrue(waitForTopology(cluster.get(0), servers.size(), 3_000)); @@ -310,11 +312,11 @@ private static String getLocalAddress() { /** * Creates a cluster service. */ - private ClusterService clusterService(String name, int port, NetworkAddress otherPeer) { + private ClusterService clusterService(TestInfo testInfo, int port, NetworkAddress otherPeer) { var nodeFinder = new StaticNodeFinder(List.of(otherPeer)); var network = ClusterServiceTestUtils.clusterService( - name, + testInfo, port, nodeFinder, SERIALIZATION_REGISTRY, @@ -331,13 +333,14 @@ private ClusterService clusterService(String name, int port, NetworkAddress othe /** * Starts a raft server. * + * @param testInfo Test info. * @param idx Server index (affects port of the server). * @return Server. */ - private JRaftServerImpl startServer(int idx) { + private JRaftServerImpl startServer(TestInfo testInfo, int idx) { var addr = new NetworkAddress(getLocalAddress(), PORT); - ClusterService service = clusterService("server" + idx, PORT + idx, addr); + ClusterService service = clusterService(testInfo, PORT + idx, addr); Path jraft = workDir.resolve("jraft" + idx); @@ -367,20 +370,20 @@ private JRaftServerImpl startServer(int idx) { * * @return Raft group service instance. */ - private RaftGroupService prepareRaftGroup() throws Exception { + private RaftGroupService prepareRaftGroup(TestInfo testInfo) throws Exception { for (int i = 0; i < INITIAL_CONF.size(); i++) - startServer(i); + startServer(testInfo, i); assertTrue(waitForTopology(cluster.get(0), servers.size(), 3_000)); - return startClient(raftGroupId(), new NetworkAddress(getLocalAddress(), PORT)); + return startClient(testInfo, raftGroupId(), new NetworkAddress(getLocalAddress(), PORT)); } /** * Starts a client with a specific address. */ - private RaftGroupService startClient(String groupId, NetworkAddress addr) throws Exception { - ClusterService clientNode = clusterService("client_" + groupId + "_", CLIENT_PORT + clients.size(), addr); + private RaftGroupService startClient(TestInfo testInfo, String groupId, NetworkAddress addr) throws Exception { + ClusterService clientNode = clusterService(testInfo, CLIENT_PORT + clients.size(), addr); RaftGroupService client = RaftGroupServiceImpl.start(groupId, clientNode, FACTORY, 10_000, List.of(new Peer(addr)), false, 200).get(3, TimeUnit.SECONDS); diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ITCliServiceTest.java b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ITCliServiceTest.java index bcc29f9db6a..202beb68c90 100644 --- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ITCliServiceTest.java +++ b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ITCliServiceTest.java @@ -103,7 +103,7 @@ public void setup(TestInfo testInfo, @WorkDirectory Path dataPath) throws Except for (int i = 0; i < 2; i++) learners.add(new PeerId(TestUtils.getLocalAddress(), TestUtils.INIT_PORT + LEARNER_PORT_STEP + i)); - cluster = new TestCluster(groupId, dataPath.toString(), peers, learners, 300); + cluster = new TestCluster(groupId, dataPath.toString(), peers, learners, 300, testInfo); for (PeerId peer : peers) cluster.start(peer.getEndpoint()); @@ -128,7 +128,7 @@ public void setup(TestInfo testInfo, @WorkDirectory Path dataPath) throws Except var registry = new MessageSerializationRegistryImpl(); ClusterService clientSvc = ClusterServiceTestUtils.clusterService( - "client", + testInfo, TestUtils.INIT_PORT - 1, nodeFinder, registry, 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 6fe6f80e17d..98057e84d21 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 @@ -156,6 +156,14 @@ public void run() { private final List appendEntriesExecutors = new ArrayList<>(); + /** Test info. */ + private final TestInfo testInfo; + + /** */ + public ITNodeTest(TestInfo testInfo) { + this.testInfo = testInfo; + } + @BeforeAll public static void setupNodeTest() { dumpThread = new DumpThread(); @@ -172,7 +180,7 @@ public static void tearNodeTest() throws Exception { } @BeforeEach - public void setup(TestInfo testInfo, @WorkDirectory Path workDir) throws Exception { + public void setup(@WorkDirectory Path workDir) throws Exception { LOG.info(">>>>>>>>>>>>>>> Start test method: " + testInfo.getDisplayName()); dataPath = workDir.toString(); @@ -182,7 +190,7 @@ public void setup(TestInfo testInfo, @WorkDirectory Path workDir) throws Excepti } @AfterEach - public void teardown(TestInfo testInfo) throws Exception { + public void teardown() throws Exception { services.forEach(service -> { try { service.shutdown(); @@ -238,7 +246,7 @@ public void testNodeTaskOverload() throws Exception { RaftGroupService service = createService("unittest", new PeerId(addr, 0), nodeOptions); - final Node node = service.start(); + Node node = service.start(); assertEquals(1, node.listPeers().size()); assertTrue(node.listPeers().contains(peer)); @@ -270,7 +278,7 @@ public void testNodeTaskOverload() throws Exception { * Test rollback stateMachine with readIndex for issue 317: https://github.com/sofastack/sofa-jraft/issues/317 */ @Test - public void testRollbackStateMachineWithReadIndex_Issue317(TestInfo testInfo) throws Exception { + public void testRollbackStateMachineWithReadIndex_Issue317() throws Exception { Endpoint addr = new Endpoint(TestUtils.getLocalAddress(), TestUtils.INIT_PORT); PeerId peer = new PeerId(addr, 0); @@ -316,7 +324,7 @@ public void onApply(Iterator iter) { RaftGroupService service = createService("unittest", peer, nodeOptions); - final Node node = service.start(); + Node node = service.start(); assertEquals(1, node.listPeers().size()); assertTrue(node.listPeers().contains(peer)); @@ -406,7 +414,7 @@ public void testSingleNode() throws Exception { public void testNoLeader() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); assertTrue(cluster.start(peers.get(0).getEndpoint())); @@ -438,7 +446,7 @@ public void testTripleNodesWithReplicatorStateListener() throws Exception { UserReplicatorStateListener listener2 = new UserReplicatorStateListener(); cluster = new TestCluster("unitest", dataPath, peers, new LinkedHashSet<>(), 300, - opts -> opts.setReplicationStateListeners(List.of(listener1, listener2))); + opts -> opts.setReplicationStateListeners(List.of(listener1, listener2)), testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -468,7 +476,7 @@ public void testTripleNodesWithReplicatorStateListener() throws Exception { public void testVoteTimedoutStepDown() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -536,7 +544,7 @@ public void testLeaderTransferWithReplicatorStateListener() throws Exception { List peers = TestUtils.generatePeers(3); cluster = new TestCluster("unitest", dataPath, peers, new LinkedHashSet<>(), 300, - opts -> opts.setReplicationStateListeners(List.of(new UserReplicatorStateListener()))); + opts -> opts.setReplicationStateListeners(List.of(new UserReplicatorStateListener())), testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -570,7 +578,7 @@ public void testLeaderTransferWithReplicatorStateListener() throws Exception { public void testTripleNodes() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -693,7 +701,7 @@ public void testResetLearners() throws Exception { for (int i = 0; i < 3; i++) learners.add(new PeerId(TestUtils.getLocalAddress(), TestUtils.INIT_PORT + 3 + i)); - cluster = new TestCluster("unittest", dataPath, peers, learners, 300); + cluster = new TestCluster("unittest", dataPath, peers, learners, 300, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -761,7 +769,7 @@ public void testResetLearners() throws Exception { public void testTripleNodesWithStaticLearners() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); LinkedHashSet learners = new LinkedHashSet<>(); PeerId learnerPeer = new PeerId(TestUtils.getLocalAddress(), TestUtils.INIT_PORT + 3); learners.add(learnerPeer); @@ -800,7 +808,7 @@ public void testTripleNodesWithStaticLearners() throws Exception { public void testTripleNodesWithLearners() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -905,7 +913,7 @@ public void testNodesWithPriorityElection() throws Exception { List peers = TestUtils.generatePriorityPeers(3, priorities); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint(), peer.getPriority())); @@ -932,7 +940,7 @@ public void testNodesWithPartPriorityElection() throws Exception { List peers = TestUtils.generatePriorityPeers(3, priorities); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint(), peer.getPriority())); @@ -958,7 +966,7 @@ public void testNodesWithSpecialPriorityElection() throws Exception { List peers = TestUtils.generatePriorityPeers(3, priorities); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint(), peer.getPriority())); @@ -984,7 +992,7 @@ public void testNodesWithZeroValPriorityElection() throws Exception { List peers = TestUtils.generatePriorityPeers(3, priorities); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint(), peer.getPriority())); @@ -1011,7 +1019,7 @@ public void testNoLeaderWithZeroValPriorityElection() throws Exception { List peers = TestUtils.generatePriorityPeers(3, priorities); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint(), peer.getPriority())); @@ -1033,7 +1041,7 @@ public void testLeaderStopAndReElectWithPriority() throws Exception { List peers = TestUtils.generatePriorityPeers(3, priorities); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint(), peer.getPriority())); @@ -1074,7 +1082,7 @@ public void testRemoveLeaderWithPriority() throws Exception { List peers = TestUtils.generatePriorityPeers(3, priorities); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint(), peer.getPriority())); @@ -1121,7 +1129,7 @@ public void testChecksum() throws Exception { // start with checksum validation { - TestCluster cluster = new TestCluster("unittest", dataPath, peers); + TestCluster cluster = new TestCluster("unittest", dataPath, peers, testInfo); try { RaftOptions raftOptions = new RaftOptions(); raftOptions.setEnableLogEntryChecksum(true); @@ -1142,7 +1150,7 @@ public void testChecksum() throws Exception { // restart with peer3 enable checksum validation { - TestCluster cluster = new TestCluster("unittest", dataPath, peers); + TestCluster cluster = new TestCluster("unittest", dataPath, peers, testInfo); try { RaftOptions raftOptions = new RaftOptions(); raftOptions.setEnableLogEntryChecksum(false); @@ -1168,7 +1176,7 @@ public void testChecksum() throws Exception { // restart with no checksum validation { - TestCluster cluster = new TestCluster("unittest", dataPath, peers); + TestCluster cluster = new TestCluster("unittest", dataPath, peers, testInfo); try { RaftOptions raftOptions = new RaftOptions(); raftOptions.setEnableLogEntryChecksum(false); @@ -1189,7 +1197,7 @@ public void testChecksum() throws Exception { // restart with all peers enable checksum validation { - TestCluster cluster = new TestCluster("unittest", dataPath, peers); + TestCluster cluster = new TestCluster("unittest", dataPath, peers, testInfo); try { RaftOptions raftOptions = new RaftOptions(); raftOptions.setEnableLogEntryChecksum(true); @@ -1214,7 +1222,7 @@ public void testChecksum() throws Exception { public void testReadIndex() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint(), false, 300, true)); @@ -1260,7 +1268,7 @@ public void run(Status status, long index, byte[] reqCtx) { public void testReadIndexTimeout() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint(), false, 300, true)); @@ -1312,7 +1320,7 @@ public void run(Status status, long index, byte[] reqCtx) { public void testReadIndexFromLearner() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint(), false, 300, true)); @@ -1350,7 +1358,7 @@ public void testReadIndexFromLearner() throws Exception { public void testReadIndexChaos() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint(), false, 300, true)); @@ -1426,7 +1434,7 @@ public void run(Status status, long index, byte[] reqCtx) { public void testNodeMetrics() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint(), false, 300, true)); @@ -1463,7 +1471,7 @@ public void testNodeMetrics() throws Exception { public void testLeaderFail() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -1560,7 +1568,7 @@ public void testJoinNodes() throws Exception { peers.add(peer0); // start single cluster - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); assertTrue(cluster.start(peer0.getEndpoint())); cluster.waitLeader(); @@ -1622,7 +1630,7 @@ public void testJoinNodes() throws Exception { public void testRemoveFollower() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -1684,7 +1692,7 @@ public void testRemoveFollower() throws Exception { public void testRemoveLeader() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -1749,7 +1757,7 @@ public void testRemoveLeader() throws Exception { public void testPreVote() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unitest", dataPath, peers); + cluster = new TestCluster("unitest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -1795,7 +1803,7 @@ public void testPreVote() throws Exception { @Test public void testSetPeer1() throws Exception { - cluster = new TestCluster("testSetPeer1", dataPath, new ArrayList<>()); + cluster = new TestCluster("testSetPeer1", dataPath, new ArrayList<>(), testInfo); PeerId bootPeer = new PeerId(TestUtils.getLocalAddress(), TestUtils.INIT_PORT); assertTrue(cluster.start(bootPeer.getEndpoint())); @@ -1814,7 +1822,7 @@ public void testSetPeer1() throws Exception { public void testSetPeer2() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unitest", dataPath, peers); + cluster = new TestCluster("unitest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -1899,7 +1907,7 @@ public void testSetPeer2() throws Exception { public void testRestoreSnapshot() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unitest", dataPath, peers); + cluster = new TestCluster("unitest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -1936,7 +1944,7 @@ public void testRestoreSnapshot() throws Exception { public void testRestoreSnapshotWithDelta() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unitest", dataPath, peers); + cluster = new TestCluster("unitest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -1979,7 +1987,7 @@ public void testRestoreSnapshotWithDelta() throws Exception { public void testInstallSnapshotWithThrottle() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unitest", dataPath, peers); + cluster = new TestCluster("unitest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint(), false, 200, false, new ThroughputSnapshotThrottle(1024, 1))); @@ -2034,7 +2042,7 @@ public void testInstallSnapshotWithThrottle() throws Exception { @Test // TODO add test for timeout on snapshot install https://issues.apache.org/jira/browse/IGNITE-14832 public void testInstallLargeSnapshotWithThrottle() throws Exception { List peers = TestUtils.generatePeers(4); - cluster = new TestCluster("unitest", dataPath, peers.subList(0, 3)); + cluster = new TestCluster("unitest", dataPath, peers.subList(0, 3), testInfo); for (int i = 0; i < peers.size() - 1; i++) { PeerId peer = peers.get(i); boolean started = cluster.start(peer.getEndpoint(), false, 200, false); @@ -2092,7 +2100,7 @@ public void testInstallLargeSnapshotWithThrottle() throws Exception { @Test public void testInstallLargeSnapshot() throws Exception { List peers = TestUtils.generatePeers(4); - cluster = new TestCluster("unitest", dataPath, peers.subList(0, 3)); + cluster = new TestCluster("unitest", dataPath, peers.subList(0, 3), testInfo); for (int i = 0; i < peers.size() - 1; i++) { PeerId peer = peers.get(i); boolean started = cluster.start(peer.getEndpoint(), false, 200, false); @@ -2152,7 +2160,7 @@ public void testInstallLargeSnapshot() throws Exception { public void testInstallSnapshot() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unitest", dataPath, peers); + cluster = new TestCluster("unitest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -2255,7 +2263,7 @@ public void testAutoSnapshot() throws Exception { public void testLeaderShouldNotChange() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unitest", dataPath, peers); + cluster = new TestCluster("unitest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -2278,7 +2286,7 @@ public void testLeaderShouldNotChange() throws Exception { public void testRecoverFollower() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unitest", dataPath, peers); + cluster = new TestCluster("unitest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -2316,7 +2324,7 @@ public void testRecoverFollower() throws Exception { public void testLeaderTransfer() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unitest", dataPath, peers, 300); + cluster = new TestCluster("unitest", dataPath, peers, 300, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -2344,7 +2352,7 @@ public void testLeaderTransfer() throws Exception { public void testLeaderTransferBeforeLogIsCompleted() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unitest", dataPath, peers, 300); + cluster = new TestCluster("unitest", dataPath, peers, 300, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint(), false, 1)); @@ -2384,7 +2392,7 @@ public void testLeaderTransferBeforeLogIsCompleted() throws Exception { public void testLeaderTransferResumeOnFailure() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unitest", dataPath, peers, 300); + cluster = new TestCluster("unitest", dataPath, peers, 300, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint(), false, 1)); @@ -2504,7 +2512,7 @@ public void testShutdownAndJoinWorkAfterInitFails() throws Exception { public void testShuttingDownLeaderTriggerTimeoutNow() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unitest", dataPath, peers, 300); + cluster = new TestCluster("unitest", dataPath, peers, 300, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -2530,7 +2538,7 @@ public void testShuttingDownLeaderTriggerTimeoutNow() throws Exception { public void testRemovingLeaderTriggerTimeoutNow() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unitest", dataPath, peers, 300); + cluster = new TestCluster("unitest", dataPath, peers, 300, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -2559,7 +2567,7 @@ public void testRemovingLeaderTriggerTimeoutNow() throws Exception { public void testTransferShouldWorkAfterInstallSnapshot() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unitest", dataPath, peers, 1000); + cluster = new TestCluster("unitest", dataPath, peers, 1000, testInfo); for (int i = 0; i < peers.size() - 1; i++) assertTrue(cluster.start(peers.get(i).getEndpoint())); @@ -2605,7 +2613,7 @@ public void testAppendEntriesWhenFollowerIsInErrorState() throws Exception { // start five nodes List peers = TestUtils.generatePeers(5); - cluster = new TestCluster("unitest", dataPath, peers, 1000); + cluster = new TestCluster("unitest", dataPath, peers, 1000, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -2656,7 +2664,7 @@ public void testFollowerStartStopFollowing() throws Exception { // start five nodes List peers = TestUtils.generatePeers(5); - cluster = new TestCluster("unitest", dataPath, peers, 1000); + cluster = new TestCluster("unitest", dataPath, peers, 1000, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -2728,7 +2736,7 @@ public void readCommittedUserLog() throws Exception { // setup cluster List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unitest", dataPath, peers, 1000); + cluster = new TestCluster("unitest", dataPath, peers, 1000, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -2901,7 +2909,7 @@ public void testBootStrapWithoutSnapshot() throws Exception { @Test public void testChangePeers() throws Exception { PeerId peer0 = new PeerId(TestUtils.getLocalAddress(), TestUtils.INIT_PORT); - cluster = new TestCluster("testChangePeers", dataPath, Collections.singletonList(peer0)); + cluster = new TestCluster("testChangePeers", dataPath, Collections.singletonList(peer0), testInfo); assertTrue(cluster.start(peer0.getEndpoint())); cluster.waitLeader(); @@ -2927,7 +2935,7 @@ public void testChangePeers() throws Exception { cluster.waitLeader(); - for (final MockStateMachine fsm : cluster.getFsms()) { + for (MockStateMachine fsm : cluster.getFsms()) { assertEquals(10, fsm.getLogs().size()); } } @@ -2935,7 +2943,7 @@ public void testChangePeers() throws Exception { @Test public void testChangePeersAddMultiNodes() throws Exception { PeerId peer0 = new PeerId(TestUtils.getLocalAddress(), TestUtils.INIT_PORT); - cluster = new TestCluster("testChangePeers", dataPath, Collections.singletonList(peer0)); + cluster = new TestCluster("testChangePeers", dataPath, Collections.singletonList(peer0), testInfo); assertTrue(cluster.start(peer0.getEndpoint())); cluster.waitLeader(); @@ -2985,7 +2993,7 @@ public void testChangePeersStepsDownInJointConsensus() throws Exception { // start single cluster peers.add(peer0); - cluster = new TestCluster("testChangePeersStepsDownInJointConsensus", dataPath, peers); + cluster = new TestCluster("testChangePeersStepsDownInJointConsensus", dataPath, peers, testInfo); assertTrue(cluster.start(peer0.getEndpoint())); cluster.waitLeader(); @@ -3102,7 +3110,7 @@ public void testChangePeersChaosWithSnapshot() throws Exception { // start cluster List peers = new ArrayList<>(); peers.add(new PeerId(TestUtils.getLocalAddress(), TestUtils.INIT_PORT)); - cluster = new TestCluster("unittest", dataPath, peers, 1000); + cluster = new TestCluster("unittest", dataPath, peers, 1000, testInfo); assertTrue(cluster.start(peers.get(0).getEndpoint(), false, 2)); // start other peers for (int i = 1; i < 10; i++) { @@ -3149,7 +3157,7 @@ public void testChangePeersChaosWithoutSnapshot() throws Exception { // start cluster List peers = new ArrayList<>(); peers.add(new PeerId(TestUtils.getLocalAddress(), TestUtils.INIT_PORT)); - cluster = new TestCluster("unittest", dataPath, peers, 1000); + cluster = new TestCluster("unittest", dataPath, peers, 1000, testInfo); assertTrue(cluster.start(peers.get(0).getEndpoint(), false, 100000)); // start other peers for (int i = 1; i < 10; i++) { @@ -3198,7 +3206,7 @@ public void testChangePeersChaosApplyTasks() throws Exception { // start cluster List peers = new ArrayList<>(); peers.add(new PeerId(TestUtils.getLocalAddress(), TestUtils.INIT_PORT)); - cluster = new TestCluster("unittest", dataPath, peers, 1000); + cluster = new TestCluster("unittest", dataPath, peers, 1000, testInfo); assertTrue(cluster.start(peers.get(0).getEndpoint(), false, 100000)); // start other peers for (int i = 1; i < 10; i++) { @@ -3273,7 +3281,7 @@ public void testChangePeersChaosApplyTasks() throws Exception { @Test public void testBlockedElection() throws Exception { List peers = TestUtils.generatePeers(3); - cluster = new TestCluster("unittest", dataPath, peers); + cluster = new TestCluster("unittest", dataPath, peers, testInfo); for (PeerId peer : peers) assertTrue(cluster.start(peer.getEndpoint())); @@ -3282,7 +3290,7 @@ public void testBlockedElection() throws Exception { Node leader = cluster.getLeader(); - LOG.warn("Current leader {}, electTimeout={}", leader.getNodeId().getPeerId(), ((NodeImpl) leader).getOptions().getElectionTimeoutMs()); + LOG.warn("Current leader {}, electTimeout={}", leader.getNodeId().getPeerId(), leader.getOptions().getElectionTimeoutMs()); List followers = cluster.getFollowers(); @@ -3453,7 +3461,7 @@ private RaftGroupService createService(String groupId, PeerId peerId, NodeOption var nodeManager = new NodeManager(); ClusterService clusterService = ClusterServiceTestUtils.clusterService( - peerId.getEndpoint().toString(), + testInfo, peerId.getEndpoint().getPort(), nodeFinder, new TestMessageSerializationRegistryImpl(), @@ -3491,9 +3499,9 @@ private RaftGroupService createService(String groupId, PeerId peerId, NodeOption /** * Creates a non-started {@link ClusterService}. */ - private static ClusterService createClusterService(Endpoint endpoint, NodeFinder nodeFinder) { + private ClusterService createClusterService(Endpoint endpoint, NodeFinder nodeFinder) { return ClusterServiceTestUtils.clusterService( - endpoint.toString(), + testInfo, endpoint.getPort(), nodeFinder, new TestMessageSerializationRegistryImpl(), @@ -3501,46 +3509,46 @@ private static ClusterService createClusterService(Endpoint endpoint, NodeFinder ); } - private void sendTestTaskAndWait(final Node node) throws InterruptedException { + private void sendTestTaskAndWait(Node node) throws InterruptedException { this.sendTestTaskAndWait(node, 0, 10, RaftError.SUCCESS); } - private void sendTestTaskAndWait(final Node node, int amount) throws InterruptedException { + private void sendTestTaskAndWait(Node node, int amount) throws InterruptedException { this.sendTestTaskAndWait(node, 0, amount, RaftError.SUCCESS); } - private void sendTestTaskAndWait(final Node node, final RaftError err) throws InterruptedException { + private void sendTestTaskAndWait(Node node, RaftError err) throws InterruptedException { this.sendTestTaskAndWait(node, 0, 10, err); } - private void sendTestTaskAndWait(final Node node, final int start, int amount, - final RaftError err) throws InterruptedException { - final CountDownLatch latch = new CountDownLatch(amount); + private void sendTestTaskAndWait(Node node, int start, int amount, + RaftError err) throws InterruptedException { + CountDownLatch latch = new CountDownLatch(amount); for (int i = start; i < start + amount; i++) { - final ByteBuffer data = ByteBuffer.wrap(("hello" + i).getBytes()); - final Task task = new Task(data, new ExpectClosure(err, latch)); + ByteBuffer data = ByteBuffer.wrap(("hello" + i).getBytes()); + Task task = new Task(data, new ExpectClosure(err, latch)); node.apply(task); } waitLatch(latch); } - private void sendTestTaskAndWait(final Node node, final int start, - final RaftError err) throws InterruptedException { + private void sendTestTaskAndWait(Node node, int start, + RaftError err) throws InterruptedException { sendTestTaskAndWait(node, start, 10, err); } @SuppressWarnings("SameParameterValue") - private void sendTestTaskAndWait(final String prefix, final Node node, final int code) throws InterruptedException { + private void sendTestTaskAndWait(String prefix, Node node, int code) throws InterruptedException { sendTestTaskAndWait(prefix, node, 10, code); } @SuppressWarnings("SameParameterValue") - private void sendTestTaskAndWait(final String prefix, final Node node, int amount, - final int code) throws InterruptedException { - final CountDownLatch latch = new CountDownLatch(10); + private void sendTestTaskAndWait(String prefix, Node node, int amount, + int code) throws InterruptedException { + CountDownLatch latch = new CountDownLatch(10); for (int i = 0; i < amount; i++) { - final ByteBuffer data = ByteBuffer.wrap((prefix + i).getBytes()); - final Task task = new Task(data, new ExpectClosure(code, null, latch)); + ByteBuffer data = ByteBuffer.wrap((prefix + i).getBytes()); + Task task = new Task(data, new ExpectClosure(code, null, latch)); node.apply(task); } waitLatch(latch); @@ -3562,7 +3570,7 @@ private void triggerLeaderSnapshot(TestCluster cluster, Node leader, int times) assertEquals(snapshotTimes + 1, cluster.getLeaderFsm().getSaveSnapshotTimes()); } - private void waitLatch(final CountDownLatch latch) throws InterruptedException { + private void waitLatch(CountDownLatch latch) throws InterruptedException { assertTrue(latch.await(30, TimeUnit.SECONDS)); } diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ITJRaftCounterServerTest.java b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ITJRaftCounterServerTest.java index b5f63725b8c..d3a87daca32 100644 --- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ITJRaftCounterServerTest.java +++ b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ITJRaftCounterServerTest.java @@ -53,7 +53,6 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.extension.ExtendWith; import static org.apache.ignite.raft.jraft.core.State.STATE_ERROR; @@ -128,13 +127,13 @@ class ITJRaftCounterServerTest extends RaftServerAbstractTest { /** */ @BeforeEach - void before(TestInfo testInfo) { + void before() { LOG.info(">>>>>>>>>>>>>>> Start test method: {}", testInfo.getTestMethod().orElseThrow().getName()); } /** */ @AfterEach - @Override protected void after(TestInfo testInfo) throws Exception { + @Override protected void after() throws Exception { LOG.info("Start client shutdown"); Iterator iterClients = clients.iterator(); @@ -159,7 +158,7 @@ void before(TestInfo testInfo) { server.stop(); } - super.after(testInfo); + super.after(); LOG.info(">>>>>>>>>>>>>>> End test method: {}", testInfo.getTestMethod().orElseThrow().getName()); } @@ -171,7 +170,7 @@ void before(TestInfo testInfo) { private JRaftServerImpl startServer(int idx, Consumer clo) { var addr = new NetworkAddress(getLocalAddress(), PORT); - ClusterService service = clusterService("server" + idx, PORT + idx, List.of(addr), true); + ClusterService service = clusterService(PORT + idx, List.of(addr), true); JRaftServerImpl server = new JRaftServerImpl(service, dataPath) { @Override public void stop() { @@ -202,8 +201,7 @@ private JRaftServerImpl startServer(int idx, Consumer clo) { private RaftGroupService startClient(String groupId) throws Exception { var addr = new NetworkAddress(getLocalAddress(), PORT); - ClusterService clientNode = clusterService( - "client_" + groupId + "_", CLIENT_PORT + clients.size(), List.of(addr), true); + ClusterService clientNode = clusterService(CLIENT_PORT + clients.size(), List.of(addr), true); RaftGroupService client = RaftGroupServiceImpl.start(groupId, clientNode, FACTORY, 10_000, List.of(new Peer(addr)), false, 200).get(3, TimeUnit.SECONDS); diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ITSimpleCounterServerTest.java b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ITSimpleCounterServerTest.java index 52894b47a5d..6122ccf2234 100644 --- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ITSimpleCounterServerTest.java +++ b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ITSimpleCounterServerTest.java @@ -33,7 +33,6 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.extension.ExtendWith; import static org.apache.ignite.raft.jraft.test.TestUtils.waitForTopology; @@ -77,16 +76,15 @@ class ITSimpleCounterServerTest extends RaftServerAbstractTest { private Path dataPath; /** - * @param testInfo Test info. * @throws Exception If failed. */ @BeforeEach - void before(TestInfo testInfo) throws Exception { + void before() throws Exception { LOG.info(">>>> Starting test {}", testInfo.getTestMethod().orElseThrow().getName()); var addr = new NetworkAddress("localhost", PORT); - ClusterService service = clusterService(addr.toString(), PORT, List.of(), true); + ClusterService service = clusterService(PORT, List.of(), true); server = new JRaftServerImpl(service, dataPath) { @Override public synchronized void stop() { @@ -103,12 +101,12 @@ void before(TestInfo testInfo) throws Exception { server.startRaftGroup(COUNTER_GROUP_ID_0, new CounterListener(), List.of(new Peer(serverNode.address()))); server.startRaftGroup(COUNTER_GROUP_ID_1, new CounterListener(), List.of(new Peer(serverNode.address()))); - ClusterService clientNode1 = clusterService("localhost:" + (PORT + 1), PORT + 1, List.of(addr), true); + ClusterService clientNode1 = clusterService(PORT + 1, List.of(addr), true); client1 = RaftGroupServiceImpl.start(COUNTER_GROUP_ID_0, clientNode1, FACTORY, 1000, List.of(new Peer(serverNode.address())), false, 200).get(3, TimeUnit.SECONDS); - ClusterService clientNode2 = clusterService("localhost:" + (PORT + 2), PORT + 2, List.of(addr), true); + ClusterService clientNode2 = clusterService(PORT + 2, List.of(addr), true); client2 = RaftGroupServiceImpl.start(COUNTER_GROUP_ID_1, clientNode2, FACTORY, 1000, List.of(new Peer(serverNode.address())), false, 200).get(3, TimeUnit.SECONDS); @@ -122,12 +120,12 @@ void before(TestInfo testInfo) throws Exception { * @throws Exception If failed. */ @AfterEach - @Override public void after(TestInfo testInfo) throws Exception { + @Override public void after() throws Exception { server.stop(); client1.shutdown(); client2.shutdown(); - super.after(testInfo); + super.after(); } /** diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/RaftServerAbstractTest.java b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/RaftServerAbstractTest.java index c606a47c695..b7c42ef944f 100644 --- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/RaftServerAbstractTest.java +++ b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/RaftServerAbstractTest.java @@ -30,6 +30,7 @@ import org.apache.ignite.raft.jraft.RaftMessagesFactory; import org.apache.ignite.utils.ClusterServiceTestUtils; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestInfo; /** @@ -53,22 +54,29 @@ abstract class RaftServerAbstractTest { /** */ private static final MessageSerializationRegistry SERIALIZATION_REGISTRY = new MessageSerializationRegistryImpl(); + /** Test info. */ + TestInfo testInfo; + private final List clusterServices = new ArrayList<>(); + @BeforeEach + void initTestInfo(TestInfo testInfo) { + this.testInfo = testInfo; + } + @AfterEach - protected void after(TestInfo testInfo) throws Exception { + protected void after() throws Exception { clusterServices.forEach(ClusterService::stop); } /** - * @param name Node name. * @param port Local port. * @param servers Server nodes of the cluster. * @return The client cluster view. */ - protected ClusterService clusterService(String name, int port, List servers, boolean start) { + protected ClusterService clusterService(int port, List servers, boolean start) { var network = ClusterServiceTestUtils.clusterService( - name, + testInfo, port, new StaticNodeFinder(servers), SERIALIZATION_REGISTRY, diff --git a/modules/raft/src/test/java/org/apache/ignite/raft/jraft/core/TestCluster.java b/modules/raft/src/test/java/org/apache/ignite/raft/jraft/core/TestCluster.java index a6d1d9f86c6..cf09c1ac42a 100644 --- a/modules/raft/src/test/java/org/apache/ignite/raft/jraft/core/TestCluster.java +++ b/modules/raft/src/test/java/org/apache/ignite/raft/jraft/core/TestCluster.java @@ -62,6 +62,7 @@ import org.apache.ignite.raft.jraft.util.concurrent.FixedThreadsExecutorGroup; import org.apache.ignite.utils.ClusterServiceTestUtils; import org.jetbrains.annotations.Nullable; +import org.junit.jupiter.api.TestInfo; import static java.util.stream.Collectors.collectingAndThen; import static java.util.stream.Collectors.toList; @@ -92,6 +93,9 @@ public class TestCluster { private final Lock lock = new ReentrantLock(); private final Consumer optsClo; + /** Test info. */ + private final TestInfo testInfo; + /** * These disruptors will be used for all RAFT servers in the cluster. */ @@ -126,7 +130,7 @@ public JRaftServiceFactory getRaftServiceFactory() { return this.raftServiceFactory; } - public void setRaftServiceFactory(final JRaftServiceFactory raftServiceFactory) { + public void setRaftServiceFactory(JRaftServiceFactory raftServiceFactory) { this.raftServiceFactory = raftServiceFactory; } @@ -134,7 +138,7 @@ public LinkedHashSet getLearners() { return this.learners; } - public void setLearners(final LinkedHashSet learners) { + public void setLearners(LinkedHashSet learners) { this.learners = learners; } @@ -142,23 +146,40 @@ public List getPeers() { return this.peers; } - public TestCluster(final String name, final String dataPath, final List peers) { - this(name, dataPath, peers, ELECTION_TIMEOUT_MILLIS); - } - - public TestCluster(final String name, final String dataPath, final List peers, - final int electionTimeoutMs) { - this(name, dataPath, peers, new LinkedHashSet<>(), ELECTION_TIMEOUT_MILLIS, null); - } - - public TestCluster(final String name, final String dataPath, final List peers, - final LinkedHashSet learners, final int electionTimeoutMs) { - this(name, dataPath, peers, learners, ELECTION_TIMEOUT_MILLIS, null); - } - - public TestCluster(final String name, final String dataPath, final List peers, - final LinkedHashSet learners, final int electionTimeoutMs, @Nullable Consumer optsClo) { - super(); + public TestCluster(String name, String dataPath, List peers, TestInfo testInfo) { + this(name, dataPath, peers, ELECTION_TIMEOUT_MILLIS, testInfo); + } + + public TestCluster( + String name, + String dataPath, + List peers, + int electionTimeoutMs, + TestInfo testInfo + ) { + this(name, dataPath, peers, new LinkedHashSet<>(), ELECTION_TIMEOUT_MILLIS, null, testInfo); + } + + public TestCluster( + String name, + String dataPath, + List peers, + LinkedHashSet learners, + int electionTimeoutMs, + TestInfo testInfo + ) { + this(name, dataPath, peers, learners, ELECTION_TIMEOUT_MILLIS, null, testInfo); + } + + public TestCluster( + String name, + String dataPath, + List peers, + LinkedHashSet learners, + int electionTimeoutMs, + @Nullable Consumer optsClo, + TestInfo testInfo + ) { this.name = name; this.dataPath = dataPath; this.peers = peers; @@ -167,47 +188,48 @@ public TestCluster(final String name, final String dataPath, final List this.electionTimeoutMs = electionTimeoutMs; this.learners = learners; this.optsClo = optsClo; + this.testInfo = testInfo; } - public boolean start(final Endpoint addr) throws Exception { + public boolean start(Endpoint addr) throws Exception { return this.start(addr, false, 300); } - public boolean start(final Endpoint addr, final int priority) throws Exception { + public boolean start(Endpoint addr, int priority) throws Exception { return this.start(addr, false, 300, false, null, null, priority); } - public boolean startLearner(final PeerId peer) throws Exception { + public boolean startLearner(PeerId peer) throws Exception { this.learners.add(peer); return this.start(peer.getEndpoint(), false, 300); } - public boolean start(final Endpoint listenAddr, final boolean emptyPeers, final int snapshotIntervalSecs) + public boolean start(Endpoint listenAddr, boolean emptyPeers, int snapshotIntervalSecs) throws IOException { return this.start(listenAddr, emptyPeers, snapshotIntervalSecs, false); } - public boolean start(final Endpoint listenAddr, final boolean emptyPeers, final int snapshotIntervalSecs, - final boolean enableMetrics) throws IOException { + public boolean start(Endpoint listenAddr, boolean emptyPeers, int snapshotIntervalSecs, + boolean enableMetrics) throws IOException { return this.start(listenAddr, emptyPeers, snapshotIntervalSecs, enableMetrics, null, null); } - public boolean start(final Endpoint listenAddr, final boolean emptyPeers, final int snapshotIntervalSecs, - final boolean enableMetrics, final SnapshotThrottle snapshotThrottle) throws IOException { + public boolean start(Endpoint listenAddr, boolean emptyPeers, int snapshotIntervalSecs, + boolean enableMetrics, SnapshotThrottle snapshotThrottle) throws IOException { return this.start(listenAddr, emptyPeers, snapshotIntervalSecs, enableMetrics, snapshotThrottle, null); } - public boolean start(final Endpoint listenAddr, final boolean emptyPeers, final int snapshotIntervalSecs, - final boolean enableMetrics, final SnapshotThrottle snapshotThrottle, - final RaftOptions raftOptions) throws IOException { + public boolean start(Endpoint listenAddr, boolean emptyPeers, int snapshotIntervalSecs, + boolean enableMetrics, SnapshotThrottle snapshotThrottle, + RaftOptions raftOptions) throws IOException { return this.start(listenAddr, emptyPeers, snapshotIntervalSecs, enableMetrics, snapshotThrottle, raftOptions, ElectionPriority.Disabled); } - public boolean start(final Endpoint listenAddr, final boolean emptyPeers, final int snapshotIntervalSecs, - final boolean enableMetrics, final SnapshotThrottle snapshotThrottle, - final RaftOptions raftOptions, final int priority) throws IOException { + public boolean start(Endpoint listenAddr, boolean emptyPeers, int snapshotIntervalSecs, + boolean enableMetrics, SnapshotThrottle snapshotThrottle, + RaftOptions raftOptions, int priority) throws IOException { this.lock.lock(); try { @@ -215,7 +237,7 @@ public boolean start(final Endpoint listenAddr, final boolean emptyPeers, final return true; } - final NodeOptions nodeOptions = new NodeOptions(); + NodeOptions nodeOptions = new NodeOptions(); nodeOptions.setServerName(listenAddr.toString()); @@ -240,7 +262,7 @@ public boolean start(final Endpoint listenAddr, final boolean emptyPeers, final if (raftOptions != null) { nodeOptions.setRaftOptions(raftOptions); } - final String serverDataPath = this.dataPath + File.separator + listenAddr.toString().replace(':', '_'); + String serverDataPath = this.dataPath + File.separator + listenAddr.toString().replace(':', '_'); new File(serverDataPath).mkdirs(); nodeOptions.setLogUri(serverDataPath + File.separator + "logs"); nodeOptions.setRaftMetaUri(serverDataPath + File.separator + "meta"); @@ -271,7 +293,7 @@ public boolean start(final Endpoint listenAddr, final boolean emptyPeers, final () -> new LogManagerImpl.StableClosureEvent(), nodeOptions.getStripes()))); - final MockStateMachine fsm = new MockStateMachine(listenAddr); + MockStateMachine fsm = new MockStateMachine(listenAddr); nodeOptions.setFsm(fsm); if (!emptyPeers) @@ -285,7 +307,7 @@ public boolean start(final Endpoint listenAddr, final boolean emptyPeers, final NodeManager nodeManager = new NodeManager(); ClusterService clusterService = ClusterServiceTestUtils.clusterService( - listenAddr.toString(), + testInfo, listenAddr.getPort(), nodeFinder, new TestMessageSerializationRegistryImpl(), @@ -307,7 +329,7 @@ public boolean start(final Endpoint listenAddr, final boolean emptyPeers, final if (optsClo != null) optsClo.accept(nodeOptions); - final RaftGroupService server = new RaftGroupService(this.name, new PeerId(listenAddr, 0, priority), + RaftGroupService server = new RaftGroupService(this.name, new PeerId(listenAddr, 0, priority), nodeOptions, rpcServer, nodeManager) { @Override public synchronized void shutdown() { super.shutdown(); @@ -320,7 +342,7 @@ public boolean start(final Endpoint listenAddr, final boolean emptyPeers, final this.serverMap.put(listenAddr, server); - final Node node = server.start(); + Node node = server.start(); this.fsms.put(new PeerId(listenAddr, 0), fsm); this.nodes.add((NodeImpl) node); @@ -350,7 +372,7 @@ public RaftGroupService getServer(Endpoint endpoint) { return serverMap.get(endpoint); } - public MockStateMachine getFsmByPeer(final PeerId peer) { + public MockStateMachine getFsmByPeer(PeerId peer) { this.lock.lock(); try { return this.fsms.get(peer); @@ -370,16 +392,16 @@ public List getFsms() { } } - public boolean stop(final Endpoint listenAddr) throws InterruptedException { + public boolean stop(Endpoint listenAddr) throws InterruptedException { removeNode(listenAddr); - final RaftGroupService raftGroupService = this.serverMap.remove(listenAddr); + RaftGroupService raftGroupService = this.serverMap.remove(listenAddr); raftGroupService.shutdown(); return true; } public void stopAll() throws InterruptedException { - final List addrs = getAllNodes(); - for (final Endpoint addr : addrs) + List addrs = getAllNodes(); + for (Endpoint addr : addrs) stop(addr); fsmCallerDusruptors.values().forEach(StripedDisruptor::shutdown); @@ -391,8 +413,8 @@ public void stopAll() throws InterruptedException { schedulers.forEach(Scheduler::shutdown); } - public void clean(final Endpoint listenAddr) { - final Path path = Paths.get(this.dataPath, listenAddr.toString().replace(':', '_')); + public void clean(Endpoint listenAddr) { + Path path = Paths.get(this.dataPath, listenAddr.toString().replace(':', '_')); LOG.info("Clean dir: {}", path); IgniteUtils.deleteIfExists(path); } @@ -401,7 +423,7 @@ public Node getLeader() { this.lock.lock(); try { for (int i = 0; i < this.nodes.size(); i++) { - final NodeImpl node = this.nodes.get(i); + NodeImpl node = this.nodes.get(i); if (node.isLeader() && this.fsms.get(node.getServerId()).getLeaderTerm() == node.getCurrentTerm()) { return node; } @@ -414,7 +436,7 @@ public Node getLeader() { } public MockStateMachine getLeaderFsm() { - final Node leader = getLeader(); + Node leader = getLeader(); if (leader != null) { return (MockStateMachine) leader.getOptions().getFsm(); } @@ -441,10 +463,10 @@ public void waitLeader() throws InterruptedException { } public List getFollowers() { - final List ret = new ArrayList<>(); + List ret = new ArrayList<>(); this.lock.lock(); try { - for (final NodeImpl node : this.nodes) { + for (NodeImpl node : this.nodes) { if (!node.isLeader() && !this.learners.contains(node.getServerId())) { ret.add(node); } @@ -462,14 +484,14 @@ public List getFollowers() { * @param node The leader. * @throws InterruptedException if interrupted */ - public void ensureLeader(final Node node) throws InterruptedException { + public void ensureLeader(Node node) throws InterruptedException { while (true) { this.lock.lock(); try { boolean wait = false; - for (final Node node0 : this.nodes) { - final PeerId leaderId = node0.getLeaderId(); + for (Node node0 : this.nodes) { + PeerId leaderId = node0.getLeaderId(); if (leaderId == null || !leaderId.equals(node.getNodeId().getPeerId())) { wait = true; @@ -513,7 +535,7 @@ public List getAllNodes() { } } - public Node removeNode(final Endpoint addr) { + public Node removeNode(Endpoint addr) { Node ret = null; this.lock.lock(); try { diff --git a/modules/raft/src/test/java/org/apache/ignite/raft/jraft/rpc/IgniteRpcTest.java b/modules/raft/src/test/java/org/apache/ignite/raft/jraft/rpc/IgniteRpcTest.java index f9bf10b9a0d..56ae92d6a58 100644 --- a/modules/raft/src/test/java/org/apache/ignite/raft/jraft/rpc/IgniteRpcTest.java +++ b/modules/raft/src/test/java/org/apache/ignite/raft/jraft/rpc/IgniteRpcTest.java @@ -33,6 +33,7 @@ import org.apache.ignite.raft.jraft.util.ExecutorServiceHelper; import org.apache.ignite.utils.ClusterServiceTestUtils; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestInfo; import static org.apache.ignite.raft.jraft.JRaftUtils.addressFromEndpoint; @@ -46,6 +47,14 @@ public class IgniteRpcTest extends AbstractRpcTest { /** Requests executor. */ private ExecutorService requestExecutor; + /** Test info. */ + private final TestInfo testInfo; + + /** */ + public IgniteRpcTest(TestInfo testInfo) { + this.testInfo = testInfo; + } + /** {@inheritDoc} */ @AfterEach @Override public void tearDown() { @@ -57,7 +66,7 @@ public class IgniteRpcTest extends AbstractRpcTest { /** {@inheritDoc} */ @Override public RpcServer createServer(Endpoint endpoint) { ClusterService service = ClusterServiceTestUtils.clusterService( - endpoint.toString(), + testInfo, endpoint.getPort(), new StaticNodeFinder(Collections.emptyList()), new MessageSerializationRegistryImpl(), @@ -86,7 +95,7 @@ public class IgniteRpcTest extends AbstractRpcTest { int i = cntr.incrementAndGet(); ClusterService service = ClusterServiceTestUtils.clusterService( - "client" + i, + testInfo, endpoint.getPort() - i, new StaticNodeFinder(List.of(addressFromEndpoint(endpoint))), new MessageSerializationRegistryImpl(), diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/ITDistributedConfigurationPropertiesTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/ITDistributedConfigurationPropertiesTest.java index fd2955c51a7..8e9c2e85e91 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/ITDistributedConfigurationPropertiesTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/ITDistributedConfigurationPropertiesTest.java @@ -51,11 +51,13 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.extension.ExtendWith; import static java.util.stream.Collectors.joining; import static java.util.stream.Collectors.toUnmodifiableList; import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.directValue; +import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName; import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe; import static org.hamcrest.MatcherAssert.assertThat; @@ -83,7 +85,7 @@ public static class DistributedConfigurationSchema { */ private static class Node { /** */ - private final List metaStorageAddrs; + private final List metaStorageNodes; /** */ private final VaultManager vaultManager; @@ -110,17 +112,18 @@ private static class Node { * Constructor that simply creates a subset of components of this node. */ Node( + TestInfo testInfo, Path workDir, NetworkAddress addr, List memberAddrs, - List metaStorageAddrs + List metaStorageNodes ) { - this.metaStorageAddrs = metaStorageAddrs; + this.metaStorageNodes = metaStorageNodes; vaultManager = new VaultManager(new InMemoryVaultService()); clusterService = ClusterServiceTestUtils.clusterService( - addr.toString(), + testInfo, addr.port(), new StaticNodeFinder(memberAddrs), new MessageSerializationRegistryImpl(), @@ -175,7 +178,7 @@ void start() throws Exception { cfgManager.start(); // metastorage configuration - String metaStorageCfg = metaStorageAddrs.stream() + String metaStorageCfg = metaStorageNodes.stream() .map(Object::toString) .collect(joining("\", \"", "\"", "\"")); @@ -225,22 +228,27 @@ void stopReceivingUpdates() { /** */ @BeforeEach - void setUp(@WorkDirectory Path workDir) throws Exception { + void setUp(@WorkDirectory Path workDir, TestInfo testInfo) throws Exception { var firstNodeAddr = new NetworkAddress("localhost", 10000); + + String firstNodeName = testNodeName(testInfo, firstNodeAddr.port()); + var secondNodeAddr = new NetworkAddress("localhost", 10001); firstNode = new Node( + testInfo, workDir.resolve("firstNode"), firstNodeAddr, List.of(firstNodeAddr, secondNodeAddr), - List.of(firstNodeAddr) + List.of(firstNodeName) ); secondNode = new Node( + testInfo, workDir.resolve("secondNode"), secondNodeAddr, List.of(firstNodeAddr, secondNodeAddr), - List.of(firstNodeAddr) + List.of(firstNodeName) ); firstNode.start(); diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ITDistributedConfigurationStorageTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ITDistributedConfigurationStorageTest.java index 3eeb8a087f8..f11dcda28bd 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ITDistributedConfigurationStorageTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ITDistributedConfigurationStorageTest.java @@ -26,9 +26,6 @@ import org.apache.ignite.configuration.RootKey; import org.apache.ignite.configuration.schemas.runner.NodeConfiguration; import org.apache.ignite.internal.configuration.ConfigurationManager; -import org.apache.ignite.internal.configuration.storage.Data; -import org.apache.ignite.internal.configuration.storage.DistributedConfigurationStorage; -import org.apache.ignite.internal.configuration.storage.LocalConfigurationStorage; import org.apache.ignite.internal.manager.IgniteComponent; import org.apache.ignite.internal.metastorage.MetaStorageManager; import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage; @@ -44,10 +41,12 @@ import org.apache.ignite.network.scalecube.TestScaleCubeClusterServiceFactory; import org.apache.ignite.utils.ClusterServiceTestUtils; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.extension.ExtendWith; import static java.util.concurrent.CompletableFuture.completedFuture; import static org.apache.ignite.internal.metastorage.MetaStorageManager.APPLIED_REV; +import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName; import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe; import static org.hamcrest.MatcherAssert.assertThat; @@ -63,7 +62,7 @@ public class ITDistributedConfigurationStorageTest { */ private static class Node { /** */ - private final NetworkAddress addr = new NetworkAddress("localhost", 10000); + private final String name; /** */ private final VaultManager vaultManager; @@ -86,11 +85,15 @@ private static class Node { /** * Constructor that simply creates a subset of components of this node. */ - Node(Path workDir) { + Node(TestInfo testInfo, Path workDir) { + var addr = new NetworkAddress("localhost", 10000); + + name = testNodeName(testInfo, addr.port()); + vaultManager = new VaultManager(new PersistentVaultService(workDir.resolve("vault"))); clusterService = ClusterServiceTestUtils.clusterService( - addr.toString(), + testInfo, addr.port(), new StaticNodeFinder(List.of(addr)), new MessageSerializationRegistryImpl(), @@ -128,7 +131,7 @@ void start() throws Exception { cfgManager.start(); // metastorage configuration - var config = String.format("{\"node\": {\"metastorageNodes\": [ \"%s\" ]}}", addr); + var config = String.format("{\"node\": {\"metastorageNodes\": [ \"%s\" ]}}", name); cfgManager.bootstrap(config); @@ -164,8 +167,8 @@ void stop() throws Exception { * @see IGNITE-15213 */ @Test - void testRestartWithPds(@WorkDirectory Path workDir) throws Exception { - var node = new Node(workDir); + void testRestartWithPds(@WorkDirectory Path workDir, TestInfo testInfo) throws Exception { + var node = new Node(testInfo, workDir); Map data = Map.of("foo", "bar"); @@ -180,7 +183,7 @@ void testRestartWithPds(@WorkDirectory Path workDir) throws Exception { node.stop(); } - var node2 = new Node(workDir); + var node2 = new Node(testInfo, workDir); try { node2.start(); diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/AbstractSchemaChangeTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/AbstractSchemaChangeTest.java index 7698c98213c..a6a0fb3a9ed 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/AbstractSchemaChangeTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/AbstractSchemaChangeTest.java @@ -37,11 +37,14 @@ import org.jetbrains.annotations.NotNull; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.extension.ExtendWith; import static org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter.convert; +import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName; /** * Ignition interface tests. @@ -51,38 +54,11 @@ abstract class AbstractSchemaChangeTest { /** Table name. */ public static final String TABLE = "PUBLIC.tbl1"; + /** Network ports of the test nodes. */ + private static final int[] PORTS = { 3344, 3345, 3346 }; + /** Nodes bootstrap configuration. */ - private final Map nodesBootstrapCfg = new LinkedHashMap<>() {{ - put("node0", "{\n" + - " \"node\": {\n" + - " \"metastorageNodes\":[ \"node0\" ]\n" + - " },\n" + - " \"network\": {\n" + - " \"port\":3344,\n" + - " \"netClusterNodes\":[ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + - " }\n" + - "}"); - - put("node1", "{\n" + - " \"node\": {\n" + - " \"metastorageNodes\":[ \"node0\" ]\n" + - " },\n" + - " \"network\": {\n" + - " \"port\":3345,\n" + - " \"netClusterNodes\":[ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + - " }\n" + - "}"); - - put("node2", "{\n" + - " \"node\": {\n" + - " \"metastorageNodes\":[ \"node0\" ]\n" + - " },\n" + - " \"network\": {\n" + - " \"port\":3346,\n" + - " \"netClusterNodes\":[ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + - " }\n" + - "}"); - }}; + private final Map nodesBootstrapCfg = new LinkedHashMap<>(); /** Cluster nodes. */ private final List clusterNodes = new ArrayList<>(); @@ -91,9 +67,48 @@ abstract class AbstractSchemaChangeTest { @WorkDirectory private Path workDir; - /** - * - */ + /** */ + @BeforeEach + void setUp(TestInfo testInfo) { + String node0Name = testNodeName(testInfo, PORTS[0]); + String node1Name = testNodeName(testInfo, PORTS[1]); + String node2Name = testNodeName(testInfo, PORTS[2]); + + nodesBootstrapCfg.put( + node0Name, + "{\n" + + " node.metastorageNodes: [ \"" + node0Name + "\" ],\n" + + " network: {\n" + + " port: " + PORTS[0] + "\n" + + " netClusterNodes: [ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + + " }\n" + + "}" + ); + + nodesBootstrapCfg.put( + node1Name, + "{\n" + + " node.metastorageNodes: [ \"" + node0Name + "\" ],\n" + + " network: {\n" + + " port: " + PORTS[1] + "\n" + + " netClusterNodes: [ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + + " }\n" + + "}" + ); + + nodesBootstrapCfg.put( + node2Name, + "{\n" + + " node.metastorageNodes: [ \"" + node0Name + "\" ],\n" + + " network: {\n" + + " port: " + PORTS[2] + "\n" + + " netClusterNodes: [ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + + " }\n" + + "}" + ); + } + + /** */ @AfterEach void afterEach() throws Exception { IgniteUtils.closeAll(Lists.reverse(clusterNodes)); diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITDynamicTableCreationTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITDynamicTableCreationTest.java index dc65651ca97..1f849b8a814 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITDynamicTableCreationTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITDynamicTableCreationTest.java @@ -38,9 +38,12 @@ import org.apache.ignite.table.Table; import org.apache.ignite.table.Tuple; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.extension.ExtendWith; +import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -50,38 +53,11 @@ */ @ExtendWith(WorkDirectoryExtension.class) class ITDynamicTableCreationTest { + /** Network ports of the test nodes. */ + private static final int[] PORTS = { 3344, 3345, 3346 }; + /** Nodes bootstrap configuration. */ - private final Map nodesBootstrapCfg = new LinkedHashMap<>() {{ - put("node0", "{\n" + - " \"node\": {\n" + - " \"metastorageNodes\":[ \"node0\" ]\n" + - " },\n" + - " \"network\": {\n" + - " \"port\":3344,\n" + - " \"netClusterNodes\":[ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + - " }\n" + - "}"); - - put("node1", "{\n" + - " \"node\": {\n" + - " \"metastorageNodes\":[ \"node0\" ]\n" + - " },\n" + - " \"network\": {\n" + - " \"port\":3345,\n" + - " \"netClusterNodes\":[ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + - " }\n" + - "}"); - - put("node2", "{\n" + - " \"node\": {\n" + - " \"metastorageNodes\":[ \"node0\"]\n" + - " },\n" + - " \"network\": {\n" + - " \"port\":3346,\n" + - " \"netClusterNodes\":[ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + - " }\n" + - "}"); - }}; + private final Map nodesBootstrapCfg = new LinkedHashMap<>(); /** */ private final List clusterNodes = new ArrayList<>(); @@ -90,6 +66,47 @@ class ITDynamicTableCreationTest { @WorkDirectory private Path workDir; + /** */ + @BeforeEach + void setUp(TestInfo testInfo) { + String node0Name = testNodeName(testInfo, PORTS[0]); + String node1Name = testNodeName(testInfo, PORTS[1]); + String node2Name = testNodeName(testInfo, PORTS[2]); + + nodesBootstrapCfg.put( + node0Name, + "{\n" + + " node.metastorageNodes: [ \"" + node0Name + "\" ],\n" + + " network: {\n" + + " port: " + PORTS[0] + "\n" + + " netClusterNodes: [ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + + " }\n" + + "}" + ); + + nodesBootstrapCfg.put( + node1Name, + "{\n" + + " node.metastorageNodes: [ \"" + node0Name + "\" ],\n" + + " network: {\n" + + " port: " + PORTS[1] + "\n" + + " netClusterNodes: [ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + + " }\n" + + "}" + ); + + nodesBootstrapCfg.put( + node2Name, + "{\n" + + " node.metastorageNodes: [ \"" + node0Name + "\" ],\n" + + " network: {\n" + + " port: " + PORTS[2] + "\n" + + " netClusterNodes: [ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + + " }\n" + + "}" + ); + } + /** */ @AfterEach void tearDown() throws Exception { diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITIgniteNodeRestartTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITIgniteNodeRestartTest.java index 62e683c3f12..6ccc8211479 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITIgniteNodeRestartTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITIgniteNodeRestartTest.java @@ -30,7 +30,9 @@ import org.apache.ignite.table.Tuple; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; +import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName; import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; @@ -39,48 +41,48 @@ * These tests check node restart scenarios. */ public class ITIgniteNodeRestartTest extends IgniteAbstractTest { - /** Test node name. */ - public static final String NODE_NAME = "TestNode"; + /** Default node port. */ + private static final int DEFAULT_NODE_PORT = 47500; /** Test table name. */ - public static final String TABLE_NAME = "Table1"; + private static final String TABLE_NAME = "Table1"; /** * Restarts empty node. - * - * @throws Exception If failed. */ @Test - public void emptyNodeTest() throws Exception { - IgniteImpl ignite = (IgniteImpl)IgnitionManager.start(NODE_NAME, null, workDir.resolve(NODE_NAME)); + public void emptyNodeTest(TestInfo testInfo) { + String nodeName = testNodeName(testInfo, DEFAULT_NODE_PORT); + + IgniteImpl ignite = (IgniteImpl)IgnitionManager.start(nodeName, null, workDir); int nodePort = ignite.nodeConfiguration().getConfiguration(NetworkConfiguration.KEY).port().value(); - assertEquals(47500, nodePort); + assertEquals(DEFAULT_NODE_PORT, nodePort); IgnitionManager.stop(ignite.name()); - ignite = (IgniteImpl)IgnitionManager.start(NODE_NAME, null, workDir.resolve(NODE_NAME)); + ignite = (IgniteImpl)IgnitionManager.start(nodeName, null, workDir); nodePort = ignite.nodeConfiguration().getConfiguration(NetworkConfiguration.KEY).port().value(); - assertEquals(47500, nodePort); + assertEquals(DEFAULT_NODE_PORT, nodePort); IgnitionManager.stop(ignite.name()); } /** * Restarts a node with changing configuration. - * - * @throws Exception If failed. */ @Test - public void changeConfigurationOnStartTest() throws Exception { - IgniteImpl ignite = (IgniteImpl)IgnitionManager.start(NODE_NAME, null, workDir.resolve(NODE_NAME)); + public void changeConfigurationOnStartTest(TestInfo testInfo) { + String nodeName = testNodeName(testInfo, DEFAULT_NODE_PORT); + + IgniteImpl ignite = (IgniteImpl)IgnitionManager.start(nodeName, null, workDir); int nodePort = ignite.nodeConfiguration().getConfiguration(NetworkConfiguration.KEY).port().value(); - assertEquals(47500, nodePort); + assertEquals(DEFAULT_NODE_PORT, nodePort); IgnitionManager.stop(ignite.name()); @@ -88,7 +90,7 @@ public void changeConfigurationOnStartTest() throws Exception { String updateCfg = "network.port=" + newPort; - ignite = (IgniteImpl)IgnitionManager.start(NODE_NAME, updateCfg, workDir.resolve(NODE_NAME)); + ignite = (IgniteImpl)IgnitionManager.start(nodeName, updateCfg, workDir); nodePort = ignite.nodeConfiguration().getConfiguration(NetworkConfiguration.KEY).port().value(); @@ -100,17 +102,17 @@ public void changeConfigurationOnStartTest() throws Exception { /** * Checks that the only one non-default property overwrites after another configuration is passed on the node * restart. - * - * @throws Exception If failed. */ @Test - public void twoCustomPropertiesTest() throws Exception { + public void twoCustomPropertiesTest(TestInfo testInfo) { + String nodeName = testNodeName(testInfo, 3344); + String startCfg = "network: {\n" + " port:3344,\n" + " netClusterNodes:[ \"localhost:3344\" ]\n" + "}"; - IgniteImpl ignite = (IgniteImpl)IgnitionManager.start(NODE_NAME, startCfg, workDir.resolve(NODE_NAME)); + IgniteImpl ignite = (IgniteImpl)IgnitionManager.start(nodeName, startCfg, workDir); assertEquals( 3344, @@ -125,9 +127,9 @@ public void twoCustomPropertiesTest() throws Exception { IgnitionManager.stop(ignite.name()); ignite = (IgniteImpl)IgnitionManager.start( - NODE_NAME, + nodeName, "network.netClusterNodes=[ \"localhost:3344\", \"localhost:3343\" ]", - workDir.resolve(NODE_NAME) + workDir ); assertEquals( @@ -148,16 +150,18 @@ public void twoCustomPropertiesTest() throws Exception { */ @Test @Disabled("https://issues.apache.org/jira/browse/IGNITE-15255") - public void nodeWithDataTest() { - Ignite ignite = IgnitionManager.start(NODE_NAME, "{\n" + + public void nodeWithDataTest(TestInfo testInfo) { + String nodeName = testNodeName(testInfo, 3344); + + Ignite ignite = IgnitionManager.start(nodeName, "{\n" + " \"node\": {\n" + - " \"metastorageNodes\":[ " + NODE_NAME + " ]\n" + + " \"metastorageNodes\":[ " + nodeName + " ]\n" + " },\n" + " \"network\": {\n" + " \"port\":3344,\n" + " \"netClusterNodes\":[ \"localhost:3344\" ]\n" + " }\n" + - "}", workDir.resolve(NODE_NAME)); + "}", workDir); TableDefinition scmTbl1 = SchemaBuilders.tableBuilder("PUBLIC", TABLE_NAME).columns( SchemaBuilders.column("id", ColumnType.INT32).asNonNull().build(), @@ -178,9 +182,9 @@ public void nodeWithDataTest() { table.keyValueView().put(key, val); } - IgnitionManager.stop(NODE_NAME); + IgnitionManager.stop(nodeName); - ignite = IgnitionManager.start(NODE_NAME, null, workDir.resolve(NODE_NAME)); + ignite = IgnitionManager.start(nodeName, null, workDir); assertNotNull(ignite.tables().table(TABLE_NAME)); @@ -190,6 +194,6 @@ public void nodeWithDataTest() { .stringValue("name")); } - IgnitionManager.stop(NODE_NAME); + IgnitionManager.stop(nodeName); } } diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITIgnitionTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITIgnitionTest.java index a63aee636da..074a1f5d17a 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITIgnitionTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITIgnitionTest.java @@ -30,46 +30,23 @@ import org.apache.ignite.internal.util.IgniteUtils; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.extension.ExtendWith; +import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName; + /** * Ignition interface tests. */ @ExtendWith(WorkDirectoryExtension.class) class ITIgnitionTest { + /** Network ports of the test nodes. */ + private static final int[] PORTS = { 3344, 3345, 3346 }; + /** Nodes bootstrap configuration. */ - private final Map nodesBootstrapCfg = new LinkedHashMap<>() {{ - put("node0", "{\n" + - " \"node\": {\n" + - " \"metastorageNodes\":[ \"node0\" ]\n" + - " },\n" + - " \"network\": {\n" + - " \"port\":3344,\n" + - " \"netClusterNodes\":[ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + - " }\n" + - "}"); - - put("node1", "{\n" + - " \"node\": {\n" + - " \"metastorageNodes\":[ \"node0\" ]\n" + - " },\n" + - " \"network\": {\n" + - " \"port\":3345,\n" + - " \"netClusterNodes\":[ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + - " }\n" + - "}"); - - put("node2", "{\n" + - " \"node\": {\n" + - " \"metastorageNodes\":[ \"node0\" ]\n" + - " },\n" + - " \"network\": {\n" + - " \"port\":3346,\n" + - " \"netClusterNodes\":[ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + - " }\n" + - "}"); - }}; + private final Map nodesBootstrapCfg = new LinkedHashMap<>(); /** */ private final List startedNodes = new ArrayList<>(); @@ -78,6 +55,47 @@ class ITIgnitionTest { @WorkDirectory private Path workDir; + /** */ + @BeforeEach + void setUp(TestInfo testInfo) { + String node0Name = testNodeName(testInfo, PORTS[0]); + String node1Name = testNodeName(testInfo, PORTS[1]); + String node2Name = testNodeName(testInfo, PORTS[2]); + + nodesBootstrapCfg.put( + node0Name, + "{\n" + + " node.metastorageNodes: [ \"" + node0Name + "\" ],\n" + + " network: {\n" + + " port: " + PORTS[0] + "\n" + + " netClusterNodes: [ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + + " }\n" + + "}" + ); + + nodesBootstrapCfg.put( + node1Name, + "{\n" + + " node.metastorageNodes: [ \"" + node0Name + "\" ],\n" + + " network: {\n" + + " port: " + PORTS[1] + "\n" + + " netClusterNodes: [ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + + " }\n" + + "}" + ); + + nodesBootstrapCfg.put( + node2Name, + "{\n" + + " node.metastorageNodes: [ \"" + node0Name + "\" ],\n" + + " network: {\n" + + " port: " + PORTS[2] + "\n" + + " netClusterNodes: [ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + + " }\n" + + "}" + ); + } + /** */ @AfterEach void tearDown() throws Exception { @@ -102,8 +120,8 @@ void testNodesStartWithBootstrapConfiguration() { * Check that Ignition.start() with bootstrap configuration returns Ignite instance. */ @Test - void testNodeStartWithoutBootstrapConfiguration() throws Exception { - startedNodes.add(IgnitionManager.start("node0", null, workDir)); + void testNodeStartWithoutBootstrapConfiguration(TestInfo testInfo) { + startedNodes.add(IgnitionManager.start(testNodeName(testInfo, 47500), null, workDir)); Assertions.assertNotNull(startedNodes.get(0)); } diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITTableCreationTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITTableCreationTest.java index b8f9c0036c3..03f5a813e94 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITTableCreationTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITTableCreationTest.java @@ -33,10 +33,13 @@ import org.apache.ignite.table.Tuple; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.extension.ExtendWith; +import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; @@ -46,114 +49,11 @@ @Disabled("https://issues.apache.org/jira/browse/IGNITE-14578") @ExtendWith(WorkDirectoryExtension.class) class ITTableCreationTest { + /** Network ports of the test nodes. */ + private static final int[] PORTS = { 3344, 3345, 3346 }; + /** Nodes bootstrap configuration with preconfigured tables. */ - private final LinkedHashMap nodesBootstrapCfg = new LinkedHashMap<>() {{ - put("node0", "{\n" + - " \"node\": {\n" + - " \"name\":node0,\n" + - " \"metastorageNodes\":[ \"node0\", \"node1\" ]\n" + - " },\n" + - " \"network\": {\n" + - " \"port\":3344,\n" + - " \"netClusterNodes\":[ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + - " },\n" + - " \"table\": {\n" + - " \"tables\": {\n" + - " \"tbl1\": {\n" + - " \"partitions\":10,\n" + - " \"replicas\":2,\n" + - " \"columns\": { \n" + - " \"key\": {\n" + - " \"type\": {" + - " \"type\":UUID\n" + - " },\n" + - " \"nullable\":false\n" + - " },\n" + - " \"affKey\": {\n" + - " \"type\": {" + - " \"type\":INT64\n" + - " },\n" + - " \"nullable\":false\n" + - " },\n" + - " \"valString\": {\n" + - " \"type\": {" + - " \"type\":String\n" + - " },\n" + - " \"nullable\":false\n" + - " },\n" + - " \"valInt\": {\n" + - " \"type\": {" + - " \"type\":INT32\n" + - " },\n" + - " \"nullable\":false\n" + - " },\n" + - " \"valNullable\": {\n" + - " \"type\": {" + - " \"type\":String\n" + - " },\n" + - " \"nullable\":true\n" + - " }\n" + - " },\n" + /* Columns. */ - " \"indices\": {\n" + - " \"PK\": {\n" + - " \"type\":PRIMARY,\n" + - " \"columns\": {\n" + - " \"key\": {\n" + - " \"asc\":true\n" + - " },\n" + - " \"affKey\": {}\n" + - " },\n" + /* Columns. */ - " \"affinityColumns\":[ \"affKey\" ]\n" + - " }\n" + - " }\n" + /* Indices. */ - " },\n" + /* Table. */ - "\n" + - " \"tbl2\": {\n" + // Table minimal configuration. - " \"columns\": { \n" + - " \"key\": {\n" + - " \"type\": {" + - " \"type\":INT64\n" + - " },\n" + - " },\n" + - " \"val\": {\n" + - " \"type\": {" + - " \"type\":INT64\n" + - " },\n" + - " }\n" + - " },\n" + /* Columns. */ - " \"indices\": {\n" + - " \"PK\": {\n" + - " \"type\":PRIMARY,\n" + - " \"columns\": {\n" + - " \"key\": {}\n" + - " },\n" + /* Columns. */ - " }\n" + - " }\n" + /* Indices. */ - " }\n" + /* Table. */ - " }\n" + /* Tables. */ - " }\n" + /* Root. */ - "}"); - - put("node1", "{\n" + - " \"node\": {\n" + - " \"metastorageNodes\":[ \"node0\", \"node1\" ]\n" + - " },\n" + - " \"network\": {\n" + - " \"port\":3345,\n" + - " \"netClusterNodes\":[ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + - " }\n" + - "}"); - - put("node2", "{\n" + - " \"node\": {\n" + - " \"metastorageNodes\":[ \"node0\", \"node1\" ]\n" + - " },\n" + - " \"network\": {\n" + - " \"port\":3346,\n" + - " \"netClusterNodes\":[ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + - " }\n" + - "}"); - }}; + private final LinkedHashMap nodesBootstrapCfg = new LinkedHashMap<>(); /** */ private final List clusterNodes = new ArrayList<>(); @@ -162,6 +62,122 @@ class ITTableCreationTest { @WorkDirectory private Path workDir; + /** */ + @BeforeEach + void setUp(TestInfo testInfo) { + String node0Name = testNodeName(testInfo, PORTS[0]); + String node1Name = testNodeName(testInfo, PORTS[1]); + String node2Name = testNodeName(testInfo, PORTS[2]); + + nodesBootstrapCfg.put( + node0Name, + "{\n" + + " node.metastorageNodes: [ \"" + node0Name + "\", \"" + node1Name + "\" ],\n" + + " network: {\n" + + " port: " + PORTS[0] + "\n" + + " netClusterNodes: [ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + + " },\n" + + " \"table\": {\n" + + " \"tables\": {\n" + + " \"tbl1\": {\n" + + " \"partitions\":10,\n" + + " \"replicas\":2,\n" + + " \"columns\": { \n" + + " \"key\": {\n" + + " \"type\": {" + + " \"type\":UUID\n" + + " },\n" + + " \"nullable\":false\n" + + " },\n" + + " \"affKey\": {\n" + + " \"type\": {" + + " \"type\":INT64\n" + + " },\n" + + " \"nullable\":false\n" + + " },\n" + + " \"valString\": {\n" + + " \"type\": {" + + " \"type\":String\n" + + " },\n" + + " \"nullable\":false\n" + + " },\n" + + " \"valInt\": {\n" + + " \"type\": {" + + " \"type\":INT32\n" + + " },\n" + + " \"nullable\":false\n" + + " },\n" + + " \"valNullable\": {\n" + + " \"type\": {" + + " \"type\":String\n" + + " },\n" + + " \"nullable\":true\n" + + " }\n" + + " },\n" + /* Columns. */ + " \"indices\": {\n" + + " \"PK\": {\n" + + " \"type\":PRIMARY,\n" + + " \"columns\": {\n" + + " \"key\": {\n" + + " \"asc\":true\n" + + " },\n" + + " \"affKey\": {}\n" + + " },\n" + /* Columns. */ + " \"affinityColumns\":[ \"affKey\" ]\n" + + " }\n" + + " }\n" + /* Indices. */ + " },\n" + /* Table. */ + "\n" + + " \"tbl2\": {\n" + // Table minimal configuration. + " \"columns\": { \n" + + " \"key\": {\n" + + " \"type\": {" + + " \"type\":INT64\n" + + " },\n" + + " },\n" + + " \"val\": {\n" + + " \"type\": {" + + " \"type\":INT64\n" + + " },\n" + + " }\n" + + " },\n" + /* Columns. */ + " \"indices\": {\n" + + " \"PK\": {\n" + + " \"type\":PRIMARY,\n" + + " \"columns\": {\n" + + " \"key\": {}\n" + + " },\n" + /* Columns. */ + " }\n" + + " }\n" + /* Indices. */ + " }\n" + /* Table. */ + " }\n" + /* Tables. */ + " }\n" + /* Root. */ + "}" + ); + + nodesBootstrapCfg.put( + node1Name, + "{\n" + + " node.metastorageNodes: [ \"" + node0Name + "\", \"" + node1Name + "\" ],\n" + + " network: {\n" + + " port: " + PORTS[1] + "\n" + + " netClusterNodes: [ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + + " }\n" + + "}" + ); + + nodesBootstrapCfg.put( + node2Name, + "{\n" + + " node.metastorageNodes: [ \"" + node0Name + "\", \"" + node1Name + "\" ],\n" + + " network: {\n" + + " port: " + PORTS[2] + "\n" + + " netClusterNodes: [ \"localhost:3344\", \"localhost:3345\", \"localhost:3346\" ]\n" + + " }\n" + + "}" + ); + } + /** */ @AfterEach void tearDown() throws Exception { diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITThinClientConnectionTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITThinClientConnectionTest.java index 08e0403d8e8..ef4d7c22011 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITThinClientConnectionTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITThinClientConnectionTest.java @@ -38,9 +38,12 @@ import org.apache.ignite.table.Table; import org.apache.ignite.table.Tuple; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.extension.ExtendWith; +import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -56,31 +59,40 @@ public class ITThinClientConnectionTest extends IgniteAbstractTest { private static final String TABLE_NAME = "tbl1"; /** Nodes bootstrap configuration. */ - private static final Map nodesBootstrapCfg = new LinkedHashMap<>() {{ - put("node0", "{\n" + - " \"node\": {\n" + - " \"metastorageNodes\":[ \"node0\" ]\n" + - " },\n" + - " \"network\": {\n" + - " \"port\":3344,\n" + - " \"netClusterNodes\":[ \"localhost:3344\", \"localhost:3345\" ]\n" + - " }\n" + - "}"); - - put("node1", "{\n" + - " \"node\": {\n" + - " \"metastorageNodes\":[ \"node0\" ]\n" + - " },\n" + - " \"network\": {\n" + - " \"port\":3345,\n" + - " \"netClusterNodes\":[ \"localhost:3344\", \"localhost:3345\" ]\n" + - " }\n" + - "}"); - }}; + private final Map nodesBootstrapCfg = new LinkedHashMap<>(); /** */ private final List startedNodes = new ArrayList<>(); + /** */ + @BeforeEach + void setup(TestInfo testInfo) { + String node0Name = testNodeName(testInfo, 3344); + String node1Name = testNodeName(testInfo, 3345); + + nodesBootstrapCfg.put( + node0Name, + "{\n" + + " node.metastorageNodes: [ \"" + node0Name + "\" ],\n" + + " network: {\n" + + " port: " + 3344 + "\n" + + " netClusterNodes: [ \"localhost:3344\", \"localhost:3345\" ]\n" + + " }\n" + + "}" + ); + + nodesBootstrapCfg.put( + node1Name, + "{\n" + + " node.metastorageNodes: [ \"" + node0Name + "\" ],\n" + + " network: {\n" + + " port: " + 3345 + "\n" + + " netClusterNodes: [ \"localhost:3344\", \"localhost:3345\" ]\n" + + " }\n" + + "}" + ); + } + /** */ @AfterEach void tearDown() throws Exception { @@ -111,8 +123,7 @@ void testThinClientConnectsToServerNodesAndExecutesBasicTableOperations() throws ); var addrs = new String[]{"127.0.0.1:" + - ((InetSocketAddress) ((IgniteImpl)startedNodes.stream().filter(node -> "node0".equals(node.name())). - findAny().get()).clientHandlerModule().localAddress()).getPort()}; + ((InetSocketAddress) ((IgniteImpl)startedNodes.get(0)).clientHandlerModule().localAddress()).getPort()}; for (var addr : addrs) { try (var client = IgniteClient.builder().addresses(addr).build()) { diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/PlatformTestNodeRunner.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/PlatformTestNodeRunner.java index 8c76ec1b230..b9416e1f697 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/PlatformTestNodeRunner.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/PlatformTestNodeRunner.java @@ -39,6 +39,9 @@ * Starts nodes, populates tables and data for tests. */ public class PlatformTestNodeRunner { + /** Test node name. */ + private static final String NODE_NAME = PlatformTestNodeRunner.class.getCanonicalName(); + /** */ private static final String SCHEMA_NAME = "PUB"; @@ -47,9 +50,9 @@ public class PlatformTestNodeRunner { /** Nodes bootstrap configuration. */ private static final Map nodesBootstrapCfg = new LinkedHashMap<>() {{ - put("node0", "{\n" + + put(NODE_NAME, "{\n" + " \"node\": {\n" + - " \"metastorageNodes\":[ \"node0\" ]\n" + + " \"metastorageNodes\":[ \"" + NODE_NAME + "\" ]\n" + " },\n" + " \"clientConnector\":{\"port\": 10942,\"portRange\":10}," + " \"network\": {\n" + diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/AbstractJdbcSelfTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/AbstractJdbcSelfTest.java index c74bebf8481..f00129bc95a 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/AbstractJdbcSelfTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/AbstractJdbcSelfTest.java @@ -21,32 +21,23 @@ import java.sql.SQLException; import java.sql.SQLFeatureNotSupportedException; import java.util.ArrayList; -import java.util.LinkedHashMap; import java.util.List; -import java.util.Map; import org.apache.ignite.app.Ignite; import org.apache.ignite.app.IgnitionManager; import org.apache.ignite.jdbc.IgniteJdbcDriver; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.function.Executable; import org.junit.jupiter.api.io.TempDir; +import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName; import static org.junit.jupiter.api.Assertions.assertThrows; public class AbstractJdbcSelfTest { /** URL. */ protected static final String URL = "jdbc:ignite:thin://127.0.1.1:10800"; - /** Nodes bootstrap configuration. */ - private static final Map nodesBootstrapCfg = new LinkedHashMap<>() {{ - put("node0", "{\n" + - " \"node\": {\n" + - " \"metastorageNodes\":[ \"node0\" ]\n" + - " }\n" + - "}"); - }}; - /** Cluster nodes. */ protected static final List clusterNodes = new ArrayList<>(); @@ -56,12 +47,14 @@ public class AbstractJdbcSelfTest { * @param temp Temporal directory. */ @BeforeAll - public static void beforeAll(@TempDir Path temp) { + public static void beforeAll(@TempDir Path temp, TestInfo testInfo) { IgniteJdbcDriver.register(); - nodesBootstrapCfg.forEach((nodeName, configStr) -> - clusterNodes.add(IgnitionManager.start(nodeName, configStr, temp.resolve(nodeName))) - ); + String nodeName = testNodeName(testInfo, 47500); + + String configStr = "node.metastorageNodes: [ \"" + nodeName + "\" ]"; + + clusterNodes.add(IgnitionManager.start(nodeName, configStr, temp.resolve(nodeName))); } /** diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITDistributedTableTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITDistributedTableTest.java index ae32af4a978..ed2068312bc 100644 --- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITDistributedTableTest.java +++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITDistributedTableTest.java @@ -72,6 +72,7 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.extension.ExtendWith; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -126,11 +127,11 @@ public class ITDistributedTableTest { * Start all cluster nodes before each test. */ @BeforeEach - public void beforeTest() { + public void beforeTest(TestInfo testInfo) { var nodeFinder = new LocalPortRangeNodeFinder(NODE_PORT_BASE, NODE_PORT_BASE + NODES); nodeFinder.findNodes().stream() - .map(addr -> startClient(addr.toString(), addr.port(), nodeFinder)) + .map(addr -> startClient(testInfo, addr.port(), nodeFinder)) .forEach(cluster::add); for (ClusterService node : cluster) @@ -138,7 +139,7 @@ public void beforeTest() { LOG.info("Cluster started."); - client = startClient("client", NODE_PORT_BASE + NODES, nodeFinder); + client = startClient(testInfo, NODE_PORT_BASE + NODES, nodeFinder); assertTrue(waitForTopology(client, NODES + 1, 1000)); @@ -491,14 +492,14 @@ public void partitionedTableKeyValueView(KeyValueView view, int ke } /** - * @param name Node name. + * @param testInfo Test info. * @param port Local port. * @param nodeFinder Node finder. * @return The client cluster view. */ - private static ClusterService startClient(String name, int port, NodeFinder nodeFinder) { + private static ClusterService startClient(TestInfo testInfo, int port, NodeFinder nodeFinder) { var network = ClusterServiceTestUtils.clusterService( - name, + testInfo, port, nodeFinder, SERIALIZATION_REGISTRY,