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

[ignite] branch ignite-2.11 updated: IGNITE-14723 Restore cache from a snapshot using CLI. (#9113)

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

mmuzaf pushed a commit to branch ignite-2.11
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/ignite-2.11 by this push:
     new 5c54001  IGNITE-14723 Restore cache from a snapshot using CLI. (#9113)
5c54001 is described below

commit 5c540010764810efefa466fd54c71ab626c7c4e1
Author: Pavel Pereslegin <xx...@gmail.com>
AuthorDate: Fri Jun 18 19:02:52 2021 +0300

    IGNITE-14723 Restore cache from a snapshot using CLI. (#9113)
---
 .../commandline/snapshot/SnapshotCommand.java      |  77 +++++++--
 .../commandline/snapshot/SnapshotSubcommand.java   |   8 +-
 .../util/GridCommandHandlerAbstractTest.java       |   8 +-
 .../util/GridCommandHandlerIndexingTest.java       |   2 +-
 .../apache/ignite/util/GridCommandHandlerTest.java | 181 +++++++++++++++++++++
 .../java/org/apache/ignite/IgniteSnapshot.java     |   9 +
 .../snapshot/IgniteSnapshotManager.java            |  78 ++++++++-
 .../snapshot/SnapshotRestoreCancelTask.java        |  47 ++++++
 .../snapshot/SnapshotRestoreManagementTask.java    |  73 +++++++++
 .../snapshot/SnapshotRestoreProcess.java           |  94 +++++++++--
 .../snapshot/SnapshotRestoreStatusTask.java        |  47 ++++++
 .../visor/snapshot/VisorSnapshotRestoreTask.java   | 121 ++++++++++++++
 .../snapshot/VisorSnapshotRestoreTaskAction.java   |  48 ++++++
 .../snapshot/VisorSnapshotRestoreTaskArg.java      |  98 +++++++++++
 .../main/resources/META-INF/classnames.properties  |   6 +
 ...ridCommandHandlerClusterByClassTest_help.output |  19 +++
 ...andHandlerClusterByClassWithSSLTest_help.output |  19 +++
 17 files changed, 888 insertions(+), 47 deletions(-)

diff --git a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotCommand.java b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotCommand.java
index 79010ee..b6a465a 100644
--- a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotCommand.java
+++ b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotCommand.java
@@ -17,6 +17,10 @@
 
 package org.apache.ignite.internal.commandline.snapshot;
 
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
 import java.util.logging.Logger;
 import org.apache.ignite.internal.client.GridClient;
 import org.apache.ignite.internal.client.GridClientConfiguration;
@@ -26,15 +30,17 @@ import org.apache.ignite.internal.commandline.CommandArgIterator;
 import org.apache.ignite.internal.commandline.CommandLogger;
 import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager;
 import org.apache.ignite.internal.processors.cache.verify.IdleVerifyResultV2;
-import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.visor.snapshot.VisorSnapshotRestoreTaskAction;
+import org.apache.ignite.internal.visor.snapshot.VisorSnapshotRestoreTaskArg;
 import org.apache.ignite.mxbean.SnapshotMXBean;
 
-import static java.util.Collections.singletonMap;
 import static org.apache.ignite.internal.commandline.CommandList.SNAPSHOT;
+import static org.apache.ignite.internal.commandline.CommandLogger.optional;
 import static org.apache.ignite.internal.commandline.TaskExecutor.executeTaskByNameOnNode;
 import static org.apache.ignite.internal.commandline.snapshot.SnapshotSubcommand.CANCEL;
 import static org.apache.ignite.internal.commandline.snapshot.SnapshotSubcommand.CHECK;
 import static org.apache.ignite.internal.commandline.snapshot.SnapshotSubcommand.CREATE;
+import static org.apache.ignite.internal.commandline.snapshot.SnapshotSubcommand.RESTORE;
 import static org.apache.ignite.internal.commandline.snapshot.SnapshotSubcommand.of;
 
 /**
@@ -45,7 +51,7 @@ import static org.apache.ignite.internal.commandline.snapshot.SnapshotSubcommand
  */
 public class SnapshotCommand extends AbstractCommand<Object> {
     /** Command argument. */
-    private String snpName;
+    private Object cmdArg;
 
     /** Snapshot sub-command to execute. */
     private SnapshotSubcommand cmd;
@@ -56,7 +62,7 @@ public class SnapshotCommand extends AbstractCommand<Object> {
             Object res = executeTaskByNameOnNode(
                 client,
                 cmd.taskName(),
-                snpName,
+                arg(),
                 null,
                 clientCfg
             );
@@ -64,6 +70,9 @@ public class SnapshotCommand extends AbstractCommand<Object> {
             if (cmd == CHECK)
                 ((IdleVerifyResultV2)res).print(log::info, true);
 
+            if (cmd == RESTORE)
+                log.info(String.valueOf(res));
+
             return res;
         }
         catch (Throwable e) {
@@ -76,28 +85,68 @@ public class SnapshotCommand extends AbstractCommand<Object> {
 
     /** {@inheritDoc} */
     @Override public Object arg() {
-        return snpName;
+        return cmdArg;
     }
 
     /** {@inheritDoc} */
     @Override public void parseArguments(CommandArgIterator argIter) {
         cmd = of(argIter.nextArg("Expected snapshot action."));
-        snpName = argIter.nextArg("Expected snapshot name.");
+        String snpName = argIter.nextArg("Expected snapshot name.");
+
+        if (cmd != RESTORE) {
+            cmdArg = snpName;
+
+            return;
+        }
+
+        VisorSnapshotRestoreTaskAction cmdAction =
+            VisorSnapshotRestoreTaskAction.fromCmdArg(argIter.nextArg("Restore action expected."));
+
+        Set<String> grpNames = null;
+
+        if (argIter.hasNextSubArg()) {
+            String arg = argIter.nextArg("");
+
+            if (cmdAction != VisorSnapshotRestoreTaskAction.START)
+                throw new IllegalArgumentException("Invalid argument \"" + arg + "\", no more arguments expected.");
+
+            grpNames = argIter.parseStringSet(arg);
+        }
 
-        if (F.isEmpty(snpName))
-            throw new IllegalArgumentException("Expected snapshot name.");
+        cmdArg = new VisorSnapshotRestoreTaskArg(cmdAction, snpName, grpNames);
     }
 
     /** {@inheritDoc} */
     @Override public void printUsage(Logger log) {
-        Command.usage(log, "Create cluster snapshot:", SNAPSHOT, singletonMap("snapshot_name", "Snapshot name."),
-            CREATE.toString(), "snapshot_name");
+        Map<String, String> commonParams = Collections.singletonMap("snapshot_name", "Snapshot name.");
+
+        Command.usage(log, "Create cluster snapshot:", SNAPSHOT, commonParams, CREATE.toString(), "snapshot_name");
+        Command.usage(log, "Cancel running snapshot:", SNAPSHOT, commonParams, CANCEL.toString(), "snapshot_name");
+        Command.usage(log, "Check snapshot:", SNAPSHOT, commonParams, CHECK.toString(), "snapshot_name");
+
+        Map<String, String> startParams = new LinkedHashMap<>(commonParams);
+
+        startParams.put("group1,...groupN", "Cache group names.");
+
+        Command.usage(log, "Restore snapshot:", SNAPSHOT, startParams, RESTORE.toString(),
+            VisorSnapshotRestoreTaskAction.START.cmdName(), "snapshot_name", optional("group1,...groupN"));
+
+        Command.usage(log, "Snapshot restore operation status:", SNAPSHOT, commonParams, RESTORE.toString(),
+            VisorSnapshotRestoreTaskAction.STATUS.cmdName(), "snapshot_name");
+
+        Command.usage(log, "Cancel snapshot restore operation:", SNAPSHOT, commonParams, RESTORE.toString(),
+            VisorSnapshotRestoreTaskAction.CANCEL.cmdName(), "snapshot_name");
+    }
+
+    /** {@inheritDoc} */
+    @Override public String confirmationPrompt() {
+        if (cmd != RESTORE)
+            return null;
 
-        Command.usage(log, "Cancel running snapshot:", SNAPSHOT, singletonMap("snapshot_name", "Snapshot name."),
-            CANCEL.toString(), "snapshot_name");
+        VisorSnapshotRestoreTaskArg arg = (VisorSnapshotRestoreTaskArg)cmdArg;
 
-        Command.usage(log, "Check snapshot:", SNAPSHOT, singletonMap("snapshot_name", "Snapshot name."),
-            CHECK.toString(), "snapshot_name");
+        return arg.jobAction() == VisorSnapshotRestoreTaskAction.START && arg.groupNames() != null ? null :
+            "Warning: command will restore ALL PUBLIC CACHE GROUPS from the snapshot " + arg.snapshotName() + '.';
     }
 
     /** {@inheritDoc} */
diff --git a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotSubcommand.java b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotSubcommand.java
index d9d3a61..55526a2 100644
--- a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotSubcommand.java
+++ b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/snapshot/SnapshotSubcommand.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.commandline.snapshot;
 import org.apache.ignite.internal.visor.snapshot.VisorSnapshotCancelTask;
 import org.apache.ignite.internal.visor.snapshot.VisorSnapshotCheckTask;
 import org.apache.ignite.internal.visor.snapshot.VisorSnapshotCreateTask;
+import org.apache.ignite.internal.visor.snapshot.VisorSnapshotRestoreTask;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -35,7 +36,10 @@ public enum SnapshotSubcommand {
     CANCEL("cancel", VisorSnapshotCancelTask.class.getName()),
 
     /** Sub-command to check snapshot. */
-    CHECK("check", VisorSnapshotCheckTask.class.getName());
+    CHECK("check", VisorSnapshotCheckTask.class.getName()),
+
+    /** Sub-command to restore snapshot. */
+    RESTORE("restore", VisorSnapshotRestoreTask.class.getName());
 
     /** Sub-command name. */
     private final String name;
@@ -53,7 +57,7 @@ public enum SnapshotSubcommand {
      * @param text Command text (case insensitive).
      * @return Command for the text. {@code Null} if there is no such command.
      */
-     @Nullable public static SnapshotSubcommand of(String text) {
+    @Nullable public static SnapshotSubcommand of(String text) {
         for (SnapshotSubcommand cmd : values()) {
             if (cmd.name.equalsIgnoreCase(text))
                 return cmd;
diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerAbstractTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerAbstractTest.java
index 14b48d1..ce84c9a 100644
--- a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerAbstractTest.java
+++ b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerAbstractTest.java
@@ -428,19 +428,21 @@ public abstract class GridCommandHandlerAbstractTest extends GridCommonAbstractT
      * </table>
      *
      * @param ignite Ignite.
+     * @param cacheName Cache name.
      * @param countEntries Count of entries.
      * @param partitions Partitions count.
      * @param filter Node filter.
      */
     protected void createCacheAndPreload(
         Ignite ignite,
+        String cacheName,
         int countEntries,
         int partitions,
         @Nullable IgnitePredicate<ClusterNode> filter
     ) {
         assert nonNull(ignite);
 
-        CacheConfiguration<?, ?> ccfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+        CacheConfiguration<?, ?> ccfg = new CacheConfiguration<>(cacheName)
             .setAffinity(new RendezvousAffinityFunction(false, partitions))
             .setBackups(1)
             .setEncryptionEnabled(encryptionEnabled);
@@ -450,7 +452,7 @@ public abstract class GridCommandHandlerAbstractTest extends GridCommonAbstractT
 
         ignite.createCache(ccfg);
 
-        IgniteCache<Object, Object> cache = ignite.cache(DEFAULT_CACHE_NAME);
+        IgniteCache<Object, Object> cache = ignite.cache(cacheName);
         for (int i = 0; i < countEntries; i++)
             cache.put(i, i);
     }
@@ -462,6 +464,6 @@ public abstract class GridCommandHandlerAbstractTest extends GridCommonAbstractT
      * @param countEntries Count of entries.
      */
     protected void createCacheAndPreload(Ignite ignite, int countEntries) {
-        createCacheAndPreload(ignite, countEntries, 32, null);
+        createCacheAndPreload(ignite, DEFAULT_CACHE_NAME, countEntries, 32, null);
     }
 }
diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java
index 504a3c1..e7bb062 100644
--- a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java
+++ b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java
@@ -144,7 +144,7 @@ public class GridCommandHandlerIndexingTest extends GridCommandHandlerClusterPer
 
         int maxItems = 100000;
 
-        createCacheAndPreload(ig, cntPreload, 1, new CachePredicate(F.asList(ig.name())));
+        createCacheAndPreload(ig, DEFAULT_CACHE_NAME, cntPreload, 1, new CachePredicate(F.asList(ig.name())));
 
         if (persistenceEnable()) {
             forceCheckpoint();
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 33683dd..74464e9 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
@@ -43,6 +43,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.atomic.LongAdder;
 import java.util.function.Function;
 import java.util.function.UnaryOperator;
@@ -103,6 +104,7 @@ import org.apache.ignite.internal.processors.cache.warmup.BlockedWarmUpConfigura
 import org.apache.ignite.internal.processors.cache.warmup.BlockedWarmUpStrategy;
 import org.apache.ignite.internal.processors.cache.warmup.WarmUpTestPluginProvider;
 import org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor;
+import org.apache.ignite.internal.util.distributed.SingleNodeMessage;
 import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.lang.GridFunc;
@@ -161,6 +163,7 @@ import static org.apache.ignite.internal.processors.cache.persistence.snapshot.I
 import static org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.resolveSnapshotWorkDirectory;
 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.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE;
 import static org.apache.ignite.testframework.GridTestUtils.assertContains;
 import static org.apache.ignite.testframework.GridTestUtils.assertThrows;
 import static org.apache.ignite.testframework.GridTestUtils.runAsync;
@@ -3110,6 +3113,184 @@ public class GridCommandHandlerTest extends GridCommandHandlerClusterPerMethodAb
         assertContains(log, sb.toString(), "The check procedure has finished, no conflicts have been found");
     }
 
+    /** @throws Exception If fails. */
+    @Test
+    public void testSnapshotRestore() throws Exception {
+        int keysCnt = 100;
+        String snpName = "snapshot_02052020";
+        String cacheName1 = "cache1";
+        String cacheName2 = "cache2";
+        String cacheName3 = "cache3";
+
+        IgniteEx ig = startGrids(2);
+
+        ig.cluster().state(ACTIVE);
+
+        injectTestSystemOut();
+
+        createCacheAndPreload(ig, cacheName1, keysCnt, 32, null);
+        createCacheAndPreload(ig, cacheName2, keysCnt, 32, null);
+        createCacheAndPreload(ig, cacheName3, keysCnt, 32, null);
+
+        ig.snapshot().createSnapshot(snpName).get(getTestTimeout());
+
+        IgniteCache<Integer, Integer> cache1 = ig.cache(cacheName1);
+        IgniteCache<Integer, Integer> cache2 = ig.cache(cacheName2);
+        IgniteCache<Integer, Integer> cache3 = ig.cache(cacheName3);
+
+        cache1.destroy();
+        cache2.destroy();
+        cache3.destroy();
+
+        awaitPartitionMapExchange();
+
+        assertNull(ig.cache(cacheName1));
+        assertNull(ig.cache(cacheName2));
+        assertNull(ig.cache(cacheName3));
+
+        CommandHandler h = new CommandHandler();
+
+        // Restore single cache group.
+        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", snpName, "--start", cacheName1));
+        assertContains(log, testOut.toString(),
+            "Snapshot cache group restore operation started [snapshot=" + snpName + ", group(s)=" + cacheName1 + ']');
+
+        waitForCondition(() -> ig.cache(cacheName1) != null, getTestTimeout());
+
+        cache1 = ig.cache(cacheName1);
+
+        assertNotNull(cache1);
+
+        for (int i = 0; i < keysCnt; i++)
+            assertEquals(cacheName1, Integer.valueOf(i), cache1.get(i));
+
+        cache1.destroy();
+
+        awaitPartitionMapExchange();
+
+        assertNull(ig.cache(cacheName1));
+        assertNull(ig.cache(cacheName2));
+        assertNull(ig.cache(cacheName3));
+
+        // Restore two (of three) groups of caches.
+        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", snpName, "--start", cacheName1 + ',' + cacheName2));
+        assertContains(log, testOut.toString(),
+            "Snapshot cache group restore operation started [snapshot=" + snpName + ", group(s)=");
+
+        waitForCondition(() -> ig.cache(cacheName1) != null, getTestTimeout());
+        waitForCondition(() -> ig.cache(cacheName2) != null, getTestTimeout());
+
+        cache1 = ig.cache(cacheName1);
+        cache2 = ig.cache(cacheName2);
+
+        assertNotNull(cache1);
+        assertNotNull(cache2);
+
+        for (int i = 0; i < keysCnt; i++) {
+            assertEquals(cacheName1, Integer.valueOf(i), cache1.get(i));
+            assertEquals(cacheName2, Integer.valueOf(i), cache2.get(i));
+        }
+
+        cache1.destroy();
+        cache2.destroy();
+
+        awaitPartitionMapExchange();
+
+        assertNull(ig.cache(cacheName1));
+        assertNull(ig.cache(cacheName2));
+        assertNull(ig.cache(cacheName3));
+
+        // Restore all public cache groups.
+        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", snpName, "--start"));
+        assertContains(log, testOut.toString(),
+            "Snapshot cache group restore operation started [snapshot=" + snpName + ']');
+
+        waitForCondition(() -> ig.cache(cacheName1) != null, getTestTimeout());
+        waitForCondition(() -> ig.cache(cacheName2) != null, getTestTimeout());
+        waitForCondition(() -> ig.cache(cacheName3) != null, getTestTimeout());
+
+        cache1 = ig.cache(cacheName1);
+        cache2 = ig.cache(cacheName2);
+        cache3 = ig.cache(cacheName3);
+
+        assertNotNull(cache1);
+        assertNotNull(cache2);
+        assertNotNull(cache3);
+
+        for (int i = 0; i < keysCnt; i++) {
+            assertEquals(cacheName1, Integer.valueOf(i), cache1.get(i));
+            assertEquals(cacheName2, Integer.valueOf(i), cache2.get(i));
+            assertEquals(cacheName3, Integer.valueOf(i), cache2.get(i));
+        }
+    }
+
+    /** @throws Exception If fails. */
+    @Test
+    public void testSnapshotRestoreCancelAndStatus() throws Exception {
+        int keysCnt = 10_000;
+        String snpName = "snapshot_25052021";
+        String missingSnpName = "snapshot_MISSING";
+
+        IgniteEx ig = startGrids(2);
+
+        ig.cluster().state(ACTIVE);
+
+        injectTestSystemOut();
+
+        createCacheAndPreload(ig, keysCnt);
+
+        ig.snapshot().createSnapshot(snpName).get(getTestTimeout());
+
+        IgniteCache<Integer, Integer> cache1 = ig.cache(DEFAULT_CACHE_NAME);
+
+        cache1.destroy();
+
+        CommandHandler h = new CommandHandler();
+
+        TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(1));
+
+        spi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage &&
+            ((SingleNodeMessage<?>)msg).type() == RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE.ordinal());
+
+        // Restore single cache group.
+        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", snpName, "--start", DEFAULT_CACHE_NAME));
+        assertContains(log, testOut.toString(),
+            "Snapshot cache group restore operation started [snapshot=" + snpName + ", group(s)=" + DEFAULT_CACHE_NAME + ']');
+
+        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", snpName, "--status"));
+        assertContains(log, testOut.toString(),
+            "Snapshot cache group restore operation is running [snapshot=" + snpName + ']');
+
+        // Check wrong snapshot name.
+        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", missingSnpName, "--status"));
+        assertContains(log, testOut.toString(),
+            "Snapshot cache group restore operation is NOT running [snapshot=" + missingSnpName + ']');
+
+        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", missingSnpName, "--cancel"));
+        assertContains(log, testOut.toString(),
+            "Snapshot cache group restore operation is not in progress [snapshot=" + missingSnpName + ']');
+
+        GridTestUtils.runAsync(() -> {
+            // Wait for the process to be interrupted.
+            AtomicReference<?> errRef = U.field((Object)U.field((Object)U.field(
+                grid(0).context().cache().context().snapshotMgr(), "restoreCacheGrpProc"), "opCtx"), "err");
+
+            waitForCondition(() -> errRef.get() != null, getTestTimeout());
+
+            spi.stopBlock();
+
+            return null;
+        });
+
+        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", snpName, "--cancel"));
+        assertContains(log, testOut.toString(),
+            "Snapshot cache group restore operation canceled [snapshot=" + snpName + ']');
+
+        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "restore", snpName, "--status"));
+        assertContains(log, testOut.toString(),
+            "Snapshot cache group restore operation is NOT running [snapshot=" + snpName + ']');
+    }
+
     /**
      * @throws Exception If failed.
      */
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java b/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java
index 0d46cbd..5945805 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java
@@ -62,4 +62,13 @@ public interface IgniteSnapshot {
      * @return Future which will be completed when restore operation finished.
      */
     public IgniteFuture<Void> restoreSnapshot(String name, @Nullable Collection<String> cacheGroupNames);
+
+    /**
+     * Cancel snapshot restore operation.
+     *
+     * @param name Snapshot name.
+     * @return Future that will be finished when the process is complete on all nodes. 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> cancelSnapshotRestore(String name);
 }
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 e2ea4d4..622df61 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
@@ -63,6 +63,7 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteSnapshot;
 import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.compute.ComputeTask;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.SnapshotEvent;
@@ -787,19 +788,17 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter
      * @return {@code True} if the snapshot restore operation is in progress.
      */
     public boolean isRestoring() {
-        return restoreCacheGrpProc.isRestoring();
+        return restoreCacheGrpProc.restoringSnapshotName() != null;
     }
 
     /**
-     * @param restoreId Restore process ID.
-     * @return Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails when
-     *         starting the cache(s), the whole procedure is rolled back.
+     * Check if snapshot restore process is currently running.
+     *
+     * @param snpName Snapshot name.
+     * @return {@code True} if the snapshot restore operation from the specified snapshot is in progress locally.
      */
-    public Set<UUID> cacheStartRequiredAliveNodes(@Nullable IgniteUuid restoreId) {
-        if (restoreId == null)
-            return Collections.emptySet();
-
-        return restoreCacheGrpProc.cacheStartRequiredAliveNodes(restoreId);
+    public boolean isRestoring(String snpName) {
+        return snpName.equals(restoreCacheGrpProc.restoringSnapshotName());
     }
 
     /**
@@ -814,6 +813,30 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter
     }
 
     /**
+     * Status of the restore operation cluster-wide.
+     *
+     * @param snpName Snapshot name.
+     * @return Future that will be completed when the status of the restore operation is received from all the server
+     * nodes. The result of this future will be {@code false} if the restore process with the specified snapshot name is
+     * not running on all nodes.
+     */
+    public IgniteFuture<Boolean> restoreStatus(String snpName) {
+        return executeRestoreManagementTask(SnapshotRestoreStatusTask.class, snpName);
+    }
+
+    /**
+     * @param restoreId Restore process ID.
+     * @return Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails when
+     *         starting the cache(s), the whole procedure is rolled back.
+     */
+    public Set<UUID> cacheStartRequiredAliveNodes(@Nullable IgniteUuid restoreId) {
+        if (restoreId == null)
+            return Collections.emptySet();
+
+        return restoreCacheGrpProc.cacheStartRequiredAliveNodes(restoreId);
+    }
+
+    /**
      * @return List of all known snapshots on the local node.
      */
     public List<String> localSnapshotNames() {
@@ -887,6 +910,21 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter
         }
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteFuture<Boolean> cancelSnapshotRestore(String name) {
+        return executeRestoreManagementTask(SnapshotRestoreCancelTask.class, name);
+    }
+
+    /**
+     * @param name 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> cancelLocalRestoreTask(String name) {
+        return restoreCacheGrpProc.cancel(new IgniteCheckedException("Operation has been canceled by the user."), name);
+    }
+
     /**
      * @param name Snapshot name.
      * @return {@code true} if snapshot is OK.
@@ -1534,6 +1572,25 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter
     }
 
     /**
+     * @param taskCls Snapshot restore operation management task class.
+     * @param snpName Snapshot name.
+     */
+    private IgniteFuture<Boolean> executeRestoreManagementTask(
+        Class<? extends ComputeTask<String, Boolean>> taskCls,
+        String snpName
+    ) {
+        cctx.kernalContext().security().authorize(ADMIN_SNAPSHOT);
+
+        Collection<ClusterNode> bltNodes = F.view(cctx.discovery().serverNodes(AffinityTopologyVersion.NONE),
+            (node) -> CU.baselineNode(node, cctx.kernalContext().state().clusterState()));
+
+        cctx.kernalContext().task().setThreadContext(TC_SKIP_AUTH, true);
+        cctx.kernalContext().task().setThreadContext(TC_SUBGRID, bltNodes);
+
+        return new IgniteFutureImpl<>(cctx.kernalContext().task().execute(taskCls, snpName));
+    }
+
+    /**
      * Ves pokrit assertami absolutely ves,
      * PageScan iterator in the ignite core est.
      */
@@ -1995,6 +2052,9 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter
         /** Snapshot finish time. */
         volatile long endTime;
 
+        /** Operation interruption exception. */
+        volatile IgniteCheckedException interruptEx;
+
         /**
          * Default constructor.
          */
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
new file mode 100644
index 0000000..10cc8d4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCancelTask.java
@@ -0,0 +1,47 @@
+/*
+ * 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
new file mode 100644
index 0000000..1cc6956
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreManagementTask.java
@@ -0,0 +1,73 @@
+/*
+ * 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 dde1bd9..f4712e6 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
@@ -172,7 +172,7 @@ public class SnapshotRestoreProcess {
                 throw new IgniteException(OP_REJECT_MSG + "A cluster snapshot operation is in progress.");
 
             synchronized (this) {
-                if (isRestoring() || fut != null)
+                if (restoringSnapshotName() != null)
                     throw new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed.");
 
                 fut = new ClusterSnapshotFuture(UUID.randomUUID(), snpName);
@@ -197,6 +197,12 @@ public class SnapshotRestoreProcess {
                 return;
             }
 
+            if (fut0.interruptEx != null) {
+                finishProcess(fut0.rqId, fut0.interruptEx);
+
+                return;
+            }
+
             Set<UUID> dataNodes = new HashSet<>();
             Set<String> snpBltNodes = null;
             Map<ClusterNode, List<SnapshotMetadata>> metas = f.result().metas();
@@ -267,12 +273,19 @@ public class SnapshotRestoreProcess {
     }
 
     /**
-     * Check if snapshot restore process is currently running.
+     * Get the name of the snapshot currently being restored
      *
-     * @return {@code True} if the snapshot restore operation is in progress.
+     * @return Name of the snapshot currently being restored or {@code null} if the restore process is not running.
      */
-    public boolean isRestoring() {
-        return isRestoring(null, null);
+    public @Nullable String restoringSnapshotName() {
+        SnapshotRestoreContext opCtx0 = opCtx;
+
+        if (opCtx0 != null)
+            return opCtx0.snpName;
+
+        ClusterSnapshotFuture fut0 = fut;
+
+        return fut0 != null ? fut0.name : null;
     }
 
     /**
@@ -282,15 +295,14 @@ public class SnapshotRestoreProcess {
      * @param grpName Cache group name.
      * @return {@code True} if the cache or group with the specified name is currently being restored.
      */
-    public boolean isRestoring(@Nullable String cacheName, @Nullable String grpName) {
+    public boolean isRestoring(String cacheName, @Nullable String grpName) {
+        assert cacheName != null;
+
         SnapshotRestoreContext opCtx0 = opCtx;
 
         if (opCtx0 == null)
             return false;
 
-        if (cacheName == null)
-            return true;
-
         Map<Integer, StoredCacheData> cacheCfgs = opCtx0.cfgs;
 
         int cacheId = CU.cacheId(cacheName);
@@ -381,22 +393,61 @@ public class SnapshotRestoreProcess {
     }
 
     /**
-     * Abort the currently running restore procedure (if any).
+     * Cancel the currently running local restore procedure.
      *
      * @param reason Interruption reason.
+     * @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 void interrupt(Exception reason) {
+    public IgniteFuture<Boolean> cancel(IgniteCheckedException reason, String snpName) {
+        SnapshotRestoreContext opCtx0;
+        ClusterSnapshotFuture fut0 = null;
+
+        synchronized (this) {
+            opCtx0 = opCtx;
+
+            if (fut != null && fut.name.equals(snpName)) {
+                fut0 = fut;
+
+                fut0.interruptEx = reason;
+            }
+        }
+
+        boolean ctxStop = opCtx0 != null && opCtx0.snpName.equals(snpName);
+
+        if (ctxStop)
+            interrupt(opCtx0, reason);
+
+        return fut0 == null ? new IgniteFinishedFutureImpl<>(ctxStop) :
+            new IgniteFutureImpl<>(fut0.chain(f -> true));
+    }
+
+    /**
+     * Interrupt the currently running local restore procedure.
+     *
+     * @param reason Interruption reason.
+     */
+    public void interrupt(IgniteCheckedException reason) {
         SnapshotRestoreContext opCtx0 = opCtx;
 
-        if (opCtx0 == null)
-            return;
+        if (opCtx0 != null)
+            interrupt(opCtx0, reason);
+    }
 
-        opCtx0.err.compareAndSet(null, reason);
+    /**
+     * Interrupt the currently running local restore procedure.
+     *
+     * @param opCtx Snapshot restore operation context.
+     * @param reason Interruption reason.
+     */
+    private void interrupt(SnapshotRestoreContext opCtx, IgniteCheckedException reason) {
+        opCtx.err.compareAndSet(null, reason);
 
         IgniteFuture<?> stopFut;
 
         synchronized (this) {
-            stopFut = opCtx0.stopFut;
+            stopFut = opCtx.stopFut;
         }
 
         if (stopFut != null)
@@ -443,9 +494,16 @@ public class SnapshotRestoreProcess {
                 }
             }
 
-            opCtx = prepareContext(req);
+            SnapshotRestoreContext opCtx0 = prepareContext(req);
 
-            SnapshotRestoreContext opCtx0 = opCtx;
+            synchronized (this) {
+                opCtx = opCtx0;
+
+                ClusterSnapshotFuture fut0 = fut;
+
+                if (fut0 != null && fut0.interruptEx != null)
+                    opCtx0.err.compareAndSet(null, fut0.interruptEx);
+            }
 
             if (opCtx0.dirs.isEmpty())
                 return new GridFinishedFuture<>();
@@ -806,7 +864,7 @@ public class SnapshotRestoreProcess {
 
         SnapshotRestoreContext opCtx0 = opCtx;
 
-        if (F.isEmpty(opCtx0.dirs))
+        if (opCtx0 == null || F.isEmpty(opCtx0.dirs))
             return new GridFinishedFuture<>();
 
         GridFutureAdapter<Boolean> retFut = new GridFutureAdapter<>();
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
new file mode 100644
index 0000000..e988042
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStatusTask.java
@@ -0,0 +1,47 @@
+/*
+ * 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 status task.
+ */
+@GridInternal
+class SnapshotRestoreStatusTask 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 Boolean execute() throws IgniteException {
+                return ignite.context().cache().context().snapshotMgr().isRestoring(snpName);
+            }
+        };
+    }
+}
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
new file mode 100644
index 0000000..8f4ab22
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotRestoreTask.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.snapshot;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.visor.VisorJob;
+import org.apache.ignite.internal.visor.VisorOneNodeTask;
+import org.apache.ignite.lang.IgniteFuture;
+
+/**
+ * Visor snapshot restore task.
+ */
+@GridInternal
+public class VisorSnapshotRestoreTask extends VisorOneNodeTask<VisorSnapshotRestoreTaskArg, String> {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override protected VisorJob<VisorSnapshotRestoreTaskArg, String> job(VisorSnapshotRestoreTaskArg arg) {
+        switch (arg.jobAction()) {
+            case START:
+                return new VisorSnapshotStartRestoreJob(arg, debug);
+
+            case CANCEL:
+                return new VisorSnapshotRestoreCancelJob(arg, debug);
+
+            case STATUS:
+                return new VisorSnapshotRestoreStatusJob(arg, debug);
+
+            default:
+                throw new IllegalArgumentException("Action is not supported: " + arg.jobAction());
+        }
+    }
+
+    /** */
+    private static class VisorSnapshotStartRestoreJob extends VisorJob<VisorSnapshotRestoreTaskArg, String> {
+        /** Serial version uid. */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         * @param arg Restore task argument.
+         * @param debug Flag indicating whether debug information should be printed into node log.
+         */
+        protected VisorSnapshotStartRestoreJob(VisorSnapshotRestoreTaskArg arg, boolean debug) {
+            super(arg, debug);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected String run(VisorSnapshotRestoreTaskArg arg) throws IgniteException {
+            IgniteFuture<Void> fut =
+                ignite.context().cache().context().snapshotMgr().restoreSnapshot(arg.snapshotName(), arg.groupNames());
+
+            if (fut.isDone())
+                fut.get();
+
+            return "Snapshot cache group restore operation started [snapshot=" + arg.snapshotName() +
+                (arg.groupNames() == null ? "" : ", group(s)=" + F.concat(arg.groupNames(), ",")) + ']';
+        }
+    }
+
+    /** */
+    private static class VisorSnapshotRestoreCancelJob extends VisorJob<VisorSnapshotRestoreTaskArg, String> {
+        /** Serial version uid. */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         * @param arg Restore task argument.
+         * @param debug Flag indicating whether debug information should be printed into node log.
+         */
+        protected VisorSnapshotRestoreCancelJob(VisorSnapshotRestoreTaskArg arg, boolean debug) {
+            super(arg, debug);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected String run(VisorSnapshotRestoreTaskArg arg) throws IgniteException {
+            boolean stopped = ignite.snapshot().cancelSnapshotRestore(arg.snapshotName()).get();
+
+            return "Snapshot cache group restore operation " +
+                (stopped ? "canceled" : "is not in progress") + " [snapshot=" + arg.snapshotName() + ']';
+        }
+    }
+
+    /** */
+    private static class VisorSnapshotRestoreStatusJob extends VisorJob<VisorSnapshotRestoreTaskArg, String> {
+        /** Serial version uid. */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         * @param arg Restore task argument.
+         * @param debug Flag indicating whether debug information should be printed into node log.
+         */
+        protected VisorSnapshotRestoreStatusJob(VisorSnapshotRestoreTaskArg arg, boolean debug) {
+            super(arg, debug);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected String run(VisorSnapshotRestoreTaskArg arg) throws IgniteException {
+            boolean state = ignite.context().cache().context().snapshotMgr().restoreStatus(arg.snapshotName()).get();
+
+            return "Snapshot cache group restore operation is " + (state ? "" : "NOT ") +
+                "running [snapshot=" + arg.snapshotName() + ']';
+        }
+    }
+}
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
new file mode 100644
index 0000000..4ca3e14
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotRestoreTaskAction.java
@@ -0,0 +1,48 @@
+/*
+ * 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;
+
+/** Snapshot restore operation management action. */
+public enum VisorSnapshotRestoreTaskAction {
+    /** Start snapshot restore operation. */
+    START,
+
+    /** Cancel snapshot restore operation. */
+    CANCEL,
+
+    /** Status of the snapshot restore operation. */
+    STATUS;
+
+    /**
+     * @param cmdArg Command line argument.
+     * @return Snapshot restore operation management action.
+     */
+    public static VisorSnapshotRestoreTaskAction fromCmdArg(String cmdArg) {
+        for (VisorSnapshotRestoreTaskAction val : values()) {
+            if (cmdArg.equalsIgnoreCase(val.cmdName()))
+                return val;
+        }
+
+        throw new IllegalArgumentException("Unexpected command line argument \"" + cmdArg + "\"");
+    }
+
+    /** @return Command line argument name. */
+    public String cmdName() {
+        return "--" + name().toLowerCase();
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotRestoreTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotRestoreTaskArg.java
new file mode 100644
index 0000000..0f5bc10
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/snapshot/VisorSnapshotRestoreTaskArg.java
@@ -0,0 +1,98 @@
+/*
+ * 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.Collection;
+import org.apache.ignite.internal.dto.IgniteDataTransferObject;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Argument for the task to manage snapshot restore operation.
+ */
+public class VisorSnapshotRestoreTaskArg extends IgniteDataTransferObject {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
+    /** Snapshot name. */
+    private String snpName;
+
+    /** Cache group names. */
+    private Collection<String> grpNames;
+
+    /** Snapshot restore operation management action. */
+    private VisorSnapshotRestoreTaskAction action;
+
+    /** Default constructor. */
+    public VisorSnapshotRestoreTaskArg() {
+        // No-op.
+    }
+
+    /**
+     * @param action Snapshot restore operation management action.
+     * @param snpName Snapshot name.
+     * @param grpNames Cache group names.
+     */
+    public VisorSnapshotRestoreTaskArg(
+        VisorSnapshotRestoreTaskAction action,
+        String snpName,
+        @Nullable Collection<String> grpNames
+    ) {
+        this.snpName = snpName;
+        this.grpNames = grpNames;
+        this.action = action;
+    }
+
+    /** @return Snapshot name. */
+    public String snapshotName() {
+        return snpName;
+    }
+
+    /** @return Cache group names. */
+    public Collection<String> groupNames() {
+        return grpNames;
+    }
+
+    /** @return Snapshot restore operation management action. */
+    public VisorSnapshotRestoreTaskAction jobAction() {
+        return action;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        U.writeEnum(out, action);
+        U.writeString(out, snpName);
+        U.writeCollection(out, grpNames);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte ver, ObjectInput in) throws IOException, ClassNotFoundException {
+        action = U.readEnum(in, VisorSnapshotRestoreTaskAction.class);
+        snpName = U.readString(in);
+        grpNames = U.readCollection(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorSnapshotRestoreTaskArg.class, this);
+    }
+}
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index c20a6c9..b787705 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -2287,6 +2287,12 @@ 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.VisorSnapshotCancelTask$VisorSnapshotCancelJob
+org.apache.ignite.internal.visor.snapshot.VisorSnapshotRestoreTask
+org.apache.ignite.internal.visor.snapshot.VisorSnapshotRestoreTask$VisorSnapshotStartRestoreJob
+org.apache.ignite.internal.visor.snapshot.VisorSnapshotRestoreTask$VisorSnapshotRestoreCancelJob
+org.apache.ignite.internal.visor.snapshot.VisorSnapshotRestoreTask$VisorSnapshotRestoreStatusJob
+org.apache.ignite.internal.visor.snapshot.VisorSnapshotRestoreTaskArg
+org.apache.ignite.internal.visor.snapshot.VisorSnapshotRestoreTaskAction
 org.apache.ignite.internal.visor.tx.FetchNearXidVersionTask
 org.apache.ignite.internal.visor.tx.FetchNearXidVersionTask$FetchNearXidVersionJob
 org.apache.ignite.internal.visor.tx.TxKeyLockType
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 2881bab..b52ea21 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
@@ -155,6 +155,25 @@ This utility can do the following commands:
     Parameters:
       snapshot_name  - Snapshot name.
 
+  Restore snapshot:
+    control.(sh|bat) --snapshot restore --start snapshot_name [group1,...groupN]
+
+    Parameters:
+      snapshot_name     - Snapshot name.
+      group1,...groupN  - Cache group names.
+
+  Snapshot restore operation status:
+    control.(sh|bat) --snapshot restore --status snapshot_name
+
+    Parameters:
+      snapshot_name  - Snapshot name.
+
+  Cancel snapshot restore operation:
+    control.(sh|bat) --snapshot restore --cancel snapshot_name
+
+    Parameters:
+      snapshot_name  - Snapshot name.
+
   Change cluster tag to new value:
     control.(sh|bat) --change-tag newTagValue [--yes]
 
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 2881bab..b52ea21 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
@@ -155,6 +155,25 @@ This utility can do the following commands:
     Parameters:
       snapshot_name  - Snapshot name.
 
+  Restore snapshot:
+    control.(sh|bat) --snapshot restore --start snapshot_name [group1,...groupN]
+
+    Parameters:
+      snapshot_name     - Snapshot name.
+      group1,...groupN  - Cache group names.
+
+  Snapshot restore operation status:
+    control.(sh|bat) --snapshot restore --status snapshot_name
+
+    Parameters:
+      snapshot_name  - Snapshot name.
+
+  Cancel snapshot restore operation:
+    control.(sh|bat) --snapshot restore --cancel snapshot_name
+
+    Parameters:
+      snapshot_name  - Snapshot name.
+
   Change cluster tag to new value:
     control.(sh|bat) --change-tag newTagValue [--yes]