IGNITE-17610 Added unified syntax for starting and canceling snapshot operations using control script (#10236)

diff --git a/docs/_docs/snapshots/snapshots.adoc b/docs/_docs/snapshots/snapshots.adoc
index ed26326..7a4e71e 100644
--- a/docs/_docs/snapshots/snapshots.adoc
+++ b/docs/_docs/snapshots/snapshots.adoc
@@ -113,12 +113,6 @@
 
 # Create a cluster snapshot named "snapshot_09062021" in the "/tmp/ignite/snapshots" folder (the full path to the snapshot files will be /tmp/ignite/snapshots/snapshot_09062021):
 control.(sh|bat) --snapshot create snapshot_09062021 -dest /tmp/ignite/snapshots
-
-# Cancel a running snapshot named "snapshot_09062021":
-control.(sh|bat) --snapshot cancel snapshot_09062021
-
-# Kill a running snapshot named "snapshot_09062021":
-control.(sh|bat) --kill SNAPSHOT snapshot_09062021
 ----
 
 === Using JMX
@@ -129,7 +123,6 @@
 |===
 |Method | Description
 |createSnapshot(String snpName) | Create a snapshot.
-|cancelSnapshot(String snpName) | Cancel a snapshot on the node initiated its creation.
 |===
 
 === Using Java API
@@ -213,7 +206,7 @@
 [source,shell]
 ----
 # Restore cache group "snapshot-cache" from the snapshot "snapshot_02092020".
-control.(sh|bat) --snapshot restore snapshot_02092020 --start --groups snapshot-cache
+control.(sh|bat) --snapshot restore snapshot_02092020 --groups snapshot-cache
 ----
 --
 
@@ -223,19 +216,16 @@
 [source,shell]
 ----
 # Start restoring all user-created cache groups from the snapshot "snapshot_09062021" in the background.
-control.(sh|bat) --snapshot restore snapshot_09062021 --start
+control.(sh|bat) --snapshot restore snapshot_09062021
 
 # Start restoring all user-created cache groups from the snapshot "snapshot_09062021" and wait for the entire operation to complete.
-control.(sh|bat) --snapshot restore snapshot_09062021 --start --sync
+control.(sh|bat) --snapshot restore snapshot_09062021 --sync
 
 # Start restoring all user-created cache groups from the snapshot "snapshot_09062021" located in the "/tmp/ignite/snapshots" folder (the full path to the snapshot files should be /tmp/ignite/snapshots/snapshot_09062021):
 control.(sh|bat) --snapshot restore snapshot_09062021 --src /tmp/ignite/snapshots
 
 # Start restoring only "cache-group1" and "cache-group2" from the snapshot "snapshot_09062021" in the background.
-control.(sh|bat) --snapshot restore snapshot_09062021 --start --groups cache-group1,cache-group2
-
-# Cancel the restore operation for "snapshot_09062021".
-control.(sh|bat) --snapshot restore snapshot_09062021 --cancel
+control.(sh|bat) --snapshot restore snapshot_09062021 --groups cache-group1,cache-group2
 ----
 
 == Getting Snapshot Operation Status
@@ -269,6 +259,44 @@
 ----
 --
 
+== Cancelling Snapshot Operation
+
+To abort create/restore snapshot operation you need to obtain an `operation request ID`.
+This identifier is displayed when you start a snapshot operation using the CLI. It can also be obtained using the status command and from snapshot metrics.
+
+[tabs]
+--
+tab:Unix[]
+[source,shell]
+----
+# Cancel a running snapshot operation with ID "9ec229f1-e0df-41ff-9434-6f08ba7d05bd":
+control.sh --snapshot cancel --id 9ec229f1-e0df-41ff-9434-6f08ba7d05bd
+
+# Kill a running snapshot operation with ID "9ec229f1-e0df-41ff-9434-6f08ba7d05bd":
+control.sh --kill SNAPSHOT 9ec229f1-e0df-41ff-9434-6f08ba7d05bd
+----
+
+tab:Windows[]
+[source,shell]
+----
+# Cancel a running snapshot operation with ID "9ec229f1-e0df-41ff-9434-6f08ba7d05bd":
+control.bat --snapshot cancel --id 9ec229f1-e0df-41ff-9434-6f08ba7d05bd
+
+# Kill a running snapshot operation with ID "9ec229f1-e0df-41ff-9434-6f08ba7d05bd":
+control.bat --kill SNAPSHOT 9ec229f1-e0df-41ff-9434-6f08ba7d05bd
+----
+
+tab:JMX[]
+You can also abort a snapshot operation via the `SnapshotMXBean` interface:
+[source,java]
+----
+SnapshotMXBean mxBean = ...;
+
+// Cancel a running snapshot operation with ID "9ec229f1-e0df-41ff-9434-6f08ba7d05bd":
+mxBean.cancelSnapshotOperation("9ec229f1-e0df-41ff-9434-6f08ba7d05bd");
+----
+--
+
 == Consistency Guarantees
 
 All snapshots are fully consistent in terms of concurrent cluster-wide operations as well as ongoing changes with Ignite.
diff --git a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/query/KillCommand.java b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/query/KillCommand.java
index 12926aa..0ddd87a 100644
--- a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/query/KillCommand.java
+++ b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/query/KillCommand.java
@@ -40,6 +40,7 @@
 import org.apache.ignite.internal.visor.service.VisorCancelServiceTask;
 import org.apache.ignite.internal.visor.service.VisorCancelServiceTaskArg;
 import org.apache.ignite.internal.visor.snapshot.VisorSnapshotCancelTask;
+import org.apache.ignite.internal.visor.snapshot.VisorSnapshotCancelTaskArg;
 import org.apache.ignite.internal.visor.tx.VisorTxOperation;
 import org.apache.ignite.internal.visor.tx.VisorTxTask;
 import org.apache.ignite.internal.visor.tx.VisorTxTaskArg;
@@ -192,7 +193,9 @@
                 break;
 
             case SNAPSHOT:
-                taskArgs = argIter.nextArg("Expected snapshot name.");
+                String arg = argIter.nextArg("Expected snapshot operation request ID.");
+
+                taskArgs = new VisorSnapshotCancelTaskArg(UUID.fromString(arg), null);
 
                 taskName = VisorSnapshotCancelTask.class.getName();
 
diff --git a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotCancelCommand.java b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotCancelCommand.java
index 5c29e9e..1e0f743 100644
--- a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotCancelCommand.java
+++ b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotCancelCommand.java
@@ -17,10 +17,21 @@
 
 package org.apache.ignite.internal.commandline.snapshot;
 
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.UUID;
 import java.util.logging.Logger;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.commandline.CommandArgIterator;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.visor.snapshot.VisorSnapshotCancelTask;
+import org.apache.ignite.internal.visor.snapshot.VisorSnapshotCancelTaskArg;
 
 import static org.apache.ignite.internal.commandline.CommandList.SNAPSHOT;
+import static org.apache.ignite.internal.commandline.CommandLogger.or;
+import static org.apache.ignite.internal.commandline.snapshot.SnapshotCancelCommandOption.ID;
+import static org.apache.ignite.internal.commandline.snapshot.SnapshotCancelCommandOption.NAME;
 
 /**
  * Sub-command to cancel running snapshot.
@@ -32,7 +43,42 @@
     }
 
     /** {@inheritDoc} */
+    @Override public Object execute(GridClientConfiguration clientCfg, Logger log) throws Exception {
+        if (!F.isEmpty(((VisorSnapshotCancelTaskArg)cmdArg).snapshotName()))
+            log.warning("'" + NAME.arg() + "' option is deprecated, please use operation request ID to cancel operation.");
+
+        return super.execute(clientCfg, log);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void parseArguments(CommandArgIterator argIter) {
+        UUID reqId = null;
+        String snpName = null;
+        String explainMsg = "One of " + Arrays.toString(SnapshotCancelCommandOption.values()) + " is expected.";
+
+        String arg = argIter.nextArg(explainMsg);
+
+        if (arg.equals(ID.argName()))
+            reqId = UUID.fromString(argIter.nextArg("Expected operation request ID."));
+        else if (arg.equals(NAME.argName()))
+            snpName = argIter.nextArg("Expected snapshot name.");
+        else
+            throw new IllegalArgumentException("Unexpected argument: " + argIter.peekNextArg() + ". " + explainMsg);
+
+        if (argIter.hasNextSubArg())
+            throw new IllegalArgumentException("No more arguments expected.");
+
+        cmdArg = new VisorSnapshotCancelTaskArg(reqId, snpName);
+    }
+
+    /** {@inheritDoc} */
     @Override public void printUsage(Logger log) {
-        usage(log, "Cancel running snapshot:", SNAPSHOT, generalUsageOptions(), name(), SNAPSHOT_NAME_ARG);
+        Map<String, String> params = new LinkedHashMap<>();
+
+        params.put(ID.argName() + " " + ID.arg(), ID.description());
+        params.put(NAME.argName() + " " + NAME.arg(), NAME.description());
+
+        usage(log, "Cancel running snapshot operation:", SNAPSHOT,
+            params, name(), or(ID.argName() + " " + ID.arg(), NAME.argName() + " " + NAME.arg()));
     }
 }
diff --git a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotCancelCommandOption.java b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotCancelCommandOption.java
new file mode 100644
index 0000000..41608d8
--- /dev/null
+++ b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotCancelCommandOption.java
@@ -0,0 +1,71 @@
+/*
+ * 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.commandline.snapshot;
+
+import org.apache.ignite.internal.commandline.argument.CommandArg;
+
+/**
+ * Snapshot cancel command options.
+ */
+public enum SnapshotCancelCommandOption implements CommandArg {
+    /** Snapshot operation request ID. */
+    ID("--id", "id", "Snapshot operation request ID."),
+
+    /** Snapshot name. */
+    NAME("--name", "name", "Snapshot name (deprecated).");
+
+    /** Name. */
+    private final String name;
+
+    /** Argument. */
+    private final String arg;
+
+    /** Description. */
+    private final String desc;
+
+    /**
+     * @param name Name.
+     * @param arg Argument.
+     * @param desc Description.
+     */
+    SnapshotCancelCommandOption(String name, String arg, String desc) {
+        this.name = name;
+        this.arg = arg;
+        this.desc = desc;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String argName() {
+        return name;
+    }
+
+    /** @return Argument. */
+    public String arg() {
+        return arg;
+    }
+
+    /** @return Description. */
+    public String description() {
+        return desc;
+    }
+
+    /** @return Argument name. */
+    @Override public String toString() {
+        return argName();
+    }
+}
diff --git a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotRestoreCommand.java b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotRestoreCommand.java
index ae90116..0770723 100644
--- a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotRestoreCommand.java
+++ b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotRestoreCommand.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.commandline.snapshot;
 
-import java.util.Collection;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Set;
@@ -37,7 +36,6 @@
 import static org.apache.ignite.internal.commandline.snapshot.SnapshotRestoreCommandOption.SYNC;
 import static org.apache.ignite.internal.commandline.snapshot.SnapshotSubcommands.RESTORE;
 import static org.apache.ignite.internal.visor.snapshot.VisorSnapshotRestoreTaskAction.START;
-import static org.apache.ignite.internal.visor.snapshot.VisorSnapshotRestoreTaskAction.STATUS;
 
 /**
  * Sub-command to restore snapshot.
@@ -50,28 +48,54 @@
 
     /** {@inheritDoc} */
     @Override public Object execute(GridClientConfiguration clientCfg, Logger log) throws Exception {
-        if (cmdArg instanceof VisorSnapshotRestoreTaskArg && ((VisorSnapshotRestoreTaskArg)cmdArg).jobAction() == STATUS)
-            log.warning("Command deprecated. Use '" + SNAPSHOT + ' ' + SnapshotSubcommands.STATUS + "' instead.");
+        explainDeprecatedOptions(cmdArg, log);
 
-        Object res = super.execute(clientCfg, log);
+        return super.execute(clientCfg, log);
+    }
 
-        log.info(String.valueOf(res));
+    /**
+     * @param cmdArg Command argument.
+     * @param log Logger.
+     */
+    private void explainDeprecatedOptions(Object cmdArg, Logger log) {
+        VisorSnapshotRestoreTaskAction action = ((VisorSnapshotRestoreTaskArg)cmdArg).jobAction();
 
-        return res;
+        if (action == null)
+            return;
+
+        switch (action) {
+            case START:
+                log.warning("Command option '--" + START.toString().toLowerCase() + "' is redundant and must be avoided.");
+
+                break;
+
+            case CANCEL:
+                log.warning("Command deprecated. Use `" + SNAPSHOT + ' ' + SnapshotSubcommands.CANCEL + "' instead.");
+
+                break;
+
+            case STATUS:
+                log.warning("Command deprecated. Use '" + SNAPSHOT + ' ' + SnapshotSubcommands.STATUS + "' instead.");
+
+                break;
+        }
     }
 
     /** {@inheritDoc} */
     @Override public void parseArguments(CommandArgIterator argIter) {
         String snpName = argIter.nextArg("Expected snapshot name.");
-        VisorSnapshotRestoreTaskAction restoreAction = parseAction(argIter);
+        VisorSnapshotRestoreTaskAction restoreAction = parseAction(argIter.peekNextArg());
         String snpPath = null;
         Set<String> grpNames = null;
         boolean sync = false;
 
+        if (restoreAction != null)
+            argIter.nextArg(null);
+
         while (argIter.hasNextSubArg()) {
             String arg = argIter.nextArg(null);
 
-            if (restoreAction != START) {
+            if (restoreAction != null && restoreAction != START) {
                 throw new IllegalArgumentException("Invalid argument: " + arg + ". " +
                     "Action \"--" + restoreAction.name().toLowerCase() + "\" does not support specified option.");
             }
@@ -124,38 +148,35 @@
         startParams.put(SOURCE.argName() + " " + SOURCE.arg(), SOURCE.description());
         startParams.put(SYNC.argName(), SYNC.description());
 
-        usage(log, "Restore snapshot:", SNAPSHOT, startParams, RESTORE.toString(), SNAPSHOT_NAME_ARG, "--start",
+        usage(log, "Restore snapshot:", SNAPSHOT, startParams, RESTORE.toString(), SNAPSHOT_NAME_ARG,
             optional(GROUPS.argName(), GROUPS.arg()), optional(SOURCE.argName(), SOURCE.arg()), optional(SYNC.argName()));
         usage(log, "Snapshot restore operation status (Command deprecated. Use '" + SNAPSHOT + ' '
-            + SnapshotSubcommands.STATUS + "' instead.):", SNAPSHOT, params, RESTORE.toString(), SNAPSHOT_NAME_ARG, "--status");
-        usage(log, "Cancel snapshot restore operation:", SNAPSHOT, params, RESTORE.toString(), SNAPSHOT_NAME_ARG, "--cancel");
+            + SnapshotSubcommands.STATUS + "' instead):", SNAPSHOT, params, RESTORE.toString(), SNAPSHOT_NAME_ARG, "--status");
+        usage(log, "Cancel snapshot restore operation (Command deprecated. Use '" + SNAPSHOT + ' '
+            + SnapshotSubcommands.CANCEL + "' instead):", SNAPSHOT, params, RESTORE.toString(), SNAPSHOT_NAME_ARG, "--cancel");
     }
 
     /** {@inheritDoc} */
     @Override public String confirmationPrompt() {
         VisorSnapshotRestoreTaskArg arg = (VisorSnapshotRestoreTaskArg)cmdArg;
 
-        return arg.jobAction() != START || arg.groupNames() != null ? null :
+        return (arg.jobAction() != null && arg.jobAction() != START) || arg.groupNames() != null ? null :
             "Warning: command will restore ALL USER-CREATED CACHE GROUPS from the snapshot " + arg.snapshotName() + '.';
     }
 
     /**
-     * @param argIter Argument iterator.
+     * @param arg Argument.
      * @return Snapshot restore operation management action.
      */
-    private VisorSnapshotRestoreTaskAction parseAction(CommandArgIterator argIter) {
-        Collection<String> cmdNames =
-            F.viewReadOnly(F.asList(VisorSnapshotRestoreTaskAction.values()), v -> "--" + v.toString().toLowerCase());
-
-        String actionErrMsg = "One of " + cmdNames + " is expected.";
-
-        String action = argIter.nextArg(actionErrMsg);
+    private VisorSnapshotRestoreTaskAction parseAction(String arg) {
+        if (arg == null)
+            return null;
 
         for (VisorSnapshotRestoreTaskAction val : VisorSnapshotRestoreTaskAction.values()) {
-            if (action.toLowerCase().equals("--" + val.name().toLowerCase()))
+            if (arg.toLowerCase().equals("--" + val.name().toLowerCase()))
                 return val;
         }
 
-        throw new IllegalArgumentException("Invalid argument: " + action + ". " + actionErrMsg);
+        return null;
     }
 }
diff --git a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotStatusCommand.java b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotStatusCommand.java
index c0f2f21..7332682 100644
--- a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotStatusCommand.java
+++ b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotStatusCommand.java
@@ -67,7 +67,7 @@
             s.a("Restore snapshot operation is in progress.").nl();
 
         s.a("Snapshot name: ").a(status.name()).nl();
-        s.a("Operation ID: ").a(status.requestId()).nl();
+        s.a("Operation request ID: ").a(status.requestId()).nl();
         s.a("Started at: ").a(DateFormat.getDateTimeInstance().format(new Date(status.startTime()))).nl();
         s.a("Duration: ").a(X.timeSpan2DHMSM(System.currentTimeMillis() - status.startTime())).nl()
                 .nl();
diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
index fc7c523..fe1be09 100644
--- a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
+++ b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
@@ -105,6 +105,7 @@
 import org.apache.ignite.internal.processors.cache.warmup.WarmUpTestPluginProvider;
 import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage;
 import org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
 import org.apache.ignite.internal.util.BasicRateLimiter;
 import org.apache.ignite.internal.util.distributed.SingleNodeMessage;
 import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
@@ -126,6 +127,7 @@
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.metric.LongMetric;
+import org.apache.ignite.spi.metric.Metric;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.WithSystemProperty;
 import org.apache.ignite.transactions.Transaction;
@@ -166,9 +168,11 @@
 import static org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.SNAPSHOT_METRICS;
 import static org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.SNAPSHOT_TRANSFER_RATE_DMS_KEY;
 import static org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.resolveSnapshotWorkDirectory;
+import static org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRestoreProcess.SNAPSHOT_RESTORE_METRICS;
 import static org.apache.ignite.internal.processors.cache.verify.IdleVerifyUtility.GRID_NOT_IDLE_MSG;
 import static org.apache.ignite.internal.processors.diagnostic.DiagnosticProcessor.DEFAULT_TARGET_FOLDER;
 import static org.apache.ignite.testframework.GridTestUtils.assertContains;
+import static org.apache.ignite.testframework.GridTestUtils.assertNotContains;
 import static org.apache.ignite.testframework.GridTestUtils.assertThrows;
 import static org.apache.ignite.testframework.GridTestUtils.runAsync;
 import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
@@ -3088,9 +3092,17 @@
 
         assertEquals(EXIT_CODE_OK, execute(h, args));
 
-        LongMetric opEndTimeMetric = ig.context().metric().registry(SNAPSHOT_METRICS).findMetric("LastSnapshotEndTime");
+        MetricRegistry metrics = ig.context().metric().registry(SNAPSHOT_METRICS);
+
+        LongMetric opEndTimeMetric = metrics.findMetric("LastSnapshotEndTime");
         BooleanSupplier endTimeMetricPredicate = () -> opEndTimeMetric.value() > 0;
 
+        String reqId = metrics.findMetric("LastRequestId").getAsString();
+        assertFalse(F.isEmpty(reqId));
+
+        // Make sure the operation ID has been shown to the user.
+        assertContains(log, testOut.toString(), reqId);
+
         if (syncMode)
             assertTrue(endTimeMetricPredicate.getAsBoolean());
         else {
@@ -3133,14 +3145,26 @@
         IgniteEx srv = startGrid(0);
         IgniteEx startCli = startClientGrid(CLIENT_NODE_NAME_PREFIX);
 
+        injectTestSystemOut();
+
         srv.cluster().state(ACTIVE);
 
         createCacheAndPreload(startCli, 100);
 
         CommandHandler h = new CommandHandler();
 
+        // Cancel snapshot using operation ID.
         doSnapshotCancellationTest(startCli, Collections.singletonList(srv), startCli.cache(DEFAULT_CACHE_NAME),
-            snpName -> assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "cancel", snpName)));
+            snpName -> {
+                String reqId = srv.context().metric().registry(SNAPSHOT_METRICS).findMetric("LastRequestId").getAsString();
+                assertFalse(F.isEmpty(reqId));
+
+                assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "cancel", "--id", reqId));
+            });
+
+        // Cancel snapshot using snapshot name.
+        doSnapshotCancellationTest(startCli, Collections.singletonList(srv), startCli.cache(DEFAULT_CACHE_NAME),
+            snpName -> assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "cancel", "--name", snpName)));
     }
 
     /** @throws Exception If fails. */
