You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "valepakh (via GitHub)" <gi...@apache.org> on 2023/06/06 13:49:34 UTC

[GitHub] [ignite-3] valepakh opened a new pull request, #2152: IGNITE-19522 Modify deploy unit command

valepakh opened a new pull request, #2152:
URL: https://github.com/apache/ignite-3/pull/2152

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


-- 
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] PakhomovAlexander commented on a diff in pull request #2152: IGNITE-19522 Modify deploy unit command

Posted by "PakhomovAlexander (via GitHub)" <gi...@apache.org>.
PakhomovAlexander commented on code in PR #2152:
URL: https://github.com/apache/ignite-3/pull/2152#discussion_r1222973642


##########
modules/cli/src/main/java/org/apache/ignite/internal/cli/commands/Options.java:
##########
@@ -240,6 +241,15 @@ public static final class Constants {
         /** Unit path option description. */
         public static final String UNIT_PATH_OPTION_DESC = "Path to deployment unit file or directory";
 
+        /** Unit nodes option long name. */
+        public static final String UNIT_NODES_OPTION = "--nodes";
+
+        /** Unit nodes option short name. */
+        public static final String UNIT_NODES_OPTION_SHORT = "-un";

Review Comment:
   if I am not mistaken, we decided not to use the two-letter symbols for short options. It might be confusing to understand if it means -u -n or -un.



##########
modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/commands/unit/ItDeploymentUnitTest.java:
##########
@@ -156,6 +156,98 @@ void deployDirectory() {
         );
     }
 
