You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2022/02/15 11:05:54 UTC

[GitHub] [ignite-3] sashapolo opened a new pull request #659: IGNITE-16528 Implement init command handling

sashapolo opened a new pull request #659:
URL: https://github.com/apache/ignite-3/pull/659


   https://issues.apache.org/jira/browse/IGNITE-16528


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811007482



##########
File path: examples/src/integrationTest/java/org/apache/ignite/example/sql/jdbc/ItSqlExamplesTest.java
##########
@@ -17,35 +17,22 @@
 
 package org.apache.ignite.example.sql.jdbc;
 
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.example.AbstractExamplesTest;
 import org.apache.ignite.example.ExampleTestUtils;
-import org.apache.ignite.internal.testframework.WorkDirectory;
-import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.extension.ExtendWith;
 
 /**
  * These tests check that all SQL JDBC examples pass correctly.
  */
-@ExtendWith(WorkDirectoryExtension.class)
-public class SqlExamplesTest {
-    /** Empty argument to invoke an example. */
-    protected static final String[] EMPTY_ARGS = new String[0];
-
+public class ItSqlExamplesTest extends AbstractExamplesTest {
     /**
      * Runs SqlJdbcExample and checks its output.
      *
      * @throws Exception If failed.
      */
     @Test
     public void testSqlJdbcExample() throws Exception {
-        ExampleTestUtils.assertConsoleOutputContains(SqlJdbcExample::main, EMPTY_ARGS,
+        ExampleTestUtils.assertConsoleOutputContains(SqlJdbcExample::main, new String[0],

Review comment:
       Never mind, I'll bring it back




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] rpuch commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
rpuch commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811667702



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> metastorageNodes = resolveNodes(clusterService, metaStorageNodeNames);
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> cmgNodes = resolveNodes(clusterService, cmgNodeNames);
+
+            // TODO: init message should only be sent to metastorageNodes and cmgNodes respectively,
+            //  https://issues.apache.org/jira/browse/IGNITE-16471
+            Collection<ClusterNode> allMembers = clusterService.topologyService().allMembers();
+
+            var msgFactory = new InitMessagesFactory();
+
+            MetastorageInitMessage metaStorageInitMessage = msgFactory.metastorageInitMessage()
+                    .metastorageNodes(metaStorageNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> metaStorageInitFuture = invokeMessage(allMembers, metaStorageInitMessage);
+
+            CmgInitMessage cmgInitMessage = msgFactory.cmgInitMessage()
+                    .cmgNodes(cmgNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> cmgInitFuture = invokeMessage(allMembers, cmgInitMessage);
+
+            return metaStorageInitFuture
+                    .thenCombine(cmgInitFuture, Leaders::new)
+                    .whenComplete((leaders, e) -> {
+                        if (e != null) {
+                            log.error("Initialization failed, rolling back", e);
+
+                            CancelInitMessage cancelMessage = msgFactory.cancelInitMessage()
+                                    .reason(e.getMessage())
+                                    .build();
+
+                            cancelInit(allMembers, cancelMessage);
+                        }
+                    });
+        } catch (Exception e) {
+            return CompletableFuture.failedFuture(e);
+        }
+    }
+
+    /**
+     * Sends a message to all provided nodes.
+     *
+     * @param nodes nodes to send message to.
+     * @param message message to send.
+     * @return future that either resolves to a leader node ID or fails if any of the nodes return an error response.
+     */
+    private CompletableFuture<String> invokeMessage(Collection<ClusterNode> nodes, NetworkMessage message) {
+        List<CompletableFuture<String>> futures = nodes.stream()
+                .map(node -> invokeMessage(node, message))
+                .collect(Collectors.toList());
+
+        return CompletableFuture.allOf(futures.toArray(CompletableFuture[]::new))
+                // all futures should return the same response, unless they fail
+                .thenCompose(v -> futures.get(0));
+    }
+
+    private CompletableFuture<String> invokeMessage(ClusterNode node, NetworkMessage message) {
+        return clusterService.messagingService()
+                .invoke(node, message, 10000)

Review comment:
       The only situation I was able to imagine is that someone has a cluster over a network with high latency (maybe due to geo-distribution), so 10 seconds could be not sufficient for them. But for such cases, the point of Ignite as a high-perf system seems to be lost, so probably we should not consider such cases (at least, for now).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sanpwc commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sanpwc commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r812112519



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/messages/InitMessageGroup.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.internal.join.messages;
+
+import org.apache.ignite.network.annotations.MessageGroup;
+
+/**
+ * Message Group for cluster initialization and CMG management.
+ */
+@MessageGroup(groupType = 6, groupName = "InitMessages")
+public class InitMessageGroup {
+    /**
+     * Message type for {@link CmgInitMessage}.
+     */
+    public static final short CMG_INIT = 1;
+
+    /**
+     * Message type for {@link MetastorageInitMessage}.
+     */
+    public static final short METASTORAGE_INIT = 2;
+
+    /**
+     * Message type for {@link LeaderElectedMessage}.
+     */
+    public static final short LEADER_ELECTED = 3;

Review comment:
       Yep




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] rpuch commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
rpuch commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810175244



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> metastorageNodes = resolveNodes(clusterService, metaStorageNodeNames);
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> cmgNodes = resolveNodes(clusterService, cmgNodeNames);
+
+            // TODO: init message should only be sent to metastorageNodes and cmgNodes respectively,
+            //  https://issues.apache.org/jira/browse/IGNITE-16471
+            Collection<ClusterNode> allMembers = clusterService.topologyService().allMembers();
+
+            var msgFactory = new InitMessagesFactory();
+
+            MetastorageInitMessage metaStorageInitMessage = msgFactory.metastorageInitMessage()
+                    .metastorageNodes(metaStorageNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> metaStorageInitFuture = invokeMessage(allMembers, metaStorageInitMessage);
+
+            CmgInitMessage cmgInitMessage = msgFactory.cmgInitMessage()
+                    .cmgNodes(cmgNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> cmgInitFuture = invokeMessage(allMembers, cmgInitMessage);
+
+            return metaStorageInitFuture
+                    .thenCombine(cmgInitFuture, Leaders::new)
+                    .whenComplete((leaders, e) -> {
+                        if (e != null) {
+                            log.error("Initialization failed, rolling back", e);
+
+                            CancelInitMessage cancelMessage = msgFactory.cancelInitMessage()
+                                    .reason(e.getMessage())
+                                    .build();
+
+                            cancelInit(allMembers, cancelMessage);
+                        }
+                    });
+        } catch (Exception e) {
+            return CompletableFuture.failedFuture(e);
+        }
+    }
+
+    /**
+     * Sends a message to all provided nodes.
+     *
+     * @param nodes nodes to send message to.
+     * @param message message to send.
+     * @return future that either resolves to a leader node ID or fails if any of the nodes return an error response.
+     */
+    private CompletableFuture<String> invokeMessage(Collection<ClusterNode> nodes, NetworkMessage message) {
+        List<CompletableFuture<String>> futures = nodes.stream()
+                .map(node -> invokeMessage(node, message))
+                .collect(Collectors.toList());

Review comment:
       If you import `toList()` statically, this starts reading as English prose: `.collect(toList())`

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> metastorageNodes = resolveNodes(clusterService, metaStorageNodeNames);

Review comment:
       Why is it resolved? Just to validate note names? Would it make sense to extract a method named like `validateNodeNames()`?

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));
+
+                    raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)
+                            .whenComplete((service, e) -> {
+                                if (e == null) {
+                                    Peer leader = service.leader();
+
+                                    assert leader != null;
+
+                                    messagingService.respond(addr, leaderElectedResponse(msgFactory, leader), correlationId);
+                                } else {
+                                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                                }
+                            });
+                }
+            } catch (Exception e) {
+                log.error("Exception when initializing the CMG", e);
+
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                }
+            } finally {
+                busyLock.leaveBusy();

Review comment:
       What's the point to take a busy lock in this message handler? Most of its methods (including `prepareRaftGroup()`) are asynchronous, so from what will the lock protect us?

##########
File path: modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
##########
@@ -874,28 +915,31 @@ private long appliedRevision() {
      *
      * @return Ignite UUID of new consolidated watch.
      */
-    private CompletableFuture<Optional<IgniteUuid>> updateWatches() {
-        long revision = appliedRevision() + 1;
+    private CompletableFuture<IgniteUuid> updateWatches() {

Review comment:
       I think it's worth mentioning in a comment that this method MUST be called under `synchronized`

##########
File path: modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
##########
@@ -122,174 +115,233 @@
 
     /**
      * Future which will be completed with {@link IgniteUuid}, when aggregated watch will be successfully deployed. Can be resolved to
-     * {@link Optional#empty()} if no watch deployed at the moment.
-     */
-    private CompletableFuture<Optional<IgniteUuid>> deployFut = new CompletableFuture<>();
-
-    /**
-     * If true - all new watches will be deployed immediately.
+     * {@code null} if no watch deployed at the moment.
      *
-     * <p>If false - all new watches will be aggregated to one batch for further deploy by {@link MetaStorageManager#deployWatches()}
+     * <p>Multithreaded access is guarded by {@code this}.
      */
-    private boolean deployed;
-
-    /** Flag indicates if meta storage nodes were set on start. */
-    private boolean metaStorageNodesOnStart;
+    private CompletableFuture<IgniteUuid> deployFut = new CompletableFuture<>();
 
     /** Actual storage for the Metastorage. */
     private final KeyValueStorage storage;
 
     /** Busy lock to stop synchronously. */
     private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
 
-    /** Prevents double stopping the component. */
-    AtomicBoolean stopGuard = new AtomicBoolean();
+    /**
+     * If true - all new watches will be deployed immediately.
+     *
+     * <p>If false - all new watches will be aggregated to one batch for further deploy by {@link MetaStorageManager#deployWatches()}.
+     *
+     * <p>Multithreaded access is guarded by {@code this}.
+     */
+    private boolean areWatchesDeployed = false;
+
+    /**
+     * Flag that indicates that the Meta Storage has been initialized.
+     */
+    private final AtomicBoolean isInitialized = new AtomicBoolean();
+
+    /**
+     * Prevents double stopping the component.
+     *
+     * <p>Multithreaded access is guarded by {@code this}.
+     */
+    private boolean isStopped = false;
 
     /**
      * The constructor.
      *
-     * @param vaultMgr      Vault manager.
-     * @param locCfgMgr     Local configuration manager.
-     * @param clusterNetSvc Cluster network service.
-     * @param raftMgr       Raft manager.
-     * @param storage       Storage. This component owns this resource and will manage its lifecycle.
+     * @param vaultMgr Vault manager.
+     * @param clusterService Cluster network service.
+     * @param raftMgr Raft manager.
+     * @param storage Storage. This component owns this resource and will manage its lifecycle.
      */
     public MetaStorageManager(
             VaultManager vaultMgr,
-            ConfigurationManager locCfgMgr,
-            ClusterService clusterNetSvc,
+            ClusterService clusterService,
             Loza raftMgr,
             KeyValueStorage storage
     ) {
         this.vaultMgr = vaultMgr;
-        this.locCfgMgr = locCfgMgr;
-        this.clusterNetSvc = clusterNetSvc;
         this.raftMgr = raftMgr;
         this.storage = storage;
-    }
 
-    /** {@inheritDoc} */
-    @Override
-    public void start() {
-        String[] metastorageNodes = this.locCfgMgr.configurationRegistry().getConfiguration(NodeConfiguration.KEY)
-                .metastorageNodes().value();
+        MessagingService messagingService = clusterService.messagingService();
 
-        Predicate<ClusterNode> metaStorageNodesContainsLocPred =
-                clusterNode -> Arrays.asList(metastorageNodes).contains(clusterNode.name());
+        var msgFactory = new InitMessagesFactory();
 
-        if (metastorageNodes.length > 0) {
-            metaStorageNodesOnStart = true;
-
-            List<ClusterNode> metaStorageMembers = clusterNetSvc.topologyService().allMembers().stream()
-                    .filter(metaStorageNodesContainsLocPred)
-                    .collect(Collectors.toList());
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {

Review comment:
       Same question concerning `busyLock` as in `ClusterManagementGroupManager`: is it ok that it covers just the start of the futures, not whole period while the futures are working?

##########
File path: examples/src/integrationTest/java/org/apache/ignite/example/sql/jdbc/ItSqlExamplesTest.java
##########
@@ -17,35 +17,22 @@
 
 package org.apache.ignite.example.sql.jdbc;
 
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.example.AbstractExamplesTest;
 import org.apache.ignite.example.ExampleTestUtils;
-import org.apache.ignite.internal.testframework.WorkDirectory;
-import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.extension.ExtendWith;
 
 /**
  * These tests check that all SQL JDBC examples pass correctly.
  */
-@ExtendWith(WorkDirectoryExtension.class)
-public class SqlExamplesTest {
-    /** Empty argument to invoke an example. */
-    protected static final String[] EMPTY_ARGS = new String[0];
-
+public class ItSqlExamplesTest extends AbstractExamplesTest {
     /**
      * Runs SqlJdbcExample and checks its output.
      *
      * @throws Exception If failed.
      */
     @Test
     public void testSqlJdbcExample() throws Exception {
-        ExampleTestUtils.assertConsoleOutputContains(SqlJdbcExample::main, EMPTY_ARGS,
+        ExampleTestUtils.assertConsoleOutputContains(SqlJdbcExample::main, new String[0],

Review comment:
       Is explicit array creation better than referencing it via a constant?

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> metastorageNodes = resolveNodes(clusterService, metaStorageNodeNames);
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> cmgNodes = resolveNodes(clusterService, cmgNodeNames);
+
+            // TODO: init message should only be sent to metastorageNodes and cmgNodes respectively,
+            //  https://issues.apache.org/jira/browse/IGNITE-16471
+            Collection<ClusterNode> allMembers = clusterService.topologyService().allMembers();
+
+            var msgFactory = new InitMessagesFactory();
+
+            MetastorageInitMessage metaStorageInitMessage = msgFactory.metastorageInitMessage()
+                    .metastorageNodes(metaStorageNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> metaStorageInitFuture = invokeMessage(allMembers, metaStorageInitMessage);
+
+            CmgInitMessage cmgInitMessage = msgFactory.cmgInitMessage()
+                    .cmgNodes(cmgNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> cmgInitFuture = invokeMessage(allMembers, cmgInitMessage);
+
+            return metaStorageInitFuture
+                    .thenCombine(cmgInitFuture, Leaders::new)
+                    .whenComplete((leaders, e) -> {
+                        if (e != null) {
+                            log.error("Initialization failed, rolling back", e);
+
+                            CancelInitMessage cancelMessage = msgFactory.cancelInitMessage()
+                                    .reason(e.getMessage())
+                                    .build();
+
+                            cancelInit(allMembers, cancelMessage);
+                        }
+                    });
+        } catch (Exception e) {
+            return CompletableFuture.failedFuture(e);
+        }
+    }
+
+    /**
+     * Sends a message to all provided nodes.
+     *
+     * @param nodes nodes to send message to.
+     * @param message message to send.
+     * @return future that either resolves to a leader node ID or fails if any of the nodes return an error response.
+     */
+    private CompletableFuture<String> invokeMessage(Collection<ClusterNode> nodes, NetworkMessage message) {
+        List<CompletableFuture<String>> futures = nodes.stream()
+                .map(node -> invokeMessage(node, message))
+                .collect(Collectors.toList());
+
+        return CompletableFuture.allOf(futures.toArray(CompletableFuture[]::new))
+                // all futures should return the same response, unless they fail
+                .thenCompose(v -> futures.get(0));
+    }
+
+    private CompletableFuture<String> invokeMessage(ClusterNode node, NetworkMessage message) {
+        return clusterService.messagingService()
+                .invoke(node, message, 10000)

Review comment:
       Should the timeout be configurable?

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> metastorageNodes = resolveNodes(clusterService, metaStorageNodeNames);
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> cmgNodes = resolveNodes(clusterService, cmgNodeNames);
+
+            // TODO: init message should only be sent to metastorageNodes and cmgNodes respectively,
+            //  https://issues.apache.org/jira/browse/IGNITE-16471
+            Collection<ClusterNode> allMembers = clusterService.topologyService().allMembers();
+
+            var msgFactory = new InitMessagesFactory();
+
+            MetastorageInitMessage metaStorageInitMessage = msgFactory.metastorageInitMessage()
+                    .metastorageNodes(metaStorageNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> metaStorageInitFuture = invokeMessage(allMembers, metaStorageInitMessage);
+
+            CmgInitMessage cmgInitMessage = msgFactory.cmgInitMessage()
+                    .cmgNodes(cmgNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> cmgInitFuture = invokeMessage(allMembers, cmgInitMessage);
+
+            return metaStorageInitFuture
+                    .thenCombine(cmgInitFuture, Leaders::new)
+                    .whenComplete((leaders, e) -> {
+                        if (e != null) {
+                            log.error("Initialization failed, rolling back", e);
+
+                            CancelInitMessage cancelMessage = msgFactory.cancelInitMessage()
+                                    .reason(e.getMessage())
+                                    .build();
+
+                            cancelInit(allMembers, cancelMessage);
+                        }
+                    });
+        } catch (Exception e) {
+            return CompletableFuture.failedFuture(e);
+        }
+    }
+
+    /**
+     * Sends a message to all provided nodes.
+     *
+     * @param nodes nodes to send message to.
+     * @param message message to send.
+     * @return future that either resolves to a leader node ID or fails if any of the nodes return an error response.
+     */
+    private CompletableFuture<String> invokeMessage(Collection<ClusterNode> nodes, NetworkMessage message) {
+        List<CompletableFuture<String>> futures = nodes.stream()
+                .map(node -> invokeMessage(node, message))
+                .collect(Collectors.toList());
+
+        return CompletableFuture.allOf(futures.toArray(CompletableFuture[]::new))
+                // all futures should return the same response, unless they fail
+                .thenCompose(v -> futures.get(0));
+    }
+
+    private CompletableFuture<String> invokeMessage(ClusterNode node, NetworkMessage message) {
+        return clusterService.messagingService()
+                .invoke(node, message, 10000)
+                .thenApply(response -> {
+                    if (response instanceof InitErrorMessage) {
+                        throw new InitException(String.format(
+                                "Got error response from node \"%s\": %s", node.name(), ((InitErrorMessage) response).cause()
+                        ));
+                    }
+
+                    if (!(response instanceof LeaderElectedMessage)) {
+                        throw new InitException(String.format(
+                                "Unexpected response from node \"%s\": %s", node.name(), response.getClass()
+                        ));
+                    }
+
+                    return ((LeaderElectedMessage) response).leaderName();
+                });
+    }
+
+    private void cancelInit(Collection<ClusterNode> nodes, NetworkMessage message) {
+        CompletableFuture<?>[] futures = nodes.stream()
+                .map(node -> clusterService.messagingService().send(node, message))
+                .toArray(CompletableFuture[]::new);
+
+        CompletableFuture.allOf(futures).whenComplete((v, e) -> log.error("Error when canceling init", e));

Review comment:
       This method is not chained, so its caller can finish working before the cancellation is finished; so it is possible for the initiator of the init command to start another attempt of init before the previous one was cleaned up.
   How about making this method return a `CompletableFuture` and using it as another step of async chain?

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));
+
+                    raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)
+                            .whenComplete((service, e) -> {
+                                if (e == null) {
+                                    Peer leader = service.leader();
+
+                                    assert leader != null;
+
+                                    messagingService.respond(addr, leaderElectedResponse(msgFactory, leader), correlationId);
+                                } else {
+                                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                                }
+                            });
+                }
+            } catch (Exception e) {
+                log.error("Exception when initializing the CMG", e);
+
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                }
+            } finally {
+                busyLock.leaveBusy();
+            }
+        });
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group.
+     */
+    public Leaders initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) throws NodeStoppingException {
+        if (!busyLock.enterBusy()) {

Review comment:
       Same busy lock encloses each of 'send init messages' and 'handle CMG init message async start'. Is this the intention?

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> metastorageNodes = resolveNodes(clusterService, metaStorageNodeNames);
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> cmgNodes = resolveNodes(clusterService, cmgNodeNames);
+
+            // TODO: init message should only be sent to metastorageNodes and cmgNodes respectively,
+            //  https://issues.apache.org/jira/browse/IGNITE-16471
+            Collection<ClusterNode> allMembers = clusterService.topologyService().allMembers();
+
+            var msgFactory = new InitMessagesFactory();
+
+            MetastorageInitMessage metaStorageInitMessage = msgFactory.metastorageInitMessage()
+                    .metastorageNodes(metaStorageNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> metaStorageInitFuture = invokeMessage(allMembers, metaStorageInitMessage);
+
+            CmgInitMessage cmgInitMessage = msgFactory.cmgInitMessage()
+                    .cmgNodes(cmgNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> cmgInitFuture = invokeMessage(allMembers, cmgInitMessage);
+
+            return metaStorageInitFuture
+                    .thenCombine(cmgInitFuture, Leaders::new)
+                    .whenComplete((leaders, e) -> {
+                        if (e != null) {
+                            log.error("Initialization failed, rolling back", e);
+
+                            CancelInitMessage cancelMessage = msgFactory.cancelInitMessage()
+                                    .reason(e.getMessage())
+                                    .build();
+
+                            cancelInit(allMembers, cancelMessage);

Review comment:
       What if we fail to send a cancellation message? Could the cluster stuck in an intermediate state? How would it be resolved (automatically or by hand)?

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/rest/InitCommand.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.internal.join.rest;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.Collection;
+
+/**
+ * REST command for initializing a cluster.
+ */
+public class InitCommand {
+    private final Collection<String> metaStorageNodes;
+
+    private final Collection<String> cmgNodes;
+
+    @JsonCreator
+    public InitCommand(
+            @JsonProperty("metaStorageNodes") Collection<String> metaStorageNodes,
+            @JsonProperty("cmgNodes") Collection<String> cmgNodes
+    ) {
+        this.metaStorageNodes = metaStorageNodes;

Review comment:
       I suggest to make a defensive copy for each of the collections

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/CmgRaftGroupListener.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.internal.join;
+
+import java.nio.file.Path;
+import java.util.Iterator;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+import org.apache.ignite.raft.client.Command;
+import org.apache.ignite.raft.client.ReadCommand;
+import org.apache.ignite.raft.client.WriteCommand;
+import org.apache.ignite.raft.client.service.CommandClosure;
+import org.apache.ignite.raft.client.service.RaftGroupListener;
+import org.jetbrains.annotations.Nullable;
+
+// TODO: implement listener, https://issues.apache.org/jira/browse/IGNITE-16471
+class CmgRaftGroupListener implements RaftGroupListener {
+    @Override
+    public void onRead(Iterator<CommandClosure<ReadCommand>> iterator) {
+
+    }
+
+    @Override
+    public void onWrite(Iterator<CommandClosure<WriteCommand>> iterator) {
+
+    }
+
+    @Override
+    public void onSnapshotSave(Path path, Consumer<Throwable> doneClo) {
+
+    }
+
+    @Override
+    public boolean onSnapshotLoad(Path path) {
+        return false;
+    }
+
+    @Override
+    public void onShutdown() {
+
+    }
+
+    @Override
+    public @Nullable CompletableFuture<Void> onBeforeApply(Command command) {
+        return null;

Review comment:
       A nullable future... this seems weird

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/rest/InitCommandHandler.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.join.rest;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import org.apache.ignite.internal.join.ClusterInitializer;
+import org.apache.ignite.internal.join.Leaders;
+import org.apache.ignite.internal.rest.ErrorResult;
+import org.apache.ignite.internal.rest.netty.RestApiHttpRequest;
+import org.apache.ignite.internal.rest.netty.RestApiHttpResponse;
+import org.apache.ignite.internal.rest.routes.RequestHandler;
+import org.apache.ignite.lang.IgniteLogger;
+
+/**
+ * REST handler for the {@link InitCommand}.
+ */
+public class InitCommandHandler implements RequestHandler {
+    private static final IgniteLogger log = IgniteLogger.forClass(InitCommandHandler.class);
+
+    private final ObjectMapper objectMapper = new ObjectMapper();
+
+    private final ClusterInitializer clusterInitializer;
+
+    public InitCommandHandler(ClusterInitializer clusterInitializer) {
+        this.clusterInitializer = clusterInitializer;
+    }
+
+    @Override
+    public CompletableFuture<RestApiHttpResponse> handle(RestApiHttpRequest request, RestApiHttpResponse response) {
+        try {
+            InitCommand command = readContent(request);
+
+            if (log.isInfoEnabled()) {
+                log.info(
+                        "Received init command:\n\tMeta Storage nodes: {}\n\tCMG nodes: {}",
+                        command.metaStorageNodes(),
+                        command.cmgNodes()
+                );
+            }
+
+            return clusterInitializer.initCluster(command.metaStorageNodes(), command.cmgNodes())
+                    .thenApply(leaders -> successResponse(response, leaders))
+                    .exceptionally(e -> errorResponse(response, e));
+        } catch (Exception e) {
+            return CompletableFuture.completedFuture(errorResponse(response, e));
+        }
+    }
+
+    private InitCommand readContent(RestApiHttpRequest restApiHttpRequest) throws IOException {
+        ByteBuf content = restApiHttpRequest.request().content();
+
+        try (InputStream is = new ByteBufInputStream(content)) {
+            return objectMapper.readValue(is, InitCommand.class);

Review comment:
       I wonder if this will block and whether we are allowed to block this thread

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/Utils.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.internal.join;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+
+/**
+ * Various utility methods.
+ */
+class Utils {

Review comment:
       This seems to be a too vague name. I suggest specifying what kind of util it is, like `NodeResolveUtils` or `NodeUtils`, or even dropping the `Utils` and calling it `NodeResolving`.

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/Leaders.java
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.internal.join;
+
+import java.util.Objects;
+
+/**
+ * Data class containing consistent IDs of leaders of the Meta Storage and the CMG.
+ */
+public class Leaders {
+    private final String metaStorageLeader;
+
+    private final String cmgLeader;
+
+    public Leaders(String metaStorageLeader, String cmgLeader) {

Review comment:
       How about checking here that both values are not null?

##########
File path: modules/join/src/test/java/org/apache/ignite/internal/join/ClusterInitializerTest.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.equalTo;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.clearInvocations;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.TopologyService;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+/**
+ * Tests for {@link ClusterInitializer}.
+ */
+@ExtendWith(MockitoExtension.class)
+public class ClusterInitializerTest {
+    @Mock
+    private MessagingService messagingService;
+
+    @Mock
+    private TopologyService topologyService;
+
+    private ClusterInitializer clusterInitializer;
+
+    private final InitMessagesFactory msgFactory = new InitMessagesFactory();
+
+    @BeforeEach
+    void setUp(@Mock ClusterService clusterService) {
+        when(clusterService.messagingService()).thenReturn(messagingService);
+        when(clusterService.topologyService()).thenReturn(topologyService);
+
+        clusterInitializer = new ClusterInitializer(clusterService);
+    }
+
+    /**
+     * Tests the happy-case scenario of cluster initialization.
+     */
+    @Test
+    void testNormalInit() {
+        ClusterNode metastorageNode = new ClusterNode("metastore", "metastore", new NetworkAddress("foo", 123));
+        ClusterNode cmgNode = new ClusterNode("cmg", "cmg", new NetworkAddress("bar", 456));
+
+        when(topologyService.getByConsistentId(metastorageNode.name())).thenReturn(metastorageNode);
+        when(topologyService.getByConsistentId(cmgNode.name())).thenReturn(cmgNode);
+        when(topologyService.allMembers()).thenReturn(List.of(metastorageNode, cmgNode));
+
+        when(messagingService.invoke(any(ClusterNode.class), any(MetastorageInitMessage.class), anyLong()))
+                .thenAnswer(invocation -> {
+                    MetastorageInitMessage message = invocation.getArgument(1);
+
+                    NetworkMessage response = msgFactory.leaderElectedMessage()
+                            .leaderName(message.metastorageNodes()[0])
+                            .build();
+
+                    return CompletableFuture.completedFuture(response);
+                });
+
+        when(messagingService.invoke(any(ClusterNode.class), any(CmgInitMessage.class), anyLong()))
+                .thenAnswer(invocation -> {
+                    CmgInitMessage message = invocation.getArgument(1);
+
+                    NetworkMessage response = msgFactory.leaderElectedMessage()
+                            .leaderName(message.cmgNodes()[0])
+                            .build();
+
+                    return CompletableFuture.completedFuture(response);
+                });
+
+        // check that leaders are different in case different node IDs are provided
+        CompletableFuture<Leaders> leaders = clusterInitializer.initCluster(List.of(metastorageNode.name()), List.of(cmgNode.name()));
+
+        verify(messagingService).invoke(eq(metastorageNode), any(MetastorageInitMessage.class), anyLong());
+        verify(messagingService).invoke(eq(cmgNode), any(MetastorageInitMessage.class), anyLong());
+        verify(messagingService).invoke(eq(metastorageNode), any(CmgInitMessage.class), anyLong());
+        verify(messagingService).invoke(eq(cmgNode), any(CmgInitMessage.class), anyLong());
+
+        var expectedLeaders = new Leaders(metastorageNode.name(), cmgNode.name());
+
+        assertThat(leaders, willBe(equalTo(expectedLeaders)));
+
+        clearInvocations(messagingService);

Review comment:
       This usually means that a test is actually testing more than one behavior. I suggest splitting the test in two: one for the case when both meta-storage and CMG nodes are specified, another for the case when only meta-storage nodes are given.

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/rest/InitCommandHandler.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.join.rest;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import org.apache.ignite.internal.join.ClusterInitializer;
+import org.apache.ignite.internal.join.Leaders;
+import org.apache.ignite.internal.rest.ErrorResult;
+import org.apache.ignite.internal.rest.netty.RestApiHttpRequest;
+import org.apache.ignite.internal.rest.netty.RestApiHttpResponse;
+import org.apache.ignite.internal.rest.routes.RequestHandler;
+import org.apache.ignite.lang.IgniteLogger;
+
+/**
+ * REST handler for the {@link InitCommand}.
+ */
+public class InitCommandHandler implements RequestHandler {
+    private static final IgniteLogger log = IgniteLogger.forClass(InitCommandHandler.class);
+
+    private final ObjectMapper objectMapper = new ObjectMapper();
+
+    private final ClusterInitializer clusterInitializer;
+
+    public InitCommandHandler(ClusterInitializer clusterInitializer) {
+        this.clusterInitializer = clusterInitializer;
+    }
+
+    @Override
+    public CompletableFuture<RestApiHttpResponse> handle(RestApiHttpRequest request, RestApiHttpResponse response) {
+        try {
+            InitCommand command = readContent(request);
+
+            if (log.isInfoEnabled()) {
+                log.info(
+                        "Received init command:\n\tMeta Storage nodes: {}\n\tCMG nodes: {}",
+                        command.metaStorageNodes(),
+                        command.cmgNodes()
+                );
+            }
+
+            return clusterInitializer.initCluster(command.metaStorageNodes(), command.cmgNodes())
+                    .thenApply(leaders -> successResponse(response, leaders))
+                    .exceptionally(e -> errorResponse(response, e));
+        } catch (Exception e) {
+            return CompletableFuture.completedFuture(errorResponse(response, e));
+        }
+    }
+
+    private InitCommand readContent(RestApiHttpRequest restApiHttpRequest) throws IOException {
+        ByteBuf content = restApiHttpRequest.request().content();
+
+        try (InputStream is = new ByteBufInputStream(content)) {
+            return objectMapper.readValue(is, InitCommand.class);
+        }
+    }
+
+    private static RestApiHttpResponse successResponse(RestApiHttpResponse response, Leaders leaders) {
+        if (log.isInfoEnabled()) {
+            log.info(
+                    "Init command executed successfully.\n\tMeta Storage leader ID: {}\n\tCMG leader ID: {}",
+                    leaders.metaStorageLeader(),
+                    leaders.cmgLeader()
+            );
+        }
+
+        response.json(Map.of(
+                "metaStorageLeaderId", leaders.metaStorageLeader(),
+                "cmgLeaderId", leaders.cmgLeader()
+        ));
+
+        return response;
+    }
+
+    private static RestApiHttpResponse errorResponse(RestApiHttpResponse response, Throwable e) {
+        if (e instanceof CompletionException) {
+            e = e.getCause();
+        }
+
+        log.error("Init command failure", e);
+
+        response.status(INTERNAL_SERVER_ERROR);
+        response.json(Map.of("error", new ErrorResult("APPLICATION_EXCEPTION", e.getMessage())));

Review comment:
       Don't we have an enum with `APPLICATION_EXCEPTION` and its friends yet?

##########
File path: modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItClusterInitTest.java
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.internal.runner.app;
+
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.not;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.join.Leaders;
+import org.apache.ignite.internal.testframework.WorkDirectory;
+import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Integration tests for initializing a cluster.
+ */
+@ExtendWith(WorkDirectoryExtension.class)
+public class ItClusterInitTest {
+    @WorkDirectory
+    private Path workDir;
+
+    private final TestInfo testInfo;
+
+    private final List<IgniteImpl> nodes = new ArrayList<>();
+
+    public ItClusterInitTest(TestInfo testInfo) {
+        this.testInfo = testInfo;
+    }
+
+    @AfterEach
+    void tearDown() throws Exception {
+        IgniteUtils.closeAll(nodes);
+    }
+
+    /**
+     * Tests the happy case for cluster initialization.
+     */
+    @Test
+    void testNormalInit() throws NodeStoppingException {
+        createCluster(4);
+
+        List<String> metastorageNodes = List.of(nodes.get(0).name(), nodes.get(1).name());
+
+        List<String> cmgNodes = List.of(nodes.get(2).name(), nodes.get(3).name());
+
+        Leaders leaders = nodes.get(0).init(metastorageNodes, cmgNodes);
+
+        assertThat(metastorageNodes, hasItem(leaders.metaStorageLeader()));
+        assertThat(metastorageNodes, not(hasItem(leaders.cmgLeader())));
+
+        assertThat(cmgNodes, hasItem(leaders.cmgLeader()));
+        assertThat(cmgNodes, not(hasItem(leaders.metaStorageLeader())));
+    }
+
+    /**
+     * Tests a scenario when a cluster is initialized twice.
+     */
+    @Test
+    void testDoubleInit() throws NodeStoppingException {
+        createCluster(1);
+
+        IgniteImpl node = nodes.get(0);
+
+        node.init(List.of(node.name()));
+
+        assertThrows(Exception.class, () -> node.init(List.of(node.name())));
+    }
+
+    /**
+     * Tests a scenario when some nodes are stopped during initialization.
+     */
+    @Test
+    void testInitStoppingNodes() {
+        createCluster(2);
+
+        IgniteImpl node1 = nodes.get(0);
+        IgniteImpl node2 = nodes.get(1);
+
+        node2.stop();
+
+        assertThrows(Exception.class, () -> node1.init(List.of(node1.name(), node2.name())));

Review comment:
       I suggest to verify expected exceptions everywhere we expect them

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));
+
+                    raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)
+                            .whenComplete((service, e) -> {
+                                if (e == null) {
+                                    Peer leader = service.leader();
+
+                                    assert leader != null;
+
+                                    messagingService.respond(addr, leaderElectedResponse(msgFactory, leader), correlationId);
+                                } else {
+                                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                                }
+                            });
+                }
+            } catch (Exception e) {
+                log.error("Exception when initializing the CMG", e);
+
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                }
+            } finally {
+                busyLock.leaveBusy();
+            }
+        });
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group.
+     */
+    public Leaders initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) throws NodeStoppingException {

Review comment:
       It seems that this method does not belong to this class. The class is about CMG, but the method is about the whole initialization (which also includes meta-storage initialization).
   
   I suggest to create another class which responsibility would be to perform the initialization as a whole; this method would be moved to that class (maybe with the initialization of a REST module, or maybe not, because a REST handler and a service seem to be in different layers).

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));
+
+                    raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)
+                            .whenComplete((service, e) -> {
+                                if (e == null) {
+                                    Peer leader = service.leader();
+
+                                    assert leader != null;
+
+                                    messagingService.respond(addr, leaderElectedResponse(msgFactory, leader), correlationId);
+                                } else {
+                                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                                }
+                            });
+                }
+            } catch (Exception e) {
+                log.error("Exception when initializing the CMG", e);
+
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                }
+            } finally {
+                busyLock.leaveBusy();
+            }
+        });
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group.
+     */
+    public Leaders initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) throws NodeStoppingException {
+        if (!busyLock.enterBusy()) {
+            throw new NodeStoppingException();
+        }
+
+        try {
+            return clusterInitializer.initCluster(metaStorageNodeNames, cmgNodeNames).get();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IgniteInternalException("Interrupted while initializing the cluster", e);
+        } catch (ExecutionException e) {
+            throw new IgniteInternalException("Unable to initialize the cluster", e.getCause());
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    private static NetworkMessage errorResponse(InitMessagesFactory msgFactory, Throwable e) {
+        log.error("Exception when starting the CMG", e);
+
+        return msgFactory.initErrorMessage()
+                .cause(e.getMessage())
+                .build();
+    }
+
+    private NetworkMessage leaderElectedResponse(InitMessagesFactory msgFactory, Peer leader) {
+        ClusterNode leaderNode = clusterService.topologyService().getByAddress(leader.address());
+
+        assert leaderNode != null;
+
+        log.info("CMG leader elected: " + leaderNode.name());
+
+        return msgFactory.leaderElectedMessage()
+                .leaderName(leaderNode.name())
+                .build();
+    }
+
+    @Override
+    public void start() {
+        restModule.addRoute(new Route(

Review comment:
       Why is init command handler initialized in `ClusterManagementGroupManager`? It seems a bit weird. Would a separate service designed specifically for init handling be better?

##########
File path: modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItClusterInitTest.java
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.internal.runner.app;
+
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.not;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.join.Leaders;
+import org.apache.ignite.internal.testframework.WorkDirectory;
+import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Integration tests for initializing a cluster.
+ */
+@ExtendWith(WorkDirectoryExtension.class)
+public class ItClusterInitTest {
+    @WorkDirectory
+    private Path workDir;
+
+    private final TestInfo testInfo;
+
+    private final List<IgniteImpl> nodes = new ArrayList<>();
+
+    public ItClusterInitTest(TestInfo testInfo) {
+        this.testInfo = testInfo;
+    }
+
+    @AfterEach
+    void tearDown() throws Exception {
+        IgniteUtils.closeAll(nodes);
+    }
+
+    /**
+     * Tests the happy case for cluster initialization.
+     */
+    @Test
+    void testNormalInit() throws NodeStoppingException {
+        createCluster(4);
+
+        List<String> metastorageNodes = List.of(nodes.get(0).name(), nodes.get(1).name());
+
+        List<String> cmgNodes = List.of(nodes.get(2).name(), nodes.get(3).name());
+
+        Leaders leaders = nodes.get(0).init(metastorageNodes, cmgNodes);
+
+        assertThat(metastorageNodes, hasItem(leaders.metaStorageLeader()));
+        assertThat(metastorageNodes, not(hasItem(leaders.cmgLeader())));
+
+        assertThat(cmgNodes, hasItem(leaders.cmgLeader()));
+        assertThat(cmgNodes, not(hasItem(leaders.metaStorageLeader())));
+    }
+
+    /**
+     * Tests a scenario when a cluster is initialized twice.
+     */
+    @Test
+    void testDoubleInit() throws NodeStoppingException {
+        createCluster(1);
+
+        IgniteImpl node = nodes.get(0);
+
+        node.init(List.of(node.name()));
+
+        assertThrows(Exception.class, () -> node.init(List.of(node.name())));

Review comment:
       Let's verify that the exception that is thrown is actually what we expect; otherwise, it could be anything like a `NullPointerException`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810918709



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> metastorageNodes = resolveNodes(clusterService, metaStorageNodeNames);

Review comment:
       "resolved" means that a node "name" has been resolved into a "ClusterNode". It's not only needed for validation, but also for obtaining the node addresses in order to send network messages




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] rpuch commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
rpuch commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810154756



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> metastorageNodes = resolveNodes(clusterService, metaStorageNodeNames);

Review comment:
       Why is it resolved? Just to validate node names? Would it make sense to extract a method named like `validateNodeNames()`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811244409



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/Leaders.java
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.internal.join;
+
+import java.util.Objects;
+
+/**
+ * Data class containing consistent IDs of leaders of the Meta Storage and the CMG.
+ */
+public class Leaders {
+    private final String metaStorageLeader;
+
+    private final String cmgLeader;
+
+    public Leaders(String metaStorageLeader, String cmgLeader) {

Review comment:
       What for? This is an internal class and static analysis should be enough for that...




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810927813



##########
File path: examples/src/integrationTest/java/org/apache/ignite/example/sql/jdbc/ItSqlExamplesTest.java
##########
@@ -62,39 +49,4 @@ public void testSqlJdbcExample() throws Exception {
                         + "    Richard, Miles, St. Petersburg\n"
         );
     }
-
-    /**
-     * Start node.
-     *
-     * @param workDir Work directory for the started node. Must not be {@code null}.
-     */
-    @BeforeEach
-    public void startNode(@WorkDirectory Path workDir) throws IOException {
-        IgnitionManager.start(
-                "my-first-node",
-                Files.readString(Path.of("config", "ignite-config.json")),
-                workDir
-        );
-    }
-
-    /**
-     * Stop node.
-     */
-    @AfterEach
-    public void stopNode() {
-        IgnitionManager.stop("my-first-node");
-    }
-
-    /**
-     * Removes a work directory created by {@link SqlExamplesTest}.
-     */
-    @BeforeEach
-    @AfterEach
-    public void removeWorkDir() {

Review comment:
       yes, WorkDirectoryExtension ensures that the work directory is removed after every test execution




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810925073



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));
+
+                    raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)
+                            .whenComplete((service, e) -> {
+                                if (e == null) {
+                                    Peer leader = service.leader();
+
+                                    assert leader != null;
+
+                                    messagingService.respond(addr, leaderElectedResponse(msgFactory, leader), correlationId);
+                                } else {
+                                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                                }
+                            });
+                }
+            } catch (Exception e) {
+                log.error("Exception when initializing the CMG", e);
+
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                }
+            } finally {
+                busyLock.leaveBusy();
+            }
+        });
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group.
+     */
+    public Leaders initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) throws NodeStoppingException {
+        if (!busyLock.enterBusy()) {
+            throw new NodeStoppingException();
+        }
+
+        try {
+            return clusterInitializer.initCluster(metaStorageNodeNames, cmgNodeNames).get();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IgniteInternalException("Interrupted while initializing the cluster", e);
+        } catch (ExecutionException e) {
+            throw new IgniteInternalException("Unable to initialize the cluster", e.getCause());
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    private static NetworkMessage errorResponse(InitMessagesFactory msgFactory, Throwable e) {
+        log.error("Exception when starting the CMG", e);
+
+        return msgFactory.initErrorMessage()
+                .cause(e.getMessage())
+                .build();
+    }
+
+    private NetworkMessage leaderElectedResponse(InitMessagesFactory msgFactory, Peer leader) {
+        ClusterNode leaderNode = clusterService.topologyService().getByAddress(leader.address());
+
+        assert leaderNode != null;
+
+        log.info("CMG leader elected: " + leaderNode.name());
+
+        return msgFactory.leaderElectedMessage()
+                .leaderName(leaderNode.name())
+                .build();
+    }
+
+    @Override
+    public void start() {
+        restModule.addRoute(new Route(

Review comment:
       > It seems a bit weird
   
   I think it's fine since this manager is responsible for the init procedure. We can introduce a separate manager for this purposes, but I think it will just bloat the overall amount of components unnecessarily. What do you think?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] rpuch commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
rpuch commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811850458



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/rest/InitCommandHandler.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.join.rest;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import org.apache.ignite.internal.join.ClusterInitializer;
+import org.apache.ignite.internal.join.Leaders;
+import org.apache.ignite.internal.rest.ErrorResult;
+import org.apache.ignite.internal.rest.netty.RestApiHttpRequest;
+import org.apache.ignite.internal.rest.netty.RestApiHttpResponse;
+import org.apache.ignite.internal.rest.routes.RequestHandler;
+import org.apache.ignite.lang.IgniteLogger;
+
+/**
+ * REST handler for the {@link InitCommand}.
+ */
+public class InitCommandHandler implements RequestHandler {
+    private static final IgniteLogger log = IgniteLogger.forClass(InitCommandHandler.class);
+
+    private final ObjectMapper objectMapper = new ObjectMapper();
+
+    private final ClusterInitializer clusterInitializer;
+
+    public InitCommandHandler(ClusterInitializer clusterInitializer) {
+        this.clusterInitializer = clusterInitializer;
+    }
+
+    @Override
+    public CompletableFuture<RestApiHttpResponse> handle(RestApiHttpRequest request, RestApiHttpResponse response) {
+        try {
+            InitCommand command = readContent(request);
+
+            if (log.isInfoEnabled()) {
+                log.info(
+                        "Received init command:\n\tMeta Storage nodes: {}\n\tCMG nodes: {}",
+                        command.metaStorageNodes(),
+                        command.cmgNodes()
+                );
+            }
+
+            return clusterInitializer.initCluster(command.metaStorageNodes(), command.cmgNodes())
+                    .thenApply(leaders -> successResponse(response, leaders))
+                    .exceptionally(e -> errorResponse(response, e));
+        } catch (Exception e) {
+            return CompletableFuture.completedFuture(errorResponse(response, e));
+        }
+    }
+
+    private InitCommand readContent(RestApiHttpRequest restApiHttpRequest) throws IOException {
+        ByteBuf content = restApiHttpRequest.request().content();
+
+        try (InputStream is = new ByteBufInputStream(content)) {
+            return objectMapper.readValue(is, InitCommand.class);
+        }
+    }
+
+    private static RestApiHttpResponse successResponse(RestApiHttpResponse response, Leaders leaders) {
+        if (log.isInfoEnabled()) {
+            log.info(
+                    "Init command executed successfully.\n\tMeta Storage leader ID: {}\n\tCMG leader ID: {}",
+                    leaders.metaStorageLeader(),
+                    leaders.cmgLeader()
+            );
+        }
+
+        response.json(Map.of(
+                "metaStorageLeaderId", leaders.metaStorageLeader(),
+                "cmgLeaderId", leaders.cmgLeader()
+        ));
+
+        return response;
+    }
+
+    private static RestApiHttpResponse errorResponse(RestApiHttpResponse response, Throwable e) {
+        if (e instanceof CompletionException) {
+            e = e.getCause();
+        }
+
+        log.error("Init command failure", e);
+
+        response.status(INTERNAL_SERVER_ERROR);
+        response.json(Map.of("error", new ErrorResult("APPLICATION_EXCEPTION", e.getMessage())));

Review comment:
       Or a ticket for enumerating the error reasons




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r820869387



##########
File path: modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.internal.cluster.management;
+
+import static io.netty.handler.codec.http.HttpHeaderValues.APPLICATION_JSON;
+import static org.apache.ignite.network.util.ClusterServiceUtils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.cluster.management.messages.CancelInitMessage;
+import org.apache.ignite.internal.cluster.management.messages.ClusterStateMessage;
+import org.apache.ignite.internal.cluster.management.messages.CmgInitMessage;
+import org.apache.ignite.internal.cluster.management.messages.CmgMessageGroup;
+import org.apache.ignite.internal.cluster.management.messages.CmgMessagesFactory;
+import org.apache.ignite.internal.cluster.management.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestComponent;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestComponent restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    private final CmgMessagesFactory msgFactory = new CmgMessagesFactory();
+
+    private final CompletableFuture<Collection<String>> metastorageNodes = new CompletableFuture<>();
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestComponent restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        messagingService.addMessageHandler(CmgMessageGroup.class, (msg, addr, correlationId) -> {

Review comment:
       Actually I will probably move this to `start` in the next PR that I'm working on - I'll probably need to check the vault state first




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r818724394



##########
File path: modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
##########
@@ -409,27 +417,55 @@ public ConfigurationRegistry clusterConfiguration() {
     }
 
     /**
-     * Returns client handler module.
+     * Returns the id of the current node.
      */
-    public ClientHandlerModule clientHandlerModule() {
-        return clientHandlerModule;
+    public String id() {
+        return clusterSvc.topologyService().localMember().id();
     }
 
     /**
-     * Returns the id of the current node.
+     * Returns the local address of REST endpoints.
      */
-    public String id() {
-        return clusterSvc.topologyService().localMember().id();
+    public NetworkAddress restAddress() {
+        return NetworkAddress.from(restModule.localAddress());
+    }
+
+    /**
+     * Returns the local address of the Thin Client.
+     */
+    public NetworkAddress clientAddress() {

Review comment:
       Added TODOs




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811734848



##########
File path: modules/join/pom.xml
##########
@@ -0,0 +1,111 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite-parent</artifactId>
+        <version>1</version>
+        <relativePath>../../parent/pom.xml</relativePath>
+    </parent>
+
+    <artifactId>ignite-join</artifactId>

Review comment:
       you are absolutely right, this is an artifact from earlier times, I forgot to rename it




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sanpwc commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sanpwc commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r812111017



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));
+
+                    raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)
+                            .whenComplete((service, e) -> {
+                                if (e == null) {
+                                    Peer leader = service.leader();
+
+                                    assert leader != null;

Review comment:
       Yep, I understand. I mean that one refreshLeader on client start with default timeout might not be enough. So I suggest adding few more iterations of refresgLeader.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810939247



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));

Review comment:
       I will update the document, thanks for noticing




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] rpuch commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
rpuch commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811675676



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> metastorageNodes = resolveNodes(clusterService, metaStorageNodeNames);

Review comment:
       I mean, the result of the resolving is not used in your code. The variable called `metastorageNodes` is even annotated as `@SuppressWarnings("unused")`. So it looks like this piece of code just resolves nodes to make sure their names are valid; hence the suggestion - make the intention to validate node names explicit.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sanpwc commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sanpwc commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811821019



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/rest/InitCommandHandler.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.join.rest;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import org.apache.ignite.internal.join.ClusterInitializer;
+import org.apache.ignite.internal.join.Leaders;
+import org.apache.ignite.internal.rest.ErrorResult;
+import org.apache.ignite.internal.rest.netty.RestApiHttpRequest;
+import org.apache.ignite.internal.rest.netty.RestApiHttpResponse;
+import org.apache.ignite.internal.rest.routes.RequestHandler;
+import org.apache.ignite.lang.IgniteLogger;
+
+/**
+ * REST handler for the {@link InitCommand}.
+ */
+public class InitCommandHandler implements RequestHandler {
+    private static final IgniteLogger log = IgniteLogger.forClass(InitCommandHandler.class);
+
+    private final ObjectMapper objectMapper = new ObjectMapper();
+
+    private final ClusterInitializer clusterInitializer;
+
+    public InitCommandHandler(ClusterInitializer clusterInitializer) {

Review comment:
       Nothing. Ok.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r818724666



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/rest/InitCommandHandler.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.join.rest;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import org.apache.ignite.internal.join.ClusterInitializer;
+import org.apache.ignite.internal.join.Leaders;
+import org.apache.ignite.internal.rest.ErrorResult;
+import org.apache.ignite.internal.rest.netty.RestApiHttpRequest;
+import org.apache.ignite.internal.rest.netty.RestApiHttpResponse;
+import org.apache.ignite.internal.rest.routes.RequestHandler;
+import org.apache.ignite.lang.IgniteLogger;
+
+/**
+ * REST handler for the {@link InitCommand}.
+ */
+public class InitCommandHandler implements RequestHandler {
+    private static final IgniteLogger log = IgniteLogger.forClass(InitCommandHandler.class);
+
+    private final ObjectMapper objectMapper = new ObjectMapper();
+
+    private final ClusterInitializer clusterInitializer;
+
+    public InitCommandHandler(ClusterInitializer clusterInitializer) {
+        this.clusterInitializer = clusterInitializer;
+    }
+
+    @Override
+    public CompletableFuture<RestApiHttpResponse> handle(RestApiHttpRequest request, RestApiHttpResponse response) {
+        try {
+            InitCommand command = readContent(request);
+
+            if (log.isInfoEnabled()) {
+                log.info(
+                        "Received init command:\n\tMeta Storage nodes: {}\n\tCMG nodes: {}",
+                        command.metaStorageNodes(),
+                        command.cmgNodes()
+                );
+            }
+
+            return clusterInitializer.initCluster(command.metaStorageNodes(), command.cmgNodes())
+                    .thenApply(leaders -> successResponse(response, leaders))

Review comment:
       removed this logic




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sanpwc commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sanpwc commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r820121959



##########
File path: modules/metastorage/pom.xml
##########
@@ -38,6 +38,11 @@
             <artifactId>ignite-configuration</artifactId>
         </dependency>
 
+        <dependency>

Review comment:
       Deprecated according to current implementation. Looks fine to me now.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810931696



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/rest/InitCommandHandler.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.join.rest;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import org.apache.ignite.internal.join.ClusterInitializer;
+import org.apache.ignite.internal.join.Leaders;
+import org.apache.ignite.internal.rest.ErrorResult;
+import org.apache.ignite.internal.rest.netty.RestApiHttpRequest;
+import org.apache.ignite.internal.rest.netty.RestApiHttpResponse;
+import org.apache.ignite.internal.rest.routes.RequestHandler;
+import org.apache.ignite.lang.IgniteLogger;
+
+/**
+ * REST handler for the {@link InitCommand}.
+ */
+public class InitCommandHandler implements RequestHandler {
+    private static final IgniteLogger log = IgniteLogger.forClass(InitCommandHandler.class);
+
+    private final ObjectMapper objectMapper = new ObjectMapper();
+
+    private final ClusterInitializer clusterInitializer;
+
+    public InitCommandHandler(ClusterInitializer clusterInitializer) {
+        this.clusterInitializer = clusterInitializer;
+    }
+
+    @Override
+    public CompletableFuture<RestApiHttpResponse> handle(RestApiHttpRequest request, RestApiHttpResponse response) {
+        try {
+            InitCommand command = readContent(request);
+
+            if (log.isInfoEnabled()) {
+                log.info(
+                        "Received init command:\n\tMeta Storage nodes: {}\n\tCMG nodes: {}",
+                        command.metaStorageNodes(),
+                        command.cmgNodes()
+                );
+            }
+
+            return clusterInitializer.initCluster(command.metaStorageNodes(), command.cmgNodes())
+                    .thenApply(leaders -> successResponse(response, leaders))

Review comment:
       > What's the point of returning leaders?
   
   It is not required by the business logic, I've added that for logging purposes, it may make debugging easier in the future. I think it's fine to have for now. If we are going to get rid of leaders, then this code can be changed accordingly.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sanpwc commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sanpwc commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811810598



##########
File path: modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
##########
@@ -409,27 +417,55 @@ public ConfigurationRegistry clusterConfiguration() {
     }
 
     /**
-     * Returns client handler module.
+     * Returns the id of the current node.
      */
-    public ClientHandlerModule clientHandlerModule() {
-        return clientHandlerModule;
+    public String id() {
+        return clusterSvc.topologyService().localMember().id();
     }
 
     /**
-     * Returns the id of the current node.
+     * Returns the local address of REST endpoints.
      */
-    public String id() {
-        return clusterSvc.topologyService().localMember().id();
+    public NetworkAddress restAddress() {
+        return NetworkAddress.from(restModule.localAddress());
+    }
+
+    /**
+     * Returns the local address of the Thin Client.
+     */
+    public NetworkAddress clientAddress() {

Review comment:
       Seems, we already have one  - IGNITE-15131. Let's just add TODO's here and there.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810945568



##########
File path: modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
##########
@@ -122,174 +115,233 @@
 
     /**
      * Future which will be completed with {@link IgniteUuid}, when aggregated watch will be successfully deployed. Can be resolved to
-     * {@link Optional#empty()} if no watch deployed at the moment.
-     */
-    private CompletableFuture<Optional<IgniteUuid>> deployFut = new CompletableFuture<>();
-
-    /**
-     * If true - all new watches will be deployed immediately.
+     * {@code null} if no watch deployed at the moment.
      *
-     * <p>If false - all new watches will be aggregated to one batch for further deploy by {@link MetaStorageManager#deployWatches()}
+     * <p>Multithreaded access is guarded by {@code this}.
      */
-    private boolean deployed;
-
-    /** Flag indicates if meta storage nodes were set on start. */
-    private boolean metaStorageNodesOnStart;
+    private CompletableFuture<IgniteUuid> deployFut = new CompletableFuture<>();
 
     /** Actual storage for the Metastorage. */
     private final KeyValueStorage storage;
 
     /** Busy lock to stop synchronously. */
     private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
 
-    /** Prevents double stopping the component. */
-    AtomicBoolean stopGuard = new AtomicBoolean();

Review comment:
       I've removed it because in this particular case the stop is performed under `synchronized`. But I'm ok with using `AtomicBoolean` everywhere for consistency.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] rpuch commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
rpuch commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811664900



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> metastorageNodes = resolveNodes(clusterService, metaStorageNodeNames);
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> cmgNodes = resolveNodes(clusterService, cmgNodeNames);
+
+            // TODO: init message should only be sent to metastorageNodes and cmgNodes respectively,
+            //  https://issues.apache.org/jira/browse/IGNITE-16471
+            Collection<ClusterNode> allMembers = clusterService.topologyService().allMembers();
+
+            var msgFactory = new InitMessagesFactory();
+
+            MetastorageInitMessage metaStorageInitMessage = msgFactory.metastorageInitMessage()
+                    .metastorageNodes(metaStorageNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> metaStorageInitFuture = invokeMessage(allMembers, metaStorageInitMessage);
+
+            CmgInitMessage cmgInitMessage = msgFactory.cmgInitMessage()
+                    .cmgNodes(cmgNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> cmgInitFuture = invokeMessage(allMembers, cmgInitMessage);
+
+            return metaStorageInitFuture
+                    .thenCombine(cmgInitFuture, Leaders::new)
+                    .whenComplete((leaders, e) -> {
+                        if (e != null) {
+                            log.error("Initialization failed, rolling back", e);
+
+                            CancelInitMessage cancelMessage = msgFactory.cancelInitMessage()
+                                    .reason(e.getMessage())
+                                    .build();
+
+                            cancelInit(allMembers, cancelMessage);

Review comment:
       Let's file a ticket (if it does not already exist)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sanpwc commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sanpwc commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r812112148



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/Utils.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.internal.join;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+
+/**
+ * Various utility methods.
+ */
+class Utils {
+    /**
+     * Resolves given node names (consistent IDs) into {@link ClusterNode}s.
+     *
+     * @param nodeNames node names.
+     * @return list of resolved {@code ClusterNode}s.
+     * @throws InitException if any of the given nodes are not present in the physical topology.
+     */
+    static List<ClusterNode> resolveNodes(ClusterService clusterService, Collection<String> nodeNames) {

Review comment:
       Not sure, but probably not, so we may create brand new))




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810923247



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/rest/InitCommandHandler.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.join.rest;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import org.apache.ignite.internal.join.ClusterInitializer;
+import org.apache.ignite.internal.join.Leaders;
+import org.apache.ignite.internal.rest.ErrorResult;
+import org.apache.ignite.internal.rest.netty.RestApiHttpRequest;
+import org.apache.ignite.internal.rest.netty.RestApiHttpResponse;
+import org.apache.ignite.internal.rest.routes.RequestHandler;
+import org.apache.ignite.lang.IgniteLogger;
+
+/**
+ * REST handler for the {@link InitCommand}.
+ */
+public class InitCommandHandler implements RequestHandler {
+    private static final IgniteLogger log = IgniteLogger.forClass(InitCommandHandler.class);
+
+    private final ObjectMapper objectMapper = new ObjectMapper();
+
+    private final ClusterInitializer clusterInitializer;
+
+    public InitCommandHandler(ClusterInitializer clusterInitializer) {
+        this.clusterInitializer = clusterInitializer;
+    }
+
+    @Override
+    public CompletableFuture<RestApiHttpResponse> handle(RestApiHttpRequest request, RestApiHttpResponse response) {
+        try {
+            InitCommand command = readContent(request);
+
+            if (log.isInfoEnabled()) {
+                log.info(
+                        "Received init command:\n\tMeta Storage nodes: {}\n\tCMG nodes: {}",
+                        command.metaStorageNodes(),
+                        command.cmgNodes()
+                );
+            }
+
+            return clusterInitializer.initCluster(command.metaStorageNodes(), command.cmgNodes())
+                    .thenApply(leaders -> successResponse(response, leaders))
+                    .exceptionally(e -> errorResponse(response, e));
+        } catch (Exception e) {
+            return CompletableFuture.completedFuture(errorResponse(response, e));
+        }
+    }
+
+    private InitCommand readContent(RestApiHttpRequest restApiHttpRequest) throws IOException {
+        ByteBuf content = restApiHttpRequest.request().content();
+
+        try (InputStream is = new ByteBufInputStream(content)) {
+            return objectMapper.readValue(is, InitCommand.class);
+        }
+    }
+
+    private static RestApiHttpResponse successResponse(RestApiHttpResponse response, Leaders leaders) {
+        if (log.isInfoEnabled()) {
+            log.info(
+                    "Init command executed successfully.\n\tMeta Storage leader ID: {}\n\tCMG leader ID: {}",
+                    leaders.metaStorageLeader(),
+                    leaders.cmgLeader()
+            );
+        }
+
+        response.json(Map.of(
+                "metaStorageLeaderId", leaders.metaStorageLeader(),
+                "cmgLeaderId", leaders.cmgLeader()
+        ));
+
+        return response;
+    }
+
+    private static RestApiHttpResponse errorResponse(RestApiHttpResponse response, Throwable e) {
+        if (e instanceof CompletionException) {
+            e = e.getCause();
+        }
+
+        log.error("Init command failure", e);
+
+        response.status(INTERNAL_SERVER_ERROR);
+        response.json(Map.of("error", new ErrorResult("APPLICATION_EXCEPTION", e.getMessage())));

Review comment:
       Sorry, what do you mean? Do you suggest creating a enum?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] rpuch commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
rpuch commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811849426



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/Utils.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.internal.join;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+
+/**
+ * Various utility methods.
+ */
+class Utils {

Review comment:
       Well, right now its name says nothing about the code that resides in it. At least now it is about resolving nodes, so now `NodeResolving` (or something similar) is the most adequate name. In the future, if other methods are added, the class can be renamed accordingly. Or the methods should be added to another class.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sanpwc commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sanpwc commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r812115376



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")

Review comment:
       Ок




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r818737948



##########
File path: modules/metastorage/pom.xml
##########
@@ -38,6 +38,11 @@
             <artifactId>ignite-configuration</artifactId>
         </dependency>
 
+        <dependency>

Review comment:
       Sorry, what do you mean by saying that `ignite-runner` should handle this message?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sanpwc commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sanpwc commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r820120884



##########
File path: modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.internal.cluster.management;
+
+import static io.netty.handler.codec.http.HttpHeaderValues.APPLICATION_JSON;
+import static org.apache.ignite.network.util.ClusterServiceUtils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.cluster.management.messages.CancelInitMessage;
+import org.apache.ignite.internal.cluster.management.messages.ClusterStateMessage;
+import org.apache.ignite.internal.cluster.management.messages.CmgInitMessage;
+import org.apache.ignite.internal.cluster.management.messages.CmgMessageGroup;
+import org.apache.ignite.internal.cluster.management.messages.CmgMessagesFactory;
+import org.apache.ignite.internal.cluster.management.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestComponent;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestComponent restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    private final CmgMessagesFactory msgFactory = new CmgMessagesFactory();
+
+    private final CompletableFuture<Collection<String>> metastorageNodes = new CompletableFuture<>();
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestComponent restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        messagingService.addMessageHandler(CmgMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    handleCancelInit((CancelInitMessage) msg);
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    handleInit((CmgInitMessage) msg, addr, correlationId);
+                } else if (msg instanceof ClusterStateMessage) {
+                    handleClusterState((ClusterStateMessage) msg);
+                }
+            } catch (Exception e) {
+                log.error("CMG message handling failed", e);
+
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                }
+            } finally {
+                busyLock.leaveBusy();
+            }
+        });
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group.
+     */
+    public void initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) throws NodeStoppingException {
+        if (!busyLock.enterBusy()) {
+            throw new NodeStoppingException();
+        }
+
+        try {
+            clusterInitializer.initCluster(metaStorageNodeNames, cmgNodeNames).get();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IgniteInternalException("Interrupted while initializing the cluster", e);
+        } catch (ExecutionException e) {
+            throw new IgniteInternalException("Unable to initialize the cluster", e.getCause());
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    private void handleInit(CmgInitMessage msg, NetworkAddress addr, long correlationId) throws NodeStoppingException {
+        List<ClusterNode> nodes = resolveNodes(clusterService, msg.cmgNodes());
+
+        raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)

Review comment:
       Could you please add todos for persisting cmgNodes and msNodes to Vault on init and corresponding cleanup on cancel?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r820871144



##########
File path: modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.internal.cluster.management;
+
+import static io.netty.handler.codec.http.HttpHeaderValues.APPLICATION_JSON;
+import static org.apache.ignite.network.util.ClusterServiceUtils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.cluster.management.messages.CancelInitMessage;
+import org.apache.ignite.internal.cluster.management.messages.ClusterStateMessage;
+import org.apache.ignite.internal.cluster.management.messages.CmgInitMessage;
+import org.apache.ignite.internal.cluster.management.messages.CmgMessageGroup;
+import org.apache.ignite.internal.cluster.management.messages.CmgMessagesFactory;
+import org.apache.ignite.internal.cluster.management.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestComponent;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestComponent restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    private final CmgMessagesFactory msgFactory = new CmgMessagesFactory();
+
+    private final CompletableFuture<Collection<String>> metastorageNodes = new CompletableFuture<>();
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestComponent restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        messagingService.addMessageHandler(CmgMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    handleCancelInit((CancelInitMessage) msg);
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    handleInit((CmgInitMessage) msg, addr, correlationId);
+                } else if (msg instanceof ClusterStateMessage) {
+                    handleClusterState((ClusterStateMessage) msg);
+                }
+            } catch (Exception e) {
+                log.error("CMG message handling failed", e);
+
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                }
+            } finally {
+                busyLock.leaveBusy();
+            }
+        });
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group.
+     */
+    public void initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) throws NodeStoppingException {
+        if (!busyLock.enterBusy()) {
+            throw new NodeStoppingException();
+        }
+
+        try {
+            clusterInitializer.initCluster(metaStorageNodeNames, cmgNodeNames).get();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IgniteInternalException("Interrupted while initializing the cluster", e);
+        } catch (ExecutionException e) {
+            throw new IgniteInternalException("Unable to initialize the cluster", e.getCause());
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    private void handleInit(CmgInitMessage msg, NetworkAddress addr, long correlationId) throws NodeStoppingException {
+        List<ClusterNode> nodes = resolveNodes(clusterService, msg.cmgNodes());
+
+        raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)
+                .whenComplete((service, e) -> {
+                    MessagingService messagingService = clusterService.messagingService();
+
+                    if (e == null) {
+                        ClusterNode leader = getLeader(service);
+
+                        ClusterNode thisNode = clusterService.topologyService().localMember();
+
+                        messagingService.respond(addr, successResponse(msgFactory), correlationId);
+
+                        if (leader.equals(thisNode)) {

Review comment:
       yes, I can't use that mechanism right now since it does not exist




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811246917



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/rest/InitCommandHandler.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.join.rest;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import org.apache.ignite.internal.join.ClusterInitializer;
+import org.apache.ignite.internal.join.Leaders;
+import org.apache.ignite.internal.rest.ErrorResult;
+import org.apache.ignite.internal.rest.netty.RestApiHttpRequest;
+import org.apache.ignite.internal.rest.netty.RestApiHttpResponse;
+import org.apache.ignite.internal.rest.routes.RequestHandler;
+import org.apache.ignite.lang.IgniteLogger;
+
+/**
+ * REST handler for the {@link InitCommand}.
+ */
+public class InitCommandHandler implements RequestHandler {
+    private static final IgniteLogger log = IgniteLogger.forClass(InitCommandHandler.class);
+
+    private final ObjectMapper objectMapper = new ObjectMapper();
+
+    private final ClusterInitializer clusterInitializer;
+
+    public InitCommandHandler(ClusterInitializer clusterInitializer) {
+        this.clusterInitializer = clusterInitializer;
+    }
+
+    @Override
+    public CompletableFuture<RestApiHttpResponse> handle(RestApiHttpRequest request, RestApiHttpResponse response) {
+        try {
+            InitCommand command = readContent(request);
+
+            if (log.isInfoEnabled()) {
+                log.info(
+                        "Received init command:\n\tMeta Storage nodes: {}\n\tCMG nodes: {}",
+                        command.metaStorageNodes(),
+                        command.cmgNodes()
+                );
+            }
+
+            return clusterInitializer.initCluster(command.metaStorageNodes(), command.cmgNodes())
+                    .thenApply(leaders -> successResponse(response, leaders))
+                    .exceptionally(e -> errorResponse(response, e));
+        } catch (Exception e) {
+            return CompletableFuture.completedFuture(errorResponse(response, e));
+        }
+    }
+
+    private InitCommand readContent(RestApiHttpRequest restApiHttpRequest) throws IOException {
+        ByteBuf content = restApiHttpRequest.request().content();
+
+        try (InputStream is = new ByteBufInputStream(content)) {
+            return objectMapper.readValue(is, InitCommand.class);

Review comment:
       I think we had a similar discussion before =) I don't think that it will block because `RestApiHttpRequest` contains a `FullHttpMessage`. Moreover I think that it is ok to block this (netty?) thread by reading from a `ByteBuf`, because that's what these threads are for, aren't they?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sk0x50 commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sk0x50 commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810627091



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/rest/InitCommandHandler.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.join.rest;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import org.apache.ignite.internal.join.ClusterInitializer;
+import org.apache.ignite.internal.join.Leaders;
+import org.apache.ignite.internal.rest.ErrorResult;
+import org.apache.ignite.internal.rest.netty.RestApiHttpRequest;
+import org.apache.ignite.internal.rest.netty.RestApiHttpResponse;
+import org.apache.ignite.internal.rest.routes.RequestHandler;
+import org.apache.ignite.lang.IgniteLogger;
+
+/**
+ * REST handler for the {@link InitCommand}.
+ */
+public class InitCommandHandler implements RequestHandler {
+    private static final IgniteLogger log = IgniteLogger.forClass(InitCommandHandler.class);
+
+    private final ObjectMapper objectMapper = new ObjectMapper();
+
+    private final ClusterInitializer clusterInitializer;
+
+    public InitCommandHandler(ClusterInitializer clusterInitializer) {

Review comment:
       I would like to see a meaningful Javadoc for internal APIs as well. However, the community has voted that Javadoc is absolutely required for public APIs (documentation of internal packages is up to contributor).
   Dev-list discussion can be found here: https://lists.apache.org/thread/y55sldhmr8mw8zj85x4jorvzb0j0djmd




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811316630



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));
+
+                    raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)
+                            .whenComplete((service, e) -> {
+                                if (e == null) {
+                                    Peer leader = service.leader();
+
+                                    assert leader != null;
+
+                                    messagingService.respond(addr, leaderElectedResponse(msgFactory, leader), correlationId);
+                                } else {
+                                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                                }
+                            });
+                }
+            } catch (Exception e) {
+                log.error("Exception when initializing the CMG", e);
+
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                }
+            } finally {
+                busyLock.leaveBusy();
+            }
+        });
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group.
+     */
+    public Leaders initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) throws NodeStoppingException {

Review comment:
       Same here, cluster initialization can be moved to a separate component but is managing yet another component worth it? Especially since we don't have any dependency injection mechanism...




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sanpwc commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sanpwc commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811807811



##########
File path: modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
##########
@@ -409,27 +417,55 @@ public ConfigurationRegistry clusterConfiguration() {
     }
 
     /**
-     * Returns client handler module.
+     * Returns the id of the current node.
      */
-    public ClientHandlerModule clientHandlerModule() {
-        return clientHandlerModule;
+    public String id() {
+        return clusterSvc.topologyService().localMember().id();
     }
 
     /**
-     * Returns the id of the current node.
+     * Returns the local address of REST endpoints.
      */
-    public String id() {
-        return clusterSvc.topologyService().localMember().id();
+    public NetworkAddress restAddress() {

Review comment:
       Yep, separate ticket with corresponding TODO's looks fine to me.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sanpwc commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sanpwc commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r812114351



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));
+
+                    raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)

Review comment:
       yep




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] rpuch commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
rpuch commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811850104



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/rest/InitCommandHandler.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.join.rest;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import org.apache.ignite.internal.join.ClusterInitializer;
+import org.apache.ignite.internal.join.Leaders;
+import org.apache.ignite.internal.rest.ErrorResult;
+import org.apache.ignite.internal.rest.netty.RestApiHttpRequest;
+import org.apache.ignite.internal.rest.netty.RestApiHttpResponse;
+import org.apache.ignite.internal.rest.routes.RequestHandler;
+import org.apache.ignite.lang.IgniteLogger;
+
+/**
+ * REST handler for the {@link InitCommand}.
+ */
+public class InitCommandHandler implements RequestHandler {
+    private static final IgniteLogger log = IgniteLogger.forClass(InitCommandHandler.class);
+
+    private final ObjectMapper objectMapper = new ObjectMapper();
+
+    private final ClusterInitializer clusterInitializer;
+
+    public InitCommandHandler(ClusterInitializer clusterInitializer) {
+        this.clusterInitializer = clusterInitializer;
+    }
+
+    @Override
+    public CompletableFuture<RestApiHttpResponse> handle(RestApiHttpRequest request, RestApiHttpResponse response) {
+        try {
+            InitCommand command = readContent(request);
+
+            if (log.isInfoEnabled()) {
+                log.info(
+                        "Received init command:\n\tMeta Storage nodes: {}\n\tCMG nodes: {}",
+                        command.metaStorageNodes(),
+                        command.cmgNodes()
+                );
+            }
+
+            return clusterInitializer.initCluster(command.metaStorageNodes(), command.cmgNodes())
+                    .thenApply(leaders -> successResponse(response, leaders))
+                    .exceptionally(e -> errorResponse(response, e));
+        } catch (Exception e) {
+            return CompletableFuture.completedFuture(errorResponse(response, e));
+        }
+    }
+
+    private InitCommand readContent(RestApiHttpRequest restApiHttpRequest) throws IOException {
+        ByteBuf content = restApiHttpRequest.request().content();
+
+        try (InputStream is = new ByteBufInputStream(content)) {
+            return objectMapper.readValue(is, InitCommand.class);

Review comment:
       Yep, I just wanted to be sure its the same case :)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sanpwc commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sanpwc commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r812041209



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {

Review comment:
       There are two main points to keep constructors simple:
   - Exception handling. If something went wrong during start, there's always a component provided by constructor as an entry point for cleanup or recovery. In other words there's always an object on which to call close
   ```
   IgniteComponent c = new Component() // No tough logic goes here;
   try {
   c.start();
   } catch(Exception e) {
   c.close(); // Cleaning all resources that might be instantiated; 
   }
   ```
   - In general there are no guaranties that un-started components will be able to register listeners etc. I'd rather prefer simple bifurcation of full-fledged started component, and only-instantiated which is only suitable for calling the start method. That'll also ease migration to DI a bit.
   
   > If message handlers are registered in "start", then the network layer is already running, and we can miss some messages. 
   
   Seems that currently it's not possible. If it is, let's introduce onBeforeStart() or similar.
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811242574



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));
+
+                    raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)
+                            .whenComplete((service, e) -> {
+                                if (e == null) {
+                                    Peer leader = service.leader();
+
+                                    assert leader != null;
+
+                                    messagingService.respond(addr, leaderElectedResponse(msgFactory, leader), correlationId);
+                                } else {
+                                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                                }
+                            });
+                }
+            } catch (Exception e) {
+                log.error("Exception when initializing the CMG", e);
+
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                }
+            } finally {
+                busyLock.leaveBusy();
+            }
+        });
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group.
+     */
+    public Leaders initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) throws NodeStoppingException {
+        if (!busyLock.enterBusy()) {

Review comment:
       yes, this lock should enclose every operation, it's only needed for a graceful component stop. Or do you mean anything else?

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));
+
+                    raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)
+                            .whenComplete((service, e) -> {
+                                if (e == null) {
+                                    Peer leader = service.leader();
+
+                                    assert leader != null;
+
+                                    messagingService.respond(addr, leaderElectedResponse(msgFactory, leader), correlationId);
+                                } else {
+                                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                                }
+                            });
+                }
+            } catch (Exception e) {
+                log.error("Exception when initializing the CMG", e);
+
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                }
+            } finally {
+                busyLock.leaveBusy();
+            }
+        });
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group.
+     */
+    public Leaders initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) throws NodeStoppingException {
+        if (!busyLock.enterBusy()) {

Review comment:
       yes, this lock should enclose every operation, it's only needed for a graceful component stop. Or do you mean something else?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] rpuch commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
rpuch commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811853397



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));
+
+                    raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)
+                            .whenComplete((service, e) -> {
+                                if (e == null) {
+                                    Peer leader = service.leader();
+
+                                    assert leader != null;
+
+                                    messagingService.respond(addr, leaderElectedResponse(msgFactory, leader), correlationId);
+                                } else {
+                                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                                }
+                            });
+                }
+            } catch (Exception e) {
+                log.error("Exception when initializing the CMG", e);
+
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                }
+            } finally {
+                busyLock.leaveBusy();
+            }
+        });
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group.
+     */
+    public Leaders initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) throws NodeStoppingException {
+        if (!busyLock.enterBusy()) {
+            throw new NodeStoppingException();
+        }
+
+        try {
+            return clusterInitializer.initCluster(metaStorageNodeNames, cmgNodeNames).get();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IgniteInternalException("Interrupted while initializing the cluster", e);
+        } catch (ExecutionException e) {
+            throw new IgniteInternalException("Unable to initialize the cluster", e.getCause());
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    private static NetworkMessage errorResponse(InitMessagesFactory msgFactory, Throwable e) {
+        log.error("Exception when starting the CMG", e);
+
+        return msgFactory.initErrorMessage()
+                .cause(e.getMessage())
+                .build();
+    }
+
+    private NetworkMessage leaderElectedResponse(InitMessagesFactory msgFactory, Peer leader) {
+        ClusterNode leaderNode = clusterService.topologyService().getByAddress(leader.address());
+
+        assert leaderNode != null;
+
+        log.info("CMG leader elected: " + leaderNode.name());
+
+        return msgFactory.leaderElectedMessage()
+                .leaderName(leaderNode.name())
+                .build();
+    }
+
+    @Override
+    public void start() {
+        restModule.addRoute(new Route(

Review comment:
       The thing that bothers me here is that, judging by the class name, it should be responsible only for a part of the init process (namely, the CMG init), but in reality it also takes part in other init activities which are not part of the CMG init. For someone who will (in the future) search for this REST handler it will be very surprising to finally find it here.
   If you don't like the idea of splitting the class, how about renaming it so that it's not CMG-specific anymore?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810942406



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")

Review comment:
       please see my answer to a similar comment by Roman: https://github.com/apache/ignite-3/pull/659#discussion_r810154756




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811317417



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> metastorageNodes = resolveNodes(clusterService, metaStorageNodeNames);
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> cmgNodes = resolveNodes(clusterService, cmgNodeNames);
+
+            // TODO: init message should only be sent to metastorageNodes and cmgNodes respectively,
+            //  https://issues.apache.org/jira/browse/IGNITE-16471
+            Collection<ClusterNode> allMembers = clusterService.topologyService().allMembers();
+
+            var msgFactory = new InitMessagesFactory();
+
+            MetastorageInitMessage metaStorageInitMessage = msgFactory.metastorageInitMessage()
+                    .metastorageNodes(metaStorageNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> metaStorageInitFuture = invokeMessage(allMembers, metaStorageInitMessage);
+
+            CmgInitMessage cmgInitMessage = msgFactory.cmgInitMessage()
+                    .cmgNodes(cmgNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> cmgInitFuture = invokeMessage(allMembers, cmgInitMessage);
+
+            return metaStorageInitFuture
+                    .thenCombine(cmgInitFuture, Leaders::new)
+                    .whenComplete((leaders, e) -> {
+                        if (e != null) {
+                            log.error("Initialization failed, rolling back", e);
+
+                            CancelInitMessage cancelMessage = msgFactory.cancelInitMessage()
+                                    .reason(e.getMessage())
+                                    .build();
+
+                            cancelInit(allMembers, cancelMessage);

Review comment:
       I think it may be stuck and this situation will require manual intervention. At least I don't know what else can we do here...




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810938369



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {

Review comment:
       > The rule of thumb is to prefer start() to constructor for registering message handlers
   
   I would argue with that, because I actually think that it is better to register handlers in constructors instead. Here are my reasons:
   
   If message handlers are registered in "start", then the network layer is already running, and we can miss some messages. Yes, the node might not have completely started yet, but at least we can either block or return a meaningful error. Otherwise these messages will simply be dropped. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810941166



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/Utils.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.internal.join;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+
+/**
+ * Various utility methods.
+ */
+class Utils {
+    /**
+     * Resolves given node names (consistent IDs) into {@link ClusterNode}s.
+     *
+     * @param nodeNames node names.
+     * @return list of resolved {@code ClusterNode}s.
+     * @throws InitException if any of the given nodes are not present in the physical topology.
+     */
+    static List<ClusterNode> resolveNodes(ClusterService clusterService, Collection<String> nodeNames) {

Review comment:
       Do we have any kind of suitable classes for that?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811748259



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/messages/InitMessageGroup.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.internal.join.messages;
+
+import org.apache.ignite.network.annotations.MessageGroup;
+
+/**
+ * Message Group for cluster initialization and CMG management.
+ */
+@MessageGroup(groupType = 6, groupName = "InitMessages")
+public class InitMessageGroup {
+    /**
+     * Message type for {@link CmgInitMessage}.
+     */
+    public static final short CMG_INIT = 1;
+
+    /**
+     * Message type for {@link MetastorageInitMessage}.
+     */
+    public static final short METASTORAGE_INIT = 2;
+
+    /**
+     * Message type for {@link LeaderElectedMessage}.
+     */
+    public static final short LEADER_ELECTED = 3;

Review comment:
       I've renamed it to `INIT_COMPLETE` to be more similar with `INIT_ERROR`. Is that ok?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810917426



##########
File path: examples/src/integrationTest/java/org/apache/ignite/example/sql/jdbc/ItSqlExamplesTest.java
##########
@@ -17,35 +17,22 @@
 
 package org.apache.ignite.example.sql.jdbc;
 
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.example.AbstractExamplesTest;
 import org.apache.ignite.example.ExampleTestUtils;
-import org.apache.ignite.internal.testframework.WorkDirectory;
-import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.extension.ExtendWith;
 
 /**
  * These tests check that all SQL JDBC examples pass correctly.
  */
-@ExtendWith(WorkDirectoryExtension.class)
-public class SqlExamplesTest {
-    /** Empty argument to invoke an example. */
-    protected static final String[] EMPTY_ARGS = new String[0];
-
+public class ItSqlExamplesTest extends AbstractExamplesTest {
     /**
      * Runs SqlJdbcExample and checks its output.
      *
      * @throws Exception If failed.
      */
     @Test
     public void testSqlJdbcExample() throws Exception {
-        ExampleTestUtils.assertConsoleOutputContains(SqlJdbcExample::main, EMPTY_ARGS,
+        ExampleTestUtils.assertConsoleOutputContains(SqlJdbcExample::main, new String[0],

Review comment:
       It's just shorter in this particular case. Previously every test had this constant, I think it's a bit too much




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810940603



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));
+
+                    raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)
+                            .whenComplete((service, e) -> {
+                                if (e == null) {
+                                    Peer leader = service.leader();
+
+                                    assert leader != null;

Review comment:
       >  I mean that if service.leader is null after service creation we should call refreshLeader that has retry logic inside.
   
   When calling `prepareRaftGroup`, `refreshLeader` is called automatically, so this assertion is just a safety check. If the leader has not been elected, then the whole future will fail, and we should not get here.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810942406



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")

Review comment:
       please see my answer to a similar comment by Roman 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811319059



##########
File path: examples/src/integrationTest/java/org/apache/ignite/example/AbstractExamplesTest.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.example;
+
+import java.nio.file.Path;
+import java.util.List;
+import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.testframework.WorkDirectory;
+import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Base class for creating tests for examples.
+ */
+@ExtendWith(WorkDirectoryExtension.class)
+public abstract class AbstractExamplesTest {
+    private static final String TEST_NODE_NAME = "ignite-node";
+
+    /**
+     * Starts a node.
+     *
+     * @param workDir Work directory for the started node. Must not be {@code null}.
+     */
+    @BeforeEach
+    public void startNode(@WorkDirectory Path workDir) throws Exception {
+        IgniteImpl ignite = (IgniteImpl) IgnitionManager.start(
+                TEST_NODE_NAME,
+                Path.of("config", "ignite-config.json"),
+                workDir,
+                null
+        );
+
+        ignite.init(List.of(ignite.name()));

Review comment:
       Here's the ticket: https://issues.apache.org/jira/browse/IGNITE-16607




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sanpwc commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sanpwc commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r820120611



##########
File path: modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.internal.cluster.management;
+
+import static io.netty.handler.codec.http.HttpHeaderValues.APPLICATION_JSON;
+import static org.apache.ignite.network.util.ClusterServiceUtils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.cluster.management.messages.CancelInitMessage;
+import org.apache.ignite.internal.cluster.management.messages.ClusterStateMessage;
+import org.apache.ignite.internal.cluster.management.messages.CmgInitMessage;
+import org.apache.ignite.internal.cluster.management.messages.CmgMessageGroup;
+import org.apache.ignite.internal.cluster.management.messages.CmgMessagesFactory;
+import org.apache.ignite.internal.cluster.management.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestComponent;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestComponent restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    private final CmgMessagesFactory msgFactory = new CmgMessagesFactory();
+
+    private final CompletableFuture<Collection<String>> metastorageNodes = new CompletableFuture<>();
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestComponent restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        messagingService.addMessageHandler(CmgMessageGroup.class, (msg, addr, correlationId) -> {

Review comment:
       I still favor registering message handlers within start().




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r818729702



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));
+
+                    raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)
+                            .whenComplete((service, e) -> {
+                                if (e == null) {
+                                    Peer leader = service.leader();
+
+                                    assert leader != null;

Review comment:
       why? Doesn't `refreshLeader` already perform several attempts?

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/Utils.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.internal.join;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+
+/**
+ * Various utility methods.
+ */
+class Utils {
+    /**
+     * Resolves given node names (consistent IDs) into {@link ClusterNode}s.
+     *
+     * @param nodeNames node names.
+     * @return list of resolved {@code ClusterNode}s.
+     * @throws InitException if any of the given nodes are not present in the physical topology.
+     */
+    static List<ClusterNode> resolveNodes(ClusterService clusterService, Collection<String> nodeNames) {

Review comment:
       Moved




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r818737948



##########
File path: modules/metastorage/pom.xml
##########
@@ -38,6 +38,11 @@
             <artifactId>ignite-configuration</artifactId>
         </dependency>
 
+        <dependency>

Review comment:
       Sorry, what do you mean?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r818750102



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {

Review comment:
       > Exception handling
   
   I don't think this is applicable to registering listeners
   
   > In general there are no guaranties that un-started components will be able to register listeners etc. I'd rather prefer simple bifurcation of full-fledged started component, and only-instantiated which is only suitable for calling the start method. That'll also ease migration to DI a bit.
   
   Why and how?
   
   > Seems that currently it's not possible.
   
   I think that it is still possible, since `ClusterService` is started before all other components 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r820870766



##########
File path: modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.internal.cluster.management;
+
+import static io.netty.handler.codec.http.HttpHeaderValues.APPLICATION_JSON;
+import static org.apache.ignite.network.util.ClusterServiceUtils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.cluster.management.messages.CancelInitMessage;
+import org.apache.ignite.internal.cluster.management.messages.ClusterStateMessage;
+import org.apache.ignite.internal.cluster.management.messages.CmgInitMessage;
+import org.apache.ignite.internal.cluster.management.messages.CmgMessageGroup;
+import org.apache.ignite.internal.cluster.management.messages.CmgMessagesFactory;
+import org.apache.ignite.internal.cluster.management.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestComponent;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestComponent restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    private final CmgMessagesFactory msgFactory = new CmgMessagesFactory();
+
+    private final CompletableFuture<Collection<String>> metastorageNodes = new CompletableFuture<>();
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestComponent restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        messagingService.addMessageHandler(CmgMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    handleCancelInit((CancelInitMessage) msg);
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    handleInit((CmgInitMessage) msg, addr, correlationId);
+                } else if (msg instanceof ClusterStateMessage) {
+                    handleClusterState((ClusterStateMessage) msg);
+                }
+            } catch (Exception e) {
+                log.error("CMG message handling failed", e);
+
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                }
+            } finally {
+                busyLock.leaveBusy();
+            }
+        });
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group.
+     */
+    public void initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) throws NodeStoppingException {
+        if (!busyLock.enterBusy()) {
+            throw new NodeStoppingException();
+        }
+
+        try {
+            clusterInitializer.initCluster(metaStorageNodeNames, cmgNodeNames).get();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IgniteInternalException("Interrupted while initializing the cluster", e);
+        } catch (ExecutionException e) {
+            throw new IgniteInternalException("Unable to initialize the cluster", e.getCause());
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    private void handleInit(CmgInitMessage msg, NetworkAddress addr, long correlationId) throws NodeStoppingException {
+        List<ClusterNode> nodes = resolveNodes(clusterService, msg.cmgNodes());
+
+        raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)

Review comment:
       > Could you please add todos for persisting cmgNodes and msNodes to Vault on init and corresponding cleanup on cancel?
   
   I've already implemented that in the next PR that I'm working on. By the way, why do we need to persist `msNodes` as well? Won't those be persisted as part of the CMG state?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sk0x50 commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sk0x50 commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810124243



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/InitException.java
##########
@@ -15,8 +15,17 @@
  * limitations under the License.
  */
 
+package org.apache.ignite.internal.join;
+
 /**
- * Configuration schemas for Cluster node.
+ * Exception thrown when cluster initialization fails for some reason.
  */
+public class InitException extends RuntimeException {

Review comment:
       It seems to me, this exception should extend `IgniteInternalException` instead of `RuntimeException`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810943984



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));
+
+                    raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)

Review comment:
       yes, I wanted to do that in a future PR, is that ok?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811730630



##########
File path: modules/cli/src/integrationTest/java/org/apache/ignite/cli/ItConfigCommandTest.java
##########
@@ -32,74 +32,50 @@
 import io.micronaut.context.ApplicationContext;
 import io.micronaut.context.env.Environment;
 import java.io.ByteArrayOutputStream;
-import java.io.IOException;
 import java.io.PrintWriter;
-import java.net.ServerSocket;
 import java.nio.file.Path;
 import net.minidev.json.JSONObject;
 import net.minidev.json.JSONValue;
-import org.apache.ignite.Ignite;
 import org.apache.ignite.IgnitionManager;
 import org.apache.ignite.cli.spec.IgniteCliSpec;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.testframework.WorkDirectory;
+import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 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 org.junit.jupiter.api.extension.ExtendWith;
 import picocli.CommandLine;
 
 /**
  * Integration test for {@code ignite config} commands.
  */
+@ExtendWith(WorkDirectoryExtension.class)
 public class ItConfigCommandTest extends AbstractCliTest {
     /** DI context. */
     private ApplicationContext ctx;
 
     /** stderr. */
-    private ByteArrayOutputStream err;
+    private final ByteArrayOutputStream err = new ByteArrayOutputStream();
 
     /** stdout. */
-    private ByteArrayOutputStream out;
-
-    /** Port for REST communication. */
-    private int restPort;
-
-    /** Port for thin client communication. */
-    private int clientPort;
-
-    /** Network port. */
-    private int networkPort;
+    private final ByteArrayOutputStream out = new ByteArrayOutputStream();
 
     /** Node. */
-    private Ignite node;
+    private IgniteImpl node;
 
     @BeforeEach
-    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.

Review comment:
       I would like to avoid fixing this ticket in this PR




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810934526



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/rest/InitCommandHandler.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.join.rest;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import org.apache.ignite.internal.join.ClusterInitializer;
+import org.apache.ignite.internal.join.Leaders;
+import org.apache.ignite.internal.rest.ErrorResult;
+import org.apache.ignite.internal.rest.netty.RestApiHttpRequest;
+import org.apache.ignite.internal.rest.netty.RestApiHttpResponse;
+import org.apache.ignite.internal.rest.routes.RequestHandler;
+import org.apache.ignite.lang.IgniteLogger;
+
+/**
+ * REST handler for the {@link InitCommand}.
+ */
+public class InitCommandHandler implements RequestHandler {
+    private static final IgniteLogger log = IgniteLogger.forClass(InitCommandHandler.class);
+
+    private final ObjectMapper objectMapper = new ObjectMapper();
+
+    private final ClusterInitializer clusterInitializer;
+
+    public InitCommandHandler(ClusterInitializer clusterInitializer) {

Review comment:
       This class already has a javadoc, what's the point of having a javadoc on a constructor with a single parameter? What meaningful things can I write there?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810928807



##########
File path: modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
##########
@@ -409,27 +417,55 @@ public ConfigurationRegistry clusterConfiguration() {
     }
 
     /**
-     * Returns client handler module.
+     * Returns the id of the current node.
      */
-    public ClientHandlerModule clientHandlerModule() {
-        return clientHandlerModule;
+    public String id() {
+        return clusterSvc.topologyService().localMember().id();
     }
 
     /**
-     * Returns the id of the current node.
+     * Returns the local address of REST endpoints.
      */
-    public String id() {
-        return clusterSvc.topologyService().localMember().id();
+    public NetworkAddress restAddress() {

Review comment:
       I agree, but I think it will make this PR even larger. Do you want me to create a ticket?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811770436



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> metastorageNodes = resolveNodes(clusterService, metaStorageNodeNames);
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> cmgNodes = resolveNodes(clusterService, cmgNodeNames);
+
+            // TODO: init message should only be sent to metastorageNodes and cmgNodes respectively,
+            //  https://issues.apache.org/jira/browse/IGNITE-16471
+            Collection<ClusterNode> allMembers = clusterService.topologyService().allMembers();
+
+            var msgFactory = new InitMessagesFactory();
+
+            MetastorageInitMessage metaStorageInitMessage = msgFactory.metastorageInitMessage()
+                    .metastorageNodes(metaStorageNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> metaStorageInitFuture = invokeMessage(allMembers, metaStorageInitMessage);
+
+            CmgInitMessage cmgInitMessage = msgFactory.cmgInitMessage()
+                    .cmgNodes(cmgNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> cmgInitFuture = invokeMessage(allMembers, cmgInitMessage);
+
+            return metaStorageInitFuture
+                    .thenCombine(cmgInitFuture, Leaders::new)
+                    .whenComplete((leaders, e) -> {
+                        if (e != null) {
+                            log.error("Initialization failed, rolling back", e);
+
+                            CancelInitMessage cancelMessage = msgFactory.cancelInitMessage()
+                                    .reason(e.getMessage())
+                                    .build();
+
+                            cancelInit(allMembers, cancelMessage);

Review comment:
       What will this ticket be about?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r818724049



##########
File path: modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
##########
@@ -409,27 +417,55 @@ public ConfigurationRegistry clusterConfiguration() {
     }
 
     /**
-     * Returns client handler module.
+     * Returns the id of the current node.
      */
-    public ClientHandlerModule clientHandlerModule() {
-        return clientHandlerModule;
+    public String id() {
+        return clusterSvc.topologyService().localMember().id();
     }
 
     /**
-     * Returns the id of the current node.
+     * Returns the local address of REST endpoints.
      */
-    public String id() {
-        return clusterSvc.topologyService().localMember().id();
+    public NetworkAddress restAddress() {

Review comment:
       Added TODOs




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sanpwc commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sanpwc commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r820124682



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {

Review comment:
       > I don't think this is applicable to registering listeners
   Why not? It's still possible to get an exception on listeners registering. My main point is to keep constructors as simple as possible moving all logic to start() or beforeStart() if it's required.
   >Why and how?
   Well, because it's not declared anywhere. I actually believe that concept of fully-fledged component after corresponding start is still simplier to use and understand.
   >I think that it is still possible, since ClusterService is started before all other components
   I mean, that currently there are no such cases.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] rpuch commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
rpuch commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811880695



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));
+
+                    raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)
+                            .whenComplete((service, e) -> {
+                                if (e == null) {
+                                    Peer leader = service.leader();
+
+                                    assert leader != null;
+
+                                    messagingService.respond(addr, leaderElectedResponse(msgFactory, leader), correlationId);
+                                } else {
+                                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                                }
+                            });
+                }
+            } catch (Exception e) {
+                log.error("Exception when initializing the CMG", e);
+
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                }
+            } finally {
+                busyLock.leaveBusy();

Review comment:
       It turns out this is only needed to forbid any method execution when the component is stopping. The code does not communicate this intention too clearly, so it makes sense to improve this part (in all components, as it is done similarly there). This should not be changed in this PR, we'll discuss with the team first and, possible, a new ticket will be filed for this refactoring.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811242574



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));
+
+                    raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)
+                            .whenComplete((service, e) -> {
+                                if (e == null) {
+                                    Peer leader = service.leader();
+
+                                    assert leader != null;
+
+                                    messagingService.respond(addr, leaderElectedResponse(msgFactory, leader), correlationId);
+                                } else {
+                                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                                }
+                            });
+                }
+            } catch (Exception e) {
+                log.error("Exception when initializing the CMG", e);
+
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                }
+            } finally {
+                busyLock.leaveBusy();
+            }
+        });
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group.
+     */
+    public Leaders initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) throws NodeStoppingException {
+        if (!busyLock.enterBusy()) {

Review comment:
       yes, this lock should enclose every operation, it's only needed for a graceful component stop




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811243478



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/CmgRaftGroupListener.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.internal.join;
+
+import java.nio.file.Path;
+import java.util.Iterator;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+import org.apache.ignite.raft.client.Command;
+import org.apache.ignite.raft.client.ReadCommand;
+import org.apache.ignite.raft.client.WriteCommand;
+import org.apache.ignite.raft.client.service.CommandClosure;
+import org.apache.ignite.raft.client.service.RaftGroupListener;
+import org.jetbrains.annotations.Nullable;
+
+// TODO: implement listener, https://issues.apache.org/jira/browse/IGNITE-16471
+class CmgRaftGroupListener implements RaftGroupListener {
+    @Override
+    public void onRead(Iterator<CommandClosure<ReadCommand>> iterator) {
+
+    }
+
+    @Override
+    public void onWrite(Iterator<CommandClosure<WriteCommand>> iterator) {
+
+    }
+
+    @Override
+    public void onSnapshotSave(Path path, Consumer<Throwable> doneClo) {
+
+    }
+
+    @Override
+    public boolean onSnapshotLoad(Path path) {
+        return false;
+    }
+
+    @Override
+    public void onShutdown() {
+
+    }
+
+    @Override
+    public @Nullable CompletableFuture<Void> onBeforeApply(Command command) {
+        return null;

Review comment:
       agree, but this is an implementation of an already existing interface




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sanpwc commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sanpwc commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811819472



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/rest/InitCommandHandler.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.join.rest;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import org.apache.ignite.internal.join.ClusterInitializer;
+import org.apache.ignite.internal.join.Leaders;
+import org.apache.ignite.internal.rest.ErrorResult;
+import org.apache.ignite.internal.rest.netty.RestApiHttpRequest;
+import org.apache.ignite.internal.rest.netty.RestApiHttpResponse;
+import org.apache.ignite.internal.rest.routes.RequestHandler;
+import org.apache.ignite.lang.IgniteLogger;
+
+/**
+ * REST handler for the {@link InitCommand}.
+ */
+public class InitCommandHandler implements RequestHandler {
+    private static final IgniteLogger log = IgniteLogger.forClass(InitCommandHandler.class);
+
+    private final ObjectMapper objectMapper = new ObjectMapper();
+
+    private final ClusterInitializer clusterInitializer;
+
+    public InitCommandHandler(ClusterInitializer clusterInitializer) {
+        this.clusterInitializer = clusterInitializer;
+    }
+
+    @Override
+    public CompletableFuture<RestApiHttpResponse> handle(RestApiHttpRequest request, RestApiHttpResponse response) {
+        try {
+            InitCommand command = readContent(request);
+
+            if (log.isInfoEnabled()) {
+                log.info(
+                        "Received init command:\n\tMeta Storage nodes: {}\n\tCMG nodes: {}",
+                        command.metaStorageNodes(),
+                        command.cmgNodes()
+                );
+            }
+
+            return clusterInitializer.initCluster(command.metaStorageNodes(), command.cmgNodes())
+                    .thenApply(leaders -> successResponse(response, leaders))
+                    .exceptionally(e -> errorResponse(response, e));
+        } catch (Exception e) {
+            return CompletableFuture.completedFuture(errorResponse(response, e));
+        }
+    }
+
+    private InitCommand readContent(RestApiHttpRequest restApiHttpRequest) throws IOException {
+        ByteBuf content = restApiHttpRequest.request().content();
+
+        try (InputStream is = new ByteBufInputStream(content)) {
+            return objectMapper.readValue(is, InitCommand.class);
+        }
+    }
+
+    private static RestApiHttpResponse successResponse(RestApiHttpResponse response, Leaders leaders) {
+        if (log.isInfoEnabled()) {
+            log.info(
+                    "Init command executed successfully.\n\tMeta Storage leader ID: {}\n\tCMG leader ID: {}",
+                    leaders.metaStorageLeader(),
+                    leaders.cmgLeader()
+            );
+        }
+
+        response.json(Map.of(
+                "metaStorageLeaderId", leaders.metaStorageLeader(),
+                "cmgLeaderId", leaders.cmgLeader()
+        ));
+
+        return response;
+    }
+
+    private static RestApiHttpResponse errorResponse(RestApiHttpResponse response, Throwable e) {
+        if (e instanceof CompletionException) {
+            e = e.getCause();

Review comment:
       Well, I mean to say that it's only valid to throw either public ignite exception or common java ones. In other words, we should prevent returning internal exceptions through our public api. But, I've missed that you've actually wrap it with APPLICATION_EXCEPTION 
   `response.json(Map.of("error", new ErrorResult("APPLICATION_EXCEPTION", e.getMessage())));`
   My fault.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sanpwc commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sanpwc commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r820121313



##########
File path: modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.internal.cluster.management;
+
+import static io.netty.handler.codec.http.HttpHeaderValues.APPLICATION_JSON;
+import static org.apache.ignite.network.util.ClusterServiceUtils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.cluster.management.messages.CancelInitMessage;
+import org.apache.ignite.internal.cluster.management.messages.ClusterStateMessage;
+import org.apache.ignite.internal.cluster.management.messages.CmgInitMessage;
+import org.apache.ignite.internal.cluster.management.messages.CmgMessageGroup;
+import org.apache.ignite.internal.cluster.management.messages.CmgMessagesFactory;
+import org.apache.ignite.internal.cluster.management.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestComponent;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestComponent restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    private final CmgMessagesFactory msgFactory = new CmgMessagesFactory();
+
+    private final CompletableFuture<Collection<String>> metastorageNodes = new CompletableFuture<>();
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestComponent restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        messagingService.addMessageHandler(CmgMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    handleCancelInit((CancelInitMessage) msg);
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    handleInit((CmgInitMessage) msg, addr, correlationId);
+                } else if (msg instanceof ClusterStateMessage) {
+                    handleClusterState((ClusterStateMessage) msg);
+                }
+            } catch (Exception e) {
+                log.error("CMG message handling failed", e);
+
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                }
+            } finally {
+                busyLock.leaveBusy();
+            }
+        });
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group.
+     */
+    public void initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) throws NodeStoppingException {
+        if (!busyLock.enterBusy()) {
+            throw new NodeStoppingException();
+        }
+
+        try {
+            clusterInitializer.initCluster(metaStorageNodeNames, cmgNodeNames).get();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IgniteInternalException("Interrupted while initializing the cluster", e);
+        } catch (ExecutionException e) {
+            throw new IgniteInternalException("Unable to initialize the cluster", e.getCause());
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    private void handleInit(CmgInitMessage msg, NetworkAddress addr, long correlationId) throws NodeStoppingException {
+        List<ClusterNode> nodes = resolveNodes(clusterService, msg.cmgNodes());
+
+        raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)
+                .whenComplete((service, e) -> {
+                    MessagingService messagingService = clusterService.messagingService();
+
+                    if (e == null) {
+                        ClusterNode leader = getLeader(service);
+
+                        ClusterNode thisNode = clusterService.topologyService().localMember();
+
+                        messagingService.respond(addr, successResponse(msgFactory), correlationId);
+
+                        if (leader.equals(thisNode)) {

Review comment:
       It'll be adjustment when onLeaderElected will be introduced, right?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] rpuch commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
rpuch commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r822668919



##########
File path: modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/messages/ClusterStateMessage.java
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.internal.cluster.management.messages;
+
+import java.util.Collection;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.annotations.Transferable;
+
+/**
+ * Message for initializing the Meta Storage.

Review comment:
       Is the javadoc description consistent with the class itself?

##########
File path: modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterInitializer.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.internal.cluster.management;
+
+import static org.apache.ignite.network.util.ClusterServiceUtils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.internal.cluster.management.messages.CancelInitMessage;
+import org.apache.ignite.internal.cluster.management.messages.CmgInitMessage;
+import org.apache.ignite.internal.cluster.management.messages.CmgMessagesFactory;
+import org.apache.ignite.internal.cluster.management.messages.InitCompleteMessage;
+import org.apache.ignite.internal.cluster.management.messages.InitErrorMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    private final CmgMessagesFactory msgFactory = new CmgMessagesFactory();
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty, in which case {@code
+     * metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Void> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must not be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            // check that provided Meta Storage nodes are present in the topology
+            resolveNodes(clusterService, metaStorageNodeNames);
+
+            List<ClusterNode> cmgNodes = resolveNodes(clusterService, cmgNodeNames);
+
+            CmgInitMessage initMessage = msgFactory.cmgInitMessage()
+                    .metaStorageNodes(metaStorageNodeNames)
+                    .cmgNodes(cmgNodeNames)
+                    .build();
+
+            return invokeMessage(cmgNodes, initMessage)
+                    .thenApply(CompletableFuture::completedFuture)
+                    .exceptionally(e -> cancelInit(cmgNodes, e))
+                    .thenCompose(Function.identity());
+        } catch (Exception e) {
+            return CompletableFuture.failedFuture(e);
+        }
+    }
+
+    private CompletableFuture<Void> cancelInit(Collection<ClusterNode> nodes, Throwable e) {
+        log.error("Initialization failed, rolling back", e);
+
+        CancelInitMessage cancelMessage = msgFactory.cancelInitMessage()
+                .reason(e.getMessage())
+                .build();
+
+        return sendMessage(nodes, cancelMessage)
+                .exceptionally(nestedEx -> {
+                    log.error("Error when canceling init", nestedEx);
+
+                    e.addSuppressed(nestedEx);
+
+                    return null;
+                })
+                .thenCompose(v -> CompletableFuture.failedFuture(e));
+    }
+
+    /**
+     * Sends a message to all provided nodes.
+     *
+     * @param nodes nodes to send message to.
+     * @param message message to send.
+     * @return future that either resolves to a leader node ID or fails if any of the nodes return an error response.
+     */
+    private CompletableFuture<Void> invokeMessage(Collection<ClusterNode> nodes, NetworkMessage message) {
+        return allOf(nodes, node ->
+                clusterService.messagingService()
+                        .invoke(node, message, 10000)

Review comment:
       Should we extract this as a constant? Just to make it visible.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r822683998



##########
File path: modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/messages/ClusterStateMessage.java
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.internal.cluster.management.messages;
+
+import java.util.Collection;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.annotations.Transferable;
+
+/**
+ * Message for initializing the Meta Storage.

Review comment:
       No, I'll change it in the next PR, ok?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sanpwc commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sanpwc commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r820124682



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {

Review comment:
       > I don't think this is applicable to registering listeners
   
   Why not? It's still possible to get an exception on listeners registering. My main point is to keep constructors as simple as possible moving all logic to start() or beforeStart() if it's required.
   >Why and how?
   
   Well, because it's not declared anywhere. I actually believe that concept of fully-fledged component after corresponding start is still simplier to use and understand.
   >I think that it is still possible, since ClusterService is started before all other components
   
   I mean, that currently there are no such cases.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810919916



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> metastorageNodes = resolveNodes(clusterService, metaStorageNodeNames);
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> cmgNodes = resolveNodes(clusterService, cmgNodeNames);
+
+            // TODO: init message should only be sent to metastorageNodes and cmgNodes respectively,
+            //  https://issues.apache.org/jira/browse/IGNITE-16471
+            Collection<ClusterNode> allMembers = clusterService.topologyService().allMembers();
+
+            var msgFactory = new InitMessagesFactory();
+
+            MetastorageInitMessage metaStorageInitMessage = msgFactory.metastorageInitMessage()
+                    .metastorageNodes(metaStorageNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> metaStorageInitFuture = invokeMessage(allMembers, metaStorageInitMessage);
+
+            CmgInitMessage cmgInitMessage = msgFactory.cmgInitMessage()
+                    .cmgNodes(cmgNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> cmgInitFuture = invokeMessage(allMembers, cmgInitMessage);
+
+            return metaStorageInitFuture
+                    .thenCombine(cmgInitFuture, Leaders::new)
+                    .whenComplete((leaders, e) -> {
+                        if (e != null) {
+                            log.error("Initialization failed, rolling back", e);
+
+                            CancelInitMessage cancelMessage = msgFactory.cancelInitMessage()
+                                    .reason(e.getMessage())
+                                    .build();
+
+                            cancelInit(allMembers, cancelMessage);
+                        }
+                    });
+        } catch (Exception e) {
+            return CompletableFuture.failedFuture(e);
+        }
+    }
+
+    /**
+     * Sends a message to all provided nodes.
+     *
+     * @param nodes nodes to send message to.
+     * @param message message to send.
+     * @return future that either resolves to a leader node ID or fails if any of the nodes return an error response.
+     */
+    private CompletableFuture<String> invokeMessage(Collection<ClusterNode> nodes, NetworkMessage message) {
+        List<CompletableFuture<String>> futures = nodes.stream()
+                .map(node -> invokeMessage(node, message))
+                .collect(Collectors.toList());
+
+        return CompletableFuture.allOf(futures.toArray(CompletableFuture[]::new))
+                // all futures should return the same response, unless they fail
+                .thenCompose(v -> futures.get(0));
+    }
+
+    private CompletableFuture<String> invokeMessage(ClusterNode node, NetworkMessage message) {
+        return clusterService.messagingService()
+                .invoke(node, message, 10000)

Review comment:
       I don't now, honestly. I can't imagine why a user would need to configure this timeout, and having a configuration for every timeout value in the system looks a bit complicated to me. Or is there any kind of a global timeout value?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810922499



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/Utils.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.internal.join;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+
+/**
+ * Various utility methods.
+ */
+class Utils {

Review comment:
       calling it `NodeResolving` will imply that all methods in this class must somehow relate to "resolving". I'm not particularly sure about the future of this class, because having utility classes is bad, and it also may obtain more methods in future PRs, not related to resolving. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810926261



##########
File path: examples/src/integrationTest/java/org/apache/ignite/example/AbstractExamplesTest.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.example;
+
+import java.nio.file.Path;
+import java.util.List;
+import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.testframework.WorkDirectory;
+import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Base class for creating tests for examples.
+ */
+@ExtendWith(WorkDirectoryExtension.class)
+public abstract class AbstractExamplesTest {
+    private static final String TEST_NODE_NAME = "ignite-node";
+
+    /**
+     * Starts a node.
+     *
+     * @param workDir Work directory for the started node. Must not be {@code null}.
+     */
+    @BeforeEach
+    public void startNode(@WorkDirectory Path workDir) throws Exception {
+        IgniteImpl ignite = (IgniteImpl) IgnitionManager.start(
+                TEST_NODE_NAME,
+                Path.of("config", "ignite-config.json"),
+                workDir,
+                null
+        );
+
+        ignite.init(List.of(ignite.name()));

Review comment:
       Yes, I remember about examples and documentation, I will do that in a separate PR




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810926911



##########
File path: examples/src/integrationTest/java/org/apache/ignite/example/sql/jdbc/ItSqlExamplesTest.java
##########
@@ -17,35 +17,22 @@
 
 package org.apache.ignite.example.sql.jdbc;
 
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.example.AbstractExamplesTest;
 import org.apache.ignite.example.ExampleTestUtils;
-import org.apache.ignite.internal.testframework.WorkDirectory;
-import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.extension.ExtendWith;
 
 /**
  * These tests check that all SQL JDBC examples pass correctly.
  */
-@ExtendWith(WorkDirectoryExtension.class)
-public class SqlExamplesTest {
-    /** Empty argument to invoke an example. */
-    protected static final String[] EMPTY_ARGS = new String[0];
-
+public class ItSqlExamplesTest extends AbstractExamplesTest {
     /**
      * Runs SqlJdbcExample and checks its output.
      *
      * @throws Exception If failed.
      */
     @Test
     public void testSqlJdbcExample() throws Exception {
-        ExampleTestUtils.assertConsoleOutputContains(SqlJdbcExample::main, EMPTY_ARGS,

Review comment:
       I don't agree, but since you are the second person noticing that, I will bring the constant back




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sanpwc commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sanpwc commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810206420



##########
File path: examples/src/integrationTest/java/org/apache/ignite/example/AbstractExamplesTest.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.example;
+
+import java.nio.file.Path;
+import java.util.List;
+import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.testframework.WorkDirectory;
+import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Base class for creating tests for examples.
+ */
+@ExtendWith(WorkDirectoryExtension.class)
+public abstract class AbstractExamplesTest {

Review comment:
       There's a mess with our test hierarchy. There are IgniteAbstractTest and BaseIgniteAbstractTest with some test logging included. Did you consider extending given classes? 

##########
File path: examples/src/integrationTest/java/org/apache/ignite/example/sql/jdbc/ItSqlExamplesTest.java
##########
@@ -62,39 +49,4 @@ public void testSqlJdbcExample() throws Exception {
                         + "    Richard, Miles, St. Petersburg\n"
         );
     }
-
-    /**
-     * Start node.
-     *
-     * @param workDir Work directory for the started node. Must not be {@code null}.
-     */
-    @BeforeEach
-    public void startNode(@WorkDirectory Path workDir) throws IOException {
-        IgnitionManager.start(
-                "my-first-node",
-                Files.readString(Path.of("config", "ignite-config.json")),
-                workDir
-        );
-    }
-
-    /**
-     * Stop node.
-     */
-    @AfterEach
-    public void stopNode() {
-        IgnitionManager.stop("my-first-node");
-    }
-
-    /**
-     * Removes a work directory created by {@link SqlExamplesTest}.
-     */
-    @BeforeEach
-    @AfterEach
-    public void removeWorkDir() {

Review comment:
       Could you please explain why we don't need such method? Is there any built-in cleanup logic within WorkDirectoryExtension?

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/rest/InitCommandHandler.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.join.rest;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import org.apache.ignite.internal.join.ClusterInitializer;
+import org.apache.ignite.internal.join.Leaders;
+import org.apache.ignite.internal.rest.ErrorResult;
+import org.apache.ignite.internal.rest.netty.RestApiHttpRequest;
+import org.apache.ignite.internal.rest.netty.RestApiHttpResponse;
+import org.apache.ignite.internal.rest.routes.RequestHandler;
+import org.apache.ignite.lang.IgniteLogger;
+
+/**
+ * REST handler for the {@link InitCommand}.
+ */
+public class InitCommandHandler implements RequestHandler {
+    private static final IgniteLogger log = IgniteLogger.forClass(InitCommandHandler.class);
+
+    private final ObjectMapper objectMapper = new ObjectMapper();
+
+    private final ClusterInitializer clusterInitializer;
+
+    public InitCommandHandler(ClusterInitializer clusterInitializer) {
+        this.clusterInitializer = clusterInitializer;
+    }
+
+    @Override
+    public CompletableFuture<RestApiHttpResponse> handle(RestApiHttpRequest request, RestApiHttpResponse response) {
+        try {
+            InitCommand command = readContent(request);
+
+            if (log.isInfoEnabled()) {
+                log.info(
+                        "Received init command:\n\tMeta Storage nodes: {}\n\tCMG nodes: {}",
+                        command.metaStorageNodes(),
+                        command.cmgNodes()
+                );
+            }
+
+            return clusterInitializer.initCluster(command.metaStorageNodes(), command.cmgNodes())
+                    .thenApply(leaders -> successResponse(response, leaders))

Review comment:
       What's the point of returning leaders? I believe that we shouldn't expose such details of realization that imposes that we use raft. We might easily move to concept of leader -> leaseholder like cockroach does or use leader-less replication framework in addition to raft.

##########
File path: modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
##########
@@ -409,27 +417,55 @@ public ConfigurationRegistry clusterConfiguration() {
     }
 
     /**
-     * Returns client handler module.
+     * Returns the id of the current node.
      */
-    public ClientHandlerModule clientHandlerModule() {
-        return clientHandlerModule;
+    public String id() {
+        return clusterSvc.topologyService().localMember().id();
     }
 
     /**
-     * Returns the id of the current node.
+     * Returns the local address of REST endpoints.
      */
-    public String id() {
-        return clusterSvc.topologyService().localMember().id();
+    public NetworkAddress restAddress() {
+        return NetworkAddress.from(restModule.localAddress());
+    }
+
+    /**
+     * Returns the local address of the Thin Client.
+     */
+    public NetworkAddress clientAddress() {

Review comment:
       - clientAddress() could throw IgniteInternalException - please add corresponding @throws clause
   - What's the proper way for client to retrieve rest/client and other ports, especially if they are specified within portRange? From my point of view we might add [rest/client/etc]Adresses to Ignite.localNode() **public** api and in that case we should wrap IgniteInternalExcpetion with proper public one.

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/messages/MetastorageInitMessage.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.internal.join.messages;
+
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.annotations.Transferable;
+
+/**
+ * Message for initializing the Meta Storage.
+ */
+@Transferable(InitMessageGroup.METASTORAGE_INIT)
+public interface MetastorageInitMessage extends NetworkMessage {

Review comment:
       Well, I believe that we don't need MetastorageInitMessage for this purpose. 
   I rather prefer following flow:
   onCMGReady cmgLeader send clusterStateMessage to every node that it sees in topology. This message besides other properties contains metaStorageNodes that are used to start metastorage raft group.
   ```
   register onLeaderElected callback:
       onLeaderElected(() -> {
           ClusterState :: CmgGroup.run(
               new adjustClusterStateCmd(
                   vaultSvc.get("CMGPeers"),
                   vaultSvc.get("MSPeers")
               )
           );
           within clusterState
               [stopSeld || trigger CMG Leader actions] :: fullValidation();
                   register clusterSvc.onAppeared(node ->
   clusterSvc.msgSvc.invoke(
   node.id, clusterState
   )
   );
                   clusterSvc.topologySvc.forEach(node ->
   clusterSvc.msgSvc.invoke(
   node.id, clusterState
   )
   );
       });
   
   ``` 

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/Utils.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.internal.join;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+
+/**
+ * Various utility methods.
+ */
+class Utils {
+    /**
+     * Resolves given node names (consistent IDs) into {@link ClusterNode}s.
+     *
+     * @param nodeNames node names.
+     * @return list of resolved {@code ClusterNode}s.
+     * @throws InitException if any of the given nodes are not present in the physical topology.
+     */
+    static List<ClusterNode> resolveNodes(ClusterService clusterService, Collection<String> nodeNames) {

Review comment:
       I'd rather move given method to some general utils or cluster service utils in order to use it in MetastorageManager.
   ```
       private CompletableFuture<String> initializeMetaStorage(ClusterService clusterService, MetastorageInitMessage initMsg)
               throws NodeStoppingException {
           TopologyService topologyService = clusterService.topologyService();
   
           List<ClusterNode> metastorageNodes = Arrays.stream(initMsg.metastorageNodes())
                   .map(nodeId -> {
                       ClusterNode node = topologyService.getByConsistentId(nodeId);
   
                       if (node == null) {
                           throw new InitException("Node " + nodeId + " is not present in the physical topology");
                       }
   
                       return node;
                   })
                   .collect(Collectors.toList());
   ```

##########
File path: modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
##########
@@ -409,27 +417,55 @@ public ConfigurationRegistry clusterConfiguration() {
     }
 
     /**
-     * Returns client handler module.
+     * Returns the id of the current node.
      */
-    public ClientHandlerModule clientHandlerModule() {
-        return clientHandlerModule;
+    public String id() {
+        return clusterSvc.topologyService().localMember().id();
     }
 
     /**
-     * Returns the id of the current node.
+     * Returns the local address of REST endpoints.
      */
-    public String id() {
-        return clusterSvc.topologyService().localMember().id();
+    public NetworkAddress restAddress() {

Review comment:
       Seems that we need better solution here.
   
   First of all we definitely need to distinguish cluster and node local attributes and methods, meaning that tables(), transactions() are cluster wide and name(), restAddress(), etc are node local. I believe that we should introduce an abstraction of Ignite.localNode() with corresponding name(), restAddress() and similar methods.

##########
File path: examples/src/integrationTest/java/org/apache/ignite/example/AbstractExamplesTest.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.example;
+
+import java.nio.file.Path;
+import java.util.List;
+import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.testframework.WorkDirectory;
+import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Base class for creating tests for examples.
+ */
+@ExtendWith(WorkDirectoryExtension.class)
+public abstract class AbstractExamplesTest {
+    private static final String TEST_NODE_NAME = "ignite-node";
+
+    /**
+     * Starts a node.
+     *
+     * @param workDir Work directory for the started node. Must not be {@code null}.
+     */
+    @BeforeEach
+    public void startNode(@WorkDirectory Path workDir) throws Exception {
+        IgniteImpl ignite = (IgniteImpl) IgnitionManager.start(
+                TEST_NODE_NAME,
+                Path.of("config", "ignite-config.json"),
+                workDir,
+                null
+        );
+
+        ignite.init(List.of(ignite.name()));

Review comment:
       I believe that we should also adjust examples in order to reflect that init phase was added.

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> metastorageNodes = resolveNodes(clusterService, metaStorageNodeNames);
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> cmgNodes = resolveNodes(clusterService, cmgNodeNames);
+
+            // TODO: init message should only be sent to metastorageNodes and cmgNodes respectively,
+            //  https://issues.apache.org/jira/browse/IGNITE-16471
+            Collection<ClusterNode> allMembers = clusterService.topologyService().allMembers();
+
+            var msgFactory = new InitMessagesFactory();
+
+            MetastorageInitMessage metaStorageInitMessage = msgFactory.metastorageInitMessage()
+                    .metastorageNodes(metaStorageNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> metaStorageInitFuture = invokeMessage(allMembers, metaStorageInitMessage);
+
+            CmgInitMessage cmgInitMessage = msgFactory.cmgInitMessage()
+                    .cmgNodes(cmgNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> cmgInitFuture = invokeMessage(allMembers, cmgInitMessage);
+
+            return metaStorageInitFuture

Review comment:
       I believe that concept of clusterStateMessage-based metastorage init is more robust and simple. Please see my comment in MetastorageInitMessage.

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));
+
+                    raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)

Review comment:
       We should also populate Vault with both cmgNodes and metaStorageNodes on init message in order to reuse given meta on cluster restart.
   ```
   if(!vault.cmgPeers.isEmpty) {
           loza.prepareRaftGroup(vault.cmgPeers);
   } else {
       restMgr.registerCmdHndl(
               initMsg,
               (initParams) ->{
                   vaultSvc.putAll(
   Map.of(
                           "CMGPeers", initMsg.cmgPeers,
                            "MSPeers", initMsg.initPeers)
                       );
                   loza.prepareRaftGroup(initParams.cmgPeers)
               }
       );
   }
   
   ```

##########
File path: modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
##########
@@ -122,174 +115,233 @@
 
     /**
      * Future which will be completed with {@link IgniteUuid}, when aggregated watch will be successfully deployed. Can be resolved to
-     * {@link Optional#empty()} if no watch deployed at the moment.
-     */
-    private CompletableFuture<Optional<IgniteUuid>> deployFut = new CompletableFuture<>();
-
-    /**
-     * If true - all new watches will be deployed immediately.
+     * {@code null} if no watch deployed at the moment.
      *
-     * <p>If false - all new watches will be aggregated to one batch for further deploy by {@link MetaStorageManager#deployWatches()}
+     * <p>Multithreaded access is guarded by {@code this}.
      */
-    private boolean deployed;
-
-    /** Flag indicates if meta storage nodes were set on start. */
-    private boolean metaStorageNodesOnStart;
+    private CompletableFuture<IgniteUuid> deployFut = new CompletableFuture<>();
 
     /** Actual storage for the Metastorage. */
     private final KeyValueStorage storage;
 
     /** Busy lock to stop synchronously. */
     private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
 
-    /** Prevents double stopping the component. */
-    AtomicBoolean stopGuard = new AtomicBoolean();
+    /**
+     * If true - all new watches will be deployed immediately.
+     *
+     * <p>If false - all new watches will be aggregated to one batch for further deploy by {@link MetaStorageManager#deployWatches()}.
+     *
+     * <p>Multithreaded access is guarded by {@code this}.
+     */
+    private boolean areWatchesDeployed = false;
+
+    /**
+     * Flag that indicates that the Meta Storage has been initialized.
+     */
+    private final AtomicBoolean isInitialized = new AtomicBoolean();
+
+    /**
+     * Prevents double stopping the component.
+     *
+     * <p>Multithreaded access is guarded by {@code this}.
+     */
+    private boolean isStopped = false;
 
     /**
      * The constructor.
      *
-     * @param vaultMgr      Vault manager.
-     * @param locCfgMgr     Local configuration manager.
-     * @param clusterNetSvc Cluster network service.
-     * @param raftMgr       Raft manager.
-     * @param storage       Storage. This component owns this resource and will manage its lifecycle.
+     * @param vaultMgr Vault manager.
+     * @param clusterService Cluster network service.
+     * @param raftMgr Raft manager.
+     * @param storage Storage. This component owns this resource and will manage its lifecycle.
      */
     public MetaStorageManager(
             VaultManager vaultMgr,
-            ConfigurationManager locCfgMgr,
-            ClusterService clusterNetSvc,
+            ClusterService clusterService,
             Loza raftMgr,
             KeyValueStorage storage
     ) {
         this.vaultMgr = vaultMgr;
-        this.locCfgMgr = locCfgMgr;
-        this.clusterNetSvc = clusterNetSvc;
         this.raftMgr = raftMgr;
         this.storage = storage;
-    }
 
-    /** {@inheritDoc} */
-    @Override
-    public void start() {
-        String[] metastorageNodes = this.locCfgMgr.configurationRegistry().getConfiguration(NodeConfiguration.KEY)
-                .metastorageNodes().value();
+        MessagingService messagingService = clusterService.messagingService();
 
-        Predicate<ClusterNode> metaStorageNodesContainsLocPred =
-                clusterNode -> Arrays.asList(metastorageNodes).contains(clusterNode.name());
+        var msgFactory = new InitMessagesFactory();
 
-        if (metastorageNodes.length > 0) {
-            metaStorageNodesOnStart = true;
-
-            List<ClusterNode> metaStorageMembers = clusterNetSvc.topologyService().allMembers().stream()
-                    .filter(metaStorageNodesContainsLocPred)
-                    .collect(Collectors.toList());
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
 
-            // TODO: This is temporary solution for providing human-readable error when you try to start single-node cluster
-            // without hosting metastorage, this will be rewritten in init phase https://issues.apache.org/jira/browse/IGNITE-15114
-            if (metaStorageMembers.isEmpty()) {
-                throw new IgniteException(
-                        "Cannot start meta storage manager because there is no node in the cluster that hosts meta storage.");
+                return;
             }
 
-            // TODO: This is temporary solution. We need to prohibit starting several metastorage nodes
-            // as far as we do not have mechanism of changing raft peers when new metastorage node is joining to cluster.
-            // This will be rewritten in init phase https://issues.apache.org/jira/browse/IGNITE-15114
-            if (metastorageNodes.length > 1) {
-                throw new IgniteException(
-                        "Cannot start meta storage manager because it is not allowed to start several metastorage nodes.");
-            }
+            try {
+                if (msg instanceof CancelInitMessage) {

Review comment:
       It's not needed here either ;)

##########
File path: modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
##########
@@ -122,174 +115,233 @@
 
     /**
      * Future which will be completed with {@link IgniteUuid}, when aggregated watch will be successfully deployed. Can be resolved to
-     * {@link Optional#empty()} if no watch deployed at the moment.
-     */
-    private CompletableFuture<Optional<IgniteUuid>> deployFut = new CompletableFuture<>();
-
-    /**
-     * If true - all new watches will be deployed immediately.
+     * {@code null} if no watch deployed at the moment.
      *
-     * <p>If false - all new watches will be aggregated to one batch for further deploy by {@link MetaStorageManager#deployWatches()}
+     * <p>Multithreaded access is guarded by {@code this}.
      */
-    private boolean deployed;
-
-    /** Flag indicates if meta storage nodes were set on start. */
-    private boolean metaStorageNodesOnStart;
+    private CompletableFuture<IgniteUuid> deployFut = new CompletableFuture<>();
 
     /** Actual storage for the Metastorage. */
     private final KeyValueStorage storage;
 
     /** Busy lock to stop synchronously. */
     private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
 
-    /** Prevents double stopping the component. */
-    AtomicBoolean stopGuard = new AtomicBoolean();
+    /**
+     * If true - all new watches will be deployed immediately.
+     *
+     * <p>If false - all new watches will be aggregated to one batch for further deploy by {@link MetaStorageManager#deployWatches()}.
+     *
+     * <p>Multithreaded access is guarded by {@code this}.
+     */
+    private boolean areWatchesDeployed = false;
+
+    /**
+     * Flag that indicates that the Meta Storage has been initialized.
+     */
+    private final AtomicBoolean isInitialized = new AtomicBoolean();
+
+    /**
+     * Prevents double stopping the component.
+     *
+     * <p>Multithreaded access is guarded by {@code this}.
+     */
+    private boolean isStopped = false;
 
     /**
      * The constructor.
      *
-     * @param vaultMgr      Vault manager.
-     * @param locCfgMgr     Local configuration manager.
-     * @param clusterNetSvc Cluster network service.
-     * @param raftMgr       Raft manager.
-     * @param storage       Storage. This component owns this resource and will manage its lifecycle.
+     * @param vaultMgr Vault manager.
+     * @param clusterService Cluster network service.
+     * @param raftMgr Raft manager.
+     * @param storage Storage. This component owns this resource and will manage its lifecycle.
      */
     public MetaStorageManager(
             VaultManager vaultMgr,
-            ConfigurationManager locCfgMgr,
-            ClusterService clusterNetSvc,
+            ClusterService clusterService,
             Loza raftMgr,
             KeyValueStorage storage
     ) {
         this.vaultMgr = vaultMgr;
-        this.locCfgMgr = locCfgMgr;
-        this.clusterNetSvc = clusterNetSvc;
         this.raftMgr = raftMgr;
         this.storage = storage;
-    }
 
-    /** {@inheritDoc} */
-    @Override
-    public void start() {
-        String[] metastorageNodes = this.locCfgMgr.configurationRegistry().getConfiguration(NodeConfiguration.KEY)
-                .metastorageNodes().value();
+        MessagingService messagingService = clusterService.messagingService();
 
-        Predicate<ClusterNode> metaStorageNodesContainsLocPred =
-                clusterNode -> Arrays.asList(metastorageNodes).contains(clusterNode.name());
+        var msgFactory = new InitMessagesFactory();
 
-        if (metastorageNodes.length > 0) {
-            metaStorageNodesOnStart = true;
-
-            List<ClusterNode> metaStorageMembers = clusterNetSvc.topologyService().allMembers().stream()
-                    .filter(metaStorageNodesContainsLocPred)
-                    .collect(Collectors.toList());
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {

Review comment:
       don't need it here, yep?

##########
File path: modules/join/pom.xml
##########
@@ -0,0 +1,111 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite-parent</artifactId>
+        <version>1</version>
+        <relativePath>../../parent/pom.xml</relativePath>
+    </parent>
+
+    <artifactId>ignite-join</artifactId>

Review comment:
       I agree with adding new module, but it's definitely not only "join" because it will also handle logical topology, metastorage configuration and adjustment,  probably rolling upgrade, etc. So I'd rather use "ignite-cluster-management" or similar here.
   
   Please also adjust package with corresponding name.

##########
File path: examples/src/integrationTest/java/org/apache/ignite/example/sql/jdbc/ItSqlExamplesTest.java
##########
@@ -17,35 +17,22 @@
 
 package org.apache.ignite.example.sql.jdbc;
 
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.example.AbstractExamplesTest;
 import org.apache.ignite.example.ExampleTestUtils;
-import org.apache.ignite.internal.testframework.WorkDirectory;
-import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.extension.ExtendWith;
 
 /**
  * These tests check that all SQL JDBC examples pass correctly.
  */
-@ExtendWith(WorkDirectoryExtension.class)
-public class SqlExamplesTest {
-    /** Empty argument to invoke an example. */
-    protected static final String[] EMPTY_ARGS = new String[0];
-
+public class ItSqlExamplesTest extends AbstractExamplesTest {
     /**
      * Runs SqlJdbcExample and checks its output.
      *
      * @throws Exception If failed.
      */
     @Test
     public void testSqlJdbcExample() throws Exception {
-        ExampleTestUtils.assertConsoleOutputContains(SqlJdbcExample::main, EMPTY_ARGS,

Review comment:
       Here and below. EMPTY_ARGS are meaningful. I'd rather add such constant to AbstractExamplesTest in order to use wherever needed.

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/rest/InitCommandHandler.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.join.rest;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import org.apache.ignite.internal.join.ClusterInitializer;
+import org.apache.ignite.internal.join.Leaders;
+import org.apache.ignite.internal.rest.ErrorResult;
+import org.apache.ignite.internal.rest.netty.RestApiHttpRequest;
+import org.apache.ignite.internal.rest.netty.RestApiHttpResponse;
+import org.apache.ignite.internal.rest.routes.RequestHandler;
+import org.apache.ignite.lang.IgniteLogger;
+
+/**
+ * REST handler for the {@link InitCommand}.
+ */
+public class InitCommandHandler implements RequestHandler {
+    private static final IgniteLogger log = IgniteLogger.forClass(InitCommandHandler.class);
+
+    private final ObjectMapper objectMapper = new ObjectMapper();
+
+    private final ClusterInitializer clusterInitializer;
+
+    public InitCommandHandler(ClusterInitializer clusterInitializer) {
+        this.clusterInitializer = clusterInitializer;
+    }
+
+    @Override
+    public CompletableFuture<RestApiHttpResponse> handle(RestApiHttpRequest request, RestApiHttpResponse response) {
+        try {
+            InitCommand command = readContent(request);
+
+            if (log.isInfoEnabled()) {
+                log.info(
+                        "Received init command:\n\tMeta Storage nodes: {}\n\tCMG nodes: {}",
+                        command.metaStorageNodes(),
+                        command.cmgNodes()
+                );
+            }
+
+            return clusterInitializer.initCluster(command.metaStorageNodes(), command.cmgNodes())
+                    .thenApply(leaders -> successResponse(response, leaders))
+                    .exceptionally(e -> errorResponse(response, e));
+        } catch (Exception e) {
+            return CompletableFuture.completedFuture(errorResponse(response, e));
+        }
+    }
+
+    private InitCommand readContent(RestApiHttpRequest restApiHttpRequest) throws IOException {
+        ByteBuf content = restApiHttpRequest.request().content();
+
+        try (InputStream is = new ByteBufInputStream(content)) {
+            return objectMapper.readValue(is, InitCommand.class);
+        }
+    }
+
+    private static RestApiHttpResponse successResponse(RestApiHttpResponse response, Leaders leaders) {
+        if (log.isInfoEnabled()) {
+            log.info(
+                    "Init command executed successfully.\n\tMeta Storage leader ID: {}\n\tCMG leader ID: {}",
+                    leaders.metaStorageLeader(),
+                    leaders.cmgLeader()
+            );
+        }
+
+        response.json(Map.of(
+                "metaStorageLeaderId", leaders.metaStorageLeader(),
+                "cmgLeaderId", leaders.cmgLeader()
+        ));
+
+        return response;
+    }
+
+    private static RestApiHttpResponse errorResponse(RestApiHttpResponse response, Throwable e) {
+        if (e instanceof CompletionException) {
+            e = e.getCause();

Review comment:
       What if e.getCause() is IgniteInternalException? Is it possible? If true, it'll be wrapped with Ignite(Public)Exception, isn't it?

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));

Review comment:
       It worth to mention that according to cluster lifecycle document initial idea assumed that cmgNodes and metaStorageNodes are public **addresses** and not names. Despite that fact, personally I believe that node names suites even better here.

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/InitException.java
##########
@@ -15,8 +15,17 @@
  * limitations under the License.
  */
 
+package org.apache.ignite.internal.join;
+
 /**
- * Configuration schemas for Cluster node.
+ * Exception thrown when cluster initialization fails for some reason.
  */
+public class InitException extends RuntimeException {

Review comment:
       Please extend either IgniteInternalException or IgniteException if it's thrown from public api.

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/rest/InitCommandHandler.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.join.rest;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import org.apache.ignite.internal.join.ClusterInitializer;
+import org.apache.ignite.internal.join.Leaders;
+import org.apache.ignite.internal.rest.ErrorResult;
+import org.apache.ignite.internal.rest.netty.RestApiHttpRequest;
+import org.apache.ignite.internal.rest.netty.RestApiHttpResponse;
+import org.apache.ignite.internal.rest.routes.RequestHandler;
+import org.apache.ignite.lang.IgniteLogger;
+
+/**
+ * REST handler for the {@link InitCommand}.
+ */
+public class InitCommandHandler implements RequestHandler {
+    private static final IgniteLogger log = IgniteLogger.forClass(InitCommandHandler.class);
+
+    private final ObjectMapper objectMapper = new ObjectMapper();
+
+    private final ClusterInitializer clusterInitializer;
+
+    public InitCommandHandler(ClusterInitializer clusterInitializer) {
+        this.clusterInitializer = clusterInitializer;
+    }
+
+    @Override
+    public CompletableFuture<RestApiHttpResponse> handle(RestApiHttpRequest request, RestApiHttpResponse response) {

Review comment:
       JavaDoc is missing.

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {

Review comment:
       The rule of thumb is to prefer start() to constructor for registering message handlers. Are there any practical reasons to do it in constructor in this particular case? 

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/rest/InitCommandHandler.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.join.rest;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import org.apache.ignite.internal.join.ClusterInitializer;
+import org.apache.ignite.internal.join.Leaders;
+import org.apache.ignite.internal.rest.ErrorResult;
+import org.apache.ignite.internal.rest.netty.RestApiHttpRequest;
+import org.apache.ignite.internal.rest.netty.RestApiHttpResponse;
+import org.apache.ignite.internal.rest.routes.RequestHandler;
+import org.apache.ignite.lang.IgniteLogger;
+
+/**
+ * REST handler for the {@link InitCommand}.
+ */
+public class InitCommandHandler implements RequestHandler {
+    private static final IgniteLogger log = IgniteLogger.forClass(InitCommandHandler.class);
+
+    private final ObjectMapper objectMapper = new ObjectMapper();
+
+    private final ClusterInitializer clusterInitializer;
+
+    public InitCommandHandler(ClusterInitializer clusterInitializer) {

Review comment:
       JavaDoc is missing. By the way, why did our javadoc check miss this?

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));
+
+                    raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)
+                            .whenComplete((service, e) -> {
+                                if (e == null) {
+                                    Peer leader = service.leader();
+
+                                    assert leader != null;
+
+                                    messagingService.respond(addr, leaderElectedResponse(msgFactory, leader), correlationId);
+                                } else {
+                                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                                }
+                            });
+                }
+            } catch (Exception e) {
+                log.error("Exception when initializing the CMG", e);
+
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                }
+            } finally {
+                busyLock.leaveBusy();
+            }
+        });
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group.
+     */
+    public Leaders initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) throws NodeStoppingException {
+        if (!busyLock.enterBusy()) {
+            throw new NodeStoppingException();
+        }
+
+        try {
+            return clusterInitializer.initCluster(metaStorageNodeNames, cmgNodeNames).get();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IgniteInternalException("Interrupted while initializing the cluster", e);
+        } catch (ExecutionException e) {
+            throw new IgniteInternalException("Unable to initialize the cluster", e.getCause());
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    private static NetworkMessage errorResponse(InitMessagesFactory msgFactory, Throwable e) {
+        log.error("Exception when starting the CMG", e);
+
+        return msgFactory.initErrorMessage()
+                .cause(e.getMessage())
+                .build();
+    }
+
+    private NetworkMessage leaderElectedResponse(InitMessagesFactory msgFactory, Peer leader) {

Review comment:
       As was mentioned earlier leader election is a detail of realization, I believe that simple successfulResponse suites better here. 

##########
File path: modules/cli/src/integrationTest/java/org/apache/ignite/cli/ItConfigCommandTest.java
##########
@@ -32,74 +32,50 @@
 import io.micronaut.context.ApplicationContext;
 import io.micronaut.context.env.Environment;
 import java.io.ByteArrayOutputStream;
-import java.io.IOException;
 import java.io.PrintWriter;
-import java.net.ServerSocket;
 import java.nio.file.Path;
 import net.minidev.json.JSONObject;
 import net.minidev.json.JSONValue;
-import org.apache.ignite.Ignite;
 import org.apache.ignite.IgnitionManager;
 import org.apache.ignite.cli.spec.IgniteCliSpec;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.testframework.WorkDirectory;
+import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 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 org.junit.jupiter.api.extension.ExtendWith;
 import picocli.CommandLine;
 
 /**
  * Integration test for {@code ignite config} commands.
  */
+@ExtendWith(WorkDirectoryExtension.class)
 public class ItConfigCommandTest extends AbstractCliTest {
     /** DI context. */
     private ApplicationContext ctx;
 
     /** stderr. */
-    private ByteArrayOutputStream err;
+    private final ByteArrayOutputStream err = new ByteArrayOutputStream();
 
     /** stdout. */
-    private ByteArrayOutputStream out;
-
-    /** Port for REST communication. */
-    private int restPort;
-
-    /** Port for thin client communication. */
-    private int clientPort;
-
-    /** Network port. */
-    private int networkPort;
+    private final ByteArrayOutputStream out = new ByteArrayOutputStream();
 
     /** Node. */
-    private Ignite node;
+    private IgniteImpl node;
 
     @BeforeEach
-    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.

Review comment:
       Please check my comment in IgniteImpl class and if you are going to fix it within current ticket please resolve IGNITE-15131.

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));
+
+                    raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)
+                            .whenComplete((service, e) -> {
+                                if (e == null) {
+                                    Peer leader = service.leader();
+
+                                    assert leader != null;

Review comment:
       We definitely need more reliable leader await logic here. It's perfectly fine for raft group not not elect leader for some time, because leader election process includes some randomized invariant inside  and because CmgInitMessage might be processed in different time frames on different nodes. I mean that if service.leader is null after service creation we should call refreshLeader that has retry logic inside.

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/messages/InitMessageGroup.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.internal.join.messages;
+
+import org.apache.ignite.network.annotations.MessageGroup;
+
+/**
+ * Message Group for cluster initialization and CMG management.
+ */
+@MessageGroup(groupType = 6, groupName = "InitMessages")
+public class InitMessageGroup {
+    /**
+     * Message type for {@link CmgInitMessage}.
+     */
+    public static final short CMG_INIT = 1;
+
+    /**
+     * Message type for {@link MetastorageInitMessage}.
+     */
+    public static final short METASTORAGE_INIT = 2;
+
+    /**
+     * Message type for {@link LeaderElectedMessage}.
+     */
+    public static final short LEADER_ELECTED = 3;

Review comment:
       Let's rename it to GROUP_READY

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/messages/LeaderElectedMessage.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.internal.join.messages;
+
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.annotations.Transferable;
+
+/**
+ * Successful response for initializing a Raft group.
+ */
+@Transferable(InitMessageGroup.LEADER_ELECTED)
+public interface LeaderElectedMessage extends NetworkMessage {

Review comment:
       Same as above, I believe that GROUP_READY is enough here, that actually means that we don't need exact leader name.

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")

Review comment:
       Could you please add a comment that it's a sort of validation, if it's really is.

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);

Review comment:
       We should also remove cmgNodes and msNodes from Vault here.

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/Leaders.java
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.internal.join;
+
+import java.util.Objects;
+
+/**
+ * Data class containing consistent IDs of leaders of the Meta Storage and the CMG.
+ */
+public class Leaders {

Review comment:
       Seems that we don't need given class at all.

##########
File path: modules/metastorage/pom.xml
##########
@@ -38,6 +38,11 @@
             <artifactId>ignite-configuration</artifactId>
         </dependency>
 
+        <dependency>

Review comment:
       I actually don't think that we need it, cause ignite-runner will handle on clusterStateMessage that will include metaStorageNodes.

##########
File path: modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
##########
@@ -122,174 +115,233 @@
 
     /**
      * Future which will be completed with {@link IgniteUuid}, when aggregated watch will be successfully deployed. Can be resolved to
-     * {@link Optional#empty()} if no watch deployed at the moment.
-     */
-    private CompletableFuture<Optional<IgniteUuid>> deployFut = new CompletableFuture<>();
-
-    /**
-     * If true - all new watches will be deployed immediately.
+     * {@code null} if no watch deployed at the moment.
      *
-     * <p>If false - all new watches will be aggregated to one batch for further deploy by {@link MetaStorageManager#deployWatches()}
+     * <p>Multithreaded access is guarded by {@code this}.
      */
-    private boolean deployed;
-
-    /** Flag indicates if meta storage nodes were set on start. */
-    private boolean metaStorageNodesOnStart;
+    private CompletableFuture<IgniteUuid> deployFut = new CompletableFuture<>();
 
     /** Actual storage for the Metastorage. */
     private final KeyValueStorage storage;
 
     /** Busy lock to stop synchronously. */
     private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
 
-    /** Prevents double stopping the component. */
-    AtomicBoolean stopGuard = new AtomicBoolean();

Review comment:
       Let's use consistent solution for all components and use either stopGuard or isStopped flag.
   Within CMGManager you've introduced stopGuard.

##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> metastorageNodes = resolveNodes(clusterService, metaStorageNodeNames);
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> cmgNodes = resolveNodes(clusterService, cmgNodeNames);
+
+            // TODO: init message should only be sent to metastorageNodes and cmgNodes respectively,
+            //  https://issues.apache.org/jira/browse/IGNITE-16471
+            Collection<ClusterNode> allMembers = clusterService.topologyService().allMembers();
+
+            var msgFactory = new InitMessagesFactory();
+
+            MetastorageInitMessage metaStorageInitMessage = msgFactory.metastorageInitMessage()
+                    .metastorageNodes(metaStorageNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> metaStorageInitFuture = invokeMessage(allMembers, metaStorageInitMessage);
+
+            CmgInitMessage cmgInitMessage = msgFactory.cmgInitMessage()
+                    .cmgNodes(cmgNodeNames.toArray(String[]::new))
+                    .build();
+
+            CompletableFuture<String> cmgInitFuture = invokeMessage(allMembers, cmgInitMessage);
+
+            return metaStorageInitFuture
+                    .thenCombine(cmgInitFuture, Leaders::new)
+                    .whenComplete((leaders, e) -> {
+                        if (e != null) {
+                            log.error("Initialization failed, rolling back", e);
+
+                            CancelInitMessage cancelMessage = msgFactory.cancelInitMessage()
+                                    .reason(e.getMessage())
+                                    .build();
+
+                            cancelInit(allMembers, cancelMessage);
+                        }
+                    });
+        } catch (Exception e) {
+            return CompletableFuture.failedFuture(e);
+        }
+    }
+
+    /**
+     * Sends a message to all provided nodes.
+     *
+     * @param nodes nodes to send message to.
+     * @param message message to send.
+     * @return future that either resolves to a leader node ID or fails if any of the nodes return an error response.
+     */
+    private CompletableFuture<String> invokeMessage(Collection<ClusterNode> nodes, NetworkMessage message) {

Review comment:
       As was mentioned above I don't think that we need leaderId, [ok] response if enough here. 

##########
File path: modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
##########
@@ -143,11 +151,10 @@
     /**
      * The Constructor.
      *
-     * @param name                       Ignite node name.
-     * @param workDir                    Work directory for the started node. Must not be {@code null}.
-     * @param serviceProviderClassLoader The class loader to be used to load provider-configuration files and provider
-     *                                   classes, or {@code null} if the system class loader (or, failing that
-     *                                   the bootstrap class loader) is to be used
+     * @param name Ignite node name.
+     * @param workDir Work directory for the started node. Must not be {@code null}.
+     * @param serviceProviderClassLoader The class loader to be used to load provider-configuration files and provider classes, or {@code

Review comment:
       {@code
   null}
   Auf

##########
File path: modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
##########
@@ -320,6 +327,7 @@ public void start(@Nullable String cfg) {
                     distributedTblMgr,
                     qryEngine,
                     restModule,
+                    cmgMgr,

Review comment:
       Order matters here, cmgMgr should start before metaStorageMgr.

##########
File path: modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
##########
@@ -351,8 +359,8 @@ public void start(@Nullable String cfg) {
      */
     public void stop() {
         if (status.getAndSet(Status.STOPPING) == Status.STARTED) {
-            doStopNode(List.of(vaultMgr, nodeCfgMgr, clusterSvc, raftMgr, txManager, metaStorageMgr, clusterCfgMgr, baselineMgr,
-                    distributedTblMgr, qryEngine, restModule, clientHandlerModule, nettyBootstrapFactory));
+            doStopNode(List.of(nettyBootstrapFactory, vaultMgr, nodeCfgMgr, clusterSvc, raftMgr, txManager, metaStorageMgr, clusterCfgMgr,

Review comment:
       Please don't forget to adjust cmgMgr position here. 

##########
File path: modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
##########
@@ -122,174 +115,233 @@
 
     /**
      * Future which will be completed with {@link IgniteUuid}, when aggregated watch will be successfully deployed. Can be resolved to
-     * {@link Optional#empty()} if no watch deployed at the moment.
-     */
-    private CompletableFuture<Optional<IgniteUuid>> deployFut = new CompletableFuture<>();
-
-    /**
-     * If true - all new watches will be deployed immediately.
+     * {@code null} if no watch deployed at the moment.
      *
-     * <p>If false - all new watches will be aggregated to one batch for further deploy by {@link MetaStorageManager#deployWatches()}
+     * <p>Multithreaded access is guarded by {@code this}.
      */
-    private boolean deployed;
-
-    /** Flag indicates if meta storage nodes were set on start. */
-    private boolean metaStorageNodesOnStart;
+    private CompletableFuture<IgniteUuid> deployFut = new CompletableFuture<>();
 
     /** Actual storage for the Metastorage. */
     private final KeyValueStorage storage;
 
     /** Busy lock to stop synchronously. */
     private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
 
-    /** Prevents double stopping the component. */
-    AtomicBoolean stopGuard = new AtomicBoolean();
+    /**
+     * If true - all new watches will be deployed immediately.
+     *
+     * <p>If false - all new watches will be aggregated to one batch for further deploy by {@link MetaStorageManager#deployWatches()}.
+     *
+     * <p>Multithreaded access is guarded by {@code this}.
+     */
+    private boolean areWatchesDeployed = false;
+
+    /**
+     * Flag that indicates that the Meta Storage has been initialized.
+     */
+    private final AtomicBoolean isInitialized = new AtomicBoolean();
+
+    /**
+     * Prevents double stopping the component.
+     *
+     * <p>Multithreaded access is guarded by {@code this}.
+     */
+    private boolean isStopped = false;
 
     /**
      * The constructor.
      *
-     * @param vaultMgr      Vault manager.
-     * @param locCfgMgr     Local configuration manager.
-     * @param clusterNetSvc Cluster network service.
-     * @param raftMgr       Raft manager.
-     * @param storage       Storage. This component owns this resource and will manage its lifecycle.
+     * @param vaultMgr Vault manager.
+     * @param clusterService Cluster network service.
+     * @param raftMgr Raft manager.
+     * @param storage Storage. This component owns this resource and will manage its lifecycle.
      */
     public MetaStorageManager(
             VaultManager vaultMgr,
-            ConfigurationManager locCfgMgr,
-            ClusterService clusterNetSvc,
+            ClusterService clusterService,
             Loza raftMgr,
             KeyValueStorage storage
     ) {
         this.vaultMgr = vaultMgr;
-        this.locCfgMgr = locCfgMgr;
-        this.clusterNetSvc = clusterNetSvc;
         this.raftMgr = raftMgr;
         this.storage = storage;
-    }
 
-    /** {@inheritDoc} */
-    @Override
-    public void start() {
-        String[] metastorageNodes = this.locCfgMgr.configurationRegistry().getConfiguration(NodeConfiguration.KEY)
-                .metastorageNodes().value();
+        MessagingService messagingService = clusterService.messagingService();
 
-        Predicate<ClusterNode> metaStorageNodesContainsLocPred =
-                clusterNode -> Arrays.asList(metastorageNodes).contains(clusterNode.name());
+        var msgFactory = new InitMessagesFactory();

Review comment:
       As was mentioned earlier, I believe that we should use clusterState from within runner process to init metastorage.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811302478



##########
File path: modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
##########
@@ -122,174 +115,233 @@
 
     /**
      * Future which will be completed with {@link IgniteUuid}, when aggregated watch will be successfully deployed. Can be resolved to
-     * {@link Optional#empty()} if no watch deployed at the moment.
-     */
-    private CompletableFuture<Optional<IgniteUuid>> deployFut = new CompletableFuture<>();
-
-    /**
-     * If true - all new watches will be deployed immediately.
+     * {@code null} if no watch deployed at the moment.
      *
-     * <p>If false - all new watches will be aggregated to one batch for further deploy by {@link MetaStorageManager#deployWatches()}
+     * <p>Multithreaded access is guarded by {@code this}.
      */
-    private boolean deployed;
-
-    /** Flag indicates if meta storage nodes were set on start. */
-    private boolean metaStorageNodesOnStart;
+    private CompletableFuture<IgniteUuid> deployFut = new CompletableFuture<>();
 
     /** Actual storage for the Metastorage. */
     private final KeyValueStorage storage;
 
     /** Busy lock to stop synchronously. */
     private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
 
-    /** Prevents double stopping the component. */
-    AtomicBoolean stopGuard = new AtomicBoolean();
+    /**
+     * If true - all new watches will be deployed immediately.
+     *
+     * <p>If false - all new watches will be aggregated to one batch for further deploy by {@link MetaStorageManager#deployWatches()}.
+     *
+     * <p>Multithreaded access is guarded by {@code this}.
+     */
+    private boolean areWatchesDeployed = false;
+
+    /**
+     * Flag that indicates that the Meta Storage has been initialized.
+     */
+    private final AtomicBoolean isInitialized = new AtomicBoolean();
+
+    /**
+     * Prevents double stopping the component.
+     *
+     * <p>Multithreaded access is guarded by {@code this}.
+     */
+    private boolean isStopped = false;
 
     /**
      * The constructor.
      *
-     * @param vaultMgr      Vault manager.
-     * @param locCfgMgr     Local configuration manager.
-     * @param clusterNetSvc Cluster network service.
-     * @param raftMgr       Raft manager.
-     * @param storage       Storage. This component owns this resource and will manage its lifecycle.
+     * @param vaultMgr Vault manager.
+     * @param clusterService Cluster network service.
+     * @param raftMgr Raft manager.
+     * @param storage Storage. This component owns this resource and will manage its lifecycle.
      */
     public MetaStorageManager(
             VaultManager vaultMgr,
-            ConfigurationManager locCfgMgr,
-            ClusterService clusterNetSvc,
+            ClusterService clusterService,
             Loza raftMgr,
             KeyValueStorage storage
     ) {
         this.vaultMgr = vaultMgr;
-        this.locCfgMgr = locCfgMgr;
-        this.clusterNetSvc = clusterNetSvc;
         this.raftMgr = raftMgr;
         this.storage = storage;
-    }
 
-    /** {@inheritDoc} */
-    @Override
-    public void start() {
-        String[] metastorageNodes = this.locCfgMgr.configurationRegistry().getConfiguration(NodeConfiguration.KEY)
-                .metastorageNodes().value();
+        MessagingService messagingService = clusterService.messagingService();
 
-        Predicate<ClusterNode> metaStorageNodesContainsLocPred =
-                clusterNode -> Arrays.asList(metastorageNodes).contains(clusterNode.name());
+        var msgFactory = new InitMessagesFactory();
 
-        if (metastorageNodes.length > 0) {
-            metaStorageNodesOnStart = true;
-
-            List<ClusterNode> metaStorageMembers = clusterNetSvc.topologyService().allMembers().stream()
-                    .filter(metaStorageNodesContainsLocPred)
-                    .collect(Collectors.toList());
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {

Review comment:
       yes, I think it is fine




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811732391



##########
File path: modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
##########
@@ -409,27 +417,55 @@ public ConfigurationRegistry clusterConfiguration() {
     }
 
     /**
-     * Returns client handler module.
+     * Returns the id of the current node.
      */
-    public ClientHandlerModule clientHandlerModule() {
-        return clientHandlerModule;
+    public String id() {
+        return clusterSvc.topologyService().localMember().id();
     }
 
     /**
-     * Returns the id of the current node.
+     * Returns the local address of REST endpoints.
      */
-    public String id() {
-        return clusterSvc.topologyService().localMember().id();
+    public NetworkAddress restAddress() {
+        return NetworkAddress.from(restModule.localAddress());
+    }
+
+    /**
+     * Returns the local address of the Thin Client.
+     */
+    public NetworkAddress clientAddress() {

Review comment:
       > clientAddress() could throw IgniteInternalException - please add corresponding @throws clause
   
   done
   
   > What's the proper way for client to retrieve rest/client and other ports, especially if they are specified within portRange? From my point of view we might add [rest/client/etc]Adresses to Ignite.localNode() public api and in that case we should wrap IgniteInternalExcpetion with proper public one.
   
   I agree, this is more of a temporary solution, I think it should be replaced in a separate ticket. What do you think?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sanpwc commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sanpwc commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r812041209



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {

Review comment:
       There are two main points to keep constructors simple:
   - Exception handling. If something went wrong during start, there's always a component provided by constructor as an entry point for cleanup or recovery. In other words there's always an object on which to call close
   ```
   IgniteComponent c = new Component() // No tough logic goes here;
   try {
   c.start();
   } catch(Exception e) {
   c.close(); // Cleaning all resources that might be instantiated; 
   }
   ```
   - In general there are no guaranties that un-started components will be able to register listeners etc. I'd rather prefer simple bifurcation of full-fledged started component, and only-instantiated which is only suitable for calling the start method. That'll also ease migration to DI a bit.
   
   > If message handlers are registered in "start", then the network layer is already running, and we can miss some messages. 
   Seems that currently it's not possible. If it is, let's introduce onBeforeStart() or similar.
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811241887



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);
+
+                    // TODO: drop the Raft storage as well, https://issues.apache.org/jira/browse/IGNITE-16471
+                } else if (msg instanceof CmgInitMessage) {
+                    assert correlationId != null;
+
+                    String[] nodeIds = ((CmgInitMessage) msg).cmgNodes();
+
+                    List<ClusterNode> nodes = resolveNodes(clusterService, Arrays.asList(nodeIds));
+
+                    raftManager.prepareRaftGroup(CMG_RAFT_GROUP_NAME, nodes, CmgRaftGroupListener::new)
+                            .whenComplete((service, e) -> {
+                                if (e == null) {
+                                    Peer leader = service.leader();
+
+                                    assert leader != null;
+
+                                    messagingService.respond(addr, leaderElectedResponse(msgFactory, leader), correlationId);
+                                } else {
+                                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                                }
+                            });
+                }
+            } catch (Exception e) {
+                log.error("Exception when initializing the CMG", e);
+
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, e), correlationId);
+                }
+            } finally {
+                busyLock.leaveBusy();

Review comment:
       `busyLock` is needed to prevent the asynchronous operations from starting if the node is stopped. Otherwise we will obtain the necessary futures and will wait for them inside the `stop` method




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r811727673



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> metastorageNodes = resolveNodes(clusterService, metaStorageNodeNames);

Review comment:
       There's a TODO below, I'm going to use these variable in the next PR. In this PR these variable are left for validation purposes, you are correct in that regard




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sanpwc commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sanpwc commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r812113925



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);

Review comment:
       https://github.com/apache/ignite-3/pull/659#discussion_r810516874




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810941935



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterManagementGroupManager.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitMessageGroup;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.rest.InitCommandHandler;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.rest.RestModule;
+import org.apache.ignite.internal.rest.routes.Route;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.raft.client.Peer;
+
+/**
+ * Ignite component responsible for cluster initialization and managing the Cluster Management Raft Group.
+ */
+public class ClusterManagementGroupManager implements IgniteComponent {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterManagementGroupManager.class);
+
+    /** CMG Raft group name. */
+    private static final String CMG_RAFT_GROUP_NAME = "cmg_raft_group";
+
+    /** Init REST endpoint path. */
+    private static final String REST_ENDPOINT = "/management/v1/cluster/init";
+
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    private final ClusterService clusterService;
+
+    private final Loza raftManager;
+
+    private final RestModule restModule;
+
+    /** Handles cluster initialization flow. */
+    private final ClusterInitializer clusterInitializer;
+
+    /** Constructor. */
+    public ClusterManagementGroupManager(ClusterService clusterService, Loza raftManager, RestModule restModule) {
+        this.clusterService = clusterService;
+        this.raftManager = raftManager;
+        this.restModule = restModule;
+        this.clusterInitializer = new ClusterInitializer(clusterService);
+
+        MessagingService messagingService = clusterService.messagingService();
+
+        var msgFactory = new InitMessagesFactory();
+
+        messagingService.addMessageHandler(InitMessageGroup.class, (msg, addr, correlationId) -> {
+            if (!busyLock.enterBusy()) {
+                if (correlationId != null) {
+                    messagingService.respond(addr, errorResponse(msgFactory, new NodeStoppingException()), correlationId);
+                }
+
+                return;
+            }
+
+            try {
+                if (msg instanceof CancelInitMessage) {
+                    log.info("CMG initialization cancelled, reason: " + ((CancelInitMessage) msg).reason());
+
+                    raftManager.stopRaftGroup(CMG_RAFT_GROUP_NAME);

Review comment:
       who puts them in the Vault? Raft manager?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810933498



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/rest/InitCommandHandler.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.join.rest;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import org.apache.ignite.internal.join.ClusterInitializer;
+import org.apache.ignite.internal.join.Leaders;
+import org.apache.ignite.internal.rest.ErrorResult;
+import org.apache.ignite.internal.rest.netty.RestApiHttpRequest;
+import org.apache.ignite.internal.rest.netty.RestApiHttpResponse;
+import org.apache.ignite.internal.rest.routes.RequestHandler;
+import org.apache.ignite.lang.IgniteLogger;
+
+/**
+ * REST handler for the {@link InitCommand}.
+ */
+public class InitCommandHandler implements RequestHandler {
+    private static final IgniteLogger log = IgniteLogger.forClass(InitCommandHandler.class);
+
+    private final ObjectMapper objectMapper = new ObjectMapper();
+
+    private final ClusterInitializer clusterInitializer;
+
+    public InitCommandHandler(ClusterInitializer clusterInitializer) {
+        this.clusterInitializer = clusterInitializer;
+    }
+
+    @Override
+    public CompletableFuture<RestApiHttpResponse> handle(RestApiHttpRequest request, RestApiHttpResponse response) {
+        try {
+            InitCommand command = readContent(request);
+
+            if (log.isInfoEnabled()) {
+                log.info(
+                        "Received init command:\n\tMeta Storage nodes: {}\n\tCMG nodes: {}",
+                        command.metaStorageNodes(),
+                        command.cmgNodes()
+                );
+            }
+
+            return clusterInitializer.initCluster(command.metaStorageNodes(), command.cmgNodes())
+                    .thenApply(leaders -> successResponse(response, leaders))
+                    .exceptionally(e -> errorResponse(response, e));
+        } catch (Exception e) {
+            return CompletableFuture.completedFuture(errorResponse(response, e));
+        }
+    }
+
+    private InitCommand readContent(RestApiHttpRequest restApiHttpRequest) throws IOException {
+        ByteBuf content = restApiHttpRequest.request().content();
+
+        try (InputStream is = new ByteBufInputStream(content)) {
+            return objectMapper.readValue(is, InitCommand.class);
+        }
+    }
+
+    private static RestApiHttpResponse successResponse(RestApiHttpResponse response, Leaders leaders) {
+        if (log.isInfoEnabled()) {
+            log.info(
+                    "Init command executed successfully.\n\tMeta Storage leader ID: {}\n\tCMG leader ID: {}",
+                    leaders.metaStorageLeader(),
+                    leaders.cmgLeader()
+            );
+        }
+
+        response.json(Map.of(
+                "metaStorageLeaderId", leaders.metaStorageLeader(),
+                "cmgLeaderId", leaders.cmgLeader()
+        ));
+
+        return response;
+    }
+
+    private static RestApiHttpResponse errorResponse(RestApiHttpResponse response, Throwable e) {
+        if (e instanceof CompletionException) {
+            e = e.getCause();

Review comment:
       > What if e.getCause() is IgniteInternalException? Is it possible?
   
   I think it can be any kind of exception
   
   > If true, it'll be wrapped with Ignite(Public)Exception, isn't it?
   
   Sorry, I don't understand the question, can you elaborate?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r810934985



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/rest/InitCommandHandler.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.internal.join.rest;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import org.apache.ignite.internal.join.ClusterInitializer;
+import org.apache.ignite.internal.join.Leaders;
+import org.apache.ignite.internal.rest.ErrorResult;
+import org.apache.ignite.internal.rest.netty.RestApiHttpRequest;
+import org.apache.ignite.internal.rest.netty.RestApiHttpResponse;
+import org.apache.ignite.internal.rest.routes.RequestHandler;
+import org.apache.ignite.lang.IgniteLogger;
+
+/**
+ * REST handler for the {@link InitCommand}.
+ */
+public class InitCommandHandler implements RequestHandler {
+    private static final IgniteLogger log = IgniteLogger.forClass(InitCommandHandler.class);
+
+    private final ObjectMapper objectMapper = new ObjectMapper();
+
+    private final ClusterInitializer clusterInitializer;
+
+    public InitCommandHandler(ClusterInitializer clusterInitializer) {
+        this.clusterInitializer = clusterInitializer;
+    }
+
+    @Override
+    public CompletableFuture<RestApiHttpResponse> handle(RestApiHttpRequest request, RestApiHttpResponse response) {

Review comment:
       This is an override, so the javadoc from the parent class will be inherited




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r818718471



##########
File path: modules/join/src/main/java/org/apache/ignite/internal/join/ClusterInitializer.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.internal.join;
+
+import static org.apache.ignite.internal.join.Utils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.join.messages.CancelInitMessage;
+import org.apache.ignite.internal.join.messages.CmgInitMessage;
+import org.apache.ignite.internal.join.messages.InitErrorMessage;
+import org.apache.ignite.internal.join.messages.InitMessagesFactory;
+import org.apache.ignite.internal.join.messages.LeaderElectedMessage;
+import org.apache.ignite.internal.join.messages.MetastorageInitMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty,
+     *      in which case {@code metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Leaders> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");
+            }
+
+            cmgNodeNames = cmgNodeNames.isEmpty() ? metaStorageNodeNames : cmgNodeNames;
+
+            @SuppressWarnings("unused")
+            List<ClusterNode> metastorageNodes = resolveNodes(clusterService, metaStorageNodeNames);

Review comment:
       no longer valid




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] SammyVimes merged pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
SammyVimes merged pull request #659:
URL: https://github.com/apache/ignite-3/pull/659


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite-3] sanpwc commented on a change in pull request #659: IGNITE-16528 Implement init command handling

Posted by GitBox <gi...@apache.org>.
sanpwc commented on a change in pull request #659:
URL: https://github.com/apache/ignite-3/pull/659#discussion_r820119634



##########
File path: modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterInitializer.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.internal.cluster.management;
+
+import static org.apache.ignite.network.util.ClusterServiceUtils.resolveNodes;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.internal.cluster.management.messages.CancelInitMessage;
+import org.apache.ignite.internal.cluster.management.messages.CmgInitMessage;
+import org.apache.ignite.internal.cluster.management.messages.CmgMessagesFactory;
+import org.apache.ignite.internal.cluster.management.messages.InitCompleteMessage;
+import org.apache.ignite.internal.cluster.management.messages.InitErrorMessage;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Class for performing cluster initialization.
+ */
+public class ClusterInitializer {
+    private static final IgniteLogger log = IgniteLogger.forClass(ClusterInitializer.class);
+
+    private final ClusterService clusterService;
+
+    private final CmgMessagesFactory msgFactory = new CmgMessagesFactory();
+
+    /** Constructor. */
+    ClusterInitializer(ClusterService clusterService) {
+        this.clusterService = clusterService;
+    }
+
+    /**
+     * Initializes the cluster that this node is present in.
+     *
+     * @param metaStorageNodeNames names of nodes that will host the Meta Storage. Cannot be empty.
+     * @param cmgNodeNames names of nodes that will host the Cluster Management Group. Can be empty, in which case {@code
+     * metaStorageNodeNames} will be used instead.
+     * @return future that resolves into leader node IDs if completed successfully.
+     */
+    public CompletableFuture<Void> initCluster(Collection<String> metaStorageNodeNames, Collection<String> cmgNodeNames) {
+        try {
+            if (metaStorageNodeNames.isEmpty()) {
+                throw new IllegalArgumentException("List of metastorage nodes must no be empty");

Review comment:
       typo?  must **no** be empty




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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