You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by pt...@apache.org on 2021/11/22 19:15:41 UTC

[ignite-3] branch ignite-15307 updated: Remove ClusterServiceFactory interface, fix tests

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

ptupitsyn pushed a commit to branch ignite-15307
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/ignite-15307 by this push:
     new 47e7c82  Remove ClusterServiceFactory interface, fix tests
47e7c82 is described below

commit 47e7c821b2bb324030af62e133699d80f83aae15
Author: Pavel Tupitsyn <pt...@apache.org>
AuthorDate: Mon Nov 22 22:15:31 2021 +0300

    Remove ClusterServiceFactory interface, fix tests
---
 .../ignite/client/handler/ItClientHandlerTest.java | 20 +++++++++---
 .../client/ItMetaStorageServiceTest.java           |  3 +-
 .../ignite/network/ClusterServiceFactory.java      | 37 ----------------------
 .../network/scalecube/ItNodeRestartsTest.java      |  3 +-
 .../scalecube/ItScaleCubeNetworkMessagingTest.java |  3 +-
 .../ignite/utils/ClusterServiceTestUtils.java      | 25 +++++++++++----
 .../scalecube/ScaleCubeClusterServiceFactory.java  | 17 ++--------
 .../apache/ignite/internal/raft/ItLozaTest.java    |  3 +-
 .../service/ItAbstractListenerSnapshotTest.java    |  3 +-
 .../ignite/raft/server/RaftServerAbstractTest.java |  3 +-
 .../ignite/distributed/ItDistributedTableTest.java |  3 +-
 .../distributed/ItInternalTableScanTest.java       |  3 +-
 12 files changed, 44 insertions(+), 79 deletions(-)

diff --git a/modules/client-handler/src/integrationTest/java/org/apache/ignite/client/handler/ItClientHandlerTest.java b/modules/client-handler/src/integrationTest/java/org/apache/ignite/client/handler/ItClientHandlerTest.java
index bc60409..e6b32f2 100644
--- a/modules/client-handler/src/integrationTest/java/org/apache/ignite/client/handler/ItClientHandlerTest.java
+++ b/modules/client-handler/src/integrationTest/java/org/apache/ignite/client/handler/ItClientHandlerTest.java
@@ -31,13 +31,16 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import org.apache.ignite.configuration.schemas.clientconnector.ClientConnectorConfiguration;
+import org.apache.ignite.configuration.schemas.network.NetworkConfiguration;
 import org.apache.ignite.internal.configuration.ConfigurationManager;
 import org.apache.ignite.internal.configuration.storage.TestConfigurationStorage;
 import org.apache.ignite.internal.processors.query.calcite.QueryProcessor;
+import org.apache.ignite.network.NettyBootstrapFactory;
 import org.apache.ignite.table.manager.IgniteTables;
 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.msgpack.core.MessagePack;
 
 /**
@@ -51,11 +54,13 @@ public class ItClientHandlerTest {
     
     private ConfigurationManager configurationManager;
     
+    private NettyBootstrapFactory bootstrapFactory;
+    
     private int serverPort;
     
     @BeforeEach
-    public void setUp() {
-        serverModule = startServer();
+    public void setUp(TestInfo testInfo) {
+        serverModule = startServer(testInfo);
         serverPort = ((InetSocketAddress) Objects.requireNonNull(serverModule.localAddress())).getPort();
     }
     
@@ -63,6 +68,7 @@ public class ItClientHandlerTest {
     public void tearDown() throws Exception {
         serverModule.stop();
         configurationManager.stop();
+        bootstrapFactory.stop();
     }
     
     @Test
@@ -176,9 +182,9 @@ public class ItClientHandlerTest {
         }
     }
     
-    private ClientHandlerModule startServer() {
+    private ClientHandlerModule startServer(TestInfo testInfo) {
         configurationManager = new ConfigurationManager(
-                List.of(ClientConnectorConfiguration.KEY),
+                List.of(ClientConnectorConfiguration.KEY, NetworkConfiguration.KEY),
                 Map.of(),
                 new TestConfigurationStorage(LOCAL),
                 List.of(),
@@ -193,7 +199,11 @@ public class ItClientHandlerTest {
                 local -> local.changePort(10800).changePortRange(10)
         ).join();
         
-        var module = new ClientHandlerModule(mock(QueryProcessor.class), mock(IgniteTables.class), registry);
+        bootstrapFactory = new NettyBootstrapFactory(registry.getConfiguration(NetworkConfiguration.KEY), testInfo.getDisplayName());
+        
+        bootstrapFactory.start();
+        
+        var module = new ClientHandlerModule(mock(QueryProcessor.class), mock(IgniteTables.class), registry, bootstrapFactory);
         
         module.start();
         
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 ff667e4..4ab19ec 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
@@ -64,7 +64,6 @@ import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.lang.IgniteLogger;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.network.ClusterService;
-import org.apache.ignite.network.ClusterServiceFactory;
 import org.apache.ignite.network.MessageSerializationRegistryImpl;
 import org.apache.ignite.network.NetworkAddress;
 import org.apache.ignite.network.StaticNodeFinder;
@@ -107,7 +106,7 @@ public class ItMetaStorageServiceTest {
     private static final RaftMessagesFactory FACTORY = new RaftMessagesFactory();
 
     /** Network factory. */