@@ -3190,20 +3214,17 @@
         autoConfirmation = false;
 
         // Invalid command syntax checks.
-        assertEquals(EXIT_CODE_INVALID_ARGUMENTS, execute(h, "--snapshot", "restore", snpName));
-        assertContains(log, testOut.toString(), "One of [--start, --cancel, --status] is expected.");
-
         assertEquals(EXIT_CODE_INVALID_ARGUMENTS, execute(h, "--snapshot", "restore", snpName, "--cancel", "--sync"));
         assertContains(log, testOut.toString(), "Invalid argument: --sync.");
 
         assertEquals(EXIT_CODE_INVALID_ARGUMENTS, execute(h, "--snapshot", "restore", snpName, "blah"));
-        assertContains(log, testOut.toString(), "Invalid argument: blah. One of [--start, --cancel, --status] is expected.");
+        assertContains(log, testOut.toString(), "Invalid argument: blah. Possible options: --groups, --src, --sync.");
 
         assertEquals(EXIT_CODE_INVALID_ARGUMENTS, execute(h, "--snapshot", "restore", snpName, "--status", "--sync"));
         assertContains(log, testOut.toString(), "Invalid argument: --sync. Action \"--status\" does not support specified option.");
 
         assertEquals(EXIT_CODE_INVALID_ARGUMENTS, execute(h, "--snapshot", "restore", snpName, "--sync", "--start"));
