You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "PakhomovAlexander (via GitHub)" <gi...@apache.org> on 2023/05/30 10:15:15 UTC

[GitHub] [ignite-3] PakhomovAlexander commented on a diff in pull request #2102: IGNITE-19476 On demand deploy units API

PakhomovAlexander commented on code in PR #2102:
URL: https://github.com/apache/ignite-3/pull/2102#discussion_r1209939241


##########
modules/code-deployment/src/main/java/org/apache/ignite/internal/deployunit/DeployTracker.java:
##########
@@ -55,7 +55,7 @@ public <T> CompletableFuture<T> track(String id, Version version, CompletableFut
      * @param version Deployment version identifier.
      */
     public void cancelIfDeploy(String id, Version version) {
-        InFlightFutures futureTracker = inFlightFutures.get(clusterStatusKey(id, version));
+        InFlightFutures futureTracker = inFlightFutures.get(ClusterStatusKey.builder().withId(id).withVersion(version).build().toKey());

Review Comment:
   Can you use record style for setters?



##########
modules/code-deployment/src/main/java/org/apache/ignite/internal/deployunit/metastore/NodeEventListener.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.deployunit.metastore;
+
+import java.util.Set;
+import org.apache.ignite.internal.deployunit.metastore.status.UnitNodeStatus;
+
+/**
+ * Listener of deployment unit node status changes.
+ */
+@FunctionalInterface
+public interface NodeEventListener {
+    /**
+     * Change event.
+     *
+     * @param status Deployment unit status.
+     * @param holders Nodes consistent id.
+     */
+    void call(UnitNodeStatus status, Set<String> holders);

Review Comment:
   ```suggestion
       void onUpdate(UnitNodeStatus status, Set<String> holders);
   ```



##########
modules/code-deployment/src/main/java/org/apache/ignite/internal/deployunit/metastore/status/ClusterStatusKey.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.deployunit.metastore.status;
+
+import static org.apache.ignite.internal.deployunit.metastore.status.UnitKey.DEPLOY_UNIT_PREFIX;
+
+import org.apache.ignite.internal.deployunit.version.Version;
+import org.apache.ignite.lang.ByteArray;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Deployment unit cluster status store key.
+ */
+public class ClusterStatusKey {
+    private static final String UNITS_PREFIX = DEPLOY_UNIT_PREFIX + "units.";
+
+    @Nullable
+    private final String id;
+
+    @Nullable
+    private final Version version;
+
+    private ClusterStatusKey(@Nullable String id, @Nullable Version version) {
+        this.id = id;
+        this.version = version;
+    }
+
+    public @Nullable String getId() {
+        return id;
+    }
+
+    public @Nullable Version getVersion() {
+        return version;
+    }
+
+    /**
+     * Serialize key instance to {@link ByteArray}.
+     *
+     * @return {@link ByteArray} instance with serialized content.
+     */
+    public ByteArray toKey() {

Review Comment:
   ```suggestion
       public ByteArray toByteArray() {
   ```



##########
modules/code-deployment/src/main/java/org/apache/ignite/internal/deployunit/metastore/status/NodeStatusKey.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.deployunit.metastore.status;
+
+import static org.apache.ignite.internal.deployunit.metastore.status.UnitKey.DEPLOY_UNIT_PREFIX;
+
+import org.apache.ignite.internal.deployunit.version.Version;
+import org.apache.ignite.lang.ByteArray;
+
+/**
+ *  Deployment unit node status store key.
+ */
+public class NodeStatusKey {
+    private static final String NODES_PREFIX = DEPLOY_UNIT_PREFIX + "nodes.";
+
+    private final String id;
+
+    private final Version version;
+
+    private final String nodeId;
+
+    /**
+     * Constructor.
+     *
+     * @param id Deployment unit identifier.
+     * @param version Deployment unit version.
+     * @param nodeId Cluster node consistent identifier.
+     */
+    public NodeStatusKey(String id, Version version, String nodeId) {
+        this.id = id;
+        this.version = version;
+        this.nodeId = nodeId;
+    }
+
+
+    public ByteArray toKey() {

Review Comment:
   ```suggestion
       public ByteArray toByteArray() {
   ```



##########
modules/code-deployment/src/main/java/org/apache/ignite/internal/deployunit/metastore/status/NodeStatusKey.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.deployunit.metastore.status;
+
+import static org.apache.ignite.internal.deployunit.metastore.status.UnitKey.DEPLOY_UNIT_PREFIX;
+
+import org.apache.ignite.internal.deployunit.version.Version;
+import org.apache.ignite.lang.ByteArray;
+
+/**
+ *  Deployment unit node status store key.
+ */
+public class NodeStatusKey {
+    private static final String NODES_PREFIX = DEPLOY_UNIT_PREFIX + "nodes.";
+
+    private final String id;
+
+    private final Version version;
+
+    private final String nodeId;
+
+    /**
+     * Constructor.
+     *
+     * @param id Deployment unit identifier.
+     * @param version Deployment unit version.
+     * @param nodeId Cluster node consistent identifier.
+     */
+    public NodeStatusKey(String id, Version version, String nodeId) {
+        this.id = id;
+        this.version = version;
+        this.nodeId = nodeId;
+    }
+
+
+    public ByteArray toKey() {
+        return UnitKey.toKey(NODES_PREFIX, id, version == null ? null : version.render(), nodeId);
+    }
+
+    /**
+     * Deserializer method.
+     *
+     * @param key Serialized node status key.
+     * @return Deserialized node status key.
+     */
+    public static NodeStatusKey fromKey(byte[] key) {

Review Comment:
   ```suggestion
       public static NodeStatusKey fromBytes(byte[] key) {
   ```



##########
modules/code-deployment/src/test/java/org/apache/ignite/deployment/FileDeployerServiceTest.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.deployment;
+
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.deployunit.FileDeployerService;
+import org.apache.ignite.internal.deployunit.UnitContent;
+import org.apache.ignite.internal.deployunit.version.Version;
+import org.apache.ignite.internal.testframework.WorkDirectory;
+import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.hamcrest.Matchers;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Test suite for {@link FileDeployerService}.
+ */
+@ExtendWith(WorkDirectoryExtension.class)
+public class FileDeployerServiceTest {
+    private final FileDeployerService service = new FileDeployerService();
+
+    @WorkDirectory
+    private Path workDir;
+
+    private Path file1;
+    private Path file2;
+    private Path file3;
+
+    @BeforeEach
+    public void setup() throws IOException {
+        service.initUnitsFolder(workDir);
+
+        file1 = workDir.resolve("file1");
+        file2 = workDir.resolve("file2");
+        file3 = workDir.resolve("file3");
+        IgniteUtils.fillDummyFile(file1, 1024);
+        IgniteUtils.fillDummyFile(file2, 1024);
+        IgniteUtils.fillDummyFile(file3, 1024);
+    }
+
+    @Test
+    public void test() throws IOException {
+        CompletableFuture<Boolean> deployed = service.deploy("id", Version.parseVersion("1.0.0"), content());
+        assertThat(deployed, willBe(true));
+
+        CompletableFuture<UnitContent> unitContent = service.getUnitContent("id", Version.parseVersion("1.0.0"));
+        assertThat(unitContent, willBe(Matchers.equalTo(content())));

Review Comment:
   Matchers can be imported statically.



##########
modules/code-deployment/src/test/java/org/apache/ignite/deployment/metastore/DeploymentUnitStoreImplTest.java:
##########
@@ -48,39 +54,46 @@
  */
 @ExtendWith(WorkDirectoryExtension.class)
 public class DeploymentUnitStoreImplTest {
+    private static final String LOCAL_NODE = "localNode";
 
     private final VaultManager vaultManager = new VaultManager(new InMemoryVaultService());
 
+    private final List<UnitNodeStatus> history = new ArrayList<>();

Review Comment:
   The usage of ArrayList requires tests to run from a single thread otherwise it is not thread-safe. Could you mention this in the comments or use thread-safe collection here?



##########
modules/code-deployment/src/main/java/org/apache/ignite/internal/deployunit/DeploymentManagerImpl.java:
##########
@@ -111,10 +115,38 @@ public DeploymentManagerImpl(ClusterService clusterService,
         this.configuration = configuration;
         this.cmgManager = cmgManager;
         this.workDir = workDir;
-        this.tracker = new DeployTracker();
-        metastore = new DeploymentUnitStoreImpl(metaStorage);
+        tracker = new DeployTracker();
         deployer = new FileDeployerService();
         messaging = new DeployMessagingService(clusterService, cmgManager, deployer, tracker);
+        deploymentUnitStore = new DeploymentUnitStoreImpl(metaStorage,
+                () -> this.clusterService.topologyService().localMember().name(),
+                this::onUnitRegister);
+    }
+
+    private void onUnitRegister(UnitNodeStatus status, Set<String> deployedNodes) {
+        if (status.status() == UPLOADING) {
+            messaging.downloadUnitContent(status.id(), status.version(), new ArrayList<>(deployedNodes))
+                    .thenCompose(content -> deployer.deploy(status.id(), status.version(), content))
+                    .thenApply(deployed -> {
+                        if (deployed) {
+                            return deploymentUnitStore.updateNodeStatus(
+                                    clusterService.topologyService().localMember().name(),
+                                    status.id(),
+                                    status.version(),
+                                    DEPLOYED);
+                        }
+                        return deployed;
+                    });
+        } else if (status.status() == DEPLOYED) {
+            deploymentUnitStore.getClusterStatus(status.id(), status.version())
+                    .thenApply(UnitClusterStatus::initialNodesToDeploy)
+                    .thenApply(deployedNodes::containsAll)
+                    .thenAccept(allRequiredDeployed -> {
+                        if (allRequiredDeployed) {
+                            deploymentUnitStore.updateClusterStatus(status.id(), status.version(), DEPLOYED);

Review Comment:
   Am I right to assume this line will be executed on every node? So, we make N calls with updated cluster status?



##########
modules/code-deployment/src/main/java/org/apache/ignite/internal/deployunit/metastore/status/ClusterStatusKey.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.deployunit.metastore.status;
+
+import static org.apache.ignite.internal.deployunit.metastore.status.UnitKey.DEPLOY_UNIT_PREFIX;
+
+import org.apache.ignite.internal.deployunit.version.Version;
+import org.apache.ignite.lang.ByteArray;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Deployment unit cluster status store key.
+ */
+public class ClusterStatusKey {
+    private static final String UNITS_PREFIX = DEPLOY_UNIT_PREFIX + "units.";
+
+    @Nullable
+    private final String id;
+
+    @Nullable
+    private final Version version;
+
+    private ClusterStatusKey(@Nullable String id, @Nullable Version version) {
+        this.id = id;
+        this.version = version;
+    }
+
+    public @Nullable String getId() {
+        return id;
+    }
+
+    public @Nullable Version getVersion() {
+        return version;
+    }
+
+    /**
+     * Serialize key instance to {@link ByteArray}.
+     *
+     * @return {@link ByteArray} instance with serialized content.
+     */
+    public ByteArray toKey() {
+        return UnitKey.toKey(UNITS_PREFIX, id, version == null ? null : version.render());
+    }
+
+    /**
+     * Deserialize key instance {@link ClusterStatusKey} from byte array.
+     *
+     * @param key Serialized key in byte array.
+     * @return Deserialized deployment unit cluster key.
+     */
+    public static ClusterStatusKey fromKey(byte[] key) {
+        String[] parse = UnitKey.fromKey(UNITS_PREFIX, key);
+        int length = parse.length;
+        String id = length > 0 ? parse[0] : null;
+        Version version = length > 1 ? Version.parseVersion(parse[1]) : null;
+
+        return builder().withId(id).withVersion(version).build();
+    }
+
+    public static ClusterStatusKeyBuilder builder() {
+        return new ClusterStatusKeyBuilder();
+    }
+
+    /**
+     * Builder for {@link ClusterStatusKey}.
+     */
+    public static class ClusterStatusKeyBuilder {
+        private String id;
+
+        private Version version;
+
+        public ClusterStatusKeyBuilder withId(String id) {

Review Comment:
   withers -> record-style



##########
modules/code-deployment/src/main/java/org/apache/ignite/internal/deployunit/metastore/status/ClusterStatusKey.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.deployunit.metastore.status;
+
+import static org.apache.ignite.internal.deployunit.metastore.status.UnitKey.DEPLOY_UNIT_PREFIX;
+
+import org.apache.ignite.internal.deployunit.version.Version;
+import org.apache.ignite.lang.ByteArray;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Deployment unit cluster status store key.
+ */
+public class ClusterStatusKey {
+    private static final String UNITS_PREFIX = DEPLOY_UNIT_PREFIX + "units.";
+
+    @Nullable
+    private final String id;
+
+    @Nullable
+    private final Version version;
+
+    private ClusterStatusKey(@Nullable String id, @Nullable Version version) {
+        this.id = id;
+        this.version = version;
+    }
+
+    public @Nullable String getId() {
+        return id;
+    }
+
+    public @Nullable Version getVersion() {
+        return version;
+    }
+
+    /**
+     * Serialize key instance to {@link ByteArray}.
+     *
+     * @return {@link ByteArray} instance with serialized content.
+     */
+    public ByteArray toKey() {
+        return UnitKey.toKey(UNITS_PREFIX, id, version == null ? null : version.render());
+    }
+
+    /**
+     * Deserialize key instance {@link ClusterStatusKey} from byte array.
+     *
+     * @param key Serialized key in byte array.
+     * @return Deserialized deployment unit cluster key.
+     */
+    public static ClusterStatusKey fromKey(byte[] key) {

Review Comment:
   ```suggestion
       public static ClusterStatusKey fromBytes(byte[] key) {
   ```



##########
modules/code-deployment/src/main/java/org/apache/ignite/internal/deployunit/metastore/status/UnitKey.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.internal.deployunit.metastore.status;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.Base64.Encoder;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.ignite.lang.ByteArray;
+
+/**
+ * Helper for deployment units metastore keys generation.
+ */
+public final class UnitKey {
+    public static final String DEPLOY_UNIT_PREFIX = "deploy-unit.";
+
+    private static final String DELIMITER = ":";
+
+    private UnitKey() {
+
+    }
+
+    static String[] fromKey(String prefix, byte[] key) {
+        String s = new String(key, StandardCharsets.UTF_8);
+
+        if (!s.startsWith(prefix)) {
+            return new String[0];
+        }
+
+        String content = s.substring(prefix.length());
+        return Arrays.stream(content.split(DELIMITER))
+                .map(e -> new String(Base64.getDecoder().decode(e), StandardCharsets.UTF_8))
+                .toArray(String[]::new);
+    }
+
+    static ByteArray toKey(String prefix, String... args) {

Review Comment:
   ```suggestion
       static ByteArray toByteArray(String prefix, String... args) {
   ```



##########
modules/code-deployment/src/main/java/org/apache/ignite/internal/deployunit/metastore/status/NodeStatusKey.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.deployunit.metastore.status;
+
+import static org.apache.ignite.internal.deployunit.metastore.status.UnitKey.DEPLOY_UNIT_PREFIX;
+
+import org.apache.ignite.internal.deployunit.version.Version;
+import org.apache.ignite.lang.ByteArray;
+
+/**
+ *  Deployment unit node status store key.
+ */
+public class NodeStatusKey {
+    private static final String NODES_PREFIX = DEPLOY_UNIT_PREFIX + "nodes.";
+
+    private final String id;
+
+    private final Version version;
+
+    private final String nodeId;
+
+    /**
+     * Constructor.
+     *
+     * @param id Deployment unit identifier.
+     * @param version Deployment unit version.
+     * @param nodeId Cluster node consistent identifier.
+     */
+    public NodeStatusKey(String id, Version version, String nodeId) {
+        this.id = id;
+        this.version = version;
+        this.nodeId = nodeId;
+    }
+
+
+    public ByteArray toKey() {
+        return UnitKey.toKey(NODES_PREFIX, id, version == null ? null : version.render(), nodeId);
+    }
+
+    /**
+     * Deserializer method.
+     *
+     * @param key Serialized node status key.
+     * @return Deserialized node status key.
+     */
+    public static NodeStatusKey fromKey(byte[] key) {
+        String[] parse = UnitKey.fromKey(NODES_PREFIX, key);
+        int length = parse.length;
+        String id = length > 0 ? parse[0] : null;
+        Version version = length > 1 ? Version.parseVersion(parse[1]) : null;
+        String nodeId = length > 2 ? parse[2] : null;
+
+        return builder().withId(id).withVersion(version).withNodeId(nodeId).build();
+    }
+
+    /**
+     * Returns builder {@link NodeStatusKeyBuilder}.
+     *
+     * @return builder {@link NodeStatusKeyBuilder}.
+     */
+    public static NodeStatusKeyBuilder builder() {
+        return new NodeStatusKeyBuilder();
+    }
+
+    public String id() {
+        return id;
+    }
+
+    public Version version() {
+        return version;
+    }
+
+    public String nodeId() {
+        return nodeId;
+    }
+
+    /**
+     * Builder for {@link NodeStatusKey}.
+     */
+    public static class NodeStatusKeyBuilder {
+        private String id;
+
+        private Version version;
+
+        private String nodeId;
+
+        public NodeStatusKeyBuilder withId(String id) {
+            this.id = id;
+            return this;
+        }
+
+        public NodeStatusKeyBuilder withVersion(Version version) {

Review Comment:
   withers -> record-style



##########
modules/code-deployment/src/main/java/org/apache/ignite/internal/deployunit/metastore/status/NodeStatusKey.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.deployunit.metastore.status;
+
+import static org.apache.ignite.internal.deployunit.metastore.status.UnitKey.DEPLOY_UNIT_PREFIX;
+
+import org.apache.ignite.internal.deployunit.version.Version;
+import org.apache.ignite.lang.ByteArray;
+
+/**
+ *  Deployment unit node status store key.
+ */
+public class NodeStatusKey {
+    private static final String NODES_PREFIX = DEPLOY_UNIT_PREFIX + "nodes.";
+
+    private final String id;
+
+    private final Version version;
+
+    private final String nodeId;
+
+    /**
+     * Constructor.
+     *
+     * @param id Deployment unit identifier.
+     * @param version Deployment unit version.
+     * @param nodeId Cluster node consistent identifier.
+     */
+    public NodeStatusKey(String id, Version version, String nodeId) {
+        this.id = id;
+        this.version = version;
+        this.nodeId = nodeId;
+    }
+
+

Review Comment:
   Extra empty line



##########
modules/code-deployment/src/main/java/org/apache/ignite/internal/deployunit/metastore/status/UnitKey.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.internal.deployunit.metastore.status;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.Base64.Encoder;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.ignite.lang.ByteArray;
+
+/**
+ * Helper for deployment units metastore keys generation.
+ */
+public final class UnitKey {
+    public static final String DEPLOY_UNIT_PREFIX = "deploy-unit.";
+
+    private static final String DELIMITER = ":";
+
+    private UnitKey() {
+
+    }
+
+    static String[] fromKey(String prefix, byte[] key) {

Review Comment:
   ```suggestion
       static String[] fromBytes(String prefix, byte[] key) {
   ```



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