+    @Test
+    @DisplayName("Should display correct status after deploy to the specified nodes")
+    void deployToNodesAndStatusCheck() {
+        // When deploy with version
+        String node = allNodeNames().get(1);
+        execute("cluster", "unit", "deploy", "test.unit.id.7", "--version", "1.0.0", "--path", testFile, "--nodes", node);
+
+        // Then
+        assertAll(
+                this::assertExitCodeIsZero,
+                this::assertErrOutputIsEmpty,
+                () -> assertOutputContains("Done")
+        );
+
+        await().untilAsserted(() -> {
+            resetOutput();
+            execute("cluster", "unit", "list", "test.unit.id.7");
+
+            // Unit is deployed on all requested nodes
+            assertAll(
+                    this::assertExitCodeIsZero,
+                    this::assertErrOutputIsEmpty,
+                    () -> assertOutputContains("1.0.0"),
+                    () -> assertOutputContains("DEPLOYED")
+            );
+
+            resetOutput();
+            execute("node", "unit", "list", "--node-url", "http://localhost:10301", "test.unit.id.7");

Review Comment:
   could you check that it is also deployed to the CMG group?



##########
modules/code-deployment/src/main/java/org/apache/ignite/internal/deployunit/IgniteDeployment.java:
##########
@@ -28,29 +28,80 @@
  */
 public interface IgniteDeployment extends IgniteComponent {
     /**
-     * Deploy provided unit to current node.
-     * After deploy finished, this deployment unit will be place to CMG group asynchronously.
+     * Deploy provided unit to the current node. After the deploy is finished, the unit will be placed to the CMG group, if
+     * {@code deployMode} is {@code MAJORITY}, or to all available units, if {@code deployMode} is {@code ALL} asynchronously.
      *
      * @param id Unit identifier. Not empty and not null.
      * @param version Unit version.
      * @param deploymentUnit Unit content.
+     * @param deployMode Initial deploy mode.
      * @return Future with success or not result.
      */
-    default CompletableFuture<Boolean> deployAsync(String id, Version version, DeploymentUnit deploymentUnit) {
-        return deployAsync(id, version, false, deploymentUnit);
+    default CompletableFuture<Boolean> deployAsync(
+            String id,
+            Version version,
+            DeploymentUnit deploymentUnit,
+            InitialDeployMode deployMode
+    ) {
+        return deployAsync(id, version, false, deploymentUnit, deployMode);
     }
 
     /**
-     * Deploy provided unit to current node.
-     * After deploy finished, this deployment unit will be place to CMG group asynchronously.
+     * Deploy provided unit to the current node. After the deploy is finished, the unit will be placed to the CMG group, if
+     * {@code deployMode} is {@code MAJORITY}, or to all available units, if {@code deployMode} is {@code ALL} asynchronously.
      *
      * @param id Unit identifier. Not empty and not null.
      * @param version Unit version.
      * @param force Force redeploy if unit with provided id and version exists.
      * @param deploymentUnit Unit content.
+     * @param deployMode Initial deploy mode.
      * @return Future with success or not result.
      */
-    CompletableFuture<Boolean> deployAsync(String id, Version version, boolean force, DeploymentUnit deploymentUnit);
+    CompletableFuture<Boolean> deployAsync(
+            String id,
+            Version version,
+            boolean force,
+            DeploymentUnit deploymentUnit,
+            InitialDeployMode deployMode
+    );
+
+    /**
+     * Deploy provided unit to the current node. After the deploy is finished, the unit will be placed to the CMG group and to the nodes
+     * passed in the @{code initialNodes} list asynchronously.
+     *
+     * @param id Unit identifier. Not empty and not null.
+     * @param version Unit version.
+     * @param deploymentUnit Unit content.
+     * @param initialNodes List of nodes to deploy to initially.
+     * @return Future with success or not result.
+     */
+    default CompletableFuture<Boolean> deployAsync(
+            String id,
+            Version version,
+            DeploymentUnit deploymentUnit,
+            List<String> initialNodes

Review Comment:
   Maybe just `nodes` is enogh?



##########
modules/code-deployment/src/main/java/org/apache/ignite/internal/deployunit/DeploymentManagerImpl.java:
##########
@@ -151,32 +155,72 @@ private void onUnitRegister(UnitNodeStatus status, Set<String> deployedNodes) {
     }
 
     @Override
-    public CompletableFuture<Boolean> deployAsync(String id, Version version, boolean force, DeploymentUnit deploymentUnit) {
+    public CompletableFuture<Boolean> deployAsync(
+            String id,
+            Version version,
+            boolean force,
+            DeploymentUnit deploymentUnit,
+            InitialDeployMode deployMode
+    ) {
         checkId(id);
         Objects.requireNonNull(version);
         Objects.requireNonNull(deploymentUnit);
 
-        return cmgManager.cmgNodes()
-                .thenCompose(cmg -> deploymentUnitStore.createClusterStatus(id, version, cmg))
+        return extractNodes(deployMode)
+                .thenCompose(nodesToDeploy ->
+                        doDeploy(id, version, force, deploymentUnit, nodesToDeploy,
+                                undeployed -> deployAsync(id, version, deploymentUnit, deployMode)
+                        )
+                );
+    }
+
+    @Override
+    public CompletableFuture<Boolean> deployAsync(
+            String id,
+            Version version,
+            boolean force,
+            DeploymentUnit deploymentUnit,
+            List<String> initialNodes
+    ) {
+        checkId(id);
+        Objects.requireNonNull(version);
+        Objects.requireNonNull(deploymentUnit);
+
+        return extractNodes(initialNodes)
+                .thenCompose(nodesToDeploy ->
+                        doDeploy(id, version, force, deploymentUnit, nodesToDeploy,
+                                undeployed -> deployAsync(id, version, deploymentUnit, initialNodes)

Review Comment:
   Looks like a recursion, am I 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] PakhomovAlexander commented on a diff in pull request #2152: IGNITE-19522 Modify deploy unit command

Posted by "PakhomovAlexander (via GitHub)" <gi...@apache.org>.
PakhomovAlexander commented on code in PR #2152:
URL: https://github.com/apache/ignite-3/pull/2152#discussion_r1226345268


##########
modules/rest-api/openapi/openapi.yaml:
##########
@@ -469,29 +469,42 @@ paths:
             application/problem+json:
               schema:
                 $ref: '#/components/schemas/Problem'
-  /management/v1/deployment/units:
+  /management/v1/deployment/units/{unitId}/{unitVersion}:
     post:
       tags:
       - deployment
-      summary: Deploy unit REST method.
-      description: Deploys provided unit to the cluster.
-      operationId: deployUnit
+      summary: Deploy unit to specified nodes REST method.
+      description: Deploys provided unit to the cluster with specified initial nodes.
+      operationId: deployUnitToNodes
+      parameters:
+      - name: unitId
+        in: path
+        required: true
+        schema:
+          type: string
+          description: The ID of the deployment unit.
+      - name: unitVersion
+        in: path
+        required: true
+        schema:
+          type: string
+          description: The version of the deployment unit.
+      - name: initialNodes
+        in: query

Review Comment:
   I think it might be moved to the body



-- 
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] PakhomovAlexander merged pull request #2152: IGNITE-19522 Modify deploy unit command

Posted by "PakhomovAlexander (via GitHub)" <gi...@apache.org>.
PakhomovAlexander merged PR #2152:
URL: https://github.com/apache/ignite-3/pull/2152


-- 
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] valepakh commented on a diff in pull request #2152: IGNITE-19522 Modify deploy unit command

Posted by "valepakh (via GitHub)" <gi...@apache.org>.
valepakh commented on code in PR #2152:
URL: https://github.com/apache/ignite-3/pull/2152#discussion_r1226333402


##########
modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/raft/CmgRaftService.java:
##########
@@ -228,6 +228,38 @@ public Set<String> nodeNames() {
                 .collect(toSet());
     }
 
+    /**
+     * Returns a set of consistent IDs of the majority of the voting nodes of the CMG, including a leader.
+     *
+     * @return Set of consistent IDs of the majority of the voting nodes of the CMG, including a leader.
+     */
+    public CompletableFuture<Set<String>> majority() {
+        Peer leader = raftService.leader();
+
+        if (leader == null) {
+            return raftService.refreshLeader().thenCompose(v -> majority());
+        }
+
+        List<Peer> peers = raftService.peers();
+
+        assert peers != null;
+
+        int peersCount = peers.size();
+        String leaderId = leader.consistentId();
+
+        // Take half of the voting peers without the leader.
+        Set<String> result = peers.stream()
+                .map(Peer::consistentId)
+                .filter(consistentId -> !consistentId.equals(leaderId))
+                .limit(peersCount / 2)
+                .collect(toSet());

Review Comment:
   It's a set mainly because the deployment APIs use sets.



-- 
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 diff in pull request #2152: IGNITE-19522 Modify deploy unit command

Posted by "sashapolo (via GitHub)" <gi...@apache.org>.
sashapolo commented on code in PR #2152:
URL: https://github.com/apache/ignite-3/pull/2152#discussion_r1226309820


##########
modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/raft/CmgRaftService.java:
##########
@@ -228,6 +228,38 @@ public Set<String> nodeNames() {
                 .collect(toSet());
     }
 
+    /**
+     * Returns a set of consistent IDs of the majority of the voting nodes of the CMG, including a leader.
+     *
+     * @return Set of consistent IDs of the majority of the voting nodes of the CMG, including a leader.
+     */
+    public CompletableFuture<Set<String>> majority() {
+        Peer leader = raftService.leader();
+
+        if (leader == null) {
+            return raftService.refreshLeader().thenCompose(v -> majority());
+        }
+
+        List<Peer> peers = raftService.peers();
+
+        assert peers != null;
+
+        int peersCount = peers.size();
+        String leaderId = leader.consistentId();
+
+        // Take half of the voting peers without the leader.
+        Set<String> result = peers.stream()
+                .map(Peer::consistentId)
+                .filter(consistentId -> !consistentId.equals(leaderId))
+                .limit(peersCount / 2)
+                .collect(toSet());

Review Comment:
   Also, why is it a set and not a list? 



##########
modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/raft/CmgRaftService.java:
##########
@@ -228,6 +228,38 @@ public Set<String> nodeNames() {
                 .collect(toSet());
     }
 
+    /**
+     * Returns a set of consistent IDs of the majority of the voting nodes of the CMG, including a leader.
+     *
+     * @return Set of consistent IDs of the majority of the voting nodes of the CMG, including a leader.
+     */
+    public CompletableFuture<Set<String>> majority() {
+        Peer leader = raftService.leader();
+
+        if (leader == null) {
+            return raftService.refreshLeader().thenCompose(v -> majority());
+        }
+
+        List<Peer> peers = raftService.peers();
+
+        assert peers != null;
+
+        int peersCount = peers.size();
+        String leaderId = leader.consistentId();
+
+        // Take half of the voting peers without the leader.
+        Set<String> result = peers.stream()
+                .map(Peer::consistentId)
+                .filter(consistentId -> !consistentId.equals(leaderId))
+                .limit(peersCount / 2)
+                .collect(toSet());

Review Comment:
   It is generally a bad idea to modify a set, returned by `toSet()`, there's no guarantee on its mutability



-- 
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] valepakh commented on a diff in pull request #2152: IGNITE-19522 Modify deploy unit command

Posted by "valepakh (via GitHub)" <gi...@apache.org>.
valepakh commented on code in PR #2152:
URL: https://github.com/apache/ignite-3/pull/2152#discussion_r1223052290


##########
modules/code-deployment/src/main/java/org/apache/ignite/internal/deployunit/DeploymentManagerImpl.java:
##########
@@ -151,32 +155,72 @@ private void onUnitRegister(UnitNodeStatus status, Set<String> deployedNodes) {
     }
 
     @Override
-    public CompletableFuture<Boolean> deployAsync(String id, Version version, boolean force, DeploymentUnit deploymentUnit) {
+    public CompletableFuture<Boolean> deployAsync(
+            String id,
+            Version version,
+            boolean force,
+            DeploymentUnit deploymentUnit,
+            InitialDeployMode deployMode
+    ) {
         checkId(id);
         Objects.requireNonNull(version);
         Objects.requireNonNull(deploymentUnit);
 
-        return cmgManager.cmgNodes()
-                .thenCompose(cmg -> deploymentUnitStore.createClusterStatus(id, version, cmg))
+        return extractNodes(deployMode)
+                .thenCompose(nodesToDeploy ->
+                        doDeploy(id, version, force, deploymentUnit, nodesToDeploy,
+                                undeployed -> deployAsync(id, version, deploymentUnit, deployMode)
+                        )
+                );
+    }
+
+    @Override
+    public CompletableFuture<Boolean> deployAsync(
+            String id,
+            Version version,
+            boolean force,
+            DeploymentUnit deploymentUnit,
+            List<String> initialNodes
+    ) {
+        checkId(id);
+        Objects.requireNonNull(version);
+        Objects.requireNonNull(deploymentUnit);
+
+        return extractNodes(initialNodes)
+                .thenCompose(nodesToDeploy ->
+                        doDeploy(id, version, force, deploymentUnit, nodesToDeploy,
+                                undeployed -> deployAsync(id, version, deploymentUnit, initialNodes)

Review Comment:
   Yes, it is a recursion, it will be called only when `force` is true and we successfully undeployed the existing unit



-- 
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] valepakh commented on a diff in pull request #2152: IGNITE-19522 Modify deploy unit command

Posted by "valepakh (via GitHub)" <gi...@apache.org>.
valepakh commented on code in PR #2152:
URL: https://github.com/apache/ignite-3/pull/2152#discussion_r1226365198


##########
modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/raft/CmgRaftService.java:
##########
@@ -228,6 +228,38 @@ public Set<String> nodeNames() {
                 .collect(toSet());
     }
 
+    /**
+     * Returns a set of consistent IDs of the majority of the voting nodes of the CMG, including a leader.
+     *
+     * @return Set of consistent IDs of the majority of the voting nodes of the CMG, including a leader.
+     */
+    public CompletableFuture<Set<String>> majority() {
+        Peer leader = raftService.leader();
+
+        if (leader == null) {
+            return raftService.refreshLeader().thenCompose(v -> majority());
+        }
+
+        List<Peer> peers = raftService.peers();
+
+        assert peers != null;
+
+        int peersCount = peers.size();
+        String leaderId = leader.consistentId();
+
+        // Take half of the voting peers without the leader.
+        Set<String> result = peers.stream()
+                .map(Peer::consistentId)
+                .filter(consistentId -> !consistentId.equals(leaderId))
+                .limit(peersCount / 2)
+                .collect(toSet());

Review Comment:
   And also because a nearby methods like `validatedNodes` or `nodeNames` also use sets



-- 
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