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

[ignite-3] branch main updated: IGNITE-15841 Starting a node should be failed if the user configuration is invalid. Fixes #423

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 5826d81  IGNITE-15841 Starting a node should be failed if the user configuration is invalid. Fixes #423
5826d81 is described below

commit 5826d81f86679158844fc6d9b4e7484e45599122
Author: sergeyuttsel <ut...@gmail.com>
AuthorDate: Tue Nov 16 00:05:16 2021 +0300

    IGNITE-15841 Starting a node should be failed if the user configuration is invalid. Fixes #423
    
    Signed-off-by: Slava Koptilin <sl...@gmail.com>
---
 .../ignite/internal/runner/app/ItIgnitionTest.java |  33 ++++-
 .../org/apache/ignite/internal/app/IgniteImpl.java | 139 ++++++++++-----------
 .../apache/ignite/internal/app/IgnitionImpl.java   |  18 ++-
 3 files changed, 105 insertions(+), 85 deletions(-)

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 66695f7..ed4ff20 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
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.runner.app;
 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;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.nio.file.Path;
 import java.util.ArrayList;
@@ -52,8 +53,10 @@ class ItIgnitionTest {
     /** Nodes bootstrap configuration. */
     private final Map<String, String> nodesBootstrapCfg = new LinkedHashMap<>();
 
+    /** Collection of started nodes. */
     private final List<Ignite> startedNodes = new ArrayList<>();
 
+    /** Path to the working directory. */
     @WorkDirectory
     private Path workDir;
 
@@ -65,7 +68,7 @@ class ItIgnitionTest {
         String node0Name = testNodeName(testInfo, PORTS[0]);
         String node1Name = testNodeName(testInfo, PORTS[1]);
         String node2Name = testNodeName(testInfo, PORTS[2]);
-    
+
         nodesBootstrapCfg.put(
                 node0Name,
                 "{\n"
@@ -78,7 +81,7 @@ class ItIgnitionTest {
                         + "  }\n"
                         + "}"
         );
-    
+
         nodesBootstrapCfg.put(
                 node1Name,
                 "{\n"
@@ -91,7 +94,7 @@ class ItIgnitionTest {
                         + "  }\n"
                         + "}"
         );
-    
+
         nodesBootstrapCfg.put(
                 node2Name,
                 "{\n"
@@ -142,7 +145,7 @@ class ItIgnitionTest {
      * Tests scenario when we try to start cluster with single node, but without any node, that hosts metastorage.
      */
     @Test
-    void testErrorWhenStartSingleNodeClusterWithoutMetastorage() throws Exception {
+    void testErrorWhenStartSingleNodeClusterWithoutMetastorage() {
         try {
             startedNodes.add(IgnitionManager.start("other-name", "{\n"
                     + "    \"node\": {\n"
@@ -188,7 +191,7 @@ class ItIgnitionTest {
                     + "      }\n"
                     + "    }\n"
                     + "}", workDir.resolve("node-0"));
-    
+
             ig2 = IgnitionManager.start("other-name", "{\n"
                     + "    \"node\": {\n"
                     + "        \"metastorageNodes\": [\n"
@@ -208,4 +211,24 @@ class ItIgnitionTest {
             IgniteUtils.closeAll(ig2, ig1);
         }
     }
+
+    /**
+     * Tests scenario when we try to start node with invalid configuration.
+     */
+    @Test
+    void testErrorWhenStartNodeWithInvalidConfiguration() {
+        try {
+            startedNodes.add(IgnitionManager.start("invalid-config-name",
+                    "{Invalid-Configuration}",
+                    workDir.resolve("invalid-config-name"))
+            );
+
+            fail();
+        } catch (Throwable t) {
+            assertTrue(IgniteTestUtils.hasCause(t,
+                    IgniteException.class,
+                    "Unable to parse user-specific configuration."
+            ));
+        }
+    }
 }
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
index db647ae..8aac3e6 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
@@ -79,61 +79,61 @@ import org.jetbrains.annotations.Nullable;
 public class IgniteImpl implements Ignite {
     /** The logger. */
     private static final IgniteLogger LOG = IgniteLogger.forClass(IgniteImpl.class);
-    
+
     /**
      * Path to the persistent storage used by the {@link VaultService} component.
      */
     private static final Path VAULT_DB_PATH = Paths.get("vault");
-    
+
     /**
      * Path to the persistent storage used by the {@link MetaStorageManager} component.
      */
     private static final Path METASTORAGE_DB_PATH = Paths.get("metastorage");
-    
+
     /**
      * Path for the partitions persistent storage.
      */
     private static final Path PARTITIONS_STORE_PATH = Paths.get("db");
-    
+
     /** Ignite node name. */
     private final String name;
-    
+
     /** Vault manager. */
     private final VaultManager vaultMgr;
-    
+
     /** Configuration manager that handles node (local) configuration. */
     private final ConfigurationManager nodeCfgMgr;
-    
+
     /** Cluster service (cluster network manager). */
     private final ClusterService clusterSvc;
-    
+
     /** Raft manager. */
     private final Loza raftMgr;
-    
+
     /** Meta storage manager. */
     private final MetaStorageManager metaStorageMgr;
-    
+
     /** Configuration manager that handles cluster (distributed) configuration. */
     private final ConfigurationManager clusterCfgMgr;
-    
+
     /** Baseline manager. */
     private final BaselineManager baselineMgr;
-    
+
     /** Distributed table manager. */
     private final TableManager distributedTblMgr;
-    
+
     /** Query engine. */
     private final SqlQueryProcessor qryEngine;
-    
+
     /** Rest module. */
     private final RestModule restModule;
-    
+
     /** Client handler module. */
     private final ClientHandlerModule clientHandlerModule;
-    
+
     /** Node status. Adds ability to stop currently starting node. */
     private final AtomicReference<Status> status = new AtomicReference<>(Status.STARTING);
-    
+
     /**
      * The Constructor.
      *
@@ -145,9 +145,9 @@ public class IgniteImpl implements Ignite {
             Path workDir
     ) {
         this.name = name;
-        
+
         vaultMgr = createVault(workDir);
-        
+
         nodeCfgMgr = new ConfigurationManager(
                 List.of(
                         NetworkConfiguration.KEY,
@@ -160,7 +160,7 @@ public class IgniteImpl implements Ignite {
                 List.of(),
                 List.of()
         );
-        
+
         clusterSvc = new ScaleCubeClusterServiceFactory().createClusterService(
                 new ClusterLocalConfiguration(
                         name,
@@ -168,9 +168,9 @@ public class IgniteImpl implements Ignite {
                 ),
                 nodeCfgMgr.configurationRegistry().getConfiguration(NetworkConfiguration.KEY)
         );
-        
+
         raftMgr = new Loza(clusterSvc, workDir);
-        
+
         metaStorageMgr = new MetaStorageManager(
                 vaultMgr,
                 nodeCfgMgr,
@@ -178,7 +178,7 @@ public class IgniteImpl implements Ignite {
                 raftMgr,
                 new RocksDbKeyValueStorage(workDir.resolve(METASTORAGE_DB_PATH))
         );
-        
+
         clusterCfgMgr = new ConfigurationManager(
                 List.of(
                         ClusterConfiguration.KEY,
@@ -193,13 +193,13 @@ public class IgniteImpl implements Ignite {
                 Collections.singletonList(ExtendedTableConfigurationSchema.class),
                 List.of()
         );
-        
+
         baselineMgr = new BaselineManager(
                 clusterCfgMgr,
                 metaStorageMgr,
                 clusterSvc
         );
-        
+
         distributedTblMgr = new TableManager(
                 clusterCfgMgr.configurationRegistry().getConfiguration(TablesConfiguration.KEY),
                 clusterCfgMgr.configurationRegistry().getConfiguration(DataStorageConfiguration.KEY),
@@ -208,17 +208,17 @@ public class IgniteImpl implements Ignite {
                 clusterSvc.topologyService(),
                 getPartitionsStorePath(workDir)
         );
-        
+
         qryEngine = new SqlQueryProcessor(
                 clusterSvc,
                 distributedTblMgr
         );
-        
+
         restModule = new RestModule(nodeCfgMgr, clusterCfgMgr);
-        
+
         clientHandlerModule = new ClientHandlerModule(qryEngine, distributedTblMgr, nodeCfgMgr.configurationRegistry());
     }
-    
+
     /**
      * Starts ignite node.
      *
@@ -239,7 +239,7 @@ public class IgniteImpl implements Ignite {
      */
     public void start(@Nullable String cfg) {
         List<IgniteComponent> startedComponents = new ArrayList<>();
-        
+
         try {
             // Vault startup.
             doStartComponent(
@@ -247,27 +247,26 @@ public class IgniteImpl implements Ignite {
                     startedComponents,
                     vaultMgr
             );
-            
+
             vaultMgr.putName(name).join();
-            
+
             // Node configuration manager startup.
             doStartComponent(
                     name,
                     startedComponents,
                     nodeCfgMgr);
-            
+
             // Node configuration manager bootstrap.
             if (cfg != null) {
                 try {
                     nodeCfgMgr.bootstrap(cfg);
                 } catch (Exception e) {
-                    LOG.warn("Unable to parse user-specific configuration, default configuration will be used: {}",
-                            e.getMessage());
+                    throw new IgniteException("Unable to parse user-specific configuration.", e);
                 }
             } else {
                 nodeCfgMgr.configurationRegistry().initializeDefaults();
             }
-            
+
             // Start the remaining components.
             List<IgniteComponent> otherComponents = List.of(
                     clusterSvc,
@@ -280,78 +279,78 @@ public class IgniteImpl implements Ignite {
                     restModule,
                     clientHandlerModule
             );
-    
+
             for (IgniteComponent component : otherComponents) {
                 doStartComponent(name, startedComponents, component);
             }
-            
+
             // Deploy all registered watches because all components are ready and have registered their listeners.
             metaStorageMgr.deployWatches();
-    
+
             if (!status.compareAndSet(Status.STARTING, Status.STARTED)) {
                 throw new NodeStoppingException();
             }
         } catch (Exception e) {
             String errMsg = "Unable to start node=[" + name + "].";
-            
+
             LOG.error(errMsg, e);
-            
+
             doStopNode(startedComponents);
-            
+
             throw new IgniteException(errMsg, e);
         }
     }
-    
+
     /**
      * Stops ignite node.
      */
     public void stop() {
         AtomicBoolean explicitStop = new AtomicBoolean();
-        
+
         status.getAndUpdate(status -> {
             if (status == Status.STARTED) {
                 explicitStop.set(true);
             } else {
                 explicitStop.set(false);
             }
-            
+
             return Status.STOPPING;
         });
-        
+
         if (explicitStop.get()) {
             doStopNode(List.of(vaultMgr, nodeCfgMgr, clusterSvc, raftMgr, metaStorageMgr, clusterCfgMgr, baselineMgr,
                     distributedTblMgr, qryEngine, restModule, clientHandlerModule));
         }
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public IgniteTables tables() {
         return distributedTblMgr;
     }
-    
+
     public QueryProcessor queryEngine() {
         return qryEngine;
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public IgniteTransactions transactions() {
         return null;
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public void close() {
         IgnitionManager.stop(name);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public String name() {
         return name;
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public void setBaseline(Set<String> baselineNodes) {
@@ -361,28 +360,28 @@ public class IgniteImpl implements Ignite {
             throw new IgniteException(e);
         }
     }
-    
+
     /**
      * Returns node configuration.
      */
     public ConfigurationRegistry nodeConfiguration() {
         return nodeCfgMgr.configurationRegistry();
     }
-    
+
     /**
      * Returns cluster configuration.
      */
     public ConfigurationRegistry clusterConfiguration() {
         return clusterCfgMgr.configurationRegistry();
     }
-    
+
     /**
      * Returns client handler module.
      */
     public ClientHandlerModule clientHandlerModule() {
         return clientHandlerModule;
     }
-    
+
     /**
      * Checks node status. If it's {@link Status#STOPPING} then prevents further starting and throws NodeStoppingException that will lead to
      * stopping already started components later on, otherwise starts component and add it to started components list.
@@ -402,11 +401,11 @@ public class IgniteImpl implements Ignite {
             throw new NodeStoppingException("Node=[" + nodeName + "] was stopped.");
         } else {
             startedComponents.add(component);
-    
+
             component.start();
         }
     }
-    
+
     /**
      * Calls {@link IgniteComponent#beforeNodeStop()} and then {@link IgniteComponent#stop()} for all components in start-reverse-order.
      * Cleanups node started components map and node status map.
@@ -416,10 +415,10 @@ public class IgniteImpl implements Ignite {
     private void doStopNode(@NotNull List<IgniteComponent> startedComponents) {
         ListIterator<IgniteComponent> beforeStopIter =
                 startedComponents.listIterator(startedComponents.size());
-        
+
         while (beforeStopIter.hasPrevious()) {
             IgniteComponent componentToExecBeforeNodeStop = beforeStopIter.previous();
-            
+
             try {
                 componentToExecBeforeNodeStop.beforeNodeStop();
             } catch (Exception e) {
@@ -427,13 +426,13 @@ public class IgniteImpl implements Ignite {
                         + componentToExecBeforeNodeStop + "] within node=[" + name + ']', e);
             }
         }
-        
+
         ListIterator<IgniteComponent> stopIter =
                 startedComponents.listIterator(startedComponents.size());
-        
+
         while (stopIter.hasPrevious()) {
             IgniteComponent componentToStop = stopIter.previous();
-            
+
             try {
                 componentToStop.stop();
             } catch (Exception e) {
@@ -441,22 +440,22 @@ public class IgniteImpl implements Ignite {
             }
         }
     }
-    
+
     /**
      * Starts the Vault component.
      */
     private static VaultManager createVault(Path workDir) {
         Path vaultPath = workDir.resolve(VAULT_DB_PATH);
-        
+
         try {
             Files.createDirectories(vaultPath);
         } catch (IOException e) {
             throw new IgniteInternalException(e);
         }
-        
+
         return new VaultManager(new PersistentVaultService(vaultPath));
     }
-    
+
     /**
      * Returns a path to the partitions store directory. Creates a directory if it doesn't exist.
      *
@@ -466,16 +465,16 @@ public class IgniteImpl implements Ignite {
     @NotNull
     private static Path getPartitionsStorePath(Path workDir) {
         Path partitionsStore = workDir.resolve(PARTITIONS_STORE_PATH);
-        
+
         try {
             Files.createDirectories(partitionsStore);
         } catch (IOException e) {
             throw new IgniteInternalException("Failed to create directory for partitions storage: " + e.getMessage(), e);
         }
-        
+
         return partitionsStore;
     }
-    
+
     /**
      * Node state.
      */
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgnitionImpl.java b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgnitionImpl.java
index 177364f..7d471df 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgnitionImpl.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgnitionImpl.java
@@ -73,9 +73,7 @@ public class IgnitionImpl implements Ignition {
                     workDir
             );
         } catch (IOException e) {
-            LOG.warn("Unable to read user specific configuration, default configuration will be used: "
-                    + e.getMessage());
-            return start(nodeName, workDir);
+            throw new IgniteException("Unable to read user specific configuration.", e);
         }
     }
 
@@ -89,9 +87,7 @@ public class IgnitionImpl implements Ignition {
                     workDir
             );
         } catch (IOException e) {
-            LOG.warn("Unable to read user specific configuration, default configuration will be used: "
-                    + e.getMessage());
-            return start(name, workDir);
+            throw new IgniteException("Unable to read user specific configuration.", e);
         }
     }
 
@@ -143,7 +139,11 @@ public class IgnitionImpl implements Ignition {
         } catch (Exception e) {
             nodes.remove(nodeName);
 
-            throw new IgniteException(e);
+            if (e instanceof IgniteException) {
+                throw e;
+            } else {
+                throw new IgniteException(e);
+            }
         }
 
         ackSuccessStart();
@@ -160,8 +160,6 @@ public class IgnitionImpl implements Ignition {
 
         String banner = String.join("\n", BANNER);
 
-        LOG.info(() ->
-                        LoggerMessageHelper.format("{}\n" + " ".repeat(22) + "Apache Ignite ver. {}\n", banner, ver),
-                null);
+        LOG.info(() -> LoggerMessageHelper.format("{}\n" + " ".repeat(22) + "Apache Ignite ver. {}\n", banner, ver), null);
     }
 }