-    private static final ClusterServiceFactory NETWORK_FACTORY = new TestScaleCubeClusterServiceFactory();
+    private static final TestScaleCubeClusterServiceFactory NETWORK_FACTORY = new TestScaleCubeClusterServiceFactory();
 
     private static final MessageSerializationRegistry SERIALIZATION_REGISTRY = new MessageSerializationRegistryImpl();
 
diff --git a/modules/network-api/src/main/java/org/apache/ignite/network/ClusterServiceFactory.java b/modules/network-api/src/main/java/org/apache/ignite/network/ClusterServiceFactory.java
deleted file mode 100644
index 46dfb9c..0000000
--- a/modules/network-api/src/main/java/org/apache/ignite/network/ClusterServiceFactory.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.network;
-
-import org.apache.ignite.configuration.schemas.network.NetworkConfiguration;
-
-/**
- * Cluster service factory.
- */
-public interface ClusterServiceFactory {
-    /**
-     * Creates a new {@link ClusterService} using the provided context. The created network will not be in the "started" state.
-     *
-     * @param context              Cluster context.
-     * @param networkConfiguration Network configuration.
-     * @return New cluster service.
-     */
-    ClusterService createClusterService(
-            ClusterLocalConfiguration context,
-            NetworkConfiguration networkConfiguration
-    );
-}
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 cd66f86..c149a09 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
@@ -25,7 +25,6 @@ import java.util.List;
 import java.util.stream.Collectors;
 import org.apache.ignite.lang.IgniteLogger;
 import org.apache.ignite.network.ClusterService;
-import org.apache.ignite.network.ClusterServiceFactory;
 import org.apache.ignite.network.NetworkAddress;
 import org.apache.ignite.network.NodeFinder;
 import org.apache.ignite.network.StaticNodeFinder;
@@ -47,7 +46,7 @@ class ItNodeRestartsTest {
     private final MessageSerializationRegistry serializationRegistry = new TestMessageSerializationRegistryImpl();
 
     /** Network factory. */
-    private final ClusterServiceFactory networkFactory = new TestScaleCubeClusterServiceFactory();
+    private final TestScaleCubeClusterServiceFactory networkFactory = new TestScaleCubeClusterServiceFactory();
 
     /** Created {@link ClusterService}s. Needed for resource management. */
     private List<ClusterService> services;
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 4f1450c..500c43d 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
@@ -46,7 +46,6 @@ import org.apache.ignite.internal.network.NetworkMessageTypes;
 import org.apache.ignite.lang.NodeStoppingException;
 import org.apache.ignite.network.ClusterNode;
 import org.apache.ignite.network.ClusterService;
-import org.apache.ignite.network.ClusterServiceFactory;
 import org.apache.ignite.network.NetworkAddress;
 import org.apache.ignite.network.NetworkMessage;
 import org.apache.ignite.network.NodeFinder;
@@ -421,7 +420,7 @@ class ItScaleCubeNetworkMessagingTest {
      */
     private static final class Cluster {
         /** Network factory. */
-        private final ClusterServiceFactory networkFactory = new TestScaleCubeClusterServiceFactory();
+        private final TestScaleCubeClusterServiceFactory networkFactory = new TestScaleCubeClusterServiceFactory();
 
         /** Serialization registry. */
         private final MessageSerializationRegistry serializationRegistry = new TestMessageSerializationRegistryImpl();
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 10e6f01..96b46c6 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,12 +31,13 @@ import org.apache.ignite.internal.configuration.ConfigurationManager;
 import org.apache.ignite.internal.configuration.storage.TestConfigurationStorage;
 import org.apache.ignite.network.ClusterLocalConfiguration;
 import org.apache.ignite.network.ClusterService;
-import org.apache.ignite.network.ClusterServiceFactory;
 import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NettyBootstrapFactory;
 import org.apache.ignite.network.NetworkAddress;
 import org.apache.ignite.network.NodeFinder;
 import org.apache.ignite.network.StaticNodeFinder;
 import org.apache.ignite.network.TopologyService;
+import org.apache.ignite.network.scalecube.TestScaleCubeClusterServiceFactory;
 import org.apache.ignite.network.serialization.MessageSerializationRegistry;
 import org.junit.jupiter.api.TestInfo;
 
@@ -60,10 +61,10 @@ public class ClusterServiceTestUtils {
             int port,
             NodeFinder nodeFinder,
             MessageSerializationRegistry msgSerializationRegistry,
-            ClusterServiceFactory clusterSvcFactory
+            TestScaleCubeClusterServiceFactory clusterSvcFactory
     ) {
         var ctx = new ClusterLocalConfiguration(testNodeName(testInfo, port), msgSerializationRegistry);
-        
+    
         ConfigurationManager nodeConfigurationMgr = new ConfigurationManager(
                 Collections.singleton(NetworkConfiguration.KEY),
                 Map.of(),
@@ -71,10 +72,15 @@ public class ClusterServiceTestUtils {
                 List.of(),
                 List.of()
         );
+    
+        NetworkConfiguration configuration = nodeConfigurationMgr.configurationRegistry().getConfiguration(NetworkConfiguration.KEY);
+        
+        var bootstrapFactory = new NettyBootstrapFactory(configuration, ctx.getName());
         
         var clusterSvc = clusterSvcFactory.createClusterService(
                 ctx,
-                nodeConfigurationMgr.configurationRegistry().getConfiguration(NetworkConfiguration.KEY));
+                configuration,
+                bootstrapFactory);
         
         assert nodeFinder instanceof StaticNodeFinder : "Only StaticNodeFinder is supported at the moment";
         
@@ -116,14 +122,21 @@ public class ClusterServiceTestUtils {
                                         )
                                 )
                 ).join();
+    
+                bootstrapFactory.start();
                 
                 clusterSvc.start();
             }
             
             @Override
             public void stop() {
-                clusterSvc.stop();
-                nodeConfigurationMgr.stop();
+                try {
+                    clusterSvc.stop();
+                    bootstrapFactory.stop();
+                    nodeConfigurationMgr.stop();
+                } catch (Exception e) {
+                    throw new Error(e);
+                }
             }
         };
     }
diff --git a/modules/network/src/main/java/org/apache/ignite/network/scalecube/ScaleCubeClusterServiceFactory.java b/modules/network/src/main/java/org/apache/ignite/network/scalecube/ScaleCubeClusterServiceFactory.java
index 1880ec0..4ba4a3f 100644
--- a/modules/network/src/main/java/org/apache/ignite/network/scalecube/ScaleCubeClusterServiceFactory.java
+++ b/modules/network/src/main/java/org/apache/ignite/network/scalecube/ScaleCubeClusterServiceFactory.java
@@ -37,7 +37,6 @@ import org.apache.ignite.internal.network.recovery.RecoveryServerHandshakeManage
 import org.apache.ignite.network.AbstractClusterService;
 import org.apache.ignite.network.ClusterLocalConfiguration;
 import org.apache.ignite.network.ClusterService;
-import org.apache.ignite.network.ClusterServiceFactory;
 import org.apache.ignite.network.NettyBootstrapFactory;
 import org.apache.ignite.network.NetworkAddress;
 import org.apache.ignite.network.NodeFinder;
@@ -45,21 +44,9 @@ import org.apache.ignite.network.NodeFinderFactory;
 import org.apache.ignite.network.serialization.MessageSerializationRegistry;
 
 /**
- * {@link ClusterServiceFactory} implementation that uses ScaleCube for messaging and topology services.
+ * Cluster service factory that uses ScaleCube for messaging and topology services.
  */