-        assertContains(log, testOut.toString(), "Invalid argument: --sync.");
+        assertContains(log, testOut.toString(), "Invalid argument: --start.");
 
         assertEquals(EXIT_CODE_INVALID_ARGUMENTS, execute(h, "--snapshot", "restore", snpName, "--start", "blah"));
         assertContains(log, testOut.toString(), "Invalid argument: blah. Possible options: --groups, --src, --sync.");
@@ -3212,13 +3233,15 @@
 
         // Cache exists.
         assertEquals(EXIT_CODE_UNEXPECTED_ERROR, execute(h, "--snapshot", "restore", snpName, "--start", "--sync"));
+        assertContains(log, testOut.toString(), "Command option '--start' is redundant and must be avoided.");
         assertContains(log, testOut.toString(), "Unable to restore cache group - directory is not empty. " +
             "Cache group should be destroyed manually before perform restore operation [group=" + cacheName);
 
         ig.cache(cacheName).destroy();
         awaitPartitionMapExchange();
 
-        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", snpName, "--start", "--sync"));
+        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", snpName, "--sync"));
+        assertNotContains(log, testOut.toString(), "Command option '--start' is redundant and must be avoided.");
         assertContains(log, testOut.toString(), "Snapshot cache group restore operation completed successfully");
 
         IgniteCache<Object, Object> cache = ig.cache(cacheName);
@@ -3269,17 +3292,22 @@
 
         CommandHandler h = new CommandHandler();
 
-        assertEquals(EXIT_CODE_INVALID_ARGUMENTS, execute(h, "--snapshot", "restore", snpName, "--start", cacheName1));
+        assertEquals(EXIT_CODE_INVALID_ARGUMENTS, execute(h, "--snapshot", "restore", snpName, cacheName1));
         assertContains(log, testOut.toString(),
             "Invalid argument: " + cacheName1 + ". Possible options: --groups, --src, --sync.");
 
         // Restore single cache group.
-        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", snpName, "--start", "--groups", cacheName1));
-        assertContains(log, testOut.toString(),
-            "Snapshot cache group restore operation started [snapshot=" + snpName + ", group(s)=" + cacheName1 + ']');
+        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", snpName, "--groups", cacheName1));
 
         waitForCondition(() -> ig.cache(cacheName1) != null, getTestTimeout());
 
+        MetricRegistry metrics = ig.context().metric().registry(SNAPSHOT_RESTORE_METRICS);
+        Metric operIdMetric = metrics.findMetric("requestId");
+        assertNotNull(operIdMetric);
+
+        assertContains(log, testOut.toString(), "Snapshot cache group restore operation started " +
+            "[name=" + snpName + ", group(s)=" + cacheName1 + ", id=" + operIdMetric.getAsString() + ']');
+
         cache1 = ig.cache(cacheName1);
 
         assertNotNull(cache1);
@@ -3298,9 +3326,9 @@
         String cacheNames = cacheName1 + ',' + cacheName2;
 
         // Restore two (of three) groups of caches.
-        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", snpName, "--start", "--groups", cacheNames));
+        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", snpName, "--groups", cacheNames));
         assertContains(log, testOut.toString(),
-            "Snapshot cache group restore operation started [snapshot=" + snpName + ", group(s)=");
+            "Snapshot cache group restore operation started [name=" + snpName + ", group(s)=");
 
         waitForCondition(() -> ig.cache(cacheName1) != null, getTestTimeout());
         waitForCondition(() -> ig.cache(cacheName2) != null, getTestTimeout());
@@ -3326,10 +3354,10 @@
         assertNull(ig.cache(cacheName3));
 
         // Restore all public cache groups.
-        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", snpName, "--start"));
+        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", snpName));
         String out = testOut.toString();
         assertContains(log, out, "Warning: command will restore ALL USER-CREATED CACHE GROUPS from the snapshot");
-        assertContains(log, out, "Snapshot cache group restore operation started [snapshot=" + snpName + ']');
+        assertContains(log, out, "Snapshot cache group restore operation started [name=" + snpName);
 
         waitForCondition(() -> ig.cache(cacheName1) != null, getTestTimeout());
         waitForCondition(() -> ig.cache(cacheName2) != null, getTestTimeout());
@@ -3376,11 +3404,11 @@
 
             ignite.destroyCache(DEFAULT_CACHE_NAME);
 
-            assertEquals(EXIT_CODE_INVALID_ARGUMENTS, execute(h, "--snapshot", "restore", snpName, "--start", "--sync"));
+            assertEquals(EXIT_CODE_INVALID_ARGUMENTS, execute(h, "--snapshot", "restore", snpName, "--sync"));
             assertContains(log, testOut.toString(), "Snapshot does not exists [snapshot=" + snpName);
 
             assertEquals(EXIT_CODE_INVALID_ARGUMENTS,
-                execute(h, "--snapshot", "restore", snpName, "--start", "--src", "A", "--src", "B"));
+                execute(h, "--snapshot", "restore", snpName, "--src", "A", "--src", "B"));
             assertContains(log, testOut.toString(), "--src arg specified twice.");
 
             // The check command simply prints the results of the check, it always ends with a zero exit code.
@@ -3391,7 +3419,7 @@
             assertContains(log, testOut.toString(), "The check procedure has finished, no conflicts have been found.");
 
             assertEquals(EXIT_CODE_OK,
-                execute(h, "--snapshot", "restore", snpName, "--start", "--sync", "--src", snpDir.getAbsolutePath()));
+                execute(h, "--snapshot", "restore", snpName, "--sync", "--src", snpDir.getAbsolutePath()));
 
             IgniteCache<Integer, Integer> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
@@ -3440,9 +3468,14 @@
         assertFalse(restoreFut.isDone());
 
         // Check the status with a control command.
-        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", snpName, "--status"));
-        assertContains(log, testOut.toString(),
-            "Snapshot cache group restore operation is running [snapshot=" + snpName + ']');
+        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "status"));
+
+        Pattern operIdPtrn = Pattern.compile("Operation request ID: (?<id>[-\\w]{36})");
+        Matcher matcher = operIdPtrn.matcher(testOut.toString());
+        assertTrue(matcher.find());
+
+        String operIdStr = matcher.group("id");
+        assertNotNull(operIdStr);
 
         // Check "status" with the wrong snapshot name.
         assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", missingSnpName, "--status"));
@@ -3455,9 +3488,9 @@
             "Snapshot cache group restore operation is NOT running [snapshot=" + missingSnpName + ']');
 
         // Cancel operation using control command.
-        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", snpName, "--cancel"));
+        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "cancel", "--id", operIdStr));
         assertContains(log, testOut.toString(),
-            "Snapshot cache group restore operation canceled [snapshot=" + snpName + ']');
+            "Snapshot operation cancelled [id=" + operIdStr + ']');
 
         GridTestUtils.assertThrowsAnyCause(log, () -> restoreFut.get(getTestTimeout()), IgniteCheckedException.class,
             "Operation has been canceled by the user.");
@@ -3468,9 +3501,8 @@
 
         assertTrue(ctxDisposed);
 
-        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", snpName, "--status"));
-        assertContains(log, testOut.toString(),
-            "Snapshot cache group restore operation is NOT running [snapshot=" + snpName + ']');
+        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "status"));
+        assertContains(log, testOut.toString(), "There is no create or restore snapshot operation in progress.");
 
         assertNull(ig.cache(DEFAULT_CACHE_NAME));
     }
@@ -3519,6 +3551,8 @@
 
         srv.destroyCache(DEFAULT_CACHE_NAME);
 
+        awaitPartitionMapExchange();
+
         spi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage);
 
         fut = srv.snapshot().restoreSnapshot(snapshotName, F.asList(DEFAULT_CACHE_NAME));