-public class ScaleCubeClusterServiceFactory implements ClusterServiceFactory {
-    /** {@inheritDoc} */
-    @Override
-    public ClusterService createClusterService(ClusterLocalConfiguration context, NetworkConfiguration networkConfiguration) {
-        final String consistentId = context.getName();
-    
-        // TOOD: How do we stop it? This method is only used from tests, we should remove it?
-        var nettyBootstrapFactory = new NettyBootstrapFactory(networkConfiguration, consistentId);
-        nettyBootstrapFactory.start();
-        
-        return createClusterService(context, networkConfiguration, nettyBootstrapFactory);
-    }
-    
+public class ScaleCubeClusterServiceFactory {
     /**
      * Creates a new {@link ClusterService} using the provided context. The created network will not be in the "started" state.
      *
diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/internal/raft/ItLozaTest.java b/modules/raft/src/integrationTest/java/org/apache/ignite/internal/raft/ItLozaTest.java
index 1d056b6..4f32e71 100644
--- a/modules/raft/src/integrationTest/java/org/apache/ignite/internal/raft/ItLozaTest.java
+++ b/modules/raft/src/integrationTest/java/org/apache/ignite/internal/raft/ItLozaTest.java
@@ -37,7 +37,6 @@ import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.apache.ignite.network.ClusterNode;
 import org.apache.ignite.network.ClusterService;
-import org.apache.ignite.network.ClusterServiceFactory;
 import org.apache.ignite.network.MessageSerializationRegistryImpl;
 import org.apache.ignite.network.MessagingService;
 import org.apache.ignite.network.NetworkAddress;
@@ -58,7 +57,7 @@ import org.junit.jupiter.api.extension.ExtendWith;
 @ExtendWith(WorkDirectoryExtension.class)
 public class ItLozaTest {
     /** Network factory. */
-    private static final ClusterServiceFactory NETWORK_FACTORY = new TestScaleCubeClusterServiceFactory();
+    private static final TestScaleCubeClusterServiceFactory NETWORK_FACTORY = new TestScaleCubeClusterServiceFactory();
 
     /** Server port offset. */
     private static final int PORT = 20010;
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 750c3d1..846b351 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
@@ -38,7 +38,6 @@ import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.apache.ignite.internal.thread.NamedThreadFactory;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.network.ClusterService;
-import org.apache.ignite.network.ClusterServiceFactory;
 import org.apache.ignite.network.MessageSerializationRegistryImpl;
 import org.apache.ignite.network.NetworkAddress;
 import org.apache.ignite.network.StaticNodeFinder;
@@ -80,7 +79,7 @@ public abstract class ItAbstractListenerSnapshotTest<T extends RaftGroupListener
     private static final RaftMessagesFactory FACTORY = new RaftMessagesFactory();
 
     /** Network factory. */
-    private static final ClusterServiceFactory NETWORK_FACTORY = new TestScaleCubeClusterServiceFactory();
+    private static final TestScaleCubeClusterServiceFactory NETWORK_FACTORY = new TestScaleCubeClusterServiceFactory();
 
     private static final MessageSerializationRegistry SERIALIZATION_REGISTRY = new MessageSerializationRegistryImpl();
 
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 77a4978..4b28d23 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
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.List;
 import org.apache.ignite.lang.IgniteLogger;
 import org.apache.ignite.network.ClusterService;
-import org.apache.ignite.network.ClusterServiceFactory;
 import org.apache.ignite.network.MessageSerializationRegistryImpl;
 import org.apache.ignite.network.NetworkAddress;
 import org.apache.ignite.network.StaticNodeFinder;
@@ -42,7 +41,7 @@ abstract class RaftServerAbstractTest {
     protected static final RaftMessagesFactory FACTORY = new RaftMessagesFactory();
 
     /** Network factory. */
-    protected static final ClusterServiceFactory NETWORK_FACTORY = new TestScaleCubeClusterServiceFactory();
+    protected static final TestScaleCubeClusterServiceFactory NETWORK_FACTORY = new TestScaleCubeClusterServiceFactory();
 
     /**
      * Server port offset.
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 8f7f04b..0d23602 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
@@ -63,7 +63,6 @@ import org.apache.ignite.lang.IgniteLogger;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.network.ClusterNode;
 import org.apache.ignite.network.ClusterService;
-import org.apache.ignite.network.ClusterServiceFactory;
 import org.apache.ignite.network.MessageSerializationRegistryImpl;
 import org.apache.ignite.network.NetworkAddress;
 import org.apache.ignite.network.NodeFinder;
@@ -108,7 +107,7 @@ public class ItDistributedTableTest {
     private static final RaftMessagesFactory FACTORY = new RaftMessagesFactory();
 
     /** Network factory. */
-    private static final ClusterServiceFactory NETWORK_FACTORY = new TestScaleCubeClusterServiceFactory();
+    private static final TestScaleCubeClusterServiceFactory NETWORK_FACTORY = new TestScaleCubeClusterServiceFactory();
 
     private static final MessageSerializationRegistry SERIALIZATION_REGISTRY = new MessageSerializationRegistryImpl();
 
diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItInternalTableScanTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItInternalTableScanTest.java
index ff44e2c..9fb7fe2 100644
--- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItInternalTableScanTest.java
+++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItInternalTableScanTest.java
@@ -64,7 +64,6 @@ import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.lang.IgniteUuidGenerator;
 import org.apache.ignite.network.ClusterService;
-import org.apache.ignite.network.ClusterServiceFactory;
 import org.apache.ignite.network.MessageSerializationRegistryImpl;
 import org.apache.ignite.network.NetworkAddress;
 import org.apache.ignite.network.StaticNodeFinder;
@@ -89,7 +88,7 @@ import org.mockito.junit.jupiter.MockitoExtension;
  */
 @ExtendWith(MockitoExtension.class)
 public class ItInternalTableScanTest {
-    private static final ClusterServiceFactory NETWORK_FACTORY = new TestScaleCubeClusterServiceFactory();
+    private static final TestScaleCubeClusterServiceFactory NETWORK_FACTORY = new TestScaleCubeClusterServiceFactory();
 
     private static final MessageSerializationRegistry SERIALIZATION_REGISTRY = new MessageSerializationRegistryImpl();