@@ -3540,7 +3574,7 @@
      * @param expName Expected snapshot name.
      */
     private void checkSnapshotStatus(boolean isCreating, boolean isRestoring, String expName) throws Exception {
-        Collection<Ignite> srvs = F.view(G.allGrids(), n -> !n.cluster().localNode().isLocal());
+        Collection<Ignite> srvs = F.view(G.allGrids(), n -> !n.cluster().localNode().isClient());
 
         assertTrue(waitForCondition(() -> srvs.stream().allMatch(
                 ignite -> {
diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/KillCommandsCommandShTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/KillCommandsCommandShTest.java
index abef2eb..fcf4c5d 100644
--- a/modules/control-utility/src/test/java/org/apache/ignite/util/KillCommandsCommandShTest.java
+++ b/modules/control-utility/src/test/java/org/apache/ignite/util/KillCommandsCommandShTest.java
@@ -46,6 +46,7 @@
 import static org.apache.ignite.internal.commandline.CommandHandler.EXIT_CODE_OK;
 import static org.apache.ignite.internal.commandline.CommandHandler.EXIT_CODE_UNEXPECTED_ERROR;
 import static org.apache.ignite.internal.processors.cache.persistence.snapshot.AbstractSnapshotSelfTest.doSnapshotCancellationTest;
+import static org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.SNAPSHOT_METRICS;
 import static org.apache.ignite.internal.processors.job.GridJobProcessor.JOBS_VIEW;
 import static org.apache.ignite.testframework.GridTestUtils.assertContains;
 import static org.apache.ignite.testframework.GridTestUtils.assertNotContains;
@@ -156,7 +157,9 @@
     @Test
     public void testCancelSnapshot() {
         doSnapshotCancellationTest(client, srvs, client.cache(DEFAULT_CACHE_NAME), snpName -> {
-            int res = execute("--kill", "snapshot", snpName);
+            String id = grid(0).context().metric().registry(SNAPSHOT_METRICS).findMetric("LastRequestId").getAsString();
+
+            int res = execute("--kill", "snapshot", id);
 
             assertEquals(EXIT_CODE_OK, res);
         });
@@ -164,14 +167,6 @@
 
     /** */
     @Test
-    public void testCancelUnknownSnapshot() {
-        int res = execute("--kill", "snapshot", "unknown");
-
-        assertEquals(EXIT_CODE_OK, res);
-    }
-
-    /** */
-    @Test
     public void testCancelUnknownScanQuery() {
         int res = execute("--kill", "scan", srvs.get(0).localNode().id().toString(), "unknown", "1");
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotFutureTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotFutureTask.java
index b4c65af..cb9486a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotFutureTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotFutureTask.java
@@ -43,6 +43,9 @@
     /** Node id which cause snapshot operation. */
     protected final UUID srcNodeId;
 
+    /** Snapshot operation request ID. */
+    protected final UUID reqId;
+
     /** Unique identifier of snapshot process. */
     protected final String snpName;
 
@@ -65,6 +68,7 @@
     /**
      * @param cctx Shared context.
      * @param srcNodeId Node id which cause snapshot task creation.
+     * @param reqId Snapshot operation request ID.
      * @param snpName Unique identifier of snapshot process.
      * @param tmpWorkDir Working directory for intermediate snapshot results.
      * @param ioFactory Factory to working with snapshot files.
@@ -74,6 +78,7 @@
     protected AbstractSnapshotFutureTask(
         GridCacheSharedContext<?, ?> cctx,
         UUID srcNodeId,
+        UUID reqId,
         String snpName,
         File tmpWorkDir,
         FileIOFactory ioFactory,
@@ -87,6 +92,7 @@
         this.cctx = cctx;
         this.log = cctx.logger(AbstractSnapshotFutureTask.class);
         this.srcNodeId = srcNodeId;
+        this.reqId = reqId;
         this.snpName = snpName;
         this.tmpSnpWorkDir = new File(tmpWorkDir, snpName);
         this.ioFactory = ioFactory;
@@ -109,6 +115,13 @@
     }
 
     /**
+     * @return Snapshot operation request ID.
+     */
+    public UUID requestId() {
+        return reqId;
+    }
+
+    /**
      * Initiates snapshot task.
      *
      * @return {@code true} if task started by this call.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotMessage.java
index e76ca16..a60c7a5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotMessage.java
@@ -30,8 +30,8 @@
  *
  */
 abstract class AbstractSnapshotMessage implements Message {
-    /** Unique request id. */
-    private String reqId;
+    /** Unique message ID. */
+    private String id;
 
     /**
      * Empty constructor required for {@link Externalizable}.
@@ -41,19 +41,19 @@
     }
 
     /**
-     * @param reqId Unique request id.
+     * @param id Unique message ID.
      */
-    protected AbstractSnapshotMessage(String reqId) {
-        assert U.alphanumericUnderscore(reqId) : reqId;
+    protected AbstractSnapshotMessage(String id) {
+        assert U.alphanumericUnderscore(id) : id;
 
-        this.reqId = reqId;
+        this.id = id;
     }
 
     /**
-     * @return Unique request id.
+     * @return Unique message ID.
      */
-    public String requestId() {
-        return reqId;
+    public String id() {
+        return id;
     }
 
     /** {@inheritDoc} */
@@ -68,7 +68,7 @@
         }
 
         if (writer.state() == 0) {
-            if (!writer.writeString("reqId", reqId))
+            if (!writer.writeString("id", id))
                 return false;
 
             writer.incrementState();
@@ -85,7 +85,7 @@
             return false;
 
         if (reader.state() == 0) {
-            reqId = reader.readString("reqId");
+            id = reader.readString("id");
 
             if (!reader.isLastRead())
                 return false;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
index b339b62..e19e5c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
@@ -49,6 +49,7 @@
 import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Objects;
+import java.util.Optional;
 import java.util.Queue;
 import java.util.Set;
 import java.util.UUID;
@@ -63,6 +64,7 @@
 import java.util.function.BiFunction;
 import java.util.function.BooleanSupplier;
 import java.util.function.Consumer;
+import java.util.function.Function;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
 import org.apache.ignite.IgniteCheckedException;
@@ -507,6 +509,8 @@
         mreg.register("LocalSnapshotNames", this::localSnapshotNames, List.class,
             "The list of names of all snapshots currently saved on the local node with respect to " +
                 "the configured via IgniteConfiguration snapshot working path.");
+        mreg.register("LastRequestId", () -> Optional.ofNullable(lastSeenSnpFut.rqId).map(UUID::toString).orElse(""),
+            String.class, "The ID of the last started snapshot operation.");
 
         mreg.register("CurrentSnapshotTotalSize", () -> {
             SnapshotFutureTask task = currentSnapshotTask();
@@ -787,6 +791,7 @@
         else {
             task0 = registerSnapshotTask(req.snapshotName(),
                 req.operationalNodeId(),
+                req.requestId(),
                 parts,
                 withMetaStorage,
                 locSndrFactory.apply(req.snapshotName(), req.snapshotPath()));
@@ -1117,13 +1122,39 @@
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Void> cancelSnapshot(String name) {
+        return new IgniteFutureImpl<>(cancelSnapshot0(name).chain(f -> null));
+    }
+
+    /**
+     * @param name Snapshot name.
+     * @return Future which will be completed when cancel operation finished.
+     */
+    private IgniteInternalFuture<Boolean> cancelSnapshot0(String name) {
         A.notNullOrEmpty(name, "Snapshot name must be not empty or null");
 
         cctx.kernalContext().security().authorize(ADMIN_SNAPSHOT);
 
-        IgniteInternalFuture<Void> fut0 = cctx.kernalContext().closure()
+        return cctx.kernalContext().closure()
             .callAsyncNoFailover(BROADCAST,
-                new CancelSnapshotCallable(name),
+                new CancelSnapshotCallable(null, name),
+                cctx.discovery().aliveServerNodes(),
+                false,
+                0,
+                true);
+    }
+
+    /**
+     * @param reqId Snapshot operation request ID.
+     * @return Future which will be completed when cancel operation finished.
+     */
+    public IgniteFuture<Boolean> cancelSnapshotOperation(UUID reqId) {
+        A.notNull(reqId, "Snapshot operation request ID must be not null");
+
+        cctx.kernalContext().security().authorize(ADMIN_SNAPSHOT);
+
+        IgniteInternalFuture<Boolean> fut0 = cctx.kernalContext().closure()
+            .callAsyncNoFailover(BROADCAST,
+                new CancelSnapshotCallable(reqId, null),
                 cctx.discovery().aliveServerNodes(),
                 false,
                 0,
@@ -1133,19 +1164,44 @@
     }
 
     /**
-     * @param name Snapshot name to cancel operation on local node.
+     * Cancel running snapshot operation (create/restore).
+     *
+     * @param reqId Snapshot operation request ID.
+     * @return {@code True} if the operation with the specified ID was canceled.
      */
-    public void cancelLocalSnapshotTask(String name) {
+    private boolean cancelLocalSnapshotOperations(UUID reqId) {
+        A.notNull(reqId, "Snapshot operation request ID must be not null");
+
+        if (cancelLocalSnapshotTask0(task -> reqId.equals(task.requestId())))
+            return true;
+
+        return restoreCacheGrpProc.cancel(reqId, null).get();
+    }
+
+    /**
+     * @param name Snapshot name to cancel operation on local node.
+     * @return {@code True} if the snapshot operation was canceled.
+     */
+    public boolean cancelLocalSnapshotTask(String name) {
         A.notNullOrEmpty(name, "Snapshot name must be not null or empty");
 
+        return cancelLocalSnapshotTask0(task -> name.equals(task.snapshotName()));
+    }
+
+    /**
+     * @param filter Snapshot task filter.
+     * @return {@code True} if the snapshot operation was canceled.
+     */
+    private boolean cancelLocalSnapshotTask0(Function<AbstractSnapshotFutureTask<?>, Boolean> filter) {
         ClusterSnapshotFuture fut0 = null;
+        boolean canceled = false;
 
         busyLock.enterBusy();
 
         try {
             for (AbstractSnapshotFutureTask<?> sctx : locSnpTasks.values()) {
-                if (sctx.snapshotName().equals(name))
-                    sctx.cancel();
+                if (filter.apply(sctx))
+                    canceled |= sctx.cancel();
             }
 
             synchronized (snpOpMux) {
@@ -1170,11 +1226,13 @@
             else
                 throw new IgniteException(e);
         }
+
+        return canceled;
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Boolean> cancelSnapshotRestore(String name) {
-        return executeRestoreManagementTask(SnapshotRestoreCancelTask.class, name);
+        return new IgniteFutureImpl<>(cancelSnapshot0(name));
     }
 
     /**
@@ -1182,9 +1240,12 @@
      *
      * @return Future that will be finished when process the process is complete. The result of this future will be
      * {@code false} if the restore process with the specified snapshot name is not running at all.
+     *
+     * @deprecated Use {@link #cancelLocalSnapshotOperations(UUID)} instead.
      */
+    @Deprecated
     public IgniteFuture<Boolean> cancelLocalRestoreTask(String name) {
-        return restoreCacheGrpProc.cancel(new IgniteCheckedException("Operation has been canceled by the user."), name);
+        return restoreCacheGrpProc.cancel(null, name);
     }
 
     /**
@@ -1451,7 +1512,7 @@
      * @param snpPath Snapshot directory path.
      * @return Future which will be completed when a process ends.
      */
-    public IgniteFuture<Void> createSnapshot(String name, @Nullable String snpPath) {
+    public IgniteFutureImpl<Void> createSnapshot(String name, @Nullable String snpPath) {
         A.notNullOrEmpty(name, "Snapshot name cannot be null or empty.");
         A.ensure(U.alphanumericUnderscore(name), "Snapshot name must satisfy the following name pattern: a-zA-Z0-9_");
 
@@ -1574,7 +1635,7 @@
      * @param grpNames Cache groups to be restored or {@code null} to restore all cache groups from the snapshot.
      * @return Future which will be completed when restore operation finished.
      */
-    public IgniteFuture<Void> restoreSnapshot(String name, @Nullable String snpPath, @Nullable Collection<String> grpNames) {
+    public IgniteFutureImpl<Void> restoreSnapshot(String name, @Nullable String snpPath, @Nullable Collection<String> grpNames) {
         A.notNullOrEmpty(name, "Snapshot name cannot be null or empty.");
         A.ensure(U.alphanumericUnderscore(name), "Snapshot name must satisfy the following name pattern: a-zA-Z0-9_");
         A.ensure(grpNames == null || !grpNames.isEmpty(), "List of cache group names cannot be empty.");
@@ -1666,6 +1727,7 @@
 
     /**
      * @param rmtNodeId The remote node to connect to.
+     * @param reqId Snapshot operation request ID.
      * @param snpName Snapshot name to request.
      * @param rmtSnpPath Snapshot directory path on the remote node.
      * @param parts Collection of pairs group and appropriate cache partition to be snapshot.
@@ -1674,6 +1736,7 @@
      */
     public IgniteInternalFuture<Void> requestRemoteSnapshotFiles(
         UUID rmtNodeId,
+        UUID reqId,
         String snpName,
         @Nullable String rmtSnpPath,
         Map<Integer, Set<Integer>> parts,
@@ -1694,7 +1757,7 @@
             throw new IgniteCheckedException("Snapshot on remote node is not supported: " + rmtNode.id());
 
         RemoteSnapshotFilesRecevier fut =
-            new RemoteSnapshotFilesRecevier(this, rmtNodeId, snpName, rmtSnpPath, parts, stopChecker, partHnd);
+            new RemoteSnapshotFilesRecevier(this, rmtNodeId, reqId, snpName, rmtSnpPath, parts, stopChecker, partHnd);
 
         snpRmtMgr.submit(fut);
 
@@ -1831,6 +1894,7 @@
     /**
      * @param snpName Unique snapshot name.
      * @param srcNodeId Node id which cause snapshot operation.
+     * @param requestId Snapshot operation request ID.
      * @param parts Collection of pairs group and appropriate cache partition to be snapshot.
      * @param withMetaStorage {@code true} if all metastorage data must be also included into snapshot.
      * @param snpSndr Factory which produces snapshot receiver instance.
@@ -1839,12 +1903,13 @@
     AbstractSnapshotFutureTask<?> registerSnapshotTask(
         String snpName,
         UUID srcNodeId,
+        UUID requestId,
         Map<Integer, Set<Integer>> parts,
         boolean withMetaStorage,
         SnapshotSender snpSndr
     ) {
-        AbstractSnapshotFutureTask<?> task = registerTask(snpName, new SnapshotFutureTask(cctx, srcNodeId, snpName,
-            tmpWorkDir, ioFactory, snpSndr, parts, withMetaStorage, locBuff));
+        AbstractSnapshotFutureTask<?> task = registerTask(snpName, new SnapshotFutureTask(cctx, srcNodeId, requestId,
+            snpName, tmpWorkDir, ioFactory, snpSndr, parts, withMetaStorage, locBuff));
 
         if (!withMetaStorage) {
             for (Integer grpId : parts.keySet()) {
@@ -2501,6 +2566,7 @@
         /**
          * @param snpMgr Ignite snapshot manager.
          * @param rmtNodeId Remote node to request snapshot from.
+         * @param reqId Snapshot operation request ID.
          * @param snpName Snapshot name to request.
          * @param rmtSnpPath Snapshot directory path on the remote node.
          * @param parts Cache group and partitions to request.
@@ -2510,14 +2576,15 @@
         public RemoteSnapshotFilesRecevier(
             IgniteSnapshotManager snpMgr,
             UUID rmtNodeId,
+            UUID reqId,
             String snpName,
             @Nullable String rmtSnpPath,
             Map<Integer, Set<Integer>> parts,
             BooleanSupplier stopChecker,
             BiConsumer<@Nullable File, @Nullable Throwable> partHnd
         ) {
-            dir = Paths.get(snpMgr.tmpWorkDir.getAbsolutePath(), reqId);
-            initMsg = new SnapshotFilesRequestMessage(reqId, snpName, rmtSnpPath, parts);
+            dir = Paths.get(snpMgr.tmpWorkDir.getAbsolutePath(), this.reqId);
+            initMsg = new SnapshotFilesRequestMessage(this.reqId, reqId, snpName, rmtSnpPath, parts);
 
             this.snpMgr = snpMgr;
             this.rmtNodeId = rmtNodeId;
@@ -2734,7 +2801,7 @@
             try {
                 if (msg instanceof SnapshotFilesRequestMessage) {
                     SnapshotFilesRequestMessage reqMsg0 = (SnapshotFilesRequestMessage)msg;
-                    String rqId = reqMsg0.requestId();
+                    String rqId = reqMsg0.id();
                     String snpName = reqMsg0.snapshotName();
 
                     try {
@@ -2753,6 +2820,7 @@
                         AbstractSnapshotFutureTask<?> task = registerTask(rqId,
                             new SnapshotResponseRemoteFutureTask(cctx,
                                 nodeId,
+                                reqMsg0.requestId(),
                                 snpName,
                                 reqMsg0.snapshotPath(),
                                 tmpWorkDir,
@@ -2770,7 +2838,7 @@
                             try {
                                 cctx.gridIO().sendToCustomTopic(nodeId,
                                     DFLT_INITIAL_SNAPSHOT_TOPIC,
-                                    new SnapshotFilesFailureMessage(reqMsg0.requestId(), f.error().getMessage()),
+                                    new SnapshotFilesFailureMessage(reqMsg0.id(), f.error().getMessage()),
                                     SYSTEM_POOL);
                             }
                             catch (IgniteCheckedException ex0) {
@@ -2787,7 +2855,7 @@
 
                         cctx.gridIO().sendToCustomTopic(nodeId,
                             DFLT_INITIAL_SNAPSHOT_TOPIC,
-                            new SnapshotFilesFailureMessage(reqMsg0.requestId(), t.getMessage()),
+                            new SnapshotFilesFailureMessage(reqMsg0.id(), t.getMessage()),
                             SYSTEM_POOL);
                     }
                 }
@@ -2796,7 +2864,7 @@
 
                     RemoteSnapshotFilesRecevier task = active;
 
-                    if (task == null || !task.reqId.equals(respMsg0.requestId())) {
+                    if (task == null || !task.reqId.equals(respMsg0.id())) {
                         if (log.isInfoEnabled()) {
                             log.info("A stale snapshot response message has been received. Will be ignored " +
                                 "[fromNodeId=" + nodeId + ", response=" + respMsg0 + ']');
@@ -3290,7 +3358,7 @@
     }
 
     /** */
-    protected static class ClusterSnapshotFuture extends GridFutureAdapter<Void> {
+    public static class ClusterSnapshotFuture extends GridFutureAdapter<Void> {
         /** Unique snapshot request id. */
         final UUID rqId;
 
@@ -3333,6 +3401,7 @@
 
         /**
          * @param rqId Unique snapshot request id.
+         * @param name Snapshot name.
          */
         public ClusterSnapshotFuture(UUID rqId, String name) {
             this.rqId = rqId;
@@ -3346,6 +3415,11 @@
 
             return super.onDone(res, err, cancel);
         }
+
+        /** @return Request ID. */
+        public UUID requestId() {
+            return rqId;
+        }
     }
 
     /** Start creation of cluster snapshot closure. */
@@ -3378,29 +3452,39 @@
 
     /** Cancel snapshot operation closure. */
     @GridInternal
-    private static class CancelSnapshotCallable implements IgniteCallable<Void> {
+    private static class CancelSnapshotCallable implements IgniteCallable<Boolean> {
         /** Serial version uid. */
         private static final long serialVersionUID = 0L;
 
         /** Snapshot name. */
         private final String snpName;
 
+        /** Snapshot operation request ID. */
+        private final UUID reqId;
+
         /** Auto-injected grid instance. */
         @IgniteInstanceResource
         private transient IgniteEx ignite;
 
         /**
+         * @param reqId Snapshot operation request ID.
          * @param snpName Snapshot name.
          */
-        public CancelSnapshotCallable(String snpName) {
+        public CancelSnapshotCallable(UUID reqId, String snpName) {
+            this.reqId = reqId;
             this.snpName = snpName;
         }
 
         /** {@inheritDoc} */
-        @Override public Void call() throws Exception {
-            ignite.context().cache().context().snapshotMgr().cancelLocalSnapshotTask(snpName);
+        @Override public Boolean call() throws Exception {
+            if (reqId != null)
+                return ignite.context().cache().context().snapshotMgr().cancelLocalSnapshotOperations(reqId);
+            else {
+                if (ignite.context().cache().context().snapshotMgr().cancelLocalSnapshotTask(snpName))
+                    return true;
 
-            return null;
+                return ignite.context().cache().context().snapshotMgr().cancelLocalRestoreTask(snpName).get();
+            }
         }
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFilesRequestMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFilesRequestMessage.java
index a06ec86..75d2d98 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFilesRequestMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFilesRequestMessage.java
@@ -24,6 +24,7 @@
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
+import java.util.UUID;
 import java.util.stream.Collectors;
 import org.apache.ignite.internal.GridDirectMap;
 import org.apache.ignite.internal.util.typedef.F;
@@ -44,6 +45,9 @@
     /** Serialization version. */
     private static final long serialVersionUID = 0L;
 
+    /** Snapshot operation request ID. */
+    private UUID requestId;
+
     /** Snapshot name to request. */
     private String snpName;
 
@@ -62,13 +66,15 @@
     }
 
     /**
-     * @param reqId Unique request id.
+     * @param reqId Unique message id.
+     * @param requestId Snapshot operation request ID.
      * @param snpName Snapshot name to request.
      * @param snpPath Snapshot directory path.
      * @param parts Map of cache group ids and corresponding set of its partition ids to be snapshot.
      */
     public SnapshotFilesRequestMessage(
         String reqId,
+        UUID requestId,
         String snpName,
         @Nullable String snpPath,
         Map<Integer, Set<Integer>> parts
@@ -77,6 +83,7 @@
 
         assert parts != null && !parts.isEmpty();
 
+        this.requestId = requestId;
         this.snpName = snpName;
         this.snpPath = snpPath;
         this.parts = new HashMap<>();
@@ -111,6 +118,13 @@
         return snpPath;
     }
 
+    /**
+     * @return Snapshot operation request ID.
+     */
+    public UUID requestId() {
+        return requestId;
+    }
+
     /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
@@ -146,6 +160,13 @@
             writer.incrementState();
         }
 
+        if (writer.state() == 4) {
+            if (!writer.writeUuid("reqId", requestId))
+                return false;
+
+            writer.incrementState();
+        }
+
         return true;
     }
 
@@ -186,6 +207,15 @@
             reader.incrementState();
         }
 
+        if (reader.state() == 4) {
+            requestId = reader.readUuid("reqId");
+
+            if (!reader.isLastRead())
+                return false;
+
+            reader.incrementState();
+        }
+
         return reader.afterMessageRead(SnapshotFilesRequestMessage.class);
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFinishedFutureTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFinishedFutureTask.java
index 561bb16..e32ddc1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFinishedFutureTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFinishedFutureTask.java
@@ -25,7 +25,7 @@
      * @param e Finished snapshot task future with particular exception.
      */
     public SnapshotFinishedFutureTask(IgniteCheckedException e) {
-        super(null, null, null, null, null, null, null);
+        super(null, null, null, null, null, null, null, null);
 
         onDone(e);
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFutureTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFutureTask.java
index c88bb4a..579da37 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFutureTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFutureTask.java
@@ -152,6 +152,7 @@
     /**
      * @param cctx Shared context.
      * @param srcNodeId Node id which cause snapshot task creation.
+     * @param reqId Snapshot operation request ID.
      * @param snpName Unique identifier of snapshot process.
      * @param tmpWorkDir Working directory for intermediate snapshot results.
      * @param ioFactory Factory to working with snapshot files.
@@ -162,6 +163,7 @@
     public SnapshotFutureTask(
         GridCacheSharedContext<?, ?> cctx,
         UUID srcNodeId,
+        UUID reqId,
         String snpName,
         File tmpWorkDir,
         FileIOFactory ioFactory,
@@ -170,7 +172,7 @@
         boolean withMetaStorage,
         ThreadLocal<ByteBuffer> locBuff
     ) {
-        super(cctx, srcNodeId, snpName, tmpWorkDir, ioFactory, snpSndr, parts);
+        super(cctx, srcNodeId, reqId, snpName, tmpWorkDir, ioFactory, snpSndr, parts);
 
         assert snpName != null : "Snapshot name cannot be empty or null.";
         assert snpSndr != null : "Snapshot sender which handles execution tasks must be not null.";
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMXBeanImpl.java
index ea0a1de..713ab2e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMXBeanImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMXBeanImpl.java
@@ -19,6 +19,7 @@
 
 import java.util.Arrays;
 import java.util.Set;
+import java.util.UUID;
 import java.util.stream.Collectors;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.metric.GridMetricManager;
@@ -61,6 +62,11 @@
     }
 
     /** {@inheritDoc} */
+    @Override public void cancelSnapshotOperation(String reqId) {
+        mgr.cancelSnapshotOperation(UUID.fromString(reqId)).get();
+    }
+
+    /** {@inheritDoc} */
     @Override public void restoreSnapshot(String name, String path, String grpNames) {
         Set<String> grpNamesSet = F.isEmpty(grpNames) ? null :
             Arrays.stream(grpNames.split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet());
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotResponseRemoteFutureTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotResponseRemoteFutureTask.java
index d18a6e3..90e562f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotResponseRemoteFutureTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotResponseRemoteFutureTask.java
@@ -45,6 +45,7 @@
     /**
      * @param cctx Shared context.
      * @param srcNodeId Node id which cause snapshot task creation.
+     * @param reqId Snapshot operation request ID.
      * @param snpName Unique identifier of snapshot process.
      * @param snpPath Snapshot directory path.
      * @param tmpWorkDir Working directory for intermediate snapshot results.
@@ -55,6 +56,7 @@
     public SnapshotResponseRemoteFutureTask(
         GridCacheSharedContext<?, ?> cctx,
         UUID srcNodeId,
+        UUID reqId,
         String snpName,
         String snpPath,
         File tmpWorkDir,
@@ -62,7 +64,7 @@
         SnapshotSender snpSndr,
         Map<Integer, Set<Integer>> parts
     ) {
-        super(cctx, srcNodeId, snpName, tmpWorkDir, ioFactory, snpSndr, parts);
+        super(cctx, srcNodeId, reqId, snpName, tmpWorkDir, ioFactory, snpSndr, parts);
 
         this.snpPath = snpPath;
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCancelTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCancelTask.java
deleted file mode 100644
index 10cc8d4..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCancelTask.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.persistence.snapshot;
-
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.compute.ComputeJob;
-import org.apache.ignite.compute.ComputeJobAdapter;
-import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.processors.task.GridInternal;
-import org.apache.ignite.resources.IgniteInstanceResource;
-
-/**
- * Snapshot restore cancel task.
- */
-@GridInternal
-class SnapshotRestoreCancelTask extends SnapshotRestoreManagementTask {
-    /** Serial version uid. */
-    private static final long serialVersionUID = 0L;
-
-    /** {@inheritDoc} */
-    @Override protected ComputeJob makeJob(String snpName) {
-        return new ComputeJobAdapter() {
-            /** Auto-injected grid instance. */
-            @IgniteInstanceResource
-            private transient IgniteEx ignite;
-
-            @Override public Object execute() throws IgniteException {
-                return ignite.context().cache().context().snapshotMgr().cancelLocalRestoreTask(snpName).get();
-            }
-        };
-    }
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreManagementTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreManagementTask.java
deleted file mode 100644
index 1cc6956..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreManagementTask.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.persistence.snapshot;
-
-import java.util.List;
-import java.util.Map;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.compute.ComputeJob;
-import org.apache.ignite.compute.ComputeJobResult;
-import org.apache.ignite.compute.ComputeJobResultPolicy;
-import org.apache.ignite.compute.ComputeTaskAdapter;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.jetbrains.annotations.NotNull;
-
-/**
- * Snapshot restore management task.
- */
-abstract class SnapshotRestoreManagementTask extends ComputeTaskAdapter<String, Boolean> {
-   /**
-     * @param param Compute job argument.
-     * @return Compute job.
-     */
-    protected abstract ComputeJob makeJob(String param);
-
-    /** {@inheritDoc} */
-    @Override public @NotNull Map<? extends ComputeJob, ClusterNode> map(
-        List<ClusterNode> subgrid,
-        String snpName
-    ) throws IgniteException {
-        Map<ComputeJob, ClusterNode> map = U.newHashMap(subgrid.size());
-
-        for (ClusterNode node : subgrid)
-            map.put(makeJob(snpName), node);
-
-        return map;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Boolean reduce(List<ComputeJobResult> results) throws IgniteException {
-        boolean ret = false;
-
-        for (ComputeJobResult r : results) {
-            if (r.getException() != null)
-                throw new IgniteException("Failed to execute job [nodeId=" + r.getNode().id() + ']', r.getException());
-
-            ret |= Boolean.TRUE.equals(r.getData());
-        }
-
-        return ret;
-    }
-
-    /** {@inheritDoc} */
-    @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws IgniteException {
-        // Handle all exceptions during the `reduce` operation.
-        return ComputeJobResultPolicy.WAIT;
-    }
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreProcess.java
index 7c28fa5..d41fdda 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreProcess.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreProcess.java
@@ -211,7 +211,7 @@
      * @param cacheGrpNames Cache groups to be restored or {@code null} to restore all cache groups from the snapshot.
      * @return Future that will be completed when the restore operation is complete and the cache groups are started.
      */
-    public IgniteFuture<Void> start(String snpName, @Nullable String snpPath, @Nullable Collection<String> cacheGrpNames) {
+    public IgniteFutureImpl<Void> start(String snpName, @Nullable String snpPath, @Nullable Collection<String> cacheGrpNames) {
         IgniteSnapshotManager snpMgr = ctx.cache().context().snapshotMgr();
         ClusterSnapshotFuture fut0;
 
@@ -465,26 +465,29 @@
     /**
      * Cancel the currently running local restore procedure.
      *
-     * @param reason Interruption reason.
+     * @param reqId Snapshot operation request ID.
      * @param snpName Snapshot name.
      * @return Future that will be finished when process the process is complete. The result of this future will be
      * {@code false} if the restore process with the specified snapshot name is not running at all.
      */
-    public IgniteFuture<Boolean> cancel(IgniteCheckedException reason, String snpName) {
+    public IgniteFuture<Boolean> cancel(UUID reqId, @Deprecated String snpName) {
+        assert (reqId == null && snpName != null) || (reqId != null && snpName == null);
+
+        IgniteCheckedException reason = new IgniteCheckedException("Operation has been canceled by the user.");
         SnapshotRestoreContext opCtx0;
         ClusterSnapshotFuture fut0 = null;
 
         synchronized (this) {
             opCtx0 = opCtx;
 
-            if (fut != null && fut.name.equals(snpName)) {
+            if (fut != null && (fut.rqId.equals(reqId) || fut.name.equals(snpName))) {
                 fut0 = fut;
 
                 fut0.interruptEx = reason;
             }
         }
 
-        boolean ctxStop = opCtx0 != null && opCtx0.snpName.equals(snpName);
+        boolean ctxStop = opCtx0 != null && (opCtx0.reqId.equals(reqId) || opCtx0.snpName.equals(snpName));
 
         if (ctxStop)
             interrupt(opCtx0, reason);
@@ -1001,6 +1004,7 @@
                 for (Map.Entry<UUID, Map<Integer, Set<Integer>>> m : snpAff.entrySet()) {
                     ctx.cache().context().snapshotMgr()
                         .requestRemoteSnapshotFiles(m.getKey(),
+                            opCtx0.reqId,
                             opCtx0.snpName,
                             opCtx0.snpPath,
                             m.getValue(),
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStatusTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStatusTask.java
index e988042..0c596d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStatusTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStatusTask.java
@@ -17,31 +17,71 @@
 
 package org.apache.ignite.internal.processors.cache.persistence.snapshot;
 
+import java.util.List;
+import java.util.Map;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.compute.ComputeJob;
 import org.apache.ignite.compute.ComputeJobAdapter;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.compute.ComputeJobResultPolicy;
+import org.apache.ignite.compute.ComputeTaskAdapter;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.snapshot.VisorSnapshotStatusTask;
 import org.apache.ignite.resources.IgniteInstanceResource;
+import org.jetbrains.annotations.NotNull;
 
 /**
  * Snapshot restore status task.
+ *
+ * @deprecated Use {@link VisorSnapshotStatusTask} instead.
  */
 @GridInternal
-class SnapshotRestoreStatusTask extends SnapshotRestoreManagementTask {
+@Deprecated
+class SnapshotRestoreStatusTask extends ComputeTaskAdapter<String, Boolean> {
     /** Serial version uid. */
     private static final long serialVersionUID = 0L;
 
     /** {@inheritDoc} */
-    @Override protected ComputeJob makeJob(String snpName) {
-        return new ComputeJobAdapter() {
-            /** Auto-injected grid instance. */
-            @IgniteInstanceResource
-            private transient IgniteEx ignite;
+    @Override public @NotNull Map<? extends ComputeJob, ClusterNode> map(
+        List<ClusterNode> subgrid,
+        String snpName
+    ) throws IgniteException {
+        Map<ComputeJob, ClusterNode> map = U.newHashMap(subgrid.size());
 
-            @Override public Boolean execute() throws IgniteException {
-                return ignite.context().cache().context().snapshotMgr().isRestoring(snpName);
-            }
-        };
+        for (ClusterNode node : subgrid) {
+            map.put(new ComputeJobAdapter() {
+                @IgniteInstanceResource
+                private transient IgniteEx ignite;
+
+                @Override public Boolean execute() throws IgniteException {
+                    return ignite.context().cache().context().snapshotMgr().isRestoring(snpName);
+                }
+            }, node);
+        }
+
+        return map;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Boolean reduce(List<ComputeJobResult> results) throws IgniteException {
+        boolean ret = false;
+
+        for (ComputeJobResult r : results) {
+            if (r.getException() != null)
+                throw new IgniteException("Failed to execute job [nodeId=" + r.getNode().id() + ']', r.getException());
+
+            ret |= Boolean.TRUE.equals(r.getData());
+        }
+
+        return ret;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws IgniteException {
+        // Handle all exceptions during the `reduce` operation.
+        return ComputeJobResultPolicy.WAIT;
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotCancelTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotCancelTask.java
index 3903215..157bc60 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotCancelTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotCancelTask.java
@@ -17,43 +17,50 @@
 
 package org.apache.ignite.internal.visor.snapshot;
 
+import java.util.UUID;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.IgniteSnapshot;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager;
 import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotMXBeanImpl;
 import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.visor.VisorJob;
 
 /**
- * @see IgniteSnapshot#cancelSnapshot(String)
+ * @see IgniteSnapshotManager#cancelSnapshotOperation(UUID)
  */
 @GridInternal
-public class VisorSnapshotCancelTask extends VisorSnapshotOneNodeTask<String, String> {
+public class VisorSnapshotCancelTask extends VisorSnapshotOneNodeTask<VisorSnapshotCancelTaskArg, String> {
     /** Serial version uid. */
     private static final long serialVersionUID = 0L;
 
     /** {@inheritDoc} */
-    @Override protected VisorJob<String, String> job(String arg) {
+    @Override protected VisorJob<VisorSnapshotCancelTaskArg, String> job(VisorSnapshotCancelTaskArg arg) {
         return new VisorSnapshotCancelJob(arg, debug);
     }
 
     /** */
-    private static class VisorSnapshotCancelJob extends VisorJob<String, String> {
+    private static class VisorSnapshotCancelJob extends VisorJob<VisorSnapshotCancelTaskArg, String> {
         /** Serial version uid. */
         private static final long serialVersionUID = 0L;
 
         /**
-         * @param name Snapshot name.
+         * @param taskArg Task argument.
          * @param debug Flag indicating whether debug information should be printed into node log.
          */
-        protected VisorSnapshotCancelJob(String name, boolean debug) {
-            super(name, debug);
+        protected VisorSnapshotCancelJob(VisorSnapshotCancelTaskArg taskArg, boolean debug) {
+            super(taskArg, debug);
         }
 
         /** {@inheritDoc} */
-        @Override protected String run(String name) throws IgniteException {
-            new SnapshotMXBeanImpl(ignite.context()).cancelSnapshot(name);
+        @Override protected String run(VisorSnapshotCancelTaskArg taskArg) throws IgniteException {
+            if (taskArg.requestId() != null) {
+                new SnapshotMXBeanImpl(ignite.context()).cancelSnapshotOperation(taskArg.requestId().toString());
 
-            return "Snapshot operation cancelled.";
+                return "Snapshot operation cancelled [id=" + taskArg.requestId() + "].";
+            }
+
+            new SnapshotMXBeanImpl(ignite.context()).cancelSnapshot(taskArg.snapshotName());
+
+            return "Snapshot operation cancelled [snapshot=" + taskArg.snapshotName() + "].";
         }
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotCancelTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotCancelTaskArg.java
new file mode 100644
index 0000000..ae7897e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotCancelTaskArg.java
@@ -0,0 +1,85 @@
+/*
+ * 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.visor.snapshot;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.UUID;
+import org.apache.ignite.internal.dto.IgniteDataTransferObject;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Argument for the task to cancel snapshot operation.
+ */
+public class VisorSnapshotCancelTaskArg extends IgniteDataTransferObject {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
+    /** Snapshot operation request ID. */
+    private UUID reqId;
+
+    /** Snapshot name. */
+    private String snpName;
+
+    /** Default constructor. */
+    public VisorSnapshotCancelTaskArg() {
+        // No-op.
+    }
+
+    /**
+     * @param reqId Snapshot operation request ID.
+     * @param snpName Snapshot name.
+     */
+    public VisorSnapshotCancelTaskArg(UUID reqId, String snpName) {
+        assert (reqId != null && snpName == null) || (reqId == null && !F.isEmpty(snpName));
+
+        this.snpName = snpName;
+        this.reqId = reqId;
+    }
+
+    /** @return Snapshot name. */
+    @Deprecated
+    public String snapshotName() {
+        return snpName;
+    }
+
+    /** @return Snapshot operation request ID. */
+    public UUID requestId() {
+        return reqId;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        U.writeUuid(out, reqId);
+        U.writeString(out, snpName);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte ver, ObjectInput in) throws IOException, ClassNotFoundException {
+        reqId = U.readUuid(in);
+        snpName = U.readString(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorSnapshotCancelTaskArg.class, this);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotCreateTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotCreateTask.java
index 2879c4a..2214a50 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotCreateTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotCreateTask.java
@@ -19,9 +19,10 @@
 
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteSnapshot;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager;
 import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
 import org.apache.ignite.internal.visor.VisorJob;
-import org.apache.ignite.lang.IgniteFuture;
 
 /**
  * @see IgniteSnapshot#createSnapshot(String)
@@ -51,14 +52,20 @@
 
         /** {@inheritDoc} */
         @Override protected String run(VisorSnapshotCreateTaskArg arg) throws IgniteException {
-            IgniteFuture<Void> fut =
+            IgniteFutureImpl<Void> fut =
                 ignite.context().cache().context().snapshotMgr().createSnapshot(arg.snapshotName(), arg.snapshotPath());
 
+            IgniteSnapshotManager.ClusterSnapshotFuture snpFut =
+                fut.internalFuture() instanceof IgniteSnapshotManager.ClusterSnapshotFuture ?
+                    (IgniteSnapshotManager.ClusterSnapshotFuture)fut.internalFuture() : null;
+
             if (arg.sync() || fut.isDone())
                 fut.get();
 
-            return "Snapshot operation " +
-                (arg.sync() ? "completed successfully" : "started") + ": " + arg.snapshotName();
+            String msgOperId = snpFut != null && snpFut.requestId() != null ? ", id=" + snpFut.requestId() : "";
+
+            return "Snapshot create operation " + (arg.sync() ? "completed successfully" : "started") +
+                " [name=" + arg.snapshotName() + msgOperId + ']';
         }
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotRestoreTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotRestoreTask.java
index 1892af8..bae3d2e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotRestoreTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotRestoreTask.java
@@ -18,10 +18,11 @@
 package org.apache.ignite.internal.visor.snapshot;
 
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager;
 import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.visor.VisorJob;
-import org.apache.ignite.lang.IgniteFuture;
 
 /**
  * Visor snapshot restore task.
@@ -33,7 +34,10 @@
 
     /** {@inheritDoc} */
     @Override protected VisorJob<VisorSnapshotRestoreTaskArg, String> job(VisorSnapshotRestoreTaskArg arg) {
-        switch (arg.jobAction()) {
+        VisorSnapshotRestoreTaskAction action =
+            arg.jobAction() == null ? VisorSnapshotRestoreTaskAction.START : arg.jobAction();
+
+        switch (action) {
             case START:
                 return new VisorSnapshotStartRestoreJob(arg, debug);
 
@@ -63,20 +67,29 @@
 
         /** {@inheritDoc} */
         @Override protected String run(VisorSnapshotRestoreTaskArg arg) throws IgniteException {
-            IgniteFuture<Void> fut = ignite.context().cache().context().snapshotMgr()
+            IgniteFutureImpl<Void> fut = ignite.context().cache().context().snapshotMgr()
                 .restoreSnapshot(arg.snapshotName(), arg.snapshotPath(), arg.groupNames());
 
+            IgniteSnapshotManager.ClusterSnapshotFuture snpFut =
+                fut.internalFuture() instanceof IgniteSnapshotManager.ClusterSnapshotFuture ?
+                    (IgniteSnapshotManager.ClusterSnapshotFuture)fut.internalFuture() : null;
+
             if (arg.sync() || fut.isDone())
                 fut.get();
-            
+
             String msgSuff = arg.sync() ? "completed successfully" : "started";
             String msgGrps = arg.groupNames() == null ? "" : ", group(s)=" + F.concat(arg.groupNames(), ",");
+            String msgId = snpFut != null && snpFut.requestId() != null ? ", id=" + snpFut.requestId() : "";
 
-            return "Snapshot cache group restore operation " + msgSuff + " [snapshot=" + arg.snapshotName() + msgGrps + ']';
+            return "Snapshot cache group restore operation " + msgSuff +
+                " [name=" + arg.snapshotName() + msgGrps + msgId + ']';
         }
     }
 
-    /** */
+    /**
+     * @deprecated Use {@link VisorSnapshotCancelTask} instead.
+     */
+    @Deprecated
     private static class VisorSnapshotRestoreCancelJob extends VisorJob<VisorSnapshotRestoreTaskArg, String> {
         /** Serial version uid. */
         private static final long serialVersionUID = 0L;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotRestoreTaskAction.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotRestoreTaskAction.java
index 04c2c74..19da78d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotRestoreTaskAction.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotRestoreTaskAction.java
@@ -18,11 +18,22 @@
 package org.apache.ignite.internal.visor.snapshot;
 
 /** Snapshot restore operation management action. */
+@Deprecated
 public enum VisorSnapshotRestoreTaskAction {
-    /** Start snapshot restore operation. */
+    /**
+     * Start snapshot restore operation.
+     *
+     * @deprecated This option is redundant and should be removed in future releases.
+     */
+    @Deprecated
     START,
 
-    /** Cancel snapshot restore operation. */
+    /**
+     * Cancel snapshot restore operation.
+     *
+     * @deprecated Use {@link VisorSnapshotCancelTask} instead.
+     */
+    @Deprecated
     CANCEL,
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/SnapshotMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/SnapshotMXBean.java
index e6cdef4..d8b06ea 100644
--- a/modules/core/src/main/java/org/apache/ignite/mxbean/SnapshotMXBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/mxbean/SnapshotMXBean.java
@@ -44,11 +44,21 @@
      * Cancel previously started snapshot operation on the node initiator.
      *
      * @param snpName Snapshot name to cancel.
+     * @deprecated Use {@link #cancelSnapshotOperation(String)} instead.
      */
     @MXBeanDescription("Cancel started cluster-wide snapshot on the node initiator.")
+    @Deprecated
     public void cancelSnapshot(@MXBeanParameter(name = "snpName", description = "Snapshot name.") String snpName);
 
     /**
+     * Cancel previously started snapshot operation.
+     *
+     * @param reqId Snapshot operation request ID.
+     */
+    @MXBeanDescription("Cancel started cluster-wide snapshot operation.")
+    public void cancelSnapshotOperation(@MXBeanParameter(name = "requestId", description = "Snapshot operation request ID.") String reqId);
+
+    /**
      * Restore cluster-wide snapshot.
      *
      * @param name Snapshot name.
@@ -71,8 +81,10 @@
      *
      * @param name Snapshot name.
      * @see IgniteSnapshot#cancelSnapshotRestore(String)
+     * @deprecated Use {@link #cancelSnapshotOperation(String)} instead.
      */
     @MXBeanDescription("Cancel previously started snapshot restore operation.")
+    @Deprecated
     public void cancelSnapshotRestore(@MXBeanParameter(name = "snpName", description = "Snapshot name.") String name);
 
     /**
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index 5675a08..e315041 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -2274,6 +2274,7 @@
 org.apache.ignite.internal.visor.snapshot.VisorSnapshotCreateTask
 org.apache.ignite.internal.visor.snapshot.VisorSnapshotCreateTask$VisorSnapshotCreateJob
 org.apache.ignite.internal.visor.snapshot.VisorSnapshotCancelTask
+org.apache.ignite.internal.visor.snapshot.VisorSnapshotCancelTaskArg
 org.apache.ignite.internal.visor.snapshot.VisorSnapshotCancelTask$VisorSnapshotCancelJob
 org.apache.ignite.internal.visor.snapshot.VisorSnapshotTaskResult
 org.apache.ignite.internal.visor.snapshot.VisorSnapshotRestoreTask
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java
index 82befa7..9aa661a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java
@@ -626,8 +626,8 @@
         boolean withMetaStorage,
         SnapshotSender snpSndr
     ) throws IgniteCheckedException {
-        AbstractSnapshotFutureTask<?> task = cctx.snapshotMgr().registerSnapshotTask(snpName, cctx.localNodeId(), parts,
-            withMetaStorage, snpSndr);
+        AbstractSnapshotFutureTask<?> task = cctx.snapshotMgr().registerSnapshotTask(snpName, cctx.localNodeId(), null,
+            parts, withMetaStorage, snpSndr);
 
         if (!(task instanceof SnapshotFutureTask))
             throw new IgniteCheckedException("Snapshot task hasn't been registered: " + task);
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/EncryptedSnapshotTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/EncryptedSnapshotTest.java
index 8912e8e..bcceee7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/EncryptedSnapshotTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/EncryptedSnapshotTest.java
@@ -283,7 +283,7 @@
 
         GridTestUtils.assertThrowsAnyCause(log,
             () -> snp(ig).registerSnapshotTask(SNAPSHOT_NAME, ig.localNode().id(),
-                F.asMap(CU.cacheId(dfltCacheCfg.getName()), null), false,
+                null, F.asMap(CU.cacheId(dfltCacheCfg.getName()), null), false,
                 snp(ig).localSnapshotSenderFactory().apply(SNAPSHOT_NAME, null)).get(TIMEOUT),
             IgniteCheckedException.class,
             "Metastore is required because it holds encryption keys");
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManagerSelfTest.java
index 919d43e..4631a82 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManagerSelfTest.java
@@ -146,6 +146,7 @@
         // Register task but not schedule it on the checkpoint.
         SnapshotFutureTask snpFutTask = (SnapshotFutureTask)mgr.registerSnapshotTask(SNAPSHOT_NAME,
             cctx.localNodeId(),
+            null,
             F.asMap(CU.cacheId(DEFAULT_CACHE_NAME), null),
             encryption,
             new DelegateSnapshotSender(log, mgr.snapshotExecutorService(), mgr.localSnapshotSenderFactory().apply(SNAPSHOT_NAME, null)) {
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotRemoteRequestTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotRemoteRequestTest.java
index 59525a2..d8ba7f1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotRemoteRequestTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotRemoteRequestTest.java
@@ -100,6 +100,7 @@
                 IgniteInternalFuture<Void> locFut;
 
                 compFut.add(locFut = snp(ignite).requestRemoteSnapshotFiles(grid(1).localNode().id(),
+                    null,
                     SNAPSHOT_NAME,
                     null,
                     parts,
@@ -142,10 +143,10 @@
             fromNode0.values().stream().mapToInt(Set::size).sum());
 
         // Snapshot must be taken on node1 and transmitted to node0.
-        IgniteInternalFuture<?> futFrom1To0 = mgr0.requestRemoteSnapshotFiles(node1, SNAPSHOT_NAME, null, fromNode1, () -> false,
-            defaultPartitionConsumer(fromNode1, latch));
-        IgniteInternalFuture<?> futFrom0To1 = mgr1.requestRemoteSnapshotFiles(node0, SNAPSHOT_NAME, null, fromNode0, () -> false,
-            defaultPartitionConsumer(fromNode0, latch));
+        IgniteInternalFuture<?> futFrom1To0 = mgr0.requestRemoteSnapshotFiles(node1, null, SNAPSHOT_NAME, null,
+            fromNode1, () -> false, defaultPartitionConsumer(fromNode1, latch));
+        IgniteInternalFuture<?> futFrom0To1 = mgr1.requestRemoteSnapshotFiles(node0, null, SNAPSHOT_NAME, null,
+            fromNode0, () -> false, defaultPartitionConsumer(fromNode0, latch));
 
         G.allGrids().forEach(g -> TestRecordingCommunicationSpi.spi(g).stopBlock());
 
@@ -192,6 +193,7 @@
         });
 
         snp(ignite).requestRemoteSnapshotFiles(grid(1).localNode().id(),
+            null,
             SNAPSHOT_NAME,
             null,
             parts,
@@ -234,6 +236,7 @@
         CountDownLatch latch = new CountDownLatch(1);
 
         IgniteInternalFuture<?> fut = snp(ignite).requestRemoteSnapshotFiles(grid(1).localNode().id(),
+            null,
             SNAPSHOT_NAME,
             null,
             parts,
@@ -281,6 +284,7 @@
         AtomicBoolean stopChecker = new AtomicBoolean();
 
         IgniteInternalFuture<Void> fut = snp(ignite).requestRemoteSnapshotFiles(grid(1).localNode().id(),
+            null,
             SNAPSHOT_NAME,
             null,
             parts,
@@ -354,7 +358,7 @@
             Map<Integer, Set<Integer>> expParts = owningParts(rcv, CU.cacheId(DEFAULT_CACHE_NAME), sndNode);
 
             IgniteInternalFuture<Void> fut = snp(rcv)
-                .requestRemoteSnapshotFiles(sndNode, SNAPSHOT_NAME, null, expParts, () -> false,
+                .requestRemoteSnapshotFiles(sndNode, null, SNAPSHOT_NAME, null, expParts, () -> false,
                     defaultPartitionConsumer(expParts, null));
 
             fut.listen(f -> expParts.values().forEach(integers -> assertTrue(integers.isEmpty())));
diff --git a/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassTest_help.output b/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassTest_help.output
index 7ef5639..c1b37a4 100644
--- a/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassTest_help.output
+++ b/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassTest_help.output
@@ -147,11 +147,12 @@
       --dest path    - Path to the directory where the snapshot will be saved. If not specified, the default configured snapshot directory will be used.
       --sync         - Run the operation synchronously, the command will wait for the entire operation to complete. Otherwise, it will be performed in the background, and the command will immediately return control.
 
-  Cancel running snapshot:
-    control.(sh|bat) --snapshot cancel snapshot_name
+  Cancel running snapshot operation:
+    control.(sh|bat) --snapshot cancel --id id|--name name
 
     Parameters:
-      snapshot_name  - Snapshot name.
+      --id id      - Snapshot operation request ID.
+      --name name  - Snapshot name (deprecated).
 
   Check snapshot:
     control.(sh|bat) --snapshot check snapshot_name [--src path]
@@ -161,7 +162,7 @@
       --src path     - Path to the directory where the snapshot files are located. If not specified, the default configured snapshot directory will be used.
 
   Restore snapshot:
-    control.(sh|bat) --snapshot restore snapshot_name --start [--groups group1,...groupN] [--src path] [--sync]
+    control.(sh|bat) --snapshot restore snapshot_name [--groups group1,...groupN] [--src path] [--sync]
 
     Parameters:
       snapshot_name              - Snapshot name.
@@ -169,13 +170,13 @@
       --src path                 - Path to the directory where the snapshot files are located. If not specified, the default configured snapshot directory will be used.
       --sync                     - Run the operation synchronously, the command will wait for the entire operation to complete. Otherwise, it will be performed in the background, and the command will immediately return control.
 
-  Snapshot restore operation status (Command deprecated. Use '--snapshot status' instead.):
+  Snapshot restore operation status (Command deprecated. Use '--snapshot status' instead):
     control.(sh|bat) --snapshot restore snapshot_name --status
 
     Parameters:
       snapshot_name  - Snapshot name.
 
-  Cancel snapshot restore operation:
+  Cancel snapshot restore operation (Command deprecated. Use '--snapshot cancel' instead):
     control.(sh|bat) --snapshot restore snapshot_name --cancel
 
     Parameters:
diff --git a/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassWithSSLTest_help.output b/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassWithSSLTest_help.output
index 7ef5639..c1b37a4 100644
--- a/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassWithSSLTest_help.output
+++ b/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassWithSSLTest_help.output
@@ -147,11 +147,12 @@
       --dest path    - Path to the directory where the snapshot will be saved. If not specified, the default configured snapshot directory will be used.
       --sync         - Run the operation synchronously, the command will wait for the entire operation to complete. Otherwise, it will be performed in the background, and the command will immediately return control.
 
-  Cancel running snapshot:
-    control.(sh|bat) --snapshot cancel snapshot_name
+  Cancel running snapshot operation:
+    control.(sh|bat) --snapshot cancel --id id|--name name
 
     Parameters:
-      snapshot_name  - Snapshot name.
+      --id id      - Snapshot operation request ID.
+      --name name  - Snapshot name (deprecated).
 
   Check snapshot:
     control.(sh|bat) --snapshot check snapshot_name [--src path]
@@ -161,7 +162,7 @@
       --src path     - Path to the directory where the snapshot files are located. If not specified, the default configured snapshot directory will be used.
 
   Restore snapshot:
-    control.(sh|bat) --snapshot restore snapshot_name --start [--groups group1,...groupN] [--src path] [--sync]
+    control.(sh|bat) --snapshot restore snapshot_name [--groups group1,...groupN] [--src path] [--sync]
 
     Parameters:
       snapshot_name              - Snapshot name.
@@ -169,13 +170,13 @@
       --src path                 - Path to the directory where the snapshot files are located. If not specified, the default configured snapshot directory will be used.
       --sync                     - Run the operation synchronously, the command will wait for the entire operation to complete. Otherwise, it will be performed in the background, and the command will immediately return control.
 
-  Snapshot restore operation status (Command deprecated. Use '--snapshot status' instead.):
+  Snapshot restore operation status (Command deprecated. Use '--snapshot status' instead):
     control.(sh|bat) --snapshot restore snapshot_name --status
 
     Parameters:
       snapshot_name  - Snapshot name.
 
-  Cancel snapshot restore operation:
+  Cancel snapshot restore operation (Command deprecated. Use '--snapshot cancel' instead):
     control.(sh|bat) --snapshot restore snapshot_name --cancel
 
     Parameters: