You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by dp...@apache.org on 2019/05/07 19:55:48 UTC

[ignite] branch master updated: IGNITE-11780 Split command handler on hierarchy of commands - Fixes #6511.

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

dpavlov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new c7bf438  IGNITE-11780 Split command handler on hierarchy of commands - Fixes #6511.
c7bf438 is described below

commit c7bf438e62284d741e3e533b7100e930065e1d23
Author: Eduard Shangareev <es...@gridgain.com>
AuthorDate: Tue May 7 22:45:47 2019 +0300

    IGNITE-11780 Split command handler on hierarchy of commands - Fixes #6511.
    
    Signed-off-by: Dmitriy Pavlov <dp...@apache.org>
---
 .../internal/commandline/ActivateCommand.java      |   63 +
 .../internal/commandline/BaselineCommand.java      |  258 ++
 .../ignite/internal/commandline/Command.java       |  103 +-
 .../internal/commandline/CommandArgIterator.java   |  152 +
 .../internal/commandline/CommandHandler.java       | 3433 ++------------------
 .../commandline/{Command.java => CommandList.java} |   40 +-
 .../ignite/internal/commandline/CommandLogger.java |  228 ++
 .../internal/commandline/CommonArgParser.java      |  338 ++
 ...uments.java => ConnectionAndSslParameters.java} |  103 +-
 .../internal/commandline/DeactivateCommand.java    |   69 +
 .../ignite/internal/commandline/StateCommand.java  |   60 +
 .../ignite/internal/commandline/TaskExecutor.java  |  198 ++
 ...AutoAdjustCommandArg.java => TxCommandArg.java} |   58 +-
 .../ignite/internal/commandline/TxCommands.java    |  582 ++++
 .../ignite/internal/commandline/WalCommands.java   |  260 ++
 .../commandline/baseline/AutoAdjustCommandArg.java |    2 +-
 .../commandline/baseline/BaselineArguments.java    |   10 +-
 ...selineCommand.java => BaselineSubcommands.java} |   37 +-
 .../internal/commandline/cache/CacheArguments.java |  361 --
 .../{CacheCommand.java => CacheCommandList.java}   |   41 +-
 .../internal/commandline/cache/CacheCommands.java  |  221 ++
 .../commandline/cache/CacheContention.java         |  139 +
 .../commandline/cache/CacheDistribution.java       |  163 +
 .../{CacheCommand.java => CacheSubcommands.java}   |   62 +-
 .../commandline/cache/CacheValidateIndexes.java    |  265 ++
 .../internal/commandline/cache/CacheViewer.java    |  430 +++
 .../commandline/cache/FindAndDeleteGarbage.java    |  185 ++
 .../internal/commandline/cache/IdleVerify.java     |  354 ++
 .../commandline/cache/ResetLostPartitions.java     |   74 +
 .../cache/argument/DistributionCommandArg.java     |    4 +-
 .../cache/argument/IdleVerifyCommandArg.java       |    4 +-
 .../commandline/cache/argument/ListCommandArg.java |    4 +-
 .../cache/argument/ValidateIndexesCommandArg.java  |    4 +-
 .../cache/verify/IdleVerifyResultV2.java           |    2 +-
 ...orFindAndDeleteGarbageInPersistenceClosure.java |    2 +-
 .../internal/commandline/CommandArgFactory.java    |   66 -
 .../commandline/CommandHandlerParsingTest.java     |  488 ++-
 .../apache/ignite/util/GridCommandHandlerTest.java |   35 +-
 .../db/IgniteCacheGroupsWithRestartsTest.java      |    4 +-
 39 files changed, 4823 insertions(+), 4079 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/ActivateCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/ActivateCommand.java
new file mode 100644
index 0000000..a61c8ad
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/ActivateCommand.java
@@ -0,0 +1,63 @@
+/*
+ * 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;
+
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientClusterState;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.client.GridClientException;
+
+import static org.apache.ignite.internal.commandline.CommandList.ACTIVATE;
+
+/**
+ * Activate cluster command.
+ */
+public class ActivateCommand implements Command<Void> {
+    /** {@inheritDoc} */
+    @Override public void printUsage(CommandLogger logger) {
+        Command.usage(logger, "Activate cluster:", ACTIVATE);
+    }
+
+    /**
+     * Activate cluster.
+     *
+     * @param cfg Client configuration.
+     * @throws GridClientException If failed to activate.
+     */
+    @Override public Object execute(GridClientConfiguration cfg, CommandLogger logger) throws Exception {
+        try (GridClient client = Command.startClient(cfg)) {
+            GridClientClusterState state = client.state();
+
+            state.active(true);
+
+            logger.log("Cluster activated");
+        }
+        catch (Throwable e) {
+            logger.log("Failed to activate cluster.");
+
+            throw e;
+        }
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Void arg() {
+        return null;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/BaselineCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/BaselineCommand.java
new file mode 100644
index 0000000..332beef
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/BaselineCommand.java
@@ -0,0 +1,258 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.commandline.argument.CommandArgUtils;
+import org.apache.ignite.internal.commandline.baseline.AutoAdjustCommandArg;
+import org.apache.ignite.internal.commandline.baseline.BaselineArguments;
+import org.apache.ignite.internal.commandline.baseline.BaselineSubcommands;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.visor.baseline.VisorBaselineAutoAdjustSettings;
+import org.apache.ignite.internal.visor.baseline.VisorBaselineNode;
+import org.apache.ignite.internal.visor.baseline.VisorBaselineTask;
+import org.apache.ignite.internal.visor.baseline.VisorBaselineTaskArg;
+import org.apache.ignite.internal.visor.baseline.VisorBaselineTaskResult;
+
+import static java.lang.Boolean.TRUE;
+import static org.apache.ignite.internal.commandline.CommandHandler.DELIM;
+import static org.apache.ignite.internal.commandline.CommandList.BASELINE;
+import static org.apache.ignite.internal.commandline.CommandLogger.optional;
+import static org.apache.ignite.internal.commandline.CommonArgParser.CMD_AUTO_CONFIRMATION;
+import static org.apache.ignite.internal.commandline.TaskExecutor.executeTask;
+import static org.apache.ignite.internal.commandline.baseline.BaselineSubcommands.of;
+
+/**
+ * Commands assosiated with baseline functionality.
+ */
+public class BaselineCommand implements Command<BaselineArguments> {
+    /** Arguments. */
+    private BaselineArguments baselineArgs;
+
+    /** {@inheritDoc} */
+    @Override public void printUsage(CommandLogger logger) {
+        final String constistIds = "consistentId1[,consistentId2,....,consistentIdN]";
+
+        Command.usage(logger, "Print cluster baseline topology:", BASELINE);
+        Command.usage(logger, "Add nodes into baseline topology:", BASELINE, BaselineSubcommands.ADD.text(),
+            constistIds, optional(CMD_AUTO_CONFIRMATION));
+        Command.usage(logger, "Remove nodes from baseline topology:", BASELINE, BaselineSubcommands.REMOVE.text(),
+            constistIds, optional(CMD_AUTO_CONFIRMATION));
+        Command.usage(logger, "Set baseline topology:", BASELINE, BaselineSubcommands.SET.text(), constistIds,
+            optional(CMD_AUTO_CONFIRMATION));
+        Command.usage(logger, "Set baseline topology based on version:", BASELINE,
+            BaselineSubcommands.VERSION.text() + " topologyVersion", optional(CMD_AUTO_CONFIRMATION));
+        Command.usage(logger, "Set baseline autoadjustment settings:", BASELINE,
+            BaselineSubcommands.AUTO_ADJUST.text(), "disable|enable timeout <timeoutValue>", optional(CMD_AUTO_CONFIRMATION));
+    }
+
+    /** {@inheritDoc} */
+    @Override public String confirmationPrompt() {
+        if (BaselineSubcommands.COLLECT != baselineArgs.getCmd())
+            return "Warning: the command will perform changes in baseline.";
+
+        return null;
+    }
+
+    /**
+     * Change baseline.
+     *
+     *
+     * @param clientCfg Client configuration.
+     * @throws Exception If failed to execute baseline action.
+     */
+    @Override public Object execute(GridClientConfiguration clientCfg, CommandLogger logger) throws Exception {
+        try (GridClient client = Command.startClient(clientCfg)) {
+            VisorBaselineTaskResult res = executeTask(client, VisorBaselineTask.class, toVisorArguments(baselineArgs), clientCfg);
+
+            baselinePrint0(res, logger);
+        }
+        catch (Throwable e) {
+            logger.error("Failed to execute baseline command='" + baselineArgs.getCmd().text() + "'", e);
+
+            throw e;
+        }
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public BaselineArguments arg() {
+        return baselineArgs;
+    }
+
+    /**
+     * Prepare task argument.
+     *
+     * @param args Argument from command line.
+     * @return Task argument.
+     */
+    private VisorBaselineTaskArg toVisorArguments(BaselineArguments args) {
+        VisorBaselineAutoAdjustSettings settings = args.getCmd() == BaselineSubcommands.AUTO_ADJUST
+            ? new VisorBaselineAutoAdjustSettings(args.getEnableAutoAdjust(), args.getSoftBaselineTimeout())
+            : null;
+
+        return new VisorBaselineTaskArg(args.getCmd().visorBaselineOperation(), args.getTopVer(), args.getConsistentIds(), settings);
+    }
+
+    /**
+     * Print baseline topology.
+     *
+     * @param res Task result with baseline topology.
+     */
+    private void baselinePrint0(VisorBaselineTaskResult res, CommandLogger logger) {
+        logger.log("Cluster state: " + (res.isActive() ? "active" : "inactive"));
+        logger.log("Current topology version: " + res.getTopologyVersion());
+        VisorBaselineAutoAdjustSettings autoAdjustSettings = res.getAutoAdjustSettings();
+
+        if (autoAdjustSettings != null) {
+            logger.log("Baseline auto adjustment " + (TRUE.equals(autoAdjustSettings.getEnabled()) ? "enabled" : "disabled")
+                + ": softTimeout=" + autoAdjustSettings.getSoftTimeout()
+            );
+        }
+
+        if (autoAdjustSettings.enabled) {
+            if (res.isBaselineAdjustInProgress())
+                logger.log("Baseline auto-adjust is in progress");
+            else if (res.getRemainingTimeToBaselineAdjust() < 0)
+                logger.log("Baseline auto-adjust are not scheduled");
+            else
+                logger.log("Baseline auto-adjust will happen in '" + res.getRemainingTimeToBaselineAdjust() + "' ms");
+        }
+
+        logger.nl();
+
+        Map<String, VisorBaselineNode> baseline = res.getBaseline();
+
+        Map<String, VisorBaselineNode> srvs = res.getServers();
+
+        // if task runs on a node with VisorBaselineNode of old version (V1) we'll get order=null for all nodes.
+
+        String crdStr = srvs.values().stream()
+            // check for not null
+            .filter(node -> node.getOrder() != null)
+            .min(Comparator.comparing(VisorBaselineNode::getOrder))
+            // format
+            .map(crd -> " (Coordinator: ConsistentId=" + crd.getConsistentId() + ", Order=" + crd.getOrder() + ")")
+            .orElse("");
+
+        logger.log("Current topology version: " + res.getTopologyVersion() + crdStr);
+        logger.nl();
+
+        if (F.isEmpty(baseline))
+            logger.log("Baseline nodes not found.");
+        else {
+            logger.log("Baseline nodes:");
+
+            for (VisorBaselineNode node : baseline.values()) {
+                VisorBaselineNode srvNode = srvs.get(node.getConsistentId());
+
+                String state = ", State=" + (srvNode != null ? "ONLINE" : "OFFLINE");
+
+                String order = srvNode != null ? ", Order=" + srvNode.getOrder() : "";
+
+                logger.logWithIndent("ConsistentId=" + node.getConsistentId() + state + order, 2);
+            }
+
+            logger.log(DELIM);
+            logger.log("Number of baseline nodes: " + baseline.size());
+
+            logger.nl();
+
+            List<VisorBaselineNode> others = new ArrayList<>();
+
+            for (VisorBaselineNode node : srvs.values()) {
+                if (!baseline.containsKey(node.getConsistentId()))
+                    others.add(node);
+            }
+
+            if (F.isEmpty(others))
+                logger.log("Other nodes not found.");
+            else {
+                logger.log("Other nodes:");
+
+                for (VisorBaselineNode node : others)
+                    logger.logWithIndent("ConsistentId=" + node.getConsistentId() + ", Order=" + node.getOrder(), 2);
+
+                logger.log("Number of other nodes: " + others.size());
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void parseArguments(CommandArgIterator argIter) {
+        if (!argIter.hasNextSubArg()) {
+            this.baselineArgs = new BaselineArguments.Builder(BaselineSubcommands.COLLECT).build();
+
+            return;
+        }
+
+        BaselineSubcommands cmd = of(argIter.nextArg("Expected baseline action"));
+
+        if (cmd == null)
+            throw new IllegalArgumentException("Expected correct baseline action");
+
+        BaselineArguments.Builder baselineArgs = new BaselineArguments.Builder(cmd);
+
+        switch (cmd) {
+            case ADD:
+            case REMOVE:
+            case SET:
+                Set<String> ids = argIter.nextStringSet("list of consistent ids");
+
+                if (F.isEmpty(ids))
+                    throw new IllegalArgumentException("Empty list of consistent IDs");
+
+                baselineArgs.withConsistentIds(new ArrayList<>(ids));
+
+                break;
+
+            case VERSION:
+                baselineArgs.withTopVer(argIter.nextLongArg("topology version"));
+
+                break;
+
+            case AUTO_ADJUST:
+                do {
+                    AutoAdjustCommandArg autoAdjustArg = CommandArgUtils.of(
+                        argIter.nextArg("Expected one of auto-adjust arguments"), AutoAdjustCommandArg.class
+                    );
+
+                    if (autoAdjustArg == null)
+                        throw new IllegalArgumentException("Expected one of auto-adjust arguments");
+
+                    if (autoAdjustArg == AutoAdjustCommandArg.ENABLE || autoAdjustArg == AutoAdjustCommandArg.DISABLE)
+                        baselineArgs.withEnable(autoAdjustArg == AutoAdjustCommandArg.ENABLE);
+
+                    if (autoAdjustArg == AutoAdjustCommandArg.TIMEOUT)
+                        baselineArgs.withSoftBaselineTimeout(argIter.nextLongArg("soft timeout"));
+                }
+                while (argIter.hasNextSubArg());
+
+                break;
+        }
+
+        this.baselineArgs = baselineArgs.build();
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/Command.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/Command.java
index 8ed4dac..cbe443c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/Command.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/Command.java
@@ -17,66 +17,83 @@
 
 package org.apache.ignite.internal.commandline;
 
-/**
- * Command.
- */
-public enum Command {
-    /** */
-    ACTIVATE("--activate"),
-
-    /** */
-    DEACTIVATE("--deactivate"),
-
-    /** */
-    STATE("--state"),
-
-    /** */
-    BASELINE("--baseline"),
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.client.GridClientFactory;
 
-    /** */
-    TX("--tx"),
+import static org.apache.ignite.internal.commandline.CommandHandler.UTILITY_NAME;
 
-    /** */
-    CACHE("--cache"),
-
-    /** */
-    WAL("--wal");
+/**
+ * Abstract class for all control.sh commands, has already implemented methods and abstract methods.
+ * Define flow how to work with command.
+ *
+ * @param <T> Generic for getArg method which should return command-specific paramters which it would be run with.
+ */
+public interface Command<T> {
+    /**
+     * Method to create thin client for communication with cluster.
+     *
+     * @param clientCfg Thin client configuration.
+     * @return Grid thin client instance which is already connected to cluster.
+     * @throws Exception If error occur.
+     */
+    public static GridClient startClient(GridClientConfiguration clientCfg) throws Exception {
+        GridClient client = GridClientFactory.start(clientCfg);
 
-    /** Private values copy so there's no need in cloning it every time. */
-    private static final Command[] VALUES = Command.values();
+        // If connection is unsuccessful, fail before doing any operations:
+        if (!client.connected())
+            client.throwLastError();
 
-    /** */
-    private final String text;
+        return client;
+    }
 
     /**
-     * @param text Text.
+     * Print command usage.
+     *
+     * @param desc Command description.
+     * @param args Arguments.
      */
-    Command(String text) {
-        this.text = text;
+    public static void usage(CommandLogger logger, String desc, CommandList cmd, String... args) {
+        logger.logWithIndent(desc);
+        logger.logWithIndent(CommandLogger.join(" ", UTILITY_NAME, cmd, CommandLogger.join(" ", args)), 2);
+        logger.nl();
     }
 
     /**
-     * @param text Command text.
-     * @return Command for the text.
+     * Actual command execution. Parameters for run should be already set by calling parseArguments method.
+     *
+     * @param clientCfg Thin client configuration if connection to cluster is necessary.
+     * @param logger Logger to use.
+     * @return Result of operation (mostly usable for tests).
+     * @throws Exception If error occur.
      */
-    public static Command of(String text) {
-        for (Command cmd : VALUES) {
-            if (cmd.text().equalsIgnoreCase(text))
-                return cmd;
-        }
+    public Object execute(GridClientConfiguration clientCfg, CommandLogger logger) throws Exception;
 
+    /**
+     * @return Message text to show user for. If null it means that confirmantion is not needed.
+     */
+    public default String confirmationPrompt() {
         return null;
     }
 
     /**
-     * @return Command text.
+     * Parse command-specific arguments.
+     *
+     * @param argIterator Argument iterator.
      */
-    public String text() {
-        return text;
+    public default void parseArguments(CommandArgIterator argIterator) {
+        //Empty block.
     }
 
-    /** {@inheritDoc} */
-    @Override public String toString() { 
-        return text; 
-    }
+    /**
+     * @return Command arguments which were parsed during {@link #parseArguments(CommandArgIterator)} call.
+     */
+    public T arg();
+
+    /**
+     * Print info for user about command (parameters, use cases and so on).
+     *
+     * @param logger Would be used as output.
+     */
+    public void printUsage(CommandLogger logger);
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandArgIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandArgIterator.java
new file mode 100644
index 0000000..4f5f57e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandArgIterator.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.ignite.internal.commandline;
+
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import org.apache.ignite.internal.util.typedef.F;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Iterator over command arguments.
+ */
+public class CommandArgIterator {
+    /** */
+    private Iterator<String> argsIt;
+
+    /** */
+    private String peekedArg;
+
+    /**
+     * Set of common arguments names and high level command name set.
+     */
+    private final Set<String> commonArgumentsAndHighLevelCommandSet;
+
+    /**
+     * @param argsIt Raw argument iterator.
+     * @param commonArgumentsAndHighLevelCommandSet All known subcomands.
+     */
+    public CommandArgIterator(Iterator<String> argsIt, Set<String> commonArgumentsAndHighLevelCommandSet) {
+        this.argsIt = argsIt;
+        this.commonArgumentsAndHighLevelCommandSet = commonArgumentsAndHighLevelCommandSet;
+    }
+
+    /**
+     * @return Returns {@code true} if the iteration has more elements.
+     */
+    public boolean hasNextArg() {
+        return peekedArg != null || argsIt.hasNext();
+    }
+
+    /**
+     * @return <code>true</code> if there's next argument for subcommand.
+     */
+    public boolean hasNextSubArg() {
+        return hasNextArg() && CommandList.of(peekNextArg()) == null &&
+            !commonArgumentsAndHighLevelCommandSet.contains(peekNextArg());
+    }
+
+    /**
+     * Extract next argument.
+     *
+     * @param err Error message.
+     * @return Next argument value.
+     */
+    public String nextArg(String err) {
+        if (peekedArg != null) {
+            String res = peekedArg;
+
+            peekedArg = null;
+
+            return res;
+        }
+
+        if (argsIt.hasNext())
+            return argsIt.next();
+
+        throw new IllegalArgumentException(err);
+    }
+
+    /**
+     * Returns the next argument in the iteration, without advancing the iteration.
+     *
+     * @return Next argument value or {@code null} if no next argument.
+     */
+    public String peekNextArg() {
+        if (peekedArg == null && argsIt.hasNext())
+            peekedArg = argsIt.next();
+
+        return peekedArg;
+    }
+
+    /**
+     * @return Numeric value.
+     */
+    public long nextLongArg(String argName) {
+        String str = nextArg("Expecting " + argName);
+
+        try {
+            long val = Long.parseLong(str);
+
+            if (val < 0)
+                throw new IllegalArgumentException("Invalid value for " + argName + ": " + val);
+
+            return val;
+        }
+        catch (NumberFormatException ignored) {
+            throw new IllegalArgumentException("Invalid value for " + argName + ": " + str);
+        }
+    }
+
+    /**
+     * @param argName Name of argument.
+     */
+    public Set<String> nextStringSet(String argName) {
+        String string = nextArg("Expected " + argName);
+
+        return parseStringSet(string);
+    }
+
+    /**
+     *
+     * @param string To scan on for string set.
+     * @return Set of string parsed from string param.
+     */
+    @NotNull public Set<String> parseStringSet(String string) {
+        Set<String> namesSet = new HashSet<>();
+
+        for (String name : string.split(",")) {
+            if (F.isEmpty(name))
+                throw new IllegalArgumentException("Non-empty string expected.");
+
+            namesSet.add(name.trim());
+        }
+        return namesSet;
+    }
+
+    /**
+     * Check if raw arg is command or option.
+     *
+     * @return {@code true} If raw arg is command, overwise {@code false}.
+     */
+    public static boolean isCommandOrOption(String raw) {
+        return raw != null && raw.contains("--");
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java
index d049d48..fa4404b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java
@@ -17,296 +17,52 @@
 
 package org.apache.ignite.internal.commandline;
 
-import java.io.IOException;
-import java.net.InetAddress;
 import java.time.LocalDateTime;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.Scanner;
-import java.util.Set;
-import java.util.UUID;
-import java.util.regex.Pattern;
-import java.util.regex.PatternSyntaxException;
 import java.util.stream.Collectors;
-import java.util.stream.Stream;
-import org.apache.ignite.IgniteSystemProperties;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.compute.ComputeTask;
-import org.apache.ignite.internal.IgniteFeatures;
-import org.apache.ignite.internal.IgniteNodeAttributes;
-import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.client.GridClientAuthenticationException;
 import org.apache.ignite.internal.client.GridClientClosedException;
-import org.apache.ignite.internal.client.GridClientClusterState;
-import org.apache.ignite.internal.client.GridClientCompute;
 import org.apache.ignite.internal.client.GridClientConfiguration;
 import org.apache.ignite.internal.client.GridClientDisconnectedException;
-import org.apache.ignite.internal.client.GridClientException;
-import org.apache.ignite.internal.client.GridClientFactory;
 import org.apache.ignite.internal.client.GridClientHandshakeException;
-import org.apache.ignite.internal.client.GridClientNode;
 import org.apache.ignite.internal.client.GridServerUnreachableException;
 import org.apache.ignite.internal.client.impl.connection.GridClientConnectionResetException;
 import org.apache.ignite.internal.client.ssl.GridSslBasicContextFactory;
-import org.apache.ignite.internal.commandline.argument.CommandArgUtils;
-import org.apache.ignite.internal.commandline.baseline.AutoAdjustCommandArg;
-import org.apache.ignite.internal.commandline.baseline.BaselineArguments;
-import org.apache.ignite.internal.commandline.baseline.BaselineCommand;
-import org.apache.ignite.internal.commandline.cache.CacheArguments;
-import org.apache.ignite.internal.commandline.cache.CacheCommand;
-import org.apache.ignite.internal.commandline.cache.argument.DistributionCommandArg;
-import org.apache.ignite.internal.commandline.cache.argument.FindAndDeleteGarbageArg;
-import org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCommandArg;
-import org.apache.ignite.internal.commandline.cache.argument.ListCommandArg;
-import org.apache.ignite.internal.commandline.cache.argument.ValidateIndexesCommandArg;
-import org.apache.ignite.internal.commandline.cache.distribution.CacheDistributionTask;
-import org.apache.ignite.internal.commandline.cache.distribution.CacheDistributionTaskArg;
-import org.apache.ignite.internal.commandline.cache.distribution.CacheDistributionTaskResult;
-import org.apache.ignite.internal.commandline.cache.reset_lost_partitions.CacheResetLostPartitionsTask;
-import org.apache.ignite.internal.commandline.cache.reset_lost_partitions.CacheResetLostPartitionsTaskArg;
-import org.apache.ignite.internal.commandline.cache.reset_lost_partitions.CacheResetLostPartitionsTaskResult;
-import org.apache.ignite.internal.processors.cache.verify.CacheInfo;
-import org.apache.ignite.internal.processors.cache.verify.ContentionInfo;
-import org.apache.ignite.internal.processors.cache.verify.IdleVerifyResultV2;
-import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecord;
-import org.apache.ignite.internal.processors.cache.verify.PartitionKey;
-import org.apache.ignite.internal.processors.cache.verify.VerifyBackupPartitionsTaskV2;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.X;
-import org.apache.ignite.internal.util.typedef.internal.SB;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.internal.visor.VisorTaskArgument;
-import org.apache.ignite.internal.visor.baseline.VisorBaselineAutoAdjustSettings;
-import org.apache.ignite.internal.visor.baseline.VisorBaselineNode;
-import org.apache.ignite.internal.visor.baseline.VisorBaselineOperation;
-import org.apache.ignite.internal.visor.baseline.VisorBaselineTask;
-import org.apache.ignite.internal.visor.baseline.VisorBaselineTaskArg;
-import org.apache.ignite.internal.visor.baseline.VisorBaselineTaskResult;
-import org.apache.ignite.internal.visor.cache.VisorCacheAffinityConfiguration;
-import org.apache.ignite.internal.visor.cache.VisorCacheConfiguration;
-import org.apache.ignite.internal.visor.cache.VisorCacheConfigurationCollectorTask;
-import org.apache.ignite.internal.visor.cache.VisorCacheConfigurationCollectorTaskArg;
-import org.apache.ignite.internal.visor.cache.VisorCacheEvictionConfiguration;
-import org.apache.ignite.internal.visor.cache.VisorCacheNearConfiguration;
-import org.apache.ignite.internal.visor.cache.VisorCacheRebalanceConfiguration;
-import org.apache.ignite.internal.visor.cache.VisorCacheStoreConfiguration;
-import org.apache.ignite.internal.visor.cache.VisorFindAndDeleteGarbageInPersistenceJobResult;
-import org.apache.ignite.internal.visor.cache.VisorFindAndDeleteGarbageInPersistenceTask;
-import org.apache.ignite.internal.visor.cache.VisorFindAndDeleteGarbageInPersistenceTaskArg;
-import org.apache.ignite.internal.visor.cache.VisorFindAndDeleteGarbageInPersistenceTaskResult;
-import org.apache.ignite.internal.visor.misc.VisorClusterNode;
-import org.apache.ignite.internal.visor.misc.VisorWalTask;
-import org.apache.ignite.internal.visor.misc.VisorWalTaskArg;
-import org.apache.ignite.internal.visor.misc.VisorWalTaskOperation;
-import org.apache.ignite.internal.visor.misc.VisorWalTaskResult;
-import org.apache.ignite.internal.visor.query.VisorQueryConfiguration;
-import org.apache.ignite.internal.visor.tx.FetchNearXidVersionTask;
-import org.apache.ignite.internal.visor.tx.TxKeyLockType;
-import org.apache.ignite.internal.visor.tx.TxMappingType;
-import org.apache.ignite.internal.visor.tx.TxVerboseId;
-import org.apache.ignite.internal.visor.tx.TxVerboseInfo;
-import org.apache.ignite.internal.visor.tx.TxVerboseKey;
-import org.apache.ignite.internal.visor.tx.VisorTxInfo;
-import org.apache.ignite.internal.visor.tx.VisorTxOperation;
-import org.apache.ignite.internal.visor.tx.VisorTxProjection;
-import org.apache.ignite.internal.visor.tx.VisorTxSortOrder;
-import org.apache.ignite.internal.visor.tx.VisorTxTask;
-import org.apache.ignite.internal.visor.tx.VisorTxTaskArg;
-import org.apache.ignite.internal.visor.tx.VisorTxTaskResult;
-import org.apache.ignite.internal.visor.verify.CacheFilterEnum;
-import org.apache.ignite.internal.visor.verify.IndexIntegrityCheckIssue;
-import org.apache.ignite.internal.visor.verify.IndexValidationIssue;
-import org.apache.ignite.internal.visor.verify.ValidateIndexesPartitionResult;
-import org.apache.ignite.internal.visor.verify.VisorContentionTask;
-import org.apache.ignite.internal.visor.verify.VisorContentionTaskArg;
-import org.apache.ignite.internal.visor.verify.VisorContentionTaskResult;
-import org.apache.ignite.internal.visor.verify.VisorIdleVerifyDumpTask;
-import org.apache.ignite.internal.visor.verify.VisorIdleVerifyDumpTaskArg;
-import org.apache.ignite.internal.visor.verify.VisorIdleVerifyTask;
-import org.apache.ignite.internal.visor.verify.VisorIdleVerifyTaskArg;
-import org.apache.ignite.internal.visor.verify.VisorIdleVerifyTaskResult;
-import org.apache.ignite.internal.visor.verify.VisorIdleVerifyTaskV2;
-import org.apache.ignite.internal.visor.verify.VisorValidateIndexesJobResult;
-import org.apache.ignite.internal.visor.verify.VisorValidateIndexesTaskArg;
-import org.apache.ignite.internal.visor.verify.VisorValidateIndexesTaskResult;
-import org.apache.ignite.internal.visor.verify.VisorViewCacheCmd;
-import org.apache.ignite.internal.visor.verify.VisorViewCacheTask;
-import org.apache.ignite.internal.visor.verify.VisorViewCacheTaskArg;
-import org.apache.ignite.internal.visor.verify.VisorViewCacheTaskResult;
-import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.plugin.security.SecurityCredentials;
 import org.apache.ignite.plugin.security.SecurityCredentialsBasicProvider;
 import org.apache.ignite.plugin.security.SecurityCredentialsProvider;
 import org.apache.ignite.ssl.SslContextFactory;
-import org.apache.ignite.transactions.TransactionState;
+import org.jetbrains.annotations.NotNull;
 
-import static java.lang.Boolean.TRUE;
-import static java.lang.String.format;
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_ENABLE_EXPERIMENTAL_COMMAND;
 import static org.apache.ignite.internal.IgniteVersionUtils.ACK_VER_STR;
 import static org.apache.ignite.internal.IgniteVersionUtils.COPYRIGHT;
-import static org.apache.ignite.internal.commandline.Command.ACTIVATE;
-import static org.apache.ignite.internal.commandline.Command.BASELINE;
-import static org.apache.ignite.internal.commandline.Command.CACHE;
-import static org.apache.ignite.internal.commandline.Command.DEACTIVATE;
-import static org.apache.ignite.internal.commandline.Command.STATE;
-import static org.apache.ignite.internal.commandline.Command.TX;
-import static org.apache.ignite.internal.commandline.Command.WAL;
-import static org.apache.ignite.internal.commandline.OutputFormat.MULTI_LINE;
-import static org.apache.ignite.internal.commandline.OutputFormat.SINGLE_LINE;
-import static org.apache.ignite.internal.commandline.baseline.BaselineCommand.of;
-import static org.apache.ignite.internal.commandline.cache.CacheCommand.CONTENTION;
-import static org.apache.ignite.internal.commandline.cache.CacheCommand.DISTRIBUTION;
-import static org.apache.ignite.internal.commandline.cache.CacheCommand.FIND_AND_DELETE_GARBAGE;
-import static org.apache.ignite.internal.commandline.cache.CacheCommand.HELP;
-import static org.apache.ignite.internal.commandline.cache.CacheCommand.IDLE_VERIFY;
-import static org.apache.ignite.internal.commandline.cache.CacheCommand.LIST;
-import static org.apache.ignite.internal.commandline.cache.CacheCommand.RESET_LOST_PARTITIONS;
-import static org.apache.ignite.internal.commandline.cache.CacheCommand.VALIDATE_INDEXES;
-import static org.apache.ignite.internal.commandline.cache.argument.DistributionCommandArg.USER_ATTRIBUTES;
-import static org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCommandArg.CACHE_FILTER;
-import static org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCommandArg.CHECK_CRC;
-import static org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCommandArg.DUMP;
-import static org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCommandArg.EXCLUDE_CACHES;
-import static org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCommandArg.SKIP_ZEROS;
-import static org.apache.ignite.internal.commandline.cache.argument.ListCommandArg.CONFIG;
-import static org.apache.ignite.internal.commandline.cache.argument.ListCommandArg.GROUP;
-import static org.apache.ignite.internal.commandline.cache.argument.ListCommandArg.OUTPUT_FORMAT;
-import static org.apache.ignite.internal.commandline.cache.argument.ListCommandArg.SEQUENCE;
-import static org.apache.ignite.internal.commandline.cache.argument.ValidateIndexesCommandArg.CHECK_FIRST;
-import static org.apache.ignite.internal.commandline.cache.argument.ValidateIndexesCommandArg.CHECK_THROUGH;
-import static org.apache.ignite.internal.visor.verify.CacheFilterEnum.ALL;
-import static org.apache.ignite.internal.visor.verify.CacheFilterEnum.NOT_PERSISTENT;
-import static org.apache.ignite.internal.visor.verify.CacheFilterEnum.PERSISTENT;
-import static org.apache.ignite.internal.visor.verify.CacheFilterEnum.SYSTEM;
-import static org.apache.ignite.internal.visor.verify.CacheFilterEnum.USER;
-import static org.apache.ignite.internal.visor.verify.VisorViewCacheCmd.CACHES;
-import static org.apache.ignite.internal.visor.verify.VisorViewCacheCmd.GROUPS;
-import static org.apache.ignite.internal.visor.verify.VisorViewCacheCmd.SEQ;
+import static org.apache.ignite.internal.commandline.CommonArgParser.CMD_AUTO_CONFIRMATION;
+import static org.apache.ignite.internal.commandline.CommonArgParser.getCommonOptions;
+import static org.apache.ignite.internal.commandline.CommandLogger.optional;
+import static org.apache.ignite.internal.commandline.TaskExecutor.DFLT_HOST;
+import static org.apache.ignite.internal.commandline.TaskExecutor.DFLT_PORT;
 import static org.apache.ignite.ssl.SslContextFactory.DFLT_SSL_PROTOCOL;
 
 /**
  * Class that execute several commands passed via command line.
  */
 public class CommandHandler {
-    /** */
-    static final String DFLT_HOST = "127.0.0.1";
-
-    /** */
-    static final String DFLT_PORT = "11211";
-
-    /** */
-    private static final String CMD_HELP = "--help";
-
-    /** */
-    private static final String CMD_HOST = "--host";
-
-    /** */
-    private static final String CMD_PORT = "--port";
-
-    /** */
-    private static final String CMD_PASSWORD = "--password";
-
-    /** */
-    private static final String CMD_USER = "--user";
-
-    /** Option is used for auto confirmation. */
-    private static final String CMD_AUTO_CONFIRMATION = "--yes";
-
-    /** */
-    private static final String CMD_PING_INTERVAL = "--ping-interval";
-
-    /** */
-    private static final String CMD_PING_TIMEOUT = "--ping-timeout";
-
-    /** One cache filter option should used message. */
-    public static final String ONE_CACHE_FILTER_OPT_SHOULD_USED_MSG = "Should use only one of option: " +
-        EXCLUDE_CACHES + ", " + CACHE_FILTER + " or pass caches explicitly";
-
-    // SSL configuration section
-
-    /** */
-    private static final String CMD_SSL_PROTOCOL = "--ssl-protocol";
-
-    /** */
-    private static final String CMD_SSL_KEY_ALGORITHM = "--ssl-key-algorithm";
-
-    /** */
-    private static final String CMD_SSL_CIPHER_SUITES = "--ssl-cipher-suites";
-
-    /** */
-    private static final String CMD_KEYSTORE = "--keystore";
-
-    /** */
-    private static final String CMD_KEYSTORE_PASSWORD = "--keystore-password";
-
-    /** */
-    private static final String CMD_KEYSTORE_TYPE = "--keystore-type";
+    /** Logger. */
+    private final CommandLogger logger = new CommandLogger();
 
     /** */
-    private static final String CMD_TRUSTSTORE = "--truststore";
-
-    /** */
-    private static final String CMD_TRUSTSTORE_PASSWORD = "--truststore-password";
-
-    /** */
-    private static final String CMD_TRUSTSTORE_TYPE = "--truststore-type";
-
-    /** List of optional auxiliary commands. */
-    private static final Set<String> AUX_COMMANDS = new HashSet<>();
-
-    static {
-        AUX_COMMANDS.add(CMD_HELP);
-
-        AUX_COMMANDS.add(CMD_HOST);
-        AUX_COMMANDS.add(CMD_PORT);
-
-        AUX_COMMANDS.add(CMD_PASSWORD);
-        AUX_COMMANDS.add(CMD_USER);
-
-        AUX_COMMANDS.add(CMD_AUTO_CONFIRMATION);
-
-        AUX_COMMANDS.add(CMD_PING_INTERVAL);
-        AUX_COMMANDS.add(CMD_PING_TIMEOUT);
-
-        AUX_COMMANDS.add(CMD_SSL_PROTOCOL);
-        AUX_COMMANDS.add(CMD_SSL_KEY_ALGORITHM);
-        AUX_COMMANDS.add(CMD_SSL_CIPHER_SUITES);
-
-        AUX_COMMANDS.add(CMD_KEYSTORE);
-        AUX_COMMANDS.add(CMD_KEYSTORE_PASSWORD);
-        AUX_COMMANDS.add(CMD_KEYSTORE_TYPE);
-
-        AUX_COMMANDS.add(CMD_TRUSTSTORE);
-        AUX_COMMANDS.add(CMD_TRUSTSTORE_PASSWORD);
-        AUX_COMMANDS.add(CMD_TRUSTSTORE_TYPE);
-    }
-
-    /** Broadcast uuid. */
-    private static final UUID BROADCAST_UUID = UUID.randomUUID();
+    static final String CMD_HELP = "--help";
 
     /** */
     public static final String CONFIRM_MSG = "y";
 
     /** */
-    static final String WAL_PRINT = "print";
-
-    /** */
-    static final String WAL_DELETE = "delete";
-
-    /** */
     static final String DELIM = "--------------------------------------------------------------------------------";
 
     /** */
@@ -333,2748 +89,330 @@ public class CommandHandler {
     /** */
     private static final Scanner IN = new Scanner(System.in);
 
-    /** Validate indexes task name. */
-    private static final String VALIDATE_INDEXES_TASK = "org.apache.ignite.internal.visor.verify.VisorValidateIndexesTask";
-
-    /** */
-    private static final String TX_LIMIT = "--limit";
-
-    /** */
-    private static final String TX_ORDER = "--order";
-
-    /** */
-    private static final String TX_SERVERS = "--servers";
-
-    /** */
-    private static final String TX_CLIENTS = "--clients";
-
-    /** */
-    private static final String TX_DURATION = "--min-duration";
+    /** Utility name. */
+    public static final String UTILITY_NAME = "control.sh";
 
     /** */
-    private static final String TX_SIZE = "--min-size";
+    public static final String NULL = "null";
 
-    /** */
-    private static final String TX_LABEL = "--label";
+    /** Console instance. Public access needs for tests. */
+    public GridConsole console = GridConsoleAdapter.getInstance();
 
     /** */
-    private static final String TX_NODES = "--nodes";
+    private Object lastOperationRes;
 
-    /** */
-    private static final String TX_XID = "--xid";
+    /**
+     * @param args Arguments to parse and apply.
+     */
+    public static void main(String[] args) {
+        CommandHandler hnd = new CommandHandler();
 
-    /** */
-    private static final String TX_KILL = "--kill";
+        System.exit(hnd.execute(Arrays.asList(args)));
+    }
 
-    /** */
-    private static final String TX_INFO = "--info";
+    /**
+     * Parse and execute command.
+     *
+     * @param rawArgs Arguments to parse and execute.
+     * @return Exit code.
+     */
+    public int execute(List<String> rawArgs) {
+        logger.log("Control utility [ver. " + ACK_VER_STR + "]");
+        logger.log(COPYRIGHT);
+        logger.log("User: " + System.getProperty("user.name"));
+        logger.log("Time: " + LocalDateTime.now());
+        logger.log(DELIM);
 
-    /** Utility name. */
-    private static final String UTILITY_NAME = "control.sh";
+        try {
+            if (F.isEmpty(rawArgs) || (rawArgs.size() == 1 && CMD_HELP.equalsIgnoreCase(rawArgs.get(0)))) {
+                printHelp();
 
-    /** Common options. */
-    private static final String COMMON_OPTIONS = j(" ", getCommonOptions());
+                return EXIT_CODE_OK;
+            }
 
-    /** Utility name with common options. */
-    private static final String UTILITY_NAME_WITH_COMMON_OPTIONS = j(" ", UTILITY_NAME, COMMON_OPTIONS);
+            ConnectionAndSslParameters args = new CommonArgParser(logger).parseAndValidate(rawArgs.iterator());
 
-    /** Indent for help output. */
-    private static final String INDENT = "  ";
+            Command command = args.command();
 
-    /** Double indent. */
-    private static final String DOUBLE_INDENT = INDENT + INDENT;
+            if (!args.autoConfirmation() && !confirm(command.confirmationPrompt())) {
+                logger.log("Operation cancelled.");
 
-    /** */
-    private static final String NULL = "null";
+                return EXIT_CODE_OK;
+            }
 
-    /** */
-    private static final String NODE_ID = "nodeId";
+            boolean tryConnectAgain = true;
 
-    /** */
-    private static final String OP_NODE_ID = op(NODE_ID);
+            int tryConnectMaxCount = 3;
 
-    /** */
-    private Iterator<String> argsIt;
+            boolean suppliedAuth = !F.isEmpty(args.userName()) && !F.isEmpty(args.password());
 
-    /** */
-    private String peekedArg;
+            GridClientConfiguration clientCfg = getClientConfiguration(args);
 
-    /** */
-    private Object lastOperationRes;
+            while (tryConnectAgain) {
+                tryConnectAgain = false;
 
-    /** */
-    private GridClientConfiguration clientCfg;
+                try {
+                    lastOperationRes = command.execute(clientCfg, logger);
+                }
+                catch (Throwable e) {
+                    if (tryConnectMaxCount > 0 && isAuthError(e)) {
+                        logger.log(suppliedAuth ?
+                            "Authentication error, please try again." :
+                            "This cluster requires authentication.");
 
-    /** Check if experimental commands are enabled. Default {@code false}. */
-    private final boolean enableExperimental = IgniteSystemProperties.getBoolean(IGNITE_ENABLE_EXPERIMENTAL_COMMAND, false);
+                        String user = clientCfg.getSecurityCredentialsProvider() == null ?
+                            requestDataFromConsole("user: ") :
+                            (String)clientCfg.getSecurityCredentialsProvider().credentials().getLogin();
 
-    /** Console instance. Public access needs for tests. */
-    public GridConsole console = GridConsoleAdapter.getInstance();
+                        clientCfg = getClientConfiguration(user, new String(requestPasswordFromConsole("password: ")),  args);
 
-    /**
-     * Creates list of common utility options.
-     *
-     * @return Array of common utility options.
-     */
-    private static String[] getCommonOptions() {
-        List<String> list = new ArrayList<>(32);
-
-        list.add(op(CMD_HOST, "HOST_OR_IP"));
-        list.add(op(CMD_PORT, "PORT"));
-        list.add(op(CMD_USER, "USER"));
-        list.add(op(CMD_PASSWORD, "PASSWORD"));
-        list.add(op(CMD_PING_INTERVAL, "PING_INTERVAL"));
-        list.add(op(CMD_PING_TIMEOUT, "PING_TIMEOUT"));
-
-        list.add(op(CMD_SSL_PROTOCOL, "SSL_PROTOCOL[, SSL_PROTOCOL_2, ..., SSL_PROTOCOL_N]"));
-        list.add(op(CMD_SSL_CIPHER_SUITES, "SSL_CIPHER_1[, SSL_CIPHER_2, ..., SSL_CIPHER_N]"));
-        list.add(op(CMD_SSL_KEY_ALGORITHM, "SSL_KEY_ALGORITHM"));
-        list.add(op(CMD_KEYSTORE_TYPE, "KEYSTORE_TYPE"));
-        list.add(op(CMD_KEYSTORE, "KEYSTORE_PATH"));
-        list.add(op(CMD_KEYSTORE_PASSWORD, "KEYSTORE_PASSWORD"));
-        list.add(op(CMD_TRUSTSTORE_TYPE, "TRUSTSTORE_TYPE"));
-        list.add(op(CMD_TRUSTSTORE, "TRUSTSTORE_PATH"));
-        list.add(op(CMD_TRUSTSTORE_PASSWORD, "TRUSTSTORE_PASSWORD"));
-
-        return list.toArray(new String[0]);
-    }
+                        tryConnectAgain = true;
 
-    /**
-     * Output specified string to console.
-     *
-     * @param s String to output.
-     */
-    private void log(String s) {
-        System.out.println(s);
-    }
+                        suppliedAuth = true;
 
-    /**
-     * Adds indent to begin of object's string representation.
-     *
-     * @param o Input object.
-     * @return Indented string.
-     */
-    private static String i(Object o) {
-        return i(o, 1);
-    }
+                        tryConnectMaxCount--;
+                    }
+                    else {
+                        if (tryConnectMaxCount == 0)
+                            throw new GridClientAuthenticationException("Authentication error, maximum number of " +
+                                "retries exceeded");
 
-    /**
-     * Adds specified indents to begin of object's string representation.
-     *
-     * @param o Input object.
-     * @param indentCnt Number of indents.
-     * @return Indented string.
-     */
-    private static String i(Object o, int indentCnt) {
-        assert indentCnt >= 0;
+                        throw e;
+                    }
+                }
+            }
 
-        String s = o == null ? null : o.toString();
+            return EXIT_CODE_OK;
+        }
+        catch (IllegalArgumentException e) {
+            logger.error("Check arguments.", e);
 
-        switch (indentCnt) {
-            case 0:
-                return s;
+            return EXIT_CODE_INVALID_ARGUMENTS;
+        }
+        catch (Throwable e) {
+            if (isAuthError(e)) {
+                logger.error("Authentication error.", e);
 
-            case 1:
-                return INDENT + s;
+                return ERR_AUTHENTICATION_FAILED;
+            }
 
-            default:
-                int sLen = s == null ? 4 : s.length();
+            if (isConnectionError(e)) {
+                logger.error("Connection to cluster failed.", e);
 
-                SB sb = new SB(sLen + indentCnt * INDENT.length());
+                return EXIT_CODE_CONNECTION_FAILED;
+            }
 
-                for (int i = 0; i < indentCnt; i++)
-                    sb.a(INDENT);
+            logger.error("", e);
 
-                return sb.a(s).toString();
+            return EXIT_CODE_UNEXPECTED_ERROR;
         }
     }
 
     /**
-     * Format and output specified string to console.
-     *
-     * @param format A format string as described in Format string syntax.
-     * @param args Arguments referenced by the format specifiers in the format string.
-     */
-    private void log(String format, Object... args) {
-        System.out.printf(format, args);
-    }
-
-    /**
-     * Provides a prompt, then reads a single line of text from the console.
-     *
-     * @param prompt text
-     * @return A string containing the line read from the console
-     */
-    private String readLine(String prompt) {
-        System.out.print(prompt);
-
-        return IN.nextLine();
-    }
-
-    /**
-     * Output empty line.
+     * @param args Common arguments.
+     * @return Thin client configuration to connect to cluster.
+     * @throws IgniteCheckedException If error occur.
      */
-    private void nl() {
-        System.out.println();
+    @NotNull private GridClientConfiguration getClientConfiguration(
+        ConnectionAndSslParameters args
+    ) throws IgniteCheckedException {
+        return getClientConfiguration(args.userName(), args.password(), args);
     }
 
     /**
-     * Print error to console.
-     *
-     * @param errCode Error code to return.
-     * @param s Optional message.
-     * @param e Error to print.
+     * @param userName User name for authorization.
+     * @param password Password for authorization.
+     * @param args Common arguments.
+     * @return Thin client configuration to connect to cluster.
+     * @throws IgniteCheckedException If error occur.
      */
-    private int error(int errCode, String s, Throwable e) {
-        if (!F.isEmpty(s))
-            log(s);
+    @NotNull private GridClientConfiguration getClientConfiguration(
+        String userName,
+        String password,
+        ConnectionAndSslParameters args
+    ) throws IgniteCheckedException {
+        GridClientConfiguration clientCfg = new GridClientConfiguration();
 
-        String msg = e.getMessage();
+        clientCfg.setPingInterval(args.pingInterval());
 
-        if (F.isEmpty(msg))
-            msg = e.getClass().getName();
+        clientCfg.setPingTimeout(args.pingTimeout());
 
-        if (msg.startsWith("Failed to handle request")) {
-            int p = msg.indexOf("err=");
+        clientCfg.setServers(Collections.singletonList(args.host() + ":" + args.port()));
 
-            msg = msg.substring(p + 4, msg.length() - 1);
-        }
+        if (!F.isEmpty(userName))
+            clientCfg.setSecurityCredentialsProvider(getSecurityCredentialsProvider(userName, password, clientCfg));
 
-        log("Error: " + msg);
+        if (!F.isEmpty(args.sslKeyStorePath()))
+            clientCfg.setSslContextFactory(createSslSupportFactory(args));
 
-        return errCode;
+        return clientCfg;
     }
 
     /**
-     * Requests interactive user confirmation if forthcoming operation is dangerous.
-     *
-     * @param args Arguments.
-     * @return {@code true} if operation confirmed (or not needed), {@code false} otherwise.
+     * @param userName User name for authorization.
+     * @param password Password for authorization.
+     * @param clientCfg Thin client configuration to connect to cluster.
+     * @return Security credentials provider with usage of given user name and password.
+     * @throws IgniteCheckedException If error occur.
      */
-    private boolean confirm(Arguments args) {
-        String prompt = confirmationPrompt(args);
+    @NotNull private SecurityCredentialsProvider getSecurityCredentialsProvider(
+        String userName,
+        String password,
+        GridClientConfiguration clientCfg
+    ) throws IgniteCheckedException {
+        SecurityCredentialsProvider securityCredential = clientCfg.getSecurityCredentialsProvider();
 
-        if (prompt == null)
-            return true;
+        if (securityCredential == null)
+            return new SecurityCredentialsBasicProvider(new SecurityCredentials(userName, password));
 
-        return CONFIRM_MSG.equalsIgnoreCase(readLine(prompt));
+        final SecurityCredentials credential = securityCredential.credentials();
+        credential.setLogin(userName);
+        credential.setPassword(password);
+
+        return securityCredential;
     }
 
     /**
-     * @param args Arguments.
-     * @return Prompt text if confirmation needed, otherwise {@code null}.
+     * @param args Commond args.
+     * @return Ssl support factory.
      */
-    private String confirmationPrompt(Arguments args) {
-        String str = null;
+    @NotNull private GridSslBasicContextFactory createSslSupportFactory(ConnectionAndSslParameters args) {
+        GridSslBasicContextFactory factory = new GridSslBasicContextFactory();
 
-        switch (args.command()) {
-            case DEACTIVATE:
-                str = "Warning: the command will deactivate a cluster.";
+        List<String> sslProtocols = split(args.sslProtocol(), ",");
 
-                break;
+        String sslProtocol = F.isEmpty(sslProtocols) ? DFLT_SSL_PROTOCOL : sslProtocols.get(0);
 
-            case BASELINE:
-                if (BaselineCommand.COLLECT != args.baselineArguments().getCmd())
-                    str = "Warning: the command will perform changes in baseline.";
+        factory.setProtocol(sslProtocol);
+        factory.setKeyAlgorithm(args.sslKeyAlgorithm());
 
-                break;
+        if (sslProtocols.size() > 1)
+            factory.setProtocols(sslProtocols);
 
-            case WAL:
-                if (WAL_DELETE.equals(args.walAction()))
-                    str = "Warning: the command will delete unused WAL segments.";
+        factory.setCipherSuites(split(args.getSslCipherSuites(), ","));
 
-                break;
+        factory.setKeyStoreFilePath(args.sslKeyStorePath());
 
-            case TX:
-                if (args.transactionArguments().getOperation() == VisorTxOperation.KILL)
-                    str = "Warning: the command will kill some transactions.";
+        if (args.sslKeyStorePassword() != null)
+            factory.setKeyStorePassword(args.sslKeyStorePassword());
+        else
+            factory.setKeyStorePassword(requestPasswordFromConsole("SSL keystore password: "));
 
-                break;
+        factory.setKeyStoreType(args.sslKeyStoreType());
 
-            default:
-                break;
-        }
+        if (F.isEmpty(args.sslTrustStorePath()))
+            factory.setTrustManagers(GridSslBasicContextFactory.getDisabledTrustManager());
+        else {
+            factory.setTrustStoreFilePath(args.sslTrustStorePath());
 
-        return str == null ? null : str + "\nPress '" + CONFIRM_MSG + "' to continue . . . ";
-    }
+            if (args.sslTrustStorePassword() != null)
+                factory.setTrustStorePassword(args.sslTrustStorePassword());
+            else
+                factory.setTrustStorePassword(requestPasswordFromConsole("SSL truststore password: "));
 
-    /**
-     * @param rawArgs Arguments.
-     */
-    private void initArgIterator(List<String> rawArgs) {
-        argsIt = rawArgs.iterator();
-        peekedArg = null;
-    }
+            factory.setTrustStoreType(args.sslTrustStoreType());
+        }
 
-    /**
-     * @return Returns {@code true} if the iteration has more elements.
-     */
-    private boolean hasNextArg() {
-        return peekedArg != null || argsIt.hasNext();
+        return factory;
     }
 
     /**
-     * Activate cluster.
+     * Used for tests.
      *
-     * @param client Client.
-     * @throws GridClientException If failed to activate.
+     * @return Last operation result;
      */
-    private void activate(GridClient client) throws Throwable {
-        try {
-            GridClientClusterState state = client.state();
-
-            state.active(true);
-
-            log("Cluster activated");
-        }
-        catch (Throwable e) {
-            log("Failed to activate cluster.");
-
-            throw e;
-        }
+    public <T> T getLastOperationResult() {
+        return (T)lastOperationRes;
     }
 
     /**
-     * Deactivate cluster.
+     * Provides a prompt, then reads a single line of text from the console.
      *
-     * @param client Client.
-     * @throws Throwable If failed to deactivate.
+     * @param prompt text
+     * @return A string containing the line read from the console
      */
-    private void deactivate(GridClient client) throws Throwable {
-        try {
-            GridClientClusterState state = client.state();
-
-            state.active(false);
-
-            log("Cluster deactivated");
-        }
-        catch (Throwable e) {
-            log("Failed to deactivate cluster.");
+    private String readLine(String prompt) {
+        System.out.print(prompt);
 
-            throw e;
-        }
+        return IN.nextLine();
     }
 
+
     /**
-     * Print cluster state.
+     * Requests interactive user confirmation if forthcoming operation is dangerous.
      *
-     * @param client Client.
-     * @throws Throwable If failed to print state.
+     * @return {@code true} if operation confirmed (or not needed), {@code false} otherwise.
      */
-    private void state(GridClient client) throws Throwable {
-        try {
-            GridClientClusterState state = client.state();
+    private <T> boolean confirm(String str) {
+        if (str == null)
+            return true;
 
-            log("Cluster is " + (state.active() ? "active" : "inactive"));
-        }
-        catch (Throwable e) {
-            log("Failed to get cluster state.");
+        String prompt = str + "\nPress '" + CONFIRM_MSG + "' to continue . . . ";
 
-            throw e;
-        }
+        return CONFIRM_MSG.equalsIgnoreCase(readLine(prompt));
     }
 
     /**
-     * @param client Client.
-     * @param taskCls Task class.
-     * @param taskArgs Task arguments.
-     * @return Task result.
-     * @throws GridClientException If failed to execute task.
+     * @param e Exception to check.
+     * @return {@code true} if specified exception is {@link GridClientAuthenticationException}.
      */
-    private <R> R executeTask(GridClient client, Class<? extends ComputeTask<?, R>> taskCls,
-        Object taskArgs) throws GridClientException {
-        return executeTaskByNameOnNode(client, taskCls.getName(), taskArgs, null);
+    private static boolean isAuthError(Throwable e) {
+        return X.hasCause(e, GridClientAuthenticationException.class);
     }
 
     /**
-     * @param client Client.
-     * @return List of hosts.
+     * @param e Exception to check.
+     * @return {@code true} if specified exception is a connection error.
      */
-    private Stream<IgniteBiTuple<GridClientNode, String>> listHosts(GridClient client) throws GridClientException {
-        return client.compute()
-            .nodes(GridClientNode::connectable)
-            .stream()
-            .flatMap(node -> Stream.concat(
-                node.tcpAddresses() == null ? Stream.empty() : node.tcpAddresses().stream(),
-                node.tcpHostNames() == null ? Stream.empty() : node.tcpHostNames().stream()
-            ).map(addr -> new IgniteBiTuple<>(node, addr + ":" + node.tcpPort())));
+    private static boolean isConnectionError(Throwable e) {
+        return e instanceof GridClientClosedException ||
+            e instanceof GridClientConnectionResetException ||
+            e instanceof GridClientDisconnectedException ||
+            e instanceof GridClientHandshakeException ||
+            e instanceof GridServerUnreachableException;
     }
 
     /**
-     * @param client Client.
-     * @return List of hosts.
+     * Requests password from console with message.
+     *
+     * @param msg Message.
+     * @return Password.
      */
-    private Stream<IgniteBiTuple<GridClientNode, List<String>>> listHostsByClientNode(
-        GridClient client
-    ) throws GridClientException {
-        return client.compute().nodes(GridClientNode::connectable).stream()
-            .map(
-                node -> new IgniteBiTuple<>(
-                    node,
-                    Stream.concat(
-                        node.tcpAddresses() == null ? Stream.empty() : node.tcpAddresses().stream(),
-                        node.tcpHostNames() == null ? Stream.empty() : node.tcpHostNames().stream()
-                    )
-                        .map(addr -> addr + ":" + node.tcpPort()).collect(Collectors.toList())
-                )
-            );
+    private char[] requestPasswordFromConsole(String msg) {
+        if (console == null)
+            throw new UnsupportedOperationException("Failed to securely read password (console is unavailable): " + msg);
+        else
+            return console.readPassword(msg);
     }
 
     /**
-     * @param client Client
-     * @param taskClsName Task class name.
-     * @param taskArgs Task args.
-     * @param nodeId Node ID to execute task at (if null, random node will be chosen by balancer).
-     * @return Task result.
-     * @throws GridClientException If failed to execute task.
+     * Requests user data from console with message.
+     *
+     * @param msg Message.
+     * @return Input user data.
      */
-    private <R> R executeTaskByNameOnNode(
-        GridClient client,
-        String taskClsName,
-        Object taskArgs,
-        UUID nodeId
-    ) throws GridClientException {
-        GridClientCompute compute = client.compute();
-
-        if (nodeId == BROADCAST_UUID) {
-            Collection<GridClientNode> nodes = compute.nodes(GridClientNode::connectable);
-
-            if (F.isEmpty(nodes))
-                throw new GridClientDisconnectedException("Connectable nodes not found", null);
+    private String requestDataFromConsole(String msg) {
+        if (console != null)
+            return console.readLine(msg);
+        else {
+            Scanner scanner = new Scanner(System.in);
 
-            List<UUID> nodeIds = nodes.stream()
-                .map(GridClientNode::nodeId)
-                .collect(Collectors.toList());
+            logger.log(msg);
 
-            return client.compute().execute(taskClsName, new VisorTaskArgument<>(nodeIds, taskArgs, false));
+            return scanner.nextLine();
         }
+    }
 
-        GridClientNode node = null;
-
-        if (nodeId == null) {
-            // Prefer node from connect string.
-            final String cfgAddr = clientCfg.getServers().iterator().next();
-
-            String[] parts = cfgAddr.split(":");
-
-            if (DFLT_HOST.equals(parts[0])) {
-                InetAddress addr;
-
-                try {
-                    addr = IgniteUtils.getLocalHost();
-                }
-                catch (IOException e) {
-                    throw new GridClientException("Can't get localhost name.", e);
-                }
-
-                if (addr.isLoopbackAddress())
-                    throw new GridClientException("Can't find localhost name.");
-
-                String origAddr = addr.getHostName() + ":" + parts[1];
-
-                node = listHosts(client).filter(tuple -> origAddr.equals(tuple.get2())).findFirst().map(IgniteBiTuple::get1).orElse(null);
-
-                if (node == null)
-                    node = listHostsByClientNode(client).filter(tuple -> tuple.get2().size() == 1 && cfgAddr.equals(tuple.get2().get(0))).
-                        findFirst().map(IgniteBiTuple::get1).orElse(null);
-            }
-            else
-                node = listHosts(client).filter(tuple -> cfgAddr.equals(tuple.get2())).findFirst().map(IgniteBiTuple::get1).orElse(null);
-
-            // Otherwise choose random node.
-            if (node == null)
-                node = getBalancedNode(compute);
-        }
-        else {
-            for (GridClientNode n : compute.nodes()) {
-                if (n.connectable() && nodeId.equals(n.nodeId())) {
-                    node = n;
-
-                    break;
-                }
-            }
-
-            if (node == null)
-                throw new IllegalArgumentException("Node with id=" + nodeId + " not found");
-        }
-
-        return compute.projection(node).execute(taskClsName, new VisorTaskArgument<>(node.nodeId(), taskArgs, false));
-    }
-
-    /**
-     * @param compute instance
-     * @return balanced node
-     */
-    private GridClientNode getBalancedNode(GridClientCompute compute) throws GridClientException {
-        Collection<GridClientNode> nodes = compute.nodes(GridClientNode::connectable);
-
-        if (F.isEmpty(nodes))
-            throw new GridClientDisconnectedException("Connectable node not found", null);
-
-        return compute.balancer().balancedNode(nodes);
-    }
-
-    /**
-     * Executes --cache subcommand.
-     *
-     * @param client Client.
-     * @param cacheArgs Cache args.
-     */
-    private Object cache(GridClient client, CacheArguments cacheArgs) throws Throwable {
-        switch (cacheArgs.command()) {
-            case HELP:
-                printCacheHelp();
-
-                break;
-
-            case IDLE_VERIFY:
-                cacheIdleVerify(client, cacheArgs);
-
-                break;
-
-            case VALIDATE_INDEXES:
-                cacheValidateIndexes(client, cacheArgs);
-
-                break;
-
-            case CONTENTION:
-                cacheContention(client, cacheArgs);
-
-                break;
-
-            case DISTRIBUTION:
-                cacheDistribution(client, cacheArgs);
-
-                break;
-
-            case RESET_LOST_PARTITIONS:
-                cacheResetLostPartitions(client, cacheArgs);
-
-                break;
-
-            case FIND_AND_DELETE_GARBAGE:
-                return findAndDeleteGarbage(client, cacheArgs);
-
-            default:
-                cacheView(client, cacheArgs);
-
-                break;
-        }
-
-        return null;
-    }
-
-    /** */
-    private void printCacheHelp() {
-        log(i("The '" + CACHE + " subcommand' is used to get information about and perform actions with caches. The command has the following syntax:"));
-        nl();
-        log(i(UTILITY_NAME_WITH_COMMON_OPTIONS + " " + CACHE + " [subcommand] <subcommand_parameters>"));
-        nl();
-        log(i("The subcommands that take " + OP_NODE_ID + " as an argument ('" + LIST + "', '" + CONTENTION + "' and '" + VALIDATE_INDEXES + "') will be executed on the given node or on all server nodes if the option is not specified. Other commands will run on a random server node."));
-        nl();
-        nl();
-        log(i("Subcommands:"));
-
-        String CACHES = "cacheName1,...,cacheNameN";
-
-        usageCache(LIST, "regexPattern", op(or(GROUP, SEQUENCE)), OP_NODE_ID, op(CONFIG), op(OUTPUT_FORMAT, MULTI_LINE));
-        usageCache(CONTENTION, "minQueueSize", OP_NODE_ID, op("maxPrint"));
-        usageCache(IDLE_VERIFY, op(DUMP), op(SKIP_ZEROS), op(CHECK_CRC), op(EXCLUDE_CACHES, CACHES),
-            op(CACHE_FILTER, or(ALL, USER, SYSTEM, PERSISTENT, NOT_PERSISTENT)), op(CACHES));
-        usageCache(VALIDATE_INDEXES, op(CACHES), OP_NODE_ID, op(or(CHECK_FIRST + " N", CHECK_THROUGH + " K")));
-        usageCache(DISTRIBUTION, or(NODE_ID, NULL), op(CACHES), op(USER_ATTRIBUTES, "attrName1,...,attrNameN"));
-        usageCache(RESET_LOST_PARTITIONS, CACHES);
-        usageCache(FIND_AND_DELETE_GARBAGE, op(GROUPS), OP_NODE_ID, op(FindAndDeleteGarbageArg.DELETE));
-        nl();
-    }
-
-    /**
-     * @param client Client.
-     * @param cacheArgs Cache args.
-     */
-    private void cacheContention(GridClient client, CacheArguments cacheArgs) throws GridClientException {
-        VisorContentionTaskArg taskArg = new VisorContentionTaskArg(
-            cacheArgs.minQueueSize(), cacheArgs.maxPrint());
-
-        UUID nodeId = cacheArgs.nodeId() == null ? BROADCAST_UUID : cacheArgs.nodeId();
-
-        VisorContentionTaskResult res = executeTaskByNameOnNode(
-            client, VisorContentionTask.class.getName(), taskArg, nodeId);
-
-        if (!F.isEmpty(res.exceptions())) {
-            log("Contention check failed on nodes:");
-
-            for (Map.Entry<UUID, Exception> e : res.exceptions().entrySet()) {
-                log("Node ID: " + e.getKey());
-
-                log("Exception message:");
-                log(e.getValue().getMessage());
-                nl();
-            }
-        }
-
-        for (ContentionInfo info : res.getInfos())
-            info.print();
-    }
-
-    /**
-     * @param client Client.
-     * @param cacheArgs Cache args.
-     */
-    private void cacheValidateIndexes(GridClient client, CacheArguments cacheArgs) throws GridClientException {
-        VisorValidateIndexesTaskArg taskArg = new VisorValidateIndexesTaskArg(
-            cacheArgs.caches(),
-            cacheArgs.nodeId() != null ? Collections.singleton(cacheArgs.nodeId()) : null,
-            cacheArgs.checkFirst(),
-            cacheArgs.checkThrough()
-        );
-
-        VisorValidateIndexesTaskResult taskRes = executeTaskByNameOnNode(
-            client, VALIDATE_INDEXES_TASK, taskArg, null);
-
-        boolean errors = printErrors(taskRes.exceptions(), "Index validation failed on nodes:");
-
-        for (Map.Entry<UUID, VisorValidateIndexesJobResult> nodeEntry : taskRes.results().entrySet()) {
-            if (!nodeEntry.getValue().hasIssues())
-                continue;
-
-            errors = true;
-
-            log("Index issues found on node " + nodeEntry.getKey() + ":");
-
-            Collection<IndexIntegrityCheckIssue> integrityCheckFailures = nodeEntry.getValue().integrityCheckFailures();
-
-            if (!integrityCheckFailures.isEmpty()) {
-                for (IndexIntegrityCheckIssue is : integrityCheckFailures)
-                    log(i(is));
-            }
-
-            Map<PartitionKey, ValidateIndexesPartitionResult> partRes = nodeEntry.getValue().partitionResult();
-
-            for (Map.Entry<PartitionKey, ValidateIndexesPartitionResult> e : partRes.entrySet()) {
-                ValidateIndexesPartitionResult res = e.getValue();
-
-                if (!res.issues().isEmpty()) {
-                    log(i(j(" ", e.getKey(), e.getValue())));
-
-                    for (IndexValidationIssue is : res.issues())
-                        log(i(is, 2));
-                }
-            }
-
-            Map<String, ValidateIndexesPartitionResult> idxRes = nodeEntry.getValue().indexResult();
-
-            for (Map.Entry<String, ValidateIndexesPartitionResult> e : idxRes.entrySet()) {
-                ValidateIndexesPartitionResult res = e.getValue();
-
-                if (!res.issues().isEmpty()) {
-                    log(i(j(" ", "SQL Index", e.getKey(), e.getValue())));
-
-                    for (IndexValidationIssue is : res.issues())
-                        log(i(is, 2));
-                }
-            }
-        }
-
-        if (!errors)
-            log("no issues found.");
-        else
-            log("issues found (listed above).");
-
-        nl();
-    }
-
-    /**
-     * @param client Client.
-     * @param cacheArgs Cache args.
-     */
-    private void cacheView(GridClient client, CacheArguments cacheArgs) throws GridClientException {
-        VisorViewCacheTaskArg taskArg = new VisorViewCacheTaskArg(cacheArgs.regex(), cacheArgs.cacheCommand());
-
-        VisorViewCacheTaskResult res = executeTaskByNameOnNode(
-            client, VisorViewCacheTask.class.getName(), taskArg, cacheArgs.nodeId());
-
-        if (cacheArgs.fullConfig() && cacheArgs.cacheCommand() == CACHES)
-            cachesConfig(client, cacheArgs, res);
-        else
-            printCacheInfos(res.cacheInfos(), cacheArgs.cacheCommand());
-
-    }
-
-    /**
-     * Executes appropriate version of idle_verify check. Old version will be used if there are old nodes in the
-     * cluster.
-     *
-     * @param client Client.
-     * @param cacheArgs Cache args.
-     */
-    private void cacheIdleVerify(GridClient client, CacheArguments cacheArgs) throws GridClientException {
-        Collection<GridClientNode> nodes = client.compute().nodes(GridClientNode::connectable);
-
-        boolean idleVerifyV2 = true;
-
-        for (GridClientNode node : nodes) {
-            String nodeVerStr = node.attribute(IgniteNodeAttributes.ATTR_BUILD_VER);
-
-            IgniteProductVersion nodeVer = IgniteProductVersion.fromString(nodeVerStr);
-
-            if (nodeVer.compareTo(VerifyBackupPartitionsTaskV2.V2_SINCE_VER) < 0) {
-                idleVerifyV2 = false;
-
-                break;
-            }
-        }
-
-        if (cacheArgs.dump())
-            cacheIdleVerifyDump(client, cacheArgs);
-        else if (idleVerifyV2)
-            cacheIdleVerifyV2(client, cacheArgs);
-        else
-            legacyCacheIdleVerify(client, cacheArgs);
-    }
-
-    /**
-     * @param client Client.
-     * @param cacheArgs Cache args.
-     */
-    private VisorFindAndDeleteGarbageInPersistenceTaskResult findAndDeleteGarbage(
-        GridClient client,
-        CacheArguments cacheArgs
-    ) throws GridClientException {
-        checkFeatureSupportedByCluster(client, IgniteFeatures.FIND_AND_DELETE_GARBAGE_COMMAND, false);
-
-        VisorFindAndDeleteGarbageInPersistenceTaskArg taskArg = new VisorFindAndDeleteGarbageInPersistenceTaskArg(
-            cacheArgs.groups(),
-            cacheArgs.delete(),
-            cacheArgs.nodeId() != null ? Collections.singleton(cacheArgs.nodeId()) : null
-        );
-
-        VisorFindAndDeleteGarbageInPersistenceTaskResult taskRes = executeTask(
-            client, VisorFindAndDeleteGarbageInPersistenceTask.class, taskArg);
-
-        printErrors(taskRes.exceptions(), "Scanning for garbage failed on nodes:");
-
-        for (Map.Entry<UUID, VisorFindAndDeleteGarbageInPersistenceJobResult> nodeEntry : taskRes.result().entrySet()) {
-            if (!nodeEntry.getValue().hasGarbage()) {
-                log("Node "+ nodeEntry.getKey() + " - garbage not found.");
-
-                continue;
-            }
-
-            log("Garbage found on node " + nodeEntry.getKey() + ":");
-
-            VisorFindAndDeleteGarbageInPersistenceJobResult value = nodeEntry.getValue();
-
-            Map<Integer, Map<Integer, Long>> grpPartErrorsCount = value.checkResult();
-
-            if (!grpPartErrorsCount.isEmpty()) {
-                for (Map.Entry<Integer, Map<Integer, Long>> entry : grpPartErrorsCount.entrySet()) {
-                    for (Map.Entry<Integer, Long> e : entry.getValue().entrySet()) {
-                        log(i("Group=" + entry.getKey() +
-                            ", partition=" + e.getKey() +
-                            ", count of keys=" + e.getValue()));
-                    }
-                }
-            }
-
-            nl();
-        }
-
-        return taskRes;
-    }
-
-    /**
-     * @param exceptions Exception per node.
-     * @param msg Message to print before errors if at least one presented.
-     * @return True if found any error.
-     */
-    private boolean printErrors(Map<UUID, Exception> exceptions, String msg) {
-        if (!F.isEmpty(exceptions)) {
-            log(msg);
-
-            for (Map.Entry<UUID, Exception> e : exceptions.entrySet()) {
-                log(i("Node ID: " + e.getKey()));
-
-                log(i("Exception message:"));
-                log(i(e.getValue().getMessage(), 2));
-                nl();
-            }
-
-            return true;
-        }
-
-        return false;
-    }
-
-    /**
-     * @param client Client.
-     * @param cacheArgs Cache args.
-     */
-    private void legacyCacheIdleVerify(GridClient client, CacheArguments cacheArgs) throws GridClientException {
-        VisorIdleVerifyTaskResult res = executeTask(
-            client,
-            VisorIdleVerifyTask.class,
-            new VisorIdleVerifyTaskArg(cacheArgs.caches(), cacheArgs.excludeCaches(), cacheArgs.idleCheckCrc())
-        );
-
-        Map<PartitionKey, List<PartitionHashRecord>> conflicts = res.getConflicts();
-
-        if (conflicts.isEmpty()) {
-            log("idle_verify check has finished, no conflicts have been found.");
-            nl();
-        }
-        else {
-            log("idle_verify check has finished, found " + conflicts.size() + " conflict partitions.");
-            nl();
-
-            for (Map.Entry<PartitionKey, List<PartitionHashRecord>> entry : conflicts.entrySet()) {
-                log("Conflict partition: " + entry.getKey());
-
-                log("Partition instances: " + entry.getValue());
-            }
-        }
-    }
-
-    /**
-     * @param client Client.
-     * @param cacheArgs Cache args.
-     */
-    private void cacheDistribution(GridClient client, CacheArguments cacheArgs) throws GridClientException {
-        CacheDistributionTaskArg taskArg = new CacheDistributionTaskArg(cacheArgs.caches(), cacheArgs.getUserAttributes());
-
-        UUID nodeId = cacheArgs.nodeId() == null ? BROADCAST_UUID : cacheArgs.nodeId();
-
-        CacheDistributionTaskResult res = executeTaskByNameOnNode(client, CacheDistributionTask.class.getName(), taskArg, nodeId);
-
-        res.print(System.out);
-    }
-
-    /**
-     * @param client Client.
-     * @param cacheArgs Cache args.
-     * @param viewRes Cache view task result.
-     */
-    private void cachesConfig(GridClient client, CacheArguments cacheArgs,
-        VisorViewCacheTaskResult viewRes) throws GridClientException {
-        VisorCacheConfigurationCollectorTaskArg taskArg = new VisorCacheConfigurationCollectorTaskArg(cacheArgs.regex());
-
-        UUID nodeId = cacheArgs.nodeId() == null ? BROADCAST_UUID : cacheArgs.nodeId();
-
-        Map<String, VisorCacheConfiguration> res =
-            executeTaskByNameOnNode(client, VisorCacheConfigurationCollectorTask.class.getName(), taskArg, nodeId);
-
-        Map<String, Integer> cacheToMapped =
-            viewRes.cacheInfos().stream().collect(Collectors.toMap(CacheInfo::getCacheName, CacheInfo::getMapped));
-
-        printCachesConfig(res, cacheArgs.outputFormat(), cacheToMapped);
-    }
-
-    /**
-     * Prints caches info.
-     *
-     * @param infos Caches info.
-     * @param cmd Command.
-     */
-    private void printCacheInfos(Collection<CacheInfo> infos, VisorViewCacheCmd cmd) {
-        for (CacheInfo info : infos) {
-            Map<String, Object> map = info.toMap(cmd);
-
-            SB sb = new SB("[");
-
-            for (Map.Entry<String, Object> e : map.entrySet())
-                sb.a(e.getKey()).a("=").a(e.getValue()).a(", ");
-
-            sb.setLength(sb.length() - 2);
-
-            sb.a("]");
-
-            log(sb.toString());
-        }
-    }
-
-    /**
-     * Prints caches config.
-     *
-     * @param caches Caches config.
-     * @param outputFormat Output format.
-     * @param cacheToMapped Map cache name to mapped.
-     */
-    private void printCachesConfig(
-        Map<String, VisorCacheConfiguration> caches,
-        OutputFormat outputFormat,
-        Map<String, Integer> cacheToMapped
-    ) {
-
-        for (Map.Entry<String, VisorCacheConfiguration> entry : caches.entrySet()) {
-            String cacheName = entry.getKey();
-
-            switch (outputFormat) {
-                case MULTI_LINE:
-                    Map<String, Object> params = mapToPairs(entry.getValue());
-
-                    params.put("Mapped", cacheToMapped.get(cacheName));
-
-                    log("[cache = '%s']%n", cacheName);
-
-                    for (Map.Entry<String, Object> innerEntry : params.entrySet())
-                        log("%s: %s%n", innerEntry.getKey(), innerEntry.getValue());
-
-                    nl();
-
-                    break;
-
-                default:
-                    int mapped = cacheToMapped.get(cacheName);
-
-                    log("%s: %s %s=%s%n", entry.getKey(), toString(entry.getValue()), "mapped", mapped);
-
-                    break;
-            }
-        }
-    }
-
-    /**
-     * Invokes toString() method and cuts class name from result string.
-     *
-     * @param cfg Visor cache configuration for invocation.
-     * @return String representation without class name in begin of string.
-     */
-    private String toString(VisorCacheConfiguration cfg) {
-        return cfg.toString().substring(cfg.getClass().getSimpleName().length() + 1);
-    }
-
-    /**
-     * @param client Client.
-     * @param cacheArgs Cache args.
-     */
-    private void cacheResetLostPartitions(GridClient client, CacheArguments cacheArgs) throws GridClientException {
-        CacheResetLostPartitionsTaskArg taskArg = new CacheResetLostPartitionsTaskArg(cacheArgs.caches());
-
-        CacheResetLostPartitionsTaskResult res = executeTaskByNameOnNode(client, CacheResetLostPartitionsTask.class.getName(), taskArg, null);
-
-        res.print(System.out);
-    }
-
-    /**
-     * @param client Client.
-     * @param cacheArgs Cache args.
-     */
-    private void cacheIdleVerifyDump(GridClient client, CacheArguments cacheArgs) throws GridClientException {
-        VisorIdleVerifyDumpTaskArg arg = new VisorIdleVerifyDumpTaskArg(
-            cacheArgs.caches(),
-            cacheArgs.excludeCaches(),
-            cacheArgs.isSkipZeros(),
-            cacheArgs.getCacheFilterEnum(),
-            cacheArgs.idleCheckCrc()
-        );
-
-        String path = executeTask(client, VisorIdleVerifyDumpTask.class, arg);
-
-        log("VisorIdleVerifyDumpTask successfully written output to '" + path + "'");
-    }
-
-    /**
-     * @param client Client.
-     * @param cacheArgs Cache args.
-     */
-    private void cacheIdleVerifyV2(GridClient client, CacheArguments cacheArgs) throws GridClientException {
-        IdleVerifyResultV2 res = executeTask(
-            client,
-            VisorIdleVerifyTaskV2.class,
-            new VisorIdleVerifyTaskArg(cacheArgs.caches(), cacheArgs.excludeCaches(), cacheArgs.idleCheckCrc())
-        );
-
-        res.print(System.out::print);
-    }
-
-    /**
-     * Change baseline.
-     *
-     * @param client Client.
-     * @param baselineArgs Baseline action arguments.
-     * @throws Throwable If failed to execute baseline action.
-     */
-    private void baseline(GridClient client, BaselineArguments baselineArgs) throws Throwable {
-        try {
-            VisorBaselineTaskResult res = executeTask(client, VisorBaselineTask.class, toVisorArguments(baselineArgs));
-
-            baselinePrint0(res);
-        }
-        catch (Throwable e) {
-            log("Failed to execute baseline command='" + baselineArgs.getCmd().text() + "'", e);
-
-            throw e;
-        }
-    }
-
-    /**
-     * Prepare task argument.
-     *
-     * @param args Argument from command line.
-     * @return Task argument.
-     */
-    private VisorBaselineTaskArg toVisorArguments(BaselineArguments args) {
-        VisorBaselineAutoAdjustSettings settings = args.getCmd() == BaselineCommand.AUTO_ADJUST
-            ? new VisorBaselineAutoAdjustSettings(args.getEnableAutoAdjust(), args.getSoftBaselineTimeout())
-            : null;
-
-        return new VisorBaselineTaskArg(toVisorOperation(args.getCmd()), args.getTopVer(), args.getConsistentIds(), settings);
-    }
-
-    /**
-     * Convert {@link BaselineCommand} to {@link VisorBaselineOperation}.
-     *
-     * @param baselineCommand Baseline command for convertion.
-     * @return {@link VisorBaselineOperation}.
-     */
-    private VisorBaselineOperation toVisorOperation(BaselineCommand baselineCommand) {
-        switch (baselineCommand) {
-            case ADD:
-                return VisorBaselineOperation.ADD;
-            case AUTO_ADJUST:
-                return VisorBaselineOperation.AUTOADJUST;
-            case REMOVE:
-                return VisorBaselineOperation.REMOVE;
-            case SET:
-                return VisorBaselineOperation.SET;
-            case VERSION:
-                return VisorBaselineOperation.VERSION;
-            case COLLECT:
-                return VisorBaselineOperation.COLLECT;
-        }
-
-        return null;
-    }
-
-    /**
-     * @param s String of consisted ids delimited by comma.
-     * @return List of consistent ids.
-     */
-    private List<String> getConsistentIds(String s) {
-        if (F.isEmpty(s))
-            throw new IllegalArgumentException("Empty list of consistent IDs");
-
-        List<String> consistentIds = new ArrayList<>();
-
-        for (String consistentId : s.split(","))
-            consistentIds.add(consistentId.trim());
-
-        return consistentIds;
-    }
-
-    /**
-     * Print baseline topology.
-     *
-     * @param res Task result with baseline topology.
-     */
-    private void baselinePrint0(VisorBaselineTaskResult res) {
-        log("Cluster state: " + (res.isActive() ? "active" : "inactive"));
-        log("Current topology version: " + res.getTopologyVersion());
-        VisorBaselineAutoAdjustSettings autoAdjustSettings = res.getAutoAdjustSettings();
-
-        if (autoAdjustSettings != null) {
-            log("Baseline auto adjustment " + (TRUE.equals(autoAdjustSettings.getEnabled()) ? "enabled" : "disabled")
-                + ": softTimeout=" + autoAdjustSettings.getSoftTimeout()
-            );
-        }
-
-        if (autoAdjustSettings.enabled) {
-            if (res.isBaselineAdjustInProgress())
-                log("Baseline auto-adjust is in progress");
-            else if (res.getRemainingTimeToBaselineAdjust() < 0)
-                log("Baseline auto-adjust are not scheduled");
-            else
-                log("Baseline auto-adjust will happen in '" + res.getRemainingTimeToBaselineAdjust() + "' ms");
-        }
-
-        nl();
-
-        Map<String, VisorBaselineNode> baseline = res.getBaseline();
-
-        Map<String, VisorBaselineNode> srvs = res.getServers();
-
-        // if task runs on a node with VisorBaselineNode of old version (V1) we'll get order=null for all nodes.
-
-        String crdStr = srvs.values().stream()
-            // check for not null
-            .filter(node -> node.getOrder() != null)
-            .min(Comparator.comparing(VisorBaselineNode::getOrder))
-            // format
-            .map(crd -> " (Coordinator: ConsistentId=" + crd.getConsistentId() + ", Order=" + crd.getOrder() + ")")
-            .orElse("");
-
-        log("Current topology version: " + res.getTopologyVersion() + crdStr);
-        nl();
-
-        if (F.isEmpty(baseline))
-            log("Baseline nodes not found.");
-        else {
-            log("Baseline nodes:");
-
-            for (VisorBaselineNode node : baseline.values()) {
-                VisorBaselineNode srvNode = srvs.get(node.getConsistentId());
-
-                String state = ", State=" + (srvNode != null ? "ONLINE" : "OFFLINE");
-
-                String order = srvNode != null ? ", Order=" + srvNode.getOrder() : "";
-
-                log(i("ConsistentId=" + node.getConsistentId() + state + order, 2));
-            }
-
-            log(DELIM);
-            log("Number of baseline nodes: " + baseline.size());
-
-            nl();
-
-            List<VisorBaselineNode> others = new ArrayList<>();
-
-            for (VisorBaselineNode node : srvs.values()) {
-                if (!baseline.containsKey(node.getConsistentId()))
-                    others.add(node);
-            }
-
-            if (F.isEmpty(others))
-                log("Other nodes not found.");
-            else {
-                log("Other nodes:");
-
-                for (VisorBaselineNode node : others)
-                    log(i("ConsistentId=" + node.getConsistentId() + ", Order=" + node.getOrder(), 2));
-
-                log("Number of other nodes: " + others.size());
-            }
-        }
-    }
-
-    /**
-     * Dump transactions information.
-     *
-     * @param client Client.
-     * @param arg Transaction search arguments
-     */
-    private void transactions(GridClient client, VisorTxTaskArg arg) throws GridClientException {
-        try {
-            if (arg.getOperation() == VisorTxOperation.INFO) {
-                transactionInfo(client, arg);
-
-                return;
-            }
-
-            Map<ClusterNode, VisorTxTaskResult> res = executeTask(client, VisorTxTask.class, arg);
-
-            lastOperationRes = res;
-
-            if (res.isEmpty())
-                log("Nothing found.");
-            else if (arg.getOperation() == VisorTxOperation.KILL)
-                log("Killed transactions:");
-            else
-                log("Matching transactions:");
-
-            for (Map.Entry<ClusterNode, VisorTxTaskResult> entry : res.entrySet()) {
-                if (entry.getValue().getInfos().isEmpty())
-                    continue;
-
-                ClusterNode key = entry.getKey();
-
-                log(nodeDescription(key));
-
-                for (VisorTxInfo info : entry.getValue().getInfos())
-                    log(info.toUserString());
-            }
-        }
-        catch (Throwable e) {
-            log("Failed to perform operation.");
-
-            throw e;
-        }
-    }
-
-    /**
-     * Provides text descrition of a cluster node.
-     *
-     * @param node Node.
-     */
-    private static String nodeDescription(ClusterNode node) {
-        return node.getClass().getSimpleName() + " [id=" + node.id() +
-            ", addrs=" + node.addresses() +
-            ", order=" + node.order() +
-            ", ver=" + node.version() +
-            ", isClient=" + node.isClient() +
-            ", consistentId=" + node.consistentId() +
-            "]";
-    }
-
-    /**
-     * Executes --tx --info command.
-     *
-     * @param client Client.
-     * @param arg Argument.
-     */
-    private void transactionInfo(GridClient client, VisorTxTaskArg arg) throws GridClientException {
-        checkFeatureSupportedByCluster(client, IgniteFeatures.TX_INFO_COMMAND, true);
-
-        GridCacheVersion nearXidVer = executeTask(client, FetchNearXidVersionTask.class, arg.txInfoArgument());
-
-        boolean histMode = false;
-
-        if (nearXidVer != null) {
-            log("Resolved transaction near XID version: " + nearXidVer);
-
-            arg.txInfoArgument(new TxVerboseId(null, nearXidVer));
-        }
-        else {
-            log("Active transactions not found.");
-
-            if (arg.txInfoArgument().gridCacheVersion() != null) {
-                log("Will try to peek history to find out whether transaction was committed / rolled back.");
-
-                histMode = true;
-            }
-            else {
-                log("You can specify transaction in GridCacheVersion format in order to peek history " +
-                    "to find out whether transaction was committed / rolled back.");
-
-                return;
-            }
-        }
-
-        Map<ClusterNode, VisorTxTaskResult> res = executeTask(client, VisorTxTask.class, arg);
-
-        lastOperationRes = res;
-
-        if (histMode)
-            printTxInfoHistoricalResult(res);
-        else
-            printTxInfoResult(res);
-
-
-    }
-
-    /**
-     * Prints result of --tx --info command to output.
-     *
-     * @param res Response.
-     */
-    private void printTxInfoResult(Map<ClusterNode, VisorTxTaskResult> res) {
-        String lb = null;
-
-        Map<Integer, String> usedCaches = new HashMap<>();
-        Map<Integer, String> usedCacheGroups = new HashMap<>();
-        VisorTxInfo firstInfo = null;
-        TxVerboseInfo firstVerboseInfo = null;
-        Set<TransactionState> states = new HashSet<>();
-
-        for (Map.Entry<ClusterNode, VisorTxTaskResult> entry : res.entrySet()) {
-            for (VisorTxInfo info : entry.getValue().getInfos()) {
-                assert info.getTxVerboseInfo() != null;
-
-                if (lb == null)
-                    lb = info.getLabel();
-
-                if (firstInfo == null) {
-                    firstInfo = info;
-                    firstVerboseInfo = info.getTxVerboseInfo();
-                }
-
-                usedCaches.putAll(info.getTxVerboseInfo().usedCaches());
-                usedCacheGroups.putAll(info.getTxVerboseInfo().usedCacheGroups());
-                states.add(info.getState());
-            }
-        }
-
-        String indent = "";
-
-        nl();
-        log(indent + "Transaction detailed info:");
-
-        printTransactionDetailedInfo(
-            res, usedCaches, usedCacheGroups, firstInfo, firstVerboseInfo, states, indent + DOUBLE_INDENT);
-    }
-
-    /**
-     * Prints detailed info about transaction to output.
-     *
-     * @param res Response.
-     * @param usedCaches Used caches.
-     * @param usedCacheGroups Used cache groups.
-     * @param firstInfo First info.
-     * @param firstVerboseInfo First verbose info.
-     * @param states States.
-     * @param indent Indent.
-     */
-    private void printTransactionDetailedInfo(Map<ClusterNode, VisorTxTaskResult> res, Map<Integer, String> usedCaches,
-        Map<Integer, String> usedCacheGroups, VisorTxInfo firstInfo, TxVerboseInfo firstVerboseInfo,
-        Set<TransactionState> states, String indent) {
-        log(indent + "Near XID version: " + firstVerboseInfo.nearXidVersion());
-        log(indent + "Near XID version (UUID): " + firstInfo.getNearXid());
-        log(indent + "Isolation: " + firstInfo.getIsolation());
-        log(indent + "Concurrency: " + firstInfo.getConcurrency());
-        log(indent + "Timeout: " + firstInfo.getTimeout());
-        log(indent + "Initiator node: " + firstVerboseInfo.nearNodeId());
-        log(indent + "Initiator node (consistent ID): " + firstVerboseInfo.nearNodeConsistentId());
-        log(indent + "Label: " + firstInfo.getLabel());
-        log(indent + "Topology version: " + firstInfo.getTopologyVersion());
-        log(indent + "Used caches (ID to name): " + usedCaches);
-        log(indent + "Used cache groups (ID to name): " + usedCacheGroups);
-        log(indent + "States across the cluster: " + states);
-        log(indent + "Transaction topology: ");
-
-        printTransactionTopology(res, indent + DOUBLE_INDENT);
-    }
-
-    /**
-     * Prints transaction topology to output.
-     *
-     * @param res Response.
-     * @param indent Indent.
-     */
-    private void printTransactionTopology(Map<ClusterNode, VisorTxTaskResult> res, String indent) {
-        for (Map.Entry<ClusterNode, VisorTxTaskResult> entry : res.entrySet()) {
-            log(indent + nodeDescription(entry.getKey()) + ':');
-
-            printTransactionMappings(indent + DOUBLE_INDENT, entry);
-        }
-    }
-
-    /**
-     * Prints transaction mappings for specific cluster node to output.
-     *
-     * @param indent Indent.
-     * @param entry Entry.
-     */
-    private void printTransactionMappings(String indent, Map.Entry<ClusterNode, VisorTxTaskResult> entry) {
-        for (VisorTxInfo info : entry.getValue().getInfos()) {
-            TxVerboseInfo verboseInfo = info.getTxVerboseInfo();
-
-            if (verboseInfo != null) {
-                log(indent + "Mapping [type=" + verboseInfo.txMappingType() + "]:");
-
-                printTransactionMapping(indent + DOUBLE_INDENT, info, verboseInfo);
-            }
-            else {
-                log(indent + "Mapping [type=HISTORICAL]:");
-
-                log(indent + DOUBLE_INDENT + "State: " + info.getState());
-            }
-        }
-    }
-
-    /**
-     * Prints specific transaction mapping to output.
-     *
-     * @param indent Indent.
-     * @param info Info.
-     * @param verboseInfo Verbose info.
-     */
-    private void printTransactionMapping(String indent, VisorTxInfo info, TxVerboseInfo verboseInfo) {
-        log(indent + "XID version (UUID): " + info.getXid());
-        log(indent + "State: " + info.getState());
-
-        if (verboseInfo.txMappingType() == TxMappingType.REMOTE) {
-            log(indent + "Primary node: " + verboseInfo.dhtNodeId());
-            log(indent + "Primary node (consistent ID): " + verboseInfo.dhtNodeConsistentId());
-        }
-
-        if (!F.isEmpty(verboseInfo.localTxKeys())) {
-            log(indent + "Mapped keys:");
-
-            printTransactionKeys(indent + DOUBLE_INDENT, verboseInfo);
-        }
-    }
-
-    /**
-     * Prints keys of specific transaction mapping to output.
-     *
-     * @param indent Indent.
-     * @param verboseInfo Verbose info.
-     */
-    private void printTransactionKeys(String indent, TxVerboseInfo verboseInfo) {
-        for (TxVerboseKey txVerboseKey : verboseInfo.localTxKeys()) {
-            log(indent + (txVerboseKey.read() ? "Read" : "Write") +
-                " [lock=" + txVerboseKey.lockType() + "]: " + txVerboseKey.txKey());
-
-            if (txVerboseKey.lockType() == TxKeyLockType.AWAITS_LOCK)
-                log(indent + DOUBLE_INDENT + "Lock owner XID: " + txVerboseKey.ownerVersion());
-        }
-    }
-
-    /**
-     * Prints results of --tx --info to output in case requested transaction is not active.
-     *
-     * @param res Response.
-     */
-    private void printTxInfoHistoricalResult(Map<ClusterNode, VisorTxTaskResult> res) {
-        if (F.isEmpty(res))
-            log("Transaction was not found in history across the cluster.");
-        else {
-            log("Transaction was found in completed versions history of the following nodes:");
-
-            for (Map.Entry<ClusterNode, VisorTxTaskResult> entry : res.entrySet()) {
-                log(DOUBLE_INDENT + nodeDescription(entry.getKey()) + ':');
-                log(DOUBLE_INDENT + DOUBLE_INDENT + "State: " + entry.getValue().getInfos().get(0).getState());
-            }
-        }
-    }
-
-    /**
-     * Checks that all cluster nodes support specified feature.
-     *
-     * @param client Client.
-     * @param feature Feature.
-     * @param validateClientNodes Whether client nodes should be checked as well.
-     */
-    private static void checkFeatureSupportedByCluster(
-        GridClient client,
-        IgniteFeatures feature,
-        boolean validateClientNodes
-    ) throws GridClientException {
-        Collection<GridClientNode> nodes = validateClientNodes ?
-            client.compute().nodes() :
-            client.compute().nodes(GridClientNode::connectable);
-
-        for (GridClientNode node : nodes) {
-            byte[] featuresAttrBytes = node.attribute(IgniteNodeAttributes.ATTR_IGNITE_FEATURES);
-
-            if (!IgniteFeatures.nodeSupports(featuresAttrBytes, feature)) {
-                throw new IllegalStateException("Failed to execute command: cluster contains node that " +
-                    "doesn't support feature [nodeId=" + node.nodeId() + ", feature=" + feature + ']');
-            }
-        }
-    }
-
-    /**
-     * Execute WAL command.
-     *
-     * @param client Client.
-     * @param walAct WAL action to execute.
-     * @param walArgs WAL args.
-     * @throws Throwable If failed to execute wal action.
-     */
-    private void wal(GridClient client, String walAct, String walArgs) throws Throwable {
-        switch (walAct) {
-            case WAL_DELETE:
-                deleteUnusedWalSegments(client, walArgs);
-
-                break;
-
-            case WAL_PRINT:
-            default:
-                printUnusedWalSegments(client, walArgs);
-
-                break;
-        }
-    }
-
-    /**
-     * Execute delete unused WAL segments task.
-     *
-     * @param client Client.
-     * @param walArgs WAL args.
-     */
-    private void deleteUnusedWalSegments(GridClient client, String walArgs) throws Throwable {
-        VisorWalTaskResult res = executeTask(client, VisorWalTask.class,
-            walArg(VisorWalTaskOperation.DELETE_UNUSED_WAL_SEGMENTS, walArgs));
-        printDeleteWalSegments0(res);
-    }
-
-    /**
-     * Execute print unused WAL segments task.
-     *
-     * @param client Client.
-     * @param walArgs Wal args.
-     */
-    private void printUnusedWalSegments(GridClient client, String walArgs) throws Throwable {
-        VisorWalTaskResult res = executeTask(client, VisorWalTask.class,
-            walArg(VisorWalTaskOperation.PRINT_UNUSED_WAL_SEGMENTS, walArgs));
-        printUnusedWalSegments0(res);
-    }
-
-    /**
-     * Prepare WAL task argument.
-     *
-     * @param op Operation.
-     * @param s Argument from command line.
-     * @return Task argument.
-     */
-    private VisorWalTaskArg walArg(VisorWalTaskOperation op, String s) {
-        List<String> consistentIds = null;
-
-        if (!F.isEmpty(s)) {
-            consistentIds = new ArrayList<>();
-
-            for (String consistentId : s.split(","))
-                consistentIds.add(consistentId.trim());
-        }
-
-        switch (op) {
-            case DELETE_UNUSED_WAL_SEGMENTS:
-            case PRINT_UNUSED_WAL_SEGMENTS:
-                return new VisorWalTaskArg(op, consistentIds);
-
-            default:
-                return new VisorWalTaskArg(VisorWalTaskOperation.PRINT_UNUSED_WAL_SEGMENTS, consistentIds);
-        }
-
-    }
-
-    /**
-     * Print list of unused wal segments.
-     *
-     * @param taskRes Task result with baseline topology.
-     */
-    private void printUnusedWalSegments0(VisorWalTaskResult taskRes) {
-        log("Unused wal segments per node:");
-        nl();
-
-        Map<String, Collection<String>> res = taskRes.results();
-        Map<String, Exception> failRes = taskRes.exceptions();
-        Map<String, VisorClusterNode> nodesInfo = taskRes.getNodesInfo();
-
-        for (Map.Entry<String, Collection<String>> entry : res.entrySet()) {
-            VisorClusterNode node = nodesInfo.get(entry.getKey());
-
-            log("Node=" + node.getConsistentId());
-            log(i("addresses " + U.addressesAsString(node.getAddresses(), node.getHostNames()), 2));
-
-            for (String fileName : entry.getValue())
-                log(i(fileName));
-
-            nl();
-        }
-
-        for (Map.Entry<String, Exception> entry : failRes.entrySet()) {
-            VisorClusterNode node = nodesInfo.get(entry.getKey());
-
-            log("Node=" + node.getConsistentId());
-            log(i("addresses " + U.addressesAsString(node.getAddresses(), node.getHostNames())), 2);
-            log(i("failed with error: " + entry.getValue().getMessage()));
-            nl();
-        }
-    }
-
-    /**
-     * Print list of unused wal segments.
-     *
-     * @param taskRes Task result with baseline topology.
-     */
-    private void printDeleteWalSegments0(VisorWalTaskResult taskRes) {
-        log("WAL segments deleted for nodes:");
-        nl();
-
-        Map<String, Collection<String>> res = taskRes.results();
-        Map<String, Exception> errors = taskRes.exceptions();
-        Map<String, VisorClusterNode> nodesInfo = taskRes.getNodesInfo();
-
-        for (Map.Entry<String, Collection<String>> entry : res.entrySet()) {
-            VisorClusterNode node = nodesInfo.get(entry.getKey());
-
-            log("Node=" + node.getConsistentId());
-            log(i("addresses " + U.addressesAsString(node.getAddresses(), node.getHostNames())), 2);
-            nl();
-        }
-
-        for (Map.Entry<String, Exception> entry : errors.entrySet()) {
-            VisorClusterNode node = nodesInfo.get(entry.getKey());
-
-            log("Node=" + node.getConsistentId());
-            log(i("addresses " + U.addressesAsString(node.getAddresses(), node.getHostNames())), 2);
-            log(i("failed with error: " + entry.getValue().getMessage()));
-            nl();
-        }
-    }
-
-    /**
-     * @param e Exception to check.
-     * @return {@code true} if specified exception is {@link GridClientAuthenticationException}.
-     */
-    private boolean isAuthError(Throwable e) {
-        return X.hasCause(e, GridClientAuthenticationException.class);
-    }
-
-    /**
-     * @param e Exception to check.
-     * @return {@code true} if specified exception is a connection error.
-     */
-    private boolean isConnectionError(Throwable e) {
-        return e instanceof GridClientClosedException ||
-            e instanceof GridClientConnectionResetException ||
-            e instanceof GridClientDisconnectedException ||
-            e instanceof GridClientHandshakeException ||
-            e instanceof GridServerUnreachableException;
-    }
-
-    /**
-     * Print command usage.
-     *
-     * @param desc Command description.
-     * @param args Arguments.
-     */
-    private void usage(String desc, Command cmd, String... args) {
-        log(desc);
-        log(i(j(" ", UTILITY_NAME, cmd, j(" ", args)), 2));
-        nl();
-    }
-
-    /**
-     * Print cache command usage with default indention.
-     *
-     * @param cmd Cache command.
-     * @param args Cache command arguments.
-     */
-    private void usageCache(CacheCommand cmd, String... args) {
-        usageCache(1, cmd, args);
-    }
-
-    /**
-     * Print cache command usage.
-     *
-     * @param indentsNum Number of indents.
-     * @param cmd Cache command.
-     * @param args Cache command arguments.
-     */
-    private void usageCache(int indentsNum, CacheCommand cmd, String... args) {
-        log(i(DELIM, indentsNum));
-        nl();
-        log(i(j(" ", CACHE, cmd, j(" ", args)), indentsNum++));
-        nl();
-        log(i(getCacheSubcommandDesc(cmd), indentsNum));
-        nl();
-
-        Map<String, String> paramsDesc = createCacheArgsDesc(cmd);
-
-        if (!paramsDesc.isEmpty()) {
-            log(i("Parameters:", indentsNum));
-
-            usageCacheParams(paramsDesc, indentsNum + 1);
-
-            nl();
-        }
-    }
-
-    /**
-     * Print cache command arguments usage.
-     *
-     * @param paramsDesc Cache command arguments description.
-     * @param indentsNum Number of indents.
-     */
-    private void usageCacheParams(Map<String, String> paramsDesc, int indentsNum) {
-        int maxParamLen = paramsDesc.keySet().stream().max(Comparator.comparingInt(String::length)).get().length();
-
-        for (Map.Entry<String, String> param : paramsDesc.entrySet())
-            log(i(extendToLen(param.getKey(), maxParamLen) + INDENT + "- " + param.getValue(), indentsNum));
-    }
-
-    /**
-     * Appends spaces to end of input string for extending to needed length.
-     *
-     * @param s Input string.
-     * @param targetLen Needed length.
-     * @return String with appended spaces on the end.
-     */
-    private String extendToLen(String s, int targetLen) {
-        assert targetLen >= 0;
-        assert s.length() <= targetLen;
-
-        if (s.length() == targetLen)
-            return s;
-
-        SB sb = new SB(targetLen);
-
-        sb.a(s);
-
-        for (int i = 0; i < targetLen - s.length(); i++)
-            sb.a(" ");
-
-        return sb.toString();
-    }
-
-    /**
-     * Gets cache command description by cache command.
-     *
-     * @param cmd Cache command.
-     * @return Cache command description.
-     */
-    private String getCacheSubcommandDesc(CacheCommand cmd) {
-        switch (cmd) {
-            case LIST:
-                return "Show information about caches, groups or sequences that match a regular expression. When executed without parameters, this subcommand prints the list of caches.";
-
-            case CONTENTION:
-                return "Show the keys that are point of contention for multiple transactions.";
-
-            case IDLE_VERIFY:
-                return "Verify counters and hash sums of primary and backup partitions for the specified caches/cache " +
-                    "groups on an idle cluster and print out the differences, if any. When no parameters are specified, " +
-                    "all user caches are verified. Cache filtering options configure the set of caches that will be " +
-                    "processed by " + IDLE_VERIFY + " command. If cache names are specified, in form of regular " +
-                    "expressions, only matching caches will be verified. Caches matched by regexes specified after " +
-                    EXCLUDE_CACHES + " parameter will be excluded from verification. Using parameter " + CACHE_FILTER +
-                    " you can verify: only " + USER + " caches, only user " + PERSISTENT + " caches, only user " +
-                    NOT_PERSISTENT + " caches, only " + SYSTEM + " caches, or " + ALL + " of the above.";
-
-            case VALIDATE_INDEXES:
-                return "Validate indexes on an idle cluster and print out the keys that are missing in the indexes.";
-
-            case DISTRIBUTION:
-                return "Prints the information about partition distribution.";
-
-            case RESET_LOST_PARTITIONS:
-                return "Reset the state of lost partitions for the specified caches.";
-
-            case FIND_AND_DELETE_GARBAGE:
-                return "Find and optionally delete garbage from shared cache groups which could be left after cache destroy.";
-
-            default:
-                throw new IllegalArgumentException("Unknown command: " + cmd);
-        }
-    }
-
-    /**
-     * Gets cache command arguments description by cache command.
-     *
-     * @param cmd Cache command.
-     * @return Cache command arguments description.
-     */
-    private Map<String, String> createCacheArgsDesc(CacheCommand cmd) {
-        Map<String, String> map = U.newLinkedHashMap(16);
-        switch (cmd) {
-            case LIST:
-                map.put(CONFIG.toString(), "print all configuration parameters for each cache.");
-                map.put(OUTPUT_FORMAT + " " + MULTI_LINE, "print configuration parameters per line. This option has effect only when used with " + CONFIG + " and without " + op(or(GROUP, SEQUENCE)) + ".");
-                map.put(GROUP.toString(), "print information about groups.");
-                map.put(SEQUENCE.toString(), "print information about sequences.");
-
-                break;
-            case VALIDATE_INDEXES:
-                map.put(CHECK_FIRST + " N", "validate only the first N keys");
-                map.put(CHECK_THROUGH + " K", "validate every Kth key");
-
-                break;
-
-            case IDLE_VERIFY:
-                map.put(CHECK_CRC.toString(), "check the CRC-sum of pages stored on disk before verifying data consistency in partitions between primary and backup nodes.");
-
-                break;
-
-            case FIND_AND_DELETE_GARBAGE:
-                map.put(FindAndDeleteGarbageArg.DELETE.toString(), "remove found garbage or not");
-
-                break;
-        }
-        return map;
-    }
-
-    /**
-     * Join input parameters with space and wrap optional braces {@code []}.
-     *
-     * @param params Other input parameter.
-     * @return Joined parameters wrapped optional braces.
-     */
-    private static String op(Object... params) {
-        return j(new SB(), "[", " ", params).a("]").toString();
-    }
-
-    /**
-     * Join input parameters with specified {@code delimeter} between them.
-     *
-     * @param delimeter Specified delimeter.
-     * @param params Other input parameter.
-     * @return Joined paramaters with specified {@code delimeter}.
-     */
-    private static String j(String delimeter, Object... params) {
-        return j(new SB(), "", delimeter, params).toString();
-    }
-
-    /**
-     * Join input parameters with specified {@code delimeter} between them and append to the end {@code delimeter}.
-     *
-     * @param sb Specified string builder.
-     * @param sbDelimeter Delimeter between {@code sb} and appended {@code param}.
-     * @param delimeter Specified delimeter.
-     * @param params Other input parameter.
-     * @return SB with appended to the end joined paramaters with specified {@code delimeter}.
-     */
-    private static SB j(SB sb, String sbDelimeter, String delimeter, Object... params) {
-        if (!F.isEmpty(params)) {
-            sb.a(sbDelimeter);
-
-            for (Object par : params)
-                sb.a(par).a(delimeter);
-
-            sb.setLength(sb.length() - delimeter.length());
-        }
-
-        return sb;
-    }
-
-    /**
-     * Concatenates input parameters to single string with OR delimiter {@code |}.
-     *
-     * @param params Remaining parameters.
-     * @return Concatenated string.
-     */
-    private static String or(Object... params) {
-        return j("|", params);
-    }
-
-    /**
-     * Join input parameters with space and wrap grouping braces {@code ()}.
-     *
-     * @param params Input parameter.
-     * @return Joined parameters wrapped grouped braces.
-     */
-    private static String g(Object... params) {
-        return j(new SB(), "(", " ", params).a(")").toString();
-    }
-
-    /**
-     * Extract next argument.
-     *
-     * @param err Error message.
-     * @return Next argument value.
-     */
-    private String nextArg(String err) {
-        if (peekedArg != null) {
-            String res = peekedArg;
-
-            peekedArg = null;
-
-            return res;
-        }
-
-        if (argsIt.hasNext())
-            return argsIt.next();
-
-        throw new IllegalArgumentException(err);
-    }
-
-    /**
-     * Returns the next argument in the iteration, without advancing the iteration.
-     *
-     * @return Next argument value or {@code null} if no next argument.
-     */
-    private String peekNextArg() {
-        if (peekedArg == null && argsIt.hasNext())
-            peekedArg = argsIt.next();
-
-        return peekedArg;
-    }
-
-    /**
-     * Parses and validates arguments.
-     *
-     * @param rawArgs Array of arguments.
-     * @return Arguments bean.
-     * @throws IllegalArgumentException In case arguments aren't valid.
-     */
-    Arguments parseAndValidate(List<String> rawArgs) {
-        String host = DFLT_HOST;
-
-        String port = DFLT_PORT;
-
-        String user = null;
-
-        String pwd = null;
-
-        Long pingInterval = DFLT_PING_INTERVAL;
-
-        Long pingTimeout = DFLT_PING_TIMEOUT;
-
-        String walAct = "";
-
-        String walArgs = "";
-
-        boolean autoConfirmation = false;
-
-        CacheArguments cacheArgs = null;
-
-        BaselineArguments baselineArgs = null;
-
-        List<Command> commands = new ArrayList<>();
-
-        initArgIterator(rawArgs);
-
-        VisorTxTaskArg txArgs = null;
-
-        String sslProtocol = DFLT_SSL_PROTOCOL;
-
-        String sslCipherSuites = "";
-
-        String sslKeyAlgorithm = SslContextFactory.DFLT_KEY_ALGORITHM;
-
-        String sslKeyStoreType = SslContextFactory.DFLT_STORE_TYPE;
-
-        String sslKeyStorePath = null;
-
-        char sslKeyStorePassword[] = null;
-
-        String sslTrustStoreType = SslContextFactory.DFLT_STORE_TYPE;
-
-        String sslTrustStorePath = null;
-
-        char sslTrustStorePassword[] = null;
-
-        final String pwdArgWarnFmt = "Warning: %s is insecure. " +
-            "Whenever possible, use interactive prompt for password (just discard %s option).";
-
-        while (hasNextArg()) {
-            String str = nextArg("").toLowerCase();
-
-            Command cmd = Command.of(str);
-
-            if (cmd != null) {
-                switch (cmd) {
-                    case ACTIVATE:
-                    case DEACTIVATE:
-                    case STATE:
-                        commands.add(cmd);
-
-                        break;
-
-                    case TX:
-                        commands.add(TX);
-
-                        txArgs = parseTransactionArguments();
-
-                        break;
-
-                    case BASELINE:
-                        commands.add(BASELINE);
-
-                        baselineArgs = parseAndValidateBaselineArgs();
-
-                        break;
-
-                    case CACHE:
-                        commands.add(CACHE);
-
-                        cacheArgs = parseAndValidateCacheArgs();
-
-                        break;
-
-                    case WAL:
-                        if (!enableExperimental)
-                            throw new IllegalArgumentException("Experimental command is disabled.");
-
-                        commands.add(WAL);
-
-                        str = nextArg("Expected arguments for " + WAL.text());
-
-                        walAct = str.toLowerCase();
-
-                        if (WAL_PRINT.equals(walAct) || WAL_DELETE.equals(walAct))
-                            walArgs = (str = peekNextArg()) != null && !isCommandOrOption(str)
-                                ? nextArg("Unexpected argument for " + WAL.text() + ": " + walAct)
-                                : "";
-                        else
-                            throw new IllegalArgumentException("Unexpected action " + walAct + " for " + WAL.text());
-
-                        break;
-                    default:
-                        throw new IllegalArgumentException("Unexpected command: " + str);
-                }
-            }
-            else {
-                switch (str) {
-                    case CMD_HOST:
-                        host = nextArg("Expected host name");
-
-                        break;
-
-                    case CMD_PORT:
-                        port = nextArg("Expected port number");
-
-                        try {
-                            int p = Integer.parseInt(port);
-
-                            if (p <= 0 || p > 65535)
-                                throw new IllegalArgumentException("Invalid value for port: " + port);
-                        }
-                        catch (NumberFormatException ignored) {
-                            throw new IllegalArgumentException("Invalid value for port: " + port);
-                        }
-
-                        break;
-
-                    case CMD_PING_INTERVAL:
-                        pingInterval = getPingParam("Expected ping interval", "Invalid value for ping interval");
-
-                        break;
-
-                    case CMD_PING_TIMEOUT:
-                        pingTimeout = getPingParam("Expected ping timeout", "Invalid value for ping timeout");
-
-                        break;
-
-                    case CMD_USER:
-                        user = nextArg("Expected user name");
-
-                        break;
-
-                    case CMD_PASSWORD:
-                        pwd = nextArg("Expected password");
-
-                        log(format(pwdArgWarnFmt, CMD_PASSWORD, CMD_PASSWORD));
-
-                        break;
-
-                    case CMD_SSL_PROTOCOL:
-                        sslProtocol = nextArg("Expected SSL protocol");
-
-                        break;
-
-                    case CMD_SSL_CIPHER_SUITES:
-                        sslCipherSuites = nextArg("Expected SSL cipher suites");
-
-                        break;
-
-                    case CMD_SSL_KEY_ALGORITHM:
-                        sslKeyAlgorithm = nextArg("Expected SSL key algorithm");
-
-                        break;
-
-                    case CMD_KEYSTORE:
-                        sslKeyStorePath = nextArg("Expected SSL key store path");
-
-                        break;
-
-                    case CMD_KEYSTORE_PASSWORD:
-                        sslKeyStorePassword = nextArg("Expected SSL key store password").toCharArray();
-
-                        log(format(pwdArgWarnFmt, CMD_KEYSTORE_PASSWORD, CMD_KEYSTORE_PASSWORD));
-
-                        break;
-
-                    case CMD_KEYSTORE_TYPE:
-                        sslKeyStoreType = nextArg("Expected SSL key store type");
-
-                        break;
-
-                    case CMD_TRUSTSTORE:
-                        sslTrustStorePath = nextArg("Expected SSL trust store path");
-
-                        break;
-
-                    case CMD_TRUSTSTORE_PASSWORD:
-                        sslTrustStorePassword = nextArg("Expected SSL trust store password").toCharArray();
-
-                        log(format(pwdArgWarnFmt, CMD_TRUSTSTORE_PASSWORD, CMD_TRUSTSTORE_PASSWORD));
-
-                        break;
-
-                    case CMD_TRUSTSTORE_TYPE:
-                        sslTrustStoreType = nextArg("Expected SSL trust store type");
-
-                        break;
-
-                    case CMD_AUTO_CONFIRMATION:
-                        autoConfirmation = true;
-
-                        break;
-
-                    default:
-                        throw new IllegalArgumentException("Unexpected argument: " + str);
-                }
-            }
-        }
-
-        int sz = commands.size();
-
-        if (sz < 1)
-            throw new IllegalArgumentException("No action was specified");
-
-        if (sz > 1)
-            throw new IllegalArgumentException("Only one action can be specified, but found: " + sz);
-
-        Command cmd = commands.get(0);
-
-        return new Arguments(cmd, host, port, user, pwd,
-            baselineArgs,
-            txArgs, cacheArgs,
-            walAct, walArgs,
-            pingTimeout, pingInterval, autoConfirmation,
-            sslProtocol, sslCipherSuites,
-            sslKeyAlgorithm, sslKeyStorePath, sslKeyStorePassword, sslKeyStoreType,
-            sslTrustStorePath, sslTrustStorePassword, sslTrustStoreType);
-    }
-
-    /**
-     * Parses and validates baseline arguments.
-     *
-     * @return --baseline subcommand arguments in case validation is successful.
-     */
-    private BaselineArguments parseAndValidateBaselineArgs() {
-        if (!hasNextSubArg())
-            return new BaselineArguments.Builder(BaselineCommand.COLLECT).build();
-
-        BaselineCommand cmd = of(nextArg("Expected baseline action"));
-
-        if (cmd == null)
-            throw new IllegalArgumentException("Expected correct baseline action");
-
-        BaselineArguments.Builder baselineArgs = new BaselineArguments.Builder(cmd);
-
-        switch (cmd) {
-            case ADD:
-            case REMOVE:
-            case SET:
-                return baselineArgs
-                    .withConsistentIds(getConsistentIds(nextArg("Expected list of consistent ids")))
-                    .build();
-            case VERSION:
-                return baselineArgs
-                    .withTopVer(nextLongArg("topology version"))
-                    .build();
-            case AUTO_ADJUST:
-                do {
-                    AutoAdjustCommandArg autoAdjustArg = CommandArgUtils.of(
-                        nextArg("Expected one of auto-adjust arguments"), AutoAdjustCommandArg.class
-                    );
-
-                    if (autoAdjustArg == null)
-                        throw new IllegalArgumentException("Expected one of auto-adjust arguments");
-
-                    if (autoAdjustArg == AutoAdjustCommandArg.ENABLE || autoAdjustArg == AutoAdjustCommandArg.DISABLE)
-                        baselineArgs.withEnable(autoAdjustArg == AutoAdjustCommandArg.ENABLE);
-
-                    if (autoAdjustArg == AutoAdjustCommandArg.TIMEOUT)
-                        baselineArgs.withSoftBaselineTimeout(nextLongArg("soft timeout"));
-                }
-                while (hasNextSubArg());
-
-                return baselineArgs.build();
-        }
-
-        return baselineArgs.build();
-    }
-
-    /**
-     * Parses and validates cache arguments.
-     *
-     * @return --cache subcommand arguments in case validation is successful.
-     */
-    private CacheArguments parseAndValidateCacheArgs() {
-        if (!hasNextSubArg()) {
-            throw new IllegalArgumentException("Arguments are expected for --cache subcommand, " +
-                "run --cache help for more info.");
-        }
-
-        CacheArguments cacheArgs = new CacheArguments();
-
-        String str = nextArg("").toLowerCase();
-
-        CacheCommand cmd = CacheCommand.of(str);
-
-        if (cmd == null)
-            cmd = CacheCommand.HELP;
-
-        cacheArgs.command(cmd);
-
-        switch (cmd) {
-            case HELP:
-                break;
-
-            case IDLE_VERIFY:
-                int idleVerifyArgsCnt = 5;
-
-                while (hasNextSubArg() && idleVerifyArgsCnt-- > 0) {
-                    String nextArg = nextArg("");
-
-                    IdleVerifyCommandArg arg = CommandArgUtils.of(nextArg, IdleVerifyCommandArg.class);
-
-                    if (arg == null)
-                        parseCacheNames(nextArg, cacheArgs);
-                    else {
-                        switch (arg) {
-                            case DUMP:
-                                cacheArgs.dump(true);
-
-                                break;
-
-                            case SKIP_ZEROS:
-                                cacheArgs.skipZeros(true);
-
-                                break;
-
-                            case CHECK_CRC:
-                                cacheArgs.idleCheckCrc(true);
-
-                                break;
-
-                            case CACHE_FILTER:
-                                String filter = nextArg("The cache filter should be specified. The following " +
-                                    "values can be used: " + Arrays.toString(CacheFilterEnum.values()) + '.');
-
-                                cacheArgs.setCacheFilterEnum(CacheFilterEnum.valueOf(filter.toUpperCase()));
-
-                                break;
-
-                            case EXCLUDE_CACHES:
-                                parseExcludeCacheNames(nextArg("Specify caches, which will be excluded."),
-                                    cacheArgs);
-
-                                break;
-                        }
-                    }
-                }
-                break;
-
-            case CONTENTION:
-                cacheArgs.minQueueSize(Integer.parseInt(nextArg("Min queue size expected")));
-
-                if (hasNextSubArg())
-                    cacheArgs.nodeId(UUID.fromString(nextArg("")));
-
-                if (hasNextSubArg())
-                    cacheArgs.maxPrint(Integer.parseInt(nextArg("")));
-                else
-                    cacheArgs.maxPrint(10);
-
-                break;
-
-            case VALIDATE_INDEXES: {
-                int argsCnt = 0;
-
-                while (hasNextSubArg() && argsCnt++ < 4) {
-                    String nextArg = nextArg("");
-
-                    ValidateIndexesCommandArg arg = CommandArgUtils.of(nextArg, ValidateIndexesCommandArg.class);
-
-                    if (arg == CHECK_FIRST || arg == CHECK_THROUGH) {
-                        if (!hasNextSubArg())
-                            throw new IllegalArgumentException("Numeric value for '" + nextArg + "' parameter expected.");
-
-                        int numVal;
-
-                        String numStr = nextArg("");
-
-                        try {
-                            numVal = Integer.parseInt(numStr);
-                        }
-                        catch (IllegalArgumentException e) {
-                            throw new IllegalArgumentException(
-                                "Not numeric value was passed for '" + nextArg + "' parameter: " + numStr
-                            );
-                        }
-
-                        if (numVal <= 0)
-                            throw new IllegalArgumentException("Value for '" + nextArg + "' property should be positive.");
-
-                        if (arg == CHECK_FIRST)
-                            cacheArgs.checkFirst(numVal);
-                        else
-                            cacheArgs.checkThrough(numVal);
-
-                        continue;
-                    }
-
-                    try {
-                        cacheArgs.nodeId(UUID.fromString(nextArg));
-
-                        continue;
-                    }
-                    catch (IllegalArgumentException ignored) {
-                        //No-op.
-                    }
-
-                    parseCacheNames(nextArg, cacheArgs);
-                }
-
-                break;
-            }
-
-            case DISTRIBUTION:
-                String nodeIdStr = nextArg("Node id expected or null");
-                if (!NULL.equals(nodeIdStr))
-                    cacheArgs.nodeId(UUID.fromString(nodeIdStr));
-
-                while (hasNextSubArg()) {
-                    String nextArg = nextArg("");
-
-                    DistributionCommandArg arg = CommandArgUtils.of(nextArg, DistributionCommandArg.class);
-
-                    if (arg == USER_ATTRIBUTES) {
-                        nextArg = nextArg("User attributes are expected to be separated by commas");
-
-                        Set<String> userAttrs = new HashSet<>();
-
-                        for (String userAttribute : nextArg.split(","))
-                            userAttrs.add(userAttribute.trim());
-
-                        cacheArgs.setUserAttributes(userAttrs);
-
-                        nextArg = (hasNextSubArg()) ? nextArg("") : null;
-
-                    }
-
-                    if (nextArg != null)
-                        parseCacheNames(nextArg, cacheArgs);
-                }
-
-                break;
-
-            case RESET_LOST_PARTITIONS:
-                parseCacheNames(nextArg("Cache name expected"), cacheArgs);
-
-                break;
-
-            case LIST:
-                cacheArgs.regex(nextArg("Regex is expected"));
-
-                VisorViewCacheCmd cacheCmd = CACHES;
-
-                OutputFormat outputFormat = SINGLE_LINE;
-
-                while (hasNextSubArg()) {
-                    String nextArg = nextArg("").toLowerCase();
-
-                    ListCommandArg arg = CommandArgUtils.of(nextArg, ListCommandArg.class);
-                    if (arg != null) {
-                        switch (arg) {
-                            case GROUP:
-                                cacheCmd = GROUPS;
-
-                                break;
-
-                            case SEQUENCE:
-                                cacheCmd = SEQ;
-
-                                break;
-
-                            case OUTPUT_FORMAT:
-                                String tmp2 = nextArg("output format must be defined!").toLowerCase();
-
-                                outputFormat = OutputFormat.fromConsoleName(tmp2);
-
-                                break;
-
-                            case CONFIG:
-                                cacheArgs.fullConfig(true);
-
-                                break;
-                        }
-                    }
-                    else
-                        cacheArgs.nodeId(UUID.fromString(nextArg));
-                }
-
-                cacheArgs.cacheCommand(cacheCmd);
-                cacheArgs.outputFormat(outputFormat);
-
-                break;
-
-            case FIND_AND_DELETE_GARBAGE: {
-                int argsCnt = 0;
-
-                while (hasNextSubArg() && argsCnt++ < 3) {
-                    String nextArg = nextArg("");
-
-                    FindAndDeleteGarbageArg arg = CommandArgUtils.of(nextArg, FindAndDeleteGarbageArg.class);
-
-                    if (arg == FindAndDeleteGarbageArg.DELETE) {
-                        cacheArgs.delete(true);
-
-                        continue;
-                    }
-
-                    try {
-                        cacheArgs.nodeId(UUID.fromString(nextArg));
-
-                        continue;
-                    }
-                    catch (IllegalArgumentException ignored) {
-                        //No-op.
-                    }
-
-                    cacheArgs.groups(parseCacheNames(nextArg));
-                }
-
-                break;
-            }
-
-            default:
-                throw new IllegalArgumentException("Unknown --cache subcommand " + cmd);
-        }
-
-        if (hasNextSubArg())
-            throw new IllegalArgumentException("Unexpected argument of --cache subcommand: " + peekNextArg());
-
-        return cacheArgs;
-    }
-
-    /**
-     * @return <code>true</code> if there's next argument for subcommand.
-     */
-    private boolean hasNextSubArg() {
-        return hasNextArg() && Command.of(peekNextArg()) == null && !AUX_COMMANDS.contains(peekNextArg());
-    }
-
-    /**
-     * @param cacheNames Cache names string.
-     * @param cacheArgs Cache args.
-     */
-    private void parseCacheNames(String cacheNames, CacheArguments cacheArgs) {
-        cacheArgs.caches(parseCacheNames(cacheNames));
-    }
-
-    /**
-     * @param cacheNames Cache names arg.
-     * @param cacheArgs Cache args.
-     */
-    private void parseExcludeCacheNames(String cacheNames, CacheArguments cacheArgs) {
-        cacheArgs.excludeCaches(parseCacheNames(cacheNames));
-    }
-
-    /**
-     * @param cacheNames Cache names string.
-     */
-    private Set<String> parseCacheNames(String cacheNames) {
-        String[] cacheNamesArr = cacheNames.split(",");
-
-        Set<String> cacheNamesSet = new HashSet<>();
-
-        for (String cacheName : cacheNamesArr) {
-            if (F.isEmpty(cacheName))
-                throw new IllegalArgumentException("Non-empty cache names expected.");
-
-            try {
-                Pattern.compile(cacheName);
-            }
-            catch (PatternSyntaxException e) {
-                throw new RuntimeException(format("Invalid cache name regexp '%s': %s", cacheName, e.getMessage()));
-            }
-
-            cacheNamesSet.add(cacheName.trim());
-        }
-
-        return cacheNamesSet;
-    }
-
-    /**
-     * Get ping param for grid client.
-     *
-     * @param nextArgErr Argument extraction error message.
-     * @param invalidErr Param validation error message.
-     */
-    private Long getPingParam(String nextArgErr, String invalidErr) {
-        String raw = nextArg(nextArgErr);
-
-        try {
-            long val = Long.valueOf(raw);
-
-            if (val <= 0)
-                throw new IllegalArgumentException(invalidErr + ": " + val);
-            else
-                return val;
-        }
-        catch (NumberFormatException ignored) {
-            throw new IllegalArgumentException(invalidErr + ": " + raw);
-        }
-    }
-
-    /**
-     * @return Transaction arguments.
-     */
-    private VisorTxTaskArg parseTransactionArguments() {
-        VisorTxProjection proj = null;
-
-        Integer limit = null;
-
-        VisorTxSortOrder sortOrder = null;
-
-        Long duration = null;
-
-        Integer size = null;
-
-        String lbRegex = null;
-
-        List<String> consistentIds = null;
-
-        VisorTxOperation op = VisorTxOperation.LIST;
-
-        String xid = null;
-
-        boolean end = false;
-
-        TxVerboseId txVerboseId = null;
-
-        do {
-            String str = peekNextArg();
-
-            if (str == null)
-                break;
-
-            switch (str) {
-                case TX_LIMIT:
-                    nextArg("");
-
-                    limit = (int)nextLongArg(TX_LIMIT);
-
-                    break;
-
-                case TX_ORDER:
-                    nextArg("");
-
-                    sortOrder = VisorTxSortOrder.valueOf(nextArg(TX_ORDER).toUpperCase());
-
-                    break;
-
-                case TX_SERVERS:
-                    nextArg("");
-
-                    proj = VisorTxProjection.SERVER;
-
-                    break;
-
-                case TX_CLIENTS:
-                    nextArg("");
-
-                    proj = VisorTxProjection.CLIENT;
-
-                    break;
-
-                case TX_NODES:
-                    nextArg("");
-
-                    consistentIds = getConsistentIds(nextArg(TX_NODES));
-
-                    break;
-
-                case TX_DURATION:
-                    nextArg("");
-
-                    duration = nextLongArg(TX_DURATION) * 1000L;
-
-                    break;
-
-                case TX_SIZE:
-                    nextArg("");
-
-                    size = (int)nextLongArg(TX_SIZE);
-
-                    break;
-
-                case TX_LABEL:
-                    nextArg("");
-
-                    lbRegex = nextArg(TX_LABEL);
-
-                    try {
-                        Pattern.compile(lbRegex);
-                    }
-                    catch (PatternSyntaxException ignored) {
-                        throw new IllegalArgumentException("Illegal regex syntax");
-                    }
-
-                    break;
-
-                case TX_XID:
-                    nextArg("");
-
-                    xid = nextArg(TX_XID);
-
-                    break;
-
-                case TX_KILL:
-                    nextArg("");
-
-                    op = VisorTxOperation.KILL;
-
-                    break;
-
-                case TX_INFO:
-                    nextArg("");
-
-                    op = VisorTxOperation.INFO;
-
-                    txVerboseId = TxVerboseId.fromString(nextArg(TX_INFO));
-
-                    break;
-
-                default:
-                    end = true;
-            }
-        }
-        while (!end);
-
-        if (proj != null && consistentIds != null)
-            throw new IllegalArgumentException("Projection can't be used together with list of consistent ids.");
-
-        return new VisorTxTaskArg(
-            op, limit, duration, size, null, proj, consistentIds, xid, lbRegex, sortOrder, txVerboseId);
-    }
-
-    /**
-     * @return Numeric value.
-     */
-    private long nextLongArg(String lb) {
-        String str = nextArg("Expecting " + lb);
-
-        try {
-            long val = Long.parseLong(str);
-
-            if (val < 0)
-                throw new IllegalArgumentException("Invalid value for " + lb + ": " + val);
-
-            return val;
-        }
-        catch (NumberFormatException ignored) {
-            throw new IllegalArgumentException("Invalid value for " + lb + ": " + str);
-        }
-    }
-
-    /**
-     * Requests password from console with message.
-     *
-     * @param msg Message.
-     * @return Password.
-     */
-    private char[] requestPasswordFromConsole(String msg) {
-        if (console == null)
-            throw new UnsupportedOperationException("Failed to securely read password (console is unavailable): " + msg);
-        else
-            return console.readPassword(msg);
-    }
-
-    /**
-     * Requests user data from console with message.
-     *
-     * @param msg Message.
-     * @return Input user data.
-     */
-    private String requestDataFromConsole(String msg) {
-        if (console != null)
-            return console.readLine(msg);
-        else {
-            Scanner scanner = new Scanner(System.in);
-
-            log(msg);
-
-            return scanner.nextLine();
-        }
-    }
-
-    /**
-     * Check if raw arg is command or option.
-     *
-     * @return {@code true} If raw arg is command, overwise {@code false}.
-     */
-    private boolean isCommandOrOption(String raw) {
-        return raw != null && raw.contains("--");
-    }
-
-    /**
-     * Maps VisorCacheConfiguration to key-value pairs.
-     *
-     * @param cfg Visor cache configuration.
-     * @return map of key-value pairs.
-     */
-    private Map<String, Object> mapToPairs(VisorCacheConfiguration cfg) {
-        Map<String, Object> params = new LinkedHashMap<>();
-
-        VisorCacheAffinityConfiguration affinityCfg = cfg.getAffinityConfiguration();
-        VisorCacheNearConfiguration nearCfg = cfg.getNearConfiguration();
-        VisorCacheRebalanceConfiguration rebalanceCfg = cfg.getRebalanceConfiguration();
-        VisorCacheEvictionConfiguration evictCfg = cfg.getEvictionConfiguration();
-        VisorCacheStoreConfiguration storeCfg = cfg.getStoreConfiguration();
-        VisorQueryConfiguration qryCfg = cfg.getQueryConfiguration();
-
-        params.put("Name", cfg.getName());
-        params.put("Group", cfg.getGroupName());
-        params.put("Dynamic Deployment ID", cfg.getDynamicDeploymentId());
-        params.put("System", cfg.isSystem());
-
-        params.put("Mode", cfg.getMode());
-        params.put("Atomicity Mode", cfg.getAtomicityMode());
-        params.put("Statistic Enabled", cfg.isStatisticsEnabled());
-        params.put("Management Enabled", cfg.isManagementEnabled());
-
-        params.put("On-heap cache enabled", cfg.isOnheapCacheEnabled());
-        params.put("Partition Loss Policy", cfg.getPartitionLossPolicy());
-        params.put("Query Parallelism", cfg.getQueryParallelism());
-        params.put("Copy On Read", cfg.isCopyOnRead());
-        params.put("Listener Configurations", cfg.getListenerConfigurations());
-        params.put("Load Previous Value", cfg.isLoadPreviousValue());
-        params.put("Memory Policy Name", cfg.getMemoryPolicyName());
-        params.put("Node Filter", cfg.getNodeFilter());
-        params.put("Read From Backup", cfg.isReadFromBackup());
-        params.put("Topology Validator", cfg.getTopologyValidator());
-
-        params.put("Time To Live Eager Flag", cfg.isEagerTtl());
-
-        params.put("Write Synchronization Mode", cfg.getWriteSynchronizationMode());
-        params.put("Invalidate", cfg.isInvalidate());
-
-        params.put("Affinity Function", affinityCfg.getFunction());
-        params.put("Affinity Backups", affinityCfg.getPartitionedBackups());
-        params.put("Affinity Partitions", affinityCfg.getPartitions());
-        params.put("Affinity Exclude Neighbors", affinityCfg.isExcludeNeighbors());
-        params.put("Affinity Mapper", affinityCfg.getMapper());
-
-        params.put("Rebalance Mode", rebalanceCfg.getMode());
-        params.put("Rebalance Batch Size", rebalanceCfg.getBatchSize());
-        params.put("Rebalance Timeout", rebalanceCfg.getTimeout());
-        params.put("Rebalance Delay", rebalanceCfg.getPartitionedDelay());
-        params.put("Time Between Rebalance Messages", rebalanceCfg.getThrottle());
-        params.put("Rebalance Batches Count", rebalanceCfg.getBatchesPrefetchCnt());
-        params.put("Rebalance Cache Order", rebalanceCfg.getRebalanceOrder());
-
-        params.put("Eviction Policy Enabled", (evictCfg.getPolicy() != null));
-        params.put("Eviction Policy Factory", evictCfg.getPolicy());
-        params.put("Eviction Policy Max Size", evictCfg.getPolicyMaxSize());
-        params.put("Eviction Filter", evictCfg.getFilter());
-
-        params.put("Near Cache Enabled", nearCfg.isNearEnabled());
-        params.put("Near Start Size", nearCfg.getNearStartSize());
-        params.put("Near Eviction Policy Factory", nearCfg.getNearEvictPolicy());
-        params.put("Near Eviction Policy Max Size", nearCfg.getNearEvictMaxSize());
-
-        params.put("Default Lock Timeout", cfg.getDefaultLockTimeout());
-        params.put("Query Entities", cfg.getQueryEntities());
-        params.put("Cache Interceptor", cfg.getInterceptor());
-
-        params.put("Store Enabled", storeCfg.isEnabled());
-        params.put("Store Class", storeCfg.getStore());
-        params.put("Store Factory Class", storeCfg.getStoreFactory());
-        params.put("Store Keep Binary", storeCfg.isStoreKeepBinary());
-        params.put("Store Read Through", storeCfg.isReadThrough());
-        params.put("Store Write Through", storeCfg.isWriteThrough());
-        params.put("Store Write Coalescing", storeCfg.getWriteBehindCoalescing());
-
-        params.put("Write-Behind Enabled", storeCfg.isWriteBehindEnabled());
-        params.put("Write-Behind Flush Size", storeCfg.getFlushSize());
-        params.put("Write-Behind Frequency", storeCfg.getFlushFrequency());
-        params.put("Write-Behind Flush Threads Count", storeCfg.getFlushThreadCount());
-        params.put("Write-Behind Batch Size", storeCfg.getBatchSize());
-
-        params.put("Concurrent Asynchronous Operations Number", cfg.getMaxConcurrentAsyncOperations());
-
-        params.put("Loader Factory Class Name", cfg.getLoaderFactory());
-        params.put("Writer Factory Class Name", cfg.getWriterFactory());
-        params.put("Expiry Policy Factory Class Name", cfg.getExpiryPolicyFactory());
-
-        params.put("Query Execution Time Threshold", qryCfg.getLongQueryWarningTimeout());
-        params.put("Query Escaped Names", qryCfg.isSqlEscapeAll());
-        params.put("Query SQL Schema", qryCfg.getSqlSchema());
-        params.put("Query SQL functions", qryCfg.getSqlFunctionClasses());
-        params.put("Query Indexed Types", qryCfg.getIndexedTypes());
-        params.put("Maximum payload size for offheap indexes", cfg.getSqlIndexMaxInlineSize());
-        params.put("Query Metrics History Size", cfg.getQueryDetailMetricsSize());
-
-        return params;
-    }
-
-    /**
-     * Split string into items.
-     *
-     * @param s String to process.
-     * @param delim Delimiter.
-     * @return List with items.
-     */
-    private List<String> split(String s, String delim) {
-        if (F.isEmpty(s))
-            return Collections.emptyList();
+    /**
+     * Split string into items.
+     *
+     * @param s String to process.
+     * @param delim Delimiter.
+     * @return List with items.
+     */
+    private static List<String> split(String s, String delim) {
+        if (F.isEmpty(s))
+            return Collections.emptyList();
 
         return Arrays.stream(s.split(delim))
             .map(String::trim)
@@ -3082,291 +420,42 @@ public class CommandHandler {
             .collect(Collectors.toList());
     }
 
-    /**
-     * @return Transaction command options.
-     */
-    private String[] getTxOptions() {
-        List<String> list = new ArrayList<>();
-
-        list.add(op(TX_XID, "XID"));
-        list.add(op(TX_DURATION, "SECONDS"));
-        list.add(op(TX_SIZE, "SIZE"));
-        list.add(op(TX_LABEL, "PATTERN_REGEX"));
-        list.add(op(or(TX_SERVERS, TX_CLIENTS)));
-        list.add(op(TX_NODES, "consistentId1[,consistentId2,....,consistentIdN]"));
-        list.add(op(TX_LIMIT, "NUMBER"));
-        list.add(op(TX_ORDER, or(VisorTxSortOrder.values())));
-        list.add(op(TX_KILL));
-        list.add(op(TX_INFO));
-        list.add(op(CMD_AUTO_CONFIRMATION));
-
-        return list.toArray(new String[list.size()]);
-    }
-
     /** */
     private void printHelp() {
-        final String constistIds = "consistentId1[,consistentId2,....,consistentIdN]";
-
-        log("Control.sh is used to execute admin commands on cluster or get common cluster info. The command has the following syntax:");
-        nl();
-
-        log(i(j(" ", UTILITY_NAME_WITH_COMMON_OPTIONS, op("command"), "<command_parameters>")));
-        nl();
-        nl();
-
-        log("This utility can do the following commands:");
-
-        usage(i("Activate cluster:"), ACTIVATE);
-        usage(i("Deactivate cluster:"), DEACTIVATE, op(CMD_AUTO_CONFIRMATION));
-        usage(i("Print current cluster state:"), STATE);
-        usage(i("Print cluster baseline topology:"), BASELINE);
-        usage(i("Add nodes into baseline topology:"), BASELINE, BaselineCommand.ADD.text(), constistIds, op(CMD_AUTO_CONFIRMATION));
-        usage(i("Remove nodes from baseline topology:"), BASELINE, BaselineCommand.REMOVE.text(), constistIds, op(CMD_AUTO_CONFIRMATION));
-        usage(i("Set baseline topology:"), BASELINE, BaselineCommand.SET.text(), constistIds, op(CMD_AUTO_CONFIRMATION));
-        usage(i("Set baseline topology based on version:"), BASELINE, BaselineCommand.VERSION.text() + " topologyVersion", op(CMD_AUTO_CONFIRMATION));
-        usage(i("Set baseline autoadjustment settings:"), BASELINE, BaselineCommand.AUTO_ADJUST.text(), "disable|enable timeout <timeoutValue>", op(CMD_AUTO_CONFIRMATION));
-        usage(i("List or kill transactions:"), TX, getTxOptions());
-        usage(i("Print detailed information (topology and key lock ownership) about specific transaction:"),
-            TX, TX_INFO, or("<TX identifier as GridCacheVersion [topVer=..., order=..., nodeOrder=...] " +
-                "(can be found in logs)>", "<TX identifier as UUID (can be retrieved via --tx command)>"));
-
-        if (enableExperimental) {
-            usage(i("Print absolute paths of unused archived wal segments on each node:"), WAL, WAL_PRINT, "[consistentId1,consistentId2,....,consistentIdN]");
-            usage(i("Delete unused archived wal segments on each node:"), WAL, WAL_DELETE, "[consistentId1,consistentId2,....,consistentIdN]", op(CMD_AUTO_CONFIRMATION));
-        }
-
-        log(i("View caches information in a cluster. For more details type:"));
-        log(i(j(" ", UTILITY_NAME, CACHE, HELP), 2));
-        nl();
-
-        log("By default commands affecting the cluster require interactive confirmation.");
-        log("Use " + CMD_AUTO_CONFIRMATION + " option to disable it.");
-        nl();
-
-        log("Default values:");
-        log(i("HOST_OR_IP=" + DFLT_HOST, 2));
-        log(i("PORT=" + DFLT_PORT, 2));
-        log(i("PING_INTERVAL=" + DFLT_PING_INTERVAL, 2));
-        log(i("PING_TIMEOUT=" + DFLT_PING_TIMEOUT, 2));
-        log(i("SSL_PROTOCOL=" + SslContextFactory.DFLT_SSL_PROTOCOL, 2));
-        log(i("SSL_KEY_ALGORITHM=" + SslContextFactory.DFLT_KEY_ALGORITHM, 2));
-        log(i("KEYSTORE_TYPE=" + SslContextFactory.DFLT_STORE_TYPE, 2));
-        log(i("TRUSTSTORE_TYPE=" + SslContextFactory.DFLT_STORE_TYPE, 2));
-
-        nl();
-
-        log("Exit codes:");
-        log(i(EXIT_CODE_OK + " - successful execution.", 2));
-        log(i(EXIT_CODE_INVALID_ARGUMENTS + " - invalid arguments.", 2));
-        log(i(EXIT_CODE_CONNECTION_FAILED + " - connection failed.", 2));
-        log(i(ERR_AUTHENTICATION_FAILED + " - authentication failed.", 2));
-        log(i(EXIT_CODE_UNEXPECTED_ERROR + " - unexpected error.", 2));
-    }
-
-    /**
-     * Parse and execute command.
-     *
-     * @param rawArgs Arguments to parse and execute.
-     * @return Exit code.
-     */
-    public int execute(List<String> rawArgs) {
-        log("Control utility [ver. " + ACK_VER_STR + "]");
-        log(COPYRIGHT);
-        log("User: " + System.getProperty("user.name"));
-        log("Time: " + LocalDateTime.now());
-        log(DELIM);
-
-        try {
-            if (F.isEmpty(rawArgs) || (rawArgs.size() == 1 && CMD_HELP.equalsIgnoreCase(rawArgs.get(0)))) {
-                printHelp();
-
-                return EXIT_CODE_OK;
-            }
-
-            Arguments args = parseAndValidate(rawArgs);
-
-            if (args.command() == CACHE && args.cacheArgs().command() == HELP) {
-                printCacheHelp();
-
-                return EXIT_CODE_OK;
-            }
-
-            if (!args.autoConfirmation() && !confirm(args)) {
-                log("Operation cancelled.");
-
-                return EXIT_CODE_OK;
-            }
-
-            clientCfg = new GridClientConfiguration();
-
-            clientCfg.setPingInterval(args.pingInterval());
-
-            clientCfg.setPingTimeout(args.pingTimeout());
-
-            clientCfg.setServers(Collections.singletonList(args.host() + ":" + args.port()));
-
-            boolean tryConnectAgain = true;
-
-            boolean suppliedAuth = !F.isEmpty(args.getUserName()) && !F.isEmpty(args.getPassword());
-
-            int tryConnectMaxCount = 3;
-
-            while (tryConnectAgain) {
-                tryConnectAgain = false;
-
-                if (!F.isEmpty(args.getUserName())) {
-                    SecurityCredentialsProvider securityCredential = clientCfg.getSecurityCredentialsProvider();
-
-                    if (securityCredential == null) {
-                        securityCredential = new SecurityCredentialsBasicProvider(
-                            new SecurityCredentials(args.getUserName(), args.getPassword()));
-
-                        clientCfg.setSecurityCredentialsProvider(securityCredential);
-                    }
-                    final SecurityCredentials credential = securityCredential.credentials();
-                    credential.setLogin(args.getUserName());
-                    credential.setPassword(args.getPassword());
-                }
-
-                if (!F.isEmpty(args.sslKeyStorePath())) {
-                    GridSslBasicContextFactory factory = new GridSslBasicContextFactory();
-
-                    List<String> sslProtocols = split(args.sslProtocol(), ",");
-
-                    String sslProtocol = F.isEmpty(sslProtocols) ? DFLT_SSL_PROTOCOL : sslProtocols.get(0);
-
-                    factory.setProtocol(sslProtocol);
-                    factory.setKeyAlgorithm(args.sslKeyAlgorithm());
-
-                    if (sslProtocols.size() > 1)
-                        factory.setProtocols(sslProtocols);
-
-                    factory.setCipherSuites(split(args.getSslCipherSuites(), ","));
-
-                    factory.setKeyStoreFilePath(args.sslKeyStorePath());
-
-                    if (args.sslKeyStorePassword() != null)
-                        factory.setKeyStorePassword(args.sslKeyStorePassword());
-                    else
-                        factory.setKeyStorePassword(requestPasswordFromConsole("SSL keystore password: "));
-
-                    factory.setKeyStoreType(args.sslKeyStoreType());
-
-                    if (F.isEmpty(args.sslTrustStorePath()))
-                        factory.setTrustManagers(GridSslBasicContextFactory.getDisabledTrustManager());
-                    else {
-                        factory.setTrustStoreFilePath(args.sslTrustStorePath());
-
-                        if (args.sslTrustStorePassword() != null)
-                            factory.setTrustStorePassword(args.sslTrustStorePassword());
-                        else
-                            factory.setTrustStorePassword(requestPasswordFromConsole("SSL truststore password: "));
-
-                        factory.setTrustStoreType(args.sslTrustStoreType());
-                    }
-
-                    clientCfg.setSslContextFactory(factory);
-                }
-
-                try (GridClient client = GridClientFactory.start(clientCfg)) {
-                    // If connection is unsuccessful, fail before doing any operations:
-                    if (!client.connected())
-                        client.throwLastError();
-
-                    switch (args.command()) {
-                        case ACTIVATE:
-                            activate(client);
-
-                            break;
-
-                        case DEACTIVATE:
-                            deactivate(client);
-
-                            break;
-
-                        case STATE:
-                            state(client);
-
-                            break;
-
-                        case BASELINE:
-                            baseline(client, args.baselineArguments());
-
-                            break;
-
-                        case TX:
-                            transactions(client, args.transactionArguments());
-
-                            break;
-
-                        case CACHE:
-                            lastOperationRes = cache(client, args.cacheArgs());
-
-                            break;
-
-                        case WAL:
-                            wal(client, args.walAction(), args.walArguments());
-
-                            break;
-                    }
-                }
-                catch (Throwable e) {
-                    if (tryConnectMaxCount > 0 && isAuthError(e)) {
-                        log(suppliedAuth ?
-                            "Authentication error, please try again." : "This cluster requires authentication.");
-
-                        if (F.isEmpty(args.getUserName()))
-                            args.setUserName(requestDataFromConsole("user: "));
-
-                        args.setPassword(new String(requestPasswordFromConsole("password: ")));
-
-                        suppliedAuth = true;
-
-                        tryConnectAgain = true;
-
-                        tryConnectMaxCount--;
-                    }
-                    else {
-                        if (tryConnectMaxCount == 0)
-                            throw new GridClientAuthenticationException("Authentication error, maximum number of " +
-                                "retries exceeded");
-
-                        throw e;
-                    }
-                }
-            }
-            return EXIT_CODE_OK;
-        }
-        catch (IllegalArgumentException e) {
-            return error(EXIT_CODE_INVALID_ARGUMENTS, "Check arguments.", e);
-        }
-        catch (Throwable e) {
-            if (isAuthError(e))
-                return error(ERR_AUTHENTICATION_FAILED, "Authentication error.", e);
-
-            if (isConnectionError(e))
-                return error(EXIT_CODE_CONNECTION_FAILED, "Connection to cluster failed.", e);
-
-            return error(EXIT_CODE_UNEXPECTED_ERROR, "", e);
-        }
-    }
-
-    /**
-     * @param args Arguments to parse and apply.
-     */
-    public static void main(String[] args) {
-        CommandHandler hnd = new CommandHandler();
-
-        System.exit(hnd.execute(Arrays.asList(args)));
-    }
-
-    /**
-     * Used for tests.
-     *
-     * @return Last operation result;
-     */
-    public <T> T getLastOperationResult() {
-        return (T)lastOperationRes;
+        logger.log("Control.sh is used to execute admin commands on cluster or get common cluster info. " +
+            "The command has the following syntax:");
+        logger.nl();
+
+        logger.logWithIndent(CommandLogger.join(" ", CommandLogger.join(" ", UTILITY_NAME, CommandLogger.join(" ", getCommonOptions())),
+            optional("command"), "<command_parameters>"));
+        logger.nl();
+        logger.nl();
+
+        logger.log("This utility can do the following commands:");
+
+        Arrays.stream(CommandList.values()).forEach(c -> c.command().printUsage(logger));
+
+        logger.log("By default commands affecting the cluster require interactive confirmation.");
+        logger.log("Use " + CMD_AUTO_CONFIRMATION + " option to disable it.");
+        logger.nl();
+
+        logger.log("Default values:");
+        logger.logWithIndent("HOST_OR_IP=" + DFLT_HOST, 2);
+        logger.logWithIndent("PORT=" + DFLT_PORT, 2);
+        logger.logWithIndent("PING_INTERVAL=" + DFLT_PING_INTERVAL, 2);
+        logger.logWithIndent("PING_TIMEOUT=" + DFLT_PING_TIMEOUT, 2);
+        logger.logWithIndent("SSL_PROTOCOL=" + SslContextFactory.DFLT_SSL_PROTOCOL, 2);
+        logger.logWithIndent("SSL_KEY_ALGORITHM=" + SslContextFactory.DFLT_KEY_ALGORITHM, 2);
+        logger.logWithIndent("KEYSTORE_TYPE=" + SslContextFactory.DFLT_STORE_TYPE, 2);
+        logger.logWithIndent("TRUSTSTORE_TYPE=" + SslContextFactory.DFLT_STORE_TYPE, 2);
+
+        logger.nl();
+
+        logger.log("Exit codes:");
+        logger.logWithIndent(EXIT_CODE_OK + " - successful execution.", 2);
+        logger.logWithIndent(EXIT_CODE_INVALID_ARGUMENTS + " - invalid arguments.", 2);
+        logger.logWithIndent(EXIT_CODE_CONNECTION_FAILED + " - connection failed.", 2);
+        logger.logWithIndent(ERR_AUTHENTICATION_FAILED + " - authentication failed.", 2);
+        logger.logWithIndent(EXIT_CODE_UNEXPECTED_ERROR + " - unexpected error.", 2);
     }
 }
-
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/Command.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandList.java
similarity index 62%
copy from modules/core/src/main/java/org/apache/ignite/internal/commandline/Command.java
copy to modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandList.java
index 8ed4dac..2b97591 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/Command.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandList.java
@@ -17,50 +17,57 @@
 
 package org.apache.ignite.internal.commandline;
 
+import org.apache.ignite.internal.commandline.cache.CacheCommands;
+
 /**
- * Command.
+ * High-level commands.
  */
-public enum Command {
+public enum CommandList {
     /** */
-    ACTIVATE("--activate"),
+    ACTIVATE("--activate", new ActivateCommand()),
 
     /** */
-    DEACTIVATE("--deactivate"),
+    DEACTIVATE("--deactivate", new DeactivateCommand()),
 
     /** */
-    STATE("--state"),
+    STATE("--state", new StateCommand()),
 
     /** */
-    BASELINE("--baseline"),
+    BASELINE("--baseline", new BaselineCommand()),
 
     /** */
-    TX("--tx"),
+    TX("--tx", new TxCommands()),
 
     /** */
-    CACHE("--cache"),
+    CACHE("--cache", new CacheCommands()),
 
     /** */
-    WAL("--wal");
+    WAL("--wal", new WalCommands());
 
     /** Private values copy so there's no need in cloning it every time. */
-    private static final Command[] VALUES = Command.values();
+    private static final CommandList[] VALUES = CommandList.values();
 
     /** */
     private final String text;
 
+    /** Command implementation. */
+    private final Command command;
+
     /**
      * @param text Text.
+     * @param command Command implementation.
      */
-    Command(String text) {
+    CommandList(String text, Command command) {
         this.text = text;
+        this.command = command;
     }
 
     /**
      * @param text Command text.
      * @return Command for the text.
      */
-    public static Command of(String text) {
-        for (Command cmd : VALUES) {
+    public static CommandList of(String text) {
+        for (CommandList cmd : VALUES) {
             if (cmd.text().equalsIgnoreCase(text))
                 return cmd;
         }
@@ -75,6 +82,13 @@ public enum Command {
         return text;
     }
 
+    /**
+     * @return Command implementation.
+     */
+    public Command command() {
+        return command;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() { 
         return text; 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandLogger.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandLogger.java
new file mode 100644
index 0000000..0ea9a5e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandLogger.java
@@ -0,0 +1,228 @@
+/*
+ * 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;
+
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.SB;
+
+/**
+ * Logger used in command handler.
+ */
+public class CommandLogger {
+    /** Indent for help output. */
+    public static final String INDENT = "  ";
+
+    /**
+     * Join input parameters with specified {@code delimeter} between them.
+     *
+     * @param delimeter Specified delimeter.
+     * @param params Other input parameter.
+     * @return Joined paramaters with specified {@code delimeter}.
+     */
+    public static String join(String delimeter, Object... params) {
+        return join(new SB(), "", delimeter, params).toString();
+    }
+
+    /**
+     * Join input parameters with specified {@code delimeter} between them and append to the end {@code delimeter}.
+     *
+     * @param sb Specified string builder.
+     * @param sbDelimeter Delimeter between {@code sb} and appended {@code param}.
+     * @param delimeter Specified delimeter.
+     * @param params Other input parameter.
+     * @return SB with appended to the end joined paramaters with specified {@code delimeter}.
+     */
+    public static SB join(SB sb, String sbDelimeter, String delimeter, Object... params) {
+        if (!F.isEmpty(params)) {
+            sb.a(sbDelimeter);
+
+            for (Object par : params)
+                sb.a(par).a(delimeter);
+
+            sb.setLength(sb.length() - delimeter.length());
+        }
+
+        return sb;
+    }
+
+
+    /**
+     * Join input parameters with space and wrap optional braces {@code []}.
+     *
+     * @param params Other input parameter.
+     * @return Joined parameters wrapped optional braces.
+     */
+    public static String optional(Object... params) {
+        return join(new SB(), "[", " ", params).a("]").toString();
+    }
+
+    /**
+     * Concatenates input parameters to single string with OR delimiter {@code |}.
+     *
+     * @param params Remaining parameters.
+     * @return Concatenated string.
+     */
+    public static String or(Object... params) {
+        return join("|", params);
+    }
+
+    /**
+     * Join input parameters with space and wrap grouping braces {@code ()}.
+     *
+     * @param params Input parameter.
+     * @return Joined parameters wrapped grouped braces.
+     */
+    public static String grouped(Object... params) {
+        return join(new SB(), "(", " ", params).a(")").toString();
+    }
+
+    /**
+     * Output specified string to console.
+     *
+     * @param s String to output.
+     */
+    public void log(String s) {
+        System.out.println(s);
+    }
+
+    /**
+     *
+     * Output specified string to console.
+     *
+     * @param s String to output.
+     */
+    public void logWithIndent(Object s) {
+        log(indent(s));
+    }
+
+    /**
+     *
+     * Output specified string to console.
+     *
+     * @param s String to output.
+     */
+    public void logWithIndent(Object s, int indentCnt) {
+        log(indent(s), indentCnt);
+    }
+
+    /**
+     * Adds indent to begin of object's string representation.
+     *
+     * @param o Input object.
+     * @return Indented string.
+     */
+    public static String indent(Object o) {
+        return indent(o, 1);
+    }
+
+
+    /**
+     * Adds specified indents to begin of object's string representation.
+     *
+     * @param o Input object.
+     * @param indentCnt Number of indents.
+     * @return Indented string.
+     */
+    public static String indent(Object o, int indentCnt) {
+        assert indentCnt >= 0;
+
+        String s = o == null ? null : o.toString();
+
+        switch (indentCnt) {
+            case 0:
+                return s;
+
+            case 1:
+                return INDENT + s;
+
+            default:
+                int sLen = s == null ? 4 : s.length();
+
+                SB sb = new SB(sLen + indentCnt * INDENT.length());
+
+                for (int i = 0; i < indentCnt; i++)
+                    sb.a(INDENT);
+
+                return sb.a(s).toString();
+        }
+    }
+
+    /**
+     * Format and output specified string to console.
+     *
+     * @param format A format string as described in Format string syntax.
+     * @param args Arguments referenced by the format specifiers in the format string.
+     */
+    public void log(String format, Object... args) {
+        System.out.printf(format, args);
+    }
+
+    /**
+     * Output empty line.
+     */
+    public void nl() {
+        System.out.println();
+    }
+
+    /**
+     * Print error to console.
+     *
+
+     * @param s Optional message.
+     * @param e Error to print.
+     */
+    public void error(String s, Throwable e) {
+        if (!F.isEmpty(s))
+            log(s);
+
+        String msg = e.getMessage();
+
+        if (F.isEmpty(msg))
+            msg = e.getClass().getName();
+
+        if (msg.startsWith("Failed to handle request")) {
+            int p = msg.indexOf("err=");
+
+            msg = msg.substring(p + 4, msg.length() - 1);
+        }
+
+        log("Error: " + msg);
+    }
+
+
+
+    public boolean printErrors(Map<UUID, Exception> exceptions, String s) {
+        if (!F.isEmpty(exceptions)) {
+            log(s);
+
+            for (Map.Entry<UUID, Exception> e : exceptions.entrySet()) {
+                logWithIndent("Node ID: " + e.getKey());
+
+                logWithIndent("Exception message:");
+                logWithIndent(e.getValue().getMessage(), 2);
+                nl();
+            }
+
+            return true;
+        }
+
+        return false;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommonArgParser.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommonArgParser.java
new file mode 100644
index 0000000..44270d4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommonArgParser.java
@@ -0,0 +1,338 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.ignite.ssl.SslContextFactory;
+
+import static org.apache.ignite.internal.client.GridClientConfiguration.DFLT_PING_INTERVAL;
+import static org.apache.ignite.internal.client.GridClientConfiguration.DFLT_PING_TIMEOUT;
+import static org.apache.ignite.internal.commandline.CommandLogger.optional;
+import static org.apache.ignite.internal.commandline.TaskExecutor.DFLT_HOST;
+import static org.apache.ignite.internal.commandline.TaskExecutor.DFLT_PORT;
+import static org.apache.ignite.ssl.SslContextFactory.DFLT_SSL_PROTOCOL;
+
+/**
+ * Common argument parser.
+ * Also would parse high-level command and delegate parsing for its argument to the command.
+ */
+public class CommonArgParser {
+    /** */
+    private final CommandLogger logger;
+
+    /** */
+    static final String CMD_HOST = "--host";
+
+    /** */
+    static final String CMD_PORT = "--port";
+
+    /** */
+    static final String CMD_PASSWORD = "--password";
+
+    /** */
+    static final String CMD_USER = "--user";
+
+    /** Option is used for auto confirmation. */
+    static final String CMD_AUTO_CONFIRMATION = "--yes";
+
+    /** */
+    static final String CMD_PING_INTERVAL = "--ping-interval";
+
+    /** */
+    static final String CMD_PING_TIMEOUT = "--ping-timeout";
+
+    // SSL configuration section
+
+    /** */
+    static final String CMD_SSL_PROTOCOL = "--ssl-protocol";
+
+    /** */
+    static final String CMD_SSL_KEY_ALGORITHM = "--ssl-key-algorithm";
+
+    /** */
+    static final String CMD_SSL_CIPHER_SUITES = "--ssl-cipher-suites";
+
+    /** */
+    static final String CMD_KEYSTORE = "--keystore";
+
+    /** */
+    static final String CMD_KEYSTORE_PASSWORD = "--keystore-password";
+
+    /** */
+    static final String CMD_KEYSTORE_TYPE = "--keystore-type";
+
+    /** */
+    static final String CMD_TRUSTSTORE = "--truststore";
+
+    /** */
+    static final String CMD_TRUSTSTORE_PASSWORD = "--truststore-password";
+
+    /** */
+    static final String CMD_TRUSTSTORE_TYPE = "--truststore-type";
+
+    /** List of optional auxiliary commands. */
+    private static final Set<String> AUX_COMMANDS = new HashSet<>();
+
+    static {
+        AUX_COMMANDS.add(CMD_HOST);
+        AUX_COMMANDS.add(CMD_PORT);
+
+        AUX_COMMANDS.add(CMD_PASSWORD);
+        AUX_COMMANDS.add(CMD_USER);
+
+        AUX_COMMANDS.add(CMD_AUTO_CONFIRMATION);
+
+        AUX_COMMANDS.add(CMD_PING_INTERVAL);
+        AUX_COMMANDS.add(CMD_PING_TIMEOUT);
+
+        AUX_COMMANDS.add(CMD_SSL_PROTOCOL);
+        AUX_COMMANDS.add(CMD_SSL_KEY_ALGORITHM);
+        AUX_COMMANDS.add(CMD_SSL_CIPHER_SUITES);
+
+        AUX_COMMANDS.add(CMD_KEYSTORE);
+        AUX_COMMANDS.add(CMD_KEYSTORE_PASSWORD);
+        AUX_COMMANDS.add(CMD_KEYSTORE_TYPE);
+
+        AUX_COMMANDS.add(CMD_TRUSTSTORE);
+        AUX_COMMANDS.add(CMD_TRUSTSTORE_PASSWORD);
+        AUX_COMMANDS.add(CMD_TRUSTSTORE_TYPE);
+    }
+
+    /**
+     * @param logger Logger.
+     */
+    public CommonArgParser(CommandLogger logger) {
+        this.logger = logger;
+    }
+
+    /**
+     * Creates list of common utility options.
+     *
+     * @return Array of common utility options.
+     */
+    public static String[] getCommonOptions() {
+        List<String> list = new ArrayList<>(32);
+
+        list.add(optional(CMD_HOST, "HOST_OR_IP"));
+        list.add(optional(CMD_PORT, "PORT"));
+        list.add(optional(CMD_USER, "USER"));
+        list.add(optional(CMD_PASSWORD, "PASSWORD"));
+        list.add(optional(CMD_PING_INTERVAL, "PING_INTERVAL"));
+        list.add(optional(CMD_PING_TIMEOUT, "PING_TIMEOUT"));
+        list.add(optional(CMD_SSL_PROTOCOL, "SSL_PROTOCOL[, SSL_PROTOCOL_2, ..., SSL_PROTOCOL_N]"));
+        list.add(optional(CMD_SSL_CIPHER_SUITES, "SSL_CIPHER_1[, SSL_CIPHER_2, ..., SSL_CIPHER_N]"));
+        list.add(optional(CMD_SSL_KEY_ALGORITHM, "SSL_KEY_ALGORITHM"));
+        list.add(optional(CMD_KEYSTORE_TYPE, "KEYSTORE_TYPE"));
+        list.add(optional(CMD_KEYSTORE, "KEYSTORE_PATH"));
+        list.add(optional(CMD_KEYSTORE_PASSWORD, "KEYSTORE_PASSWORD"));
+        list.add(optional(CMD_TRUSTSTORE_TYPE, "TRUSTSTORE_TYPE"));
+        list.add(optional(CMD_TRUSTSTORE, "TRUSTSTORE_PATH"));
+        list.add(optional(CMD_TRUSTSTORE_PASSWORD, "TRUSTSTORE_PASSWORD"));
+
+        return list.toArray(new String[0]);
+    }
+
+    /**
+     * Parses and validates arguments.
+     *
+     * @param rawArgIter Iterator of arguments.
+     * @return Arguments bean.
+     * @throws IllegalArgumentException In case arguments aren't valid.
+     */
+    ConnectionAndSslParameters parseAndValidate(Iterator<String> rawArgIter) {
+        String host = DFLT_HOST;
+
+        String port = DFLT_PORT;
+
+        String user = null;
+
+        String pwd = null;
+
+        Long pingInterval = DFLT_PING_INTERVAL;
+
+        Long pingTimeout = DFLT_PING_TIMEOUT;
+
+        boolean autoConfirmation = false;
+
+        String sslProtocol = DFLT_SSL_PROTOCOL;
+
+        String sslCipherSuites = "";
+
+        String sslKeyAlgorithm = SslContextFactory.DFLT_KEY_ALGORITHM;
+
+        String sslKeyStoreType = SslContextFactory.DFLT_STORE_TYPE;
+
+        String sslKeyStorePath = null;
+
+        char sslKeyStorePassword[] = null;
+
+        String sslTrustStoreType = SslContextFactory.DFLT_STORE_TYPE;
+
+        String sslTrustStorePath = null;
+
+        char sslTrustStorePassword[] = null;
+
+        CommandArgIterator argIter = new CommandArgIterator(rawArgIter, AUX_COMMANDS);
+
+        CommandList command = null;
+
+        while (argIter.hasNextArg()) {
+            String str = argIter.nextArg("").toLowerCase();
+
+            CommandList cmd = CommandList.of(str);
+
+            if (cmd != null) {
+                if (command != null)
+                    throw new IllegalArgumentException("Only one action can be specified, but found at least two:" +
+                        cmd.toString() + ", " + command.toString());
+
+                cmd.command().parseArguments(argIter);
+
+                command = cmd;
+            }
+            else {
+
+                switch (str) {
+                    case CMD_HOST:
+                        host = argIter.nextArg("Expected host name");
+
+                        break;
+
+                    case CMD_PORT:
+                        port = argIter.nextArg("Expected port number");
+
+                        try {
+                            int p = Integer.parseInt(port);
+
+                            if (p <= 0 || p > 65535)
+                                throw new IllegalArgumentException("Invalid value for port: " + port);
+                        }
+                        catch (NumberFormatException ignored) {
+                            throw new IllegalArgumentException("Invalid value for port: " + port);
+                        }
+
+                        break;
+
+                    case CMD_PING_INTERVAL:
+                        pingInterval = argIter.nextLongArg("ping interval");
+
+                        break;
+
+                    case CMD_PING_TIMEOUT:
+                        pingTimeout = argIter.nextLongArg("ping timeout");
+
+                        break;
+
+                    case CMD_USER:
+                        user = argIter.nextArg("Expected user name");
+
+                        break;
+
+                    case CMD_PASSWORD:
+                        pwd = argIter.nextArg("Expected password");
+
+                        logger.log(securityWarningMessage(CMD_PASSWORD));
+
+                        break;
+
+                    case CMD_SSL_PROTOCOL:
+                        sslProtocol = argIter.nextArg("Expected SSL protocol");
+
+                        break;
+
+                    case CMD_SSL_CIPHER_SUITES:
+                        sslCipherSuites = argIter.nextArg("Expected SSL cipher suites");
+
+                        break;
+
+                    case CMD_SSL_KEY_ALGORITHM:
+                        sslKeyAlgorithm = argIter.nextArg("Expected SSL key algorithm");
+
+                        break;
+
+                    case CMD_KEYSTORE:
+                        sslKeyStorePath = argIter.nextArg("Expected SSL key store path");
+
+                        break;
+
+                    case CMD_KEYSTORE_PASSWORD:
+                        sslKeyStorePassword = argIter.nextArg("Expected SSL key store password").toCharArray();
+
+                        logger.log(securityWarningMessage(CMD_KEYSTORE_PASSWORD));
+
+                        break;
+
+                    case CMD_KEYSTORE_TYPE:
+                        sslKeyStoreType = argIter.nextArg("Expected SSL key store type");
+
+                        break;
+
+                    case CMD_TRUSTSTORE:
+                        sslTrustStorePath = argIter.nextArg("Expected SSL trust store path");
+
+                        break;
+
+                    case CMD_TRUSTSTORE_PASSWORD:
+                        sslTrustStorePassword = argIter.nextArg("Expected SSL trust store password").toCharArray();
+
+                        logger.log(securityWarningMessage(CMD_TRUSTSTORE_PASSWORD));
+
+                        break;
+
+                    case CMD_TRUSTSTORE_TYPE:
+                        sslTrustStoreType = argIter.nextArg("Expected SSL trust store type");
+
+                        break;
+
+                    case CMD_AUTO_CONFIRMATION:
+                        autoConfirmation = true;
+
+                        break;
+
+                    default:
+                        throw new IllegalArgumentException("Unexpected argument: " + str);
+                }
+            }
+        }
+
+        if (command == null)
+            throw new IllegalArgumentException("No action was specified");
+
+        return new ConnectionAndSslParameters(command.command(), host, port, user, pwd,
+                pingTimeout, pingInterval, autoConfirmation,
+                sslProtocol, sslCipherSuites,
+                sslKeyAlgorithm, sslKeyStorePath, sslKeyStorePassword, sslKeyStoreType,
+                sslTrustStorePath, sslTrustStorePassword, sslTrustStoreType);
+    }
+
+    /**
+     * @param password Parsed password.
+     * @return String with warning to show for user.
+     */
+    private String securityWarningMessage(String password) {
+        final String pwdArgWarnFmt = "Warning: %s is insecure. " +
+            "Whenever possible, use interactive prompt for password (just discard %s option).";
+
+        return String.format(pwdArgWarnFmt, password, password);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/Arguments.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/ConnectionAndSslParameters.java
similarity index 73%
rename from modules/core/src/main/java/org/apache/ignite/internal/commandline/Arguments.java
rename to modules/core/src/main/java/org/apache/ignite/internal/commandline/ConnectionAndSslParameters.java
index b16a142..befe451 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/Arguments.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/ConnectionAndSslParameters.java
@@ -18,17 +18,11 @@
 package org.apache.ignite.internal.commandline;
 
 import org.apache.ignite.internal.client.GridClientConfiguration;
-import org.apache.ignite.internal.commandline.baseline.BaselineArguments;
-import org.apache.ignite.internal.commandline.cache.CacheArguments;
-import org.apache.ignite.internal.visor.tx.VisorTxTaskArg;
 
 /**
- * Bean with all parsed and validated arguments.
+ * Container with common parsed and validated arguments.
  */
-public class Arguments {
-    /** Command. */
-    private Command cmd;
-
+public class ConnectionAndSslParameters {
     /** Host. */
     private String host;
 
@@ -44,29 +38,6 @@ public class Arguments {
     /** Force option is used for auto confirmation. */
     private boolean autoConfirmation;
 
-    /**
-     * Arguments for baseline command.
-     */
-    private BaselineArguments baselineArgs;
-
-    /** Transaction arguments. */
-    private final VisorTxTaskArg txArg;
-
-    /**
-     * Arguments for --cache subcommand.
-     */
-    private CacheArguments cacheArgs;
-
-    /**
-     * Action for WAL command.
-     */
-    private String walAct;
-
-    /**
-     * Arguments for WAL command.
-     */
-    private String walArgs;
-
     /** Ping timeout for grid client. See {@link GridClientConfiguration#getPingTimeout()}. */
     private long pingTimeout;
 
@@ -100,17 +71,15 @@ public class Arguments {
     /** Truststore Password. */
     private char[] sslTrustStorePassword;
 
+    /** High-level command. */
+    private Command command;
+
     /**
-     * @param cmd Command.
+     * @param command Command.
      * @param host Host.
      * @param port Port.
      * @param user User.
      * @param pwd Password.
-     * @param baselineArgs Baseline args.
-     * @param txArg TX arg.
-     * @param cacheArgs --cache subcommand arguments.
-     * @param walAct WAL action.
-     * @param walArgs WAL args.
      * @param pingTimeout Ping timeout. See {@link GridClientConfiguration#getPingTimeout()}.
      * @param pingInterval Ping interval. See {@link GridClientConfiguration#getPingInterval()}.
      * @param autoConfirmation Auto confirmation flag.
@@ -124,27 +93,18 @@ public class Arguments {
      * @param sslTrustStorePassword Truststore Password.
      * @param sslTrustStoreType Truststore Type.
      */
-    public Arguments(Command cmd, String host, String port, String user, String pwd,
-        BaselineArguments baselineArgs, VisorTxTaskArg txArg, CacheArguments cacheArgs, String walAct, String walArgs,
+    public ConnectionAndSslParameters(Command command, String host, String port, String user, String pwd,
         Long pingTimeout, Long pingInterval, boolean autoConfirmation,
         String sslProtocol, String sslCipherSuites, String sslKeyAlgorithm,
         String sslKeyStorePath, char[] sslKeyStorePassword, String sslKeyStoreType,
         String sslTrustStorePath, char[] sslTrustStorePassword, String sslTrustStoreType
     ) {
-        this.cmd = cmd;
+        this.command = command;
         this.host = host;
         this.port = port;
         this.user = user;
         this.pwd = pwd;
 
-        this.baselineArgs = baselineArgs;
-
-        this.txArg = txArg;
-        this.cacheArgs = cacheArgs;
-
-        this.walAct = walAct;
-        this.walArgs = walArgs;
-
         this.pingTimeout = pingTimeout;
         this.pingInterval = pingInterval;
 
@@ -164,10 +124,10 @@ public class Arguments {
     }
 
     /**
-     * @return command
+     * @return High-level command which were defined by user to run.
      */
     public Command command() {
-        return cmd;
+        return command;
     }
 
     /**
@@ -187,67 +147,32 @@ public class Arguments {
     /**
      * @return user name
      */
-    public String getUserName() {
+    public String userName() {
         return user;
     }
 
     /**
      * @param user New user name.
      */
-    public void setUserName(String user) {
+    public void userName(String user) {
         this.user = user;
     }
 
     /**
      * @return password
      */
-    public String getPassword() {
+    public String password() {
         return pwd;
     }
 
     /**
      * @param pwd New password.
      */
-    public void setPassword(String pwd) {
+    public void password(String pwd) {
         this.pwd = pwd;
     }
 
     /**
-     * @return Baseline arguments.
-     */
-    public BaselineArguments baselineArguments() {
-        return baselineArgs;
-    }
-
-    /**
-     * @return Transaction arguments.
-     */
-    public VisorTxTaskArg transactionArguments() {
-        return txArg;
-    }
-
-    /**
-     * @return Arguments for --cache subcommand.
-     */
-    public CacheArguments cacheArgs() {
-        return cacheArgs;
-    }
-
-    /**
-     * @return WAL action.
-     */
-    public String walAction() {
-        return walAct;
-    }
-
-    /**
-     * @return WAL arguments.
-     */
-    public String walArguments() {
-        return walArgs;
-    }
-
-    /**
      * See {@link GridClientConfiguration#getPingInterval()}.
      *
      * @return Ping timeout.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/DeactivateCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/DeactivateCommand.java
new file mode 100644
index 0000000..ab2ee39
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/DeactivateCommand.java
@@ -0,0 +1,69 @@
+/*
+ * 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;
+
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientClusterState;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+
+import static org.apache.ignite.internal.commandline.CommandList.DEACTIVATE;
+import static org.apache.ignite.internal.commandline.CommandLogger.optional;
+import static org.apache.ignite.internal.commandline.CommonArgParser.CMD_AUTO_CONFIRMATION;
+
+/**
+ * Command to deactivate cluster.
+ */
+public class DeactivateCommand implements Command<Void> {
+    /** {@inheritDoc} */
+    @Override public void printUsage(CommandLogger logger) {
+        Command.usage(logger, "Deactivate cluster:", DEACTIVATE, optional(CMD_AUTO_CONFIRMATION));
+    }
+
+    /** {@inheritDoc} */
+    @Override public String confirmationPrompt() {
+        return "Warning: the command will deactivate a cluster.";
+    }
+
+    /**
+     * Deactivate cluster.
+     *
+     * @param clientCfg Client configuration.
+     * @throws Exception If failed to deactivate.
+     */
+    @Override public Object execute(GridClientConfiguration clientCfg, CommandLogger logger) throws Exception {
+        try (GridClient client = Command.startClient(clientCfg)) {
+            GridClientClusterState state = client.state();
+
+            state.active(false);
+
+            logger.log("Cluster deactivated");
+        }
+        catch (Exception e) {
+            logger.log("Failed to deactivate cluster.");
+
+            throw e;
+        }
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Void arg() {
+        return null;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/StateCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/StateCommand.java
new file mode 100644
index 0000000..484c25d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/StateCommand.java
@@ -0,0 +1,60 @@
+/*
+ * 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;
+
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientClusterState;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+
+import static org.apache.ignite.internal.commandline.CommandList.STATE;
+
+/**
+ * Command to print cluster state.
+ */
+public class StateCommand implements Command<Void> {
+    /** {@inheritDoc} */
+    @Override public void printUsage(CommandLogger logger) {
+        Command.usage(logger, "Print current cluster state:", STATE);
+    }
+
+    /**
+     * Print cluster state.
+     *
+     * @param clientCfg Client configuration.
+     * @throws Exception If failed to print state.
+     */
+    @Override public Object execute(GridClientConfiguration clientCfg, CommandLogger logger) throws Exception {
+        try (GridClient client = Command.startClient(clientCfg)){
+            GridClientClusterState state = client.state();
+
+            logger.log("Cluster is " + (state.active() ? "active" : "inactive"));
+        }
+        catch (Throwable e) {
+            logger.log("Failed to get cluster state.");
+
+            throw e;
+        }
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Void arg() {
+        return null;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/TaskExecutor.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/TaskExecutor.java
new file mode 100644
index 0000000..c798d45
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/TaskExecutor.java
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.commandline;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.ignite.compute.ComputeTask;
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientCompute;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.client.GridClientDisconnectedException;
+import org.apache.ignite.internal.client.GridClientException;
+import org.apache.ignite.internal.client.GridClientNode;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.visor.VisorTaskArgument;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Visor task executor.
+ */
+public class TaskExecutor {
+    /** */
+    public static final String DFLT_HOST = "127.0.0.1";
+
+    /** */
+    public static final String DFLT_PORT = "11211";
+
+    /** Broadcast uuid. */
+    public static final UUID BROADCAST_UUID = UUID.randomUUID();
+
+    /**
+     * @param client Client
+     * @param taskClsName Task class name.
+     * @param taskArgs Task args.
+     * @param nodeId Node ID to execute task at (if null, random node will be chosen by balancer).
+     * @param clientCfg
+     * @return Task result.
+     * @throws GridClientException If failed to execute task.
+     */
+    public static <R> R executeTaskByNameOnNode(
+        GridClient client,
+        String taskClsName,
+        Object taskArgs,
+        UUID nodeId,
+        GridClientConfiguration clientCfg
+    ) throws GridClientException {
+        GridClientCompute compute = client.compute();
+
+        if (nodeId == BROADCAST_UUID) {
+            Collection<GridClientNode> nodes = compute.nodes(GridClientNode::connectable);
+
+            if (F.isEmpty(nodes))
+                throw new GridClientDisconnectedException("Connectable nodes not found", null);
+
+            List<UUID> nodeIds = nodes.stream()
+                .map(GridClientNode::nodeId)
+                .collect(Collectors.toList());
+
+            return client.compute().execute(taskClsName, new VisorTaskArgument<>(nodeIds, taskArgs, false));
+        }
+
+        GridClientNode node = null;
+
+        if (nodeId == null) {
+            // Prefer node from connect string.
+            final String cfgAddr = clientCfg.getServers().iterator().next();
+
+            String[] parts = cfgAddr.split(":");
+
+            if (DFLT_HOST.equals(parts[0])) {
+                InetAddress addr;
+
+                try {
+                    addr = IgniteUtils.getLocalHost();
+                }
+                catch (IOException e) {
+                    throw new GridClientException("Can't get localhost name.", e);
+                }
+
+                if (addr.isLoopbackAddress())
+                    throw new GridClientException("Can't find localhost name.");
+
+                String origAddr = addr.getHostName() + ":" + parts[1];
+
+                node = listHosts(client).filter(tuple -> origAddr.equals(tuple.get2())).findFirst().map(IgniteBiTuple::get1).orElse(null);
+
+                if (node == null)
+                    node = listHostsByClientNode(client).filter(tuple -> tuple.get2().size() == 1 && cfgAddr.equals(tuple.get2().get(0))).
+                        findFirst().map(IgniteBiTuple::get1).orElse(null);
+            }
+            else
+                node = listHosts(client).filter(tuple -> cfgAddr.equals(tuple.get2())).findFirst().map(IgniteBiTuple::get1).orElse(null);
+
+            // Otherwise choose random node.
+            if (node == null)
+                node = getBalancedNode(compute);
+        }
+        else {
+            for (GridClientNode n : compute.nodes()) {
+                if (n.connectable() && nodeId.equals(n.nodeId())) {
+                    node = n;
+
+                    break;
+                }
+            }
+
+            if (node == null)
+                throw new IllegalArgumentException("Node with id=" + nodeId + " not found");
+        }
+
+        return compute.projection(node).execute(taskClsName, new VisorTaskArgument<>(node.nodeId(), taskArgs, false));
+    }
+
+    /**
+     * @param client Client.
+     * @param taskCls Task class.
+     * @param taskArgs Task arguments.
+     * @param clientCfg Client configuration.
+     * @return Task result.
+     * @throws GridClientException If failed to execute task.
+     */
+    public static <R> R executeTask(
+        GridClient client,
+        Class<? extends ComputeTask<?, R>> taskCls,
+        Object taskArgs,
+        GridClientConfiguration clientCfg
+    ) throws GridClientException {
+        return executeTaskByNameOnNode(client, taskCls.getName(), taskArgs, null, clientCfg);
+    }
+
+    /**
+     * @param client Client.
+     * @return List of hosts.
+     */
+    private static Stream<IgniteBiTuple<GridClientNode, String>> listHosts(GridClient client) throws GridClientException {
+        return client.compute()
+            .nodes(GridClientNode::connectable)
+            .stream()
+            .flatMap(node -> Stream.concat(
+                node.tcpAddresses() == null ? Stream.empty() : node.tcpAddresses().stream(),
+                node.tcpHostNames() == null ? Stream.empty() : node.tcpHostNames().stream()
+            ).map(addr -> new IgniteBiTuple<>(node, addr + ":" + node.tcpPort())));
+    }
+
+    /**
+     * @param client Client.
+     * @return List of hosts.
+     */
+    private static Stream<IgniteBiTuple<GridClientNode, List<String>>> listHostsByClientNode(
+        GridClient client
+    ) throws GridClientException {
+        return client.compute().nodes(GridClientNode::connectable).stream()
+            .map(
+                node -> new IgniteBiTuple<>(
+                    node,
+                    Stream.concat(
+                        node.tcpAddresses() == null ? Stream.empty() : node.tcpAddresses().stream(),
+                        node.tcpHostNames() == null ? Stream.empty() : node.tcpHostNames().stream()
+                    )
+                        .map(addr -> addr + ":" + node.tcpPort()).collect(Collectors.toList())
+                )
+            );
+    }
+
+    /**
+     * @param compute instance
+     * @return balanced node
+     */
+    private static GridClientNode getBalancedNode(GridClientCompute compute) throws GridClientException {
+        Collection<GridClientNode> nodes = compute.nodes(GridClientNode::connectable);
+
+        if (F.isEmpty(nodes))
+            throw new GridClientDisconnectedException("Connectable node not found", null);
+
+        return compute.balancer().balancedNode(nodes);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/baseline/AutoAdjustCommandArg.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/TxCommandArg.java
similarity index 55%
copy from modules/core/src/main/java/org/apache/ignite/internal/commandline/baseline/AutoAdjustCommandArg.java
copy to modules/core/src/main/java/org/apache/ignite/internal/commandline/TxCommandArg.java
index 7e97d1d..4f4165b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/baseline/AutoAdjustCommandArg.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/TxCommandArg.java
@@ -1,12 +1,12 @@
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
+ * 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
+ * the License. You may obtain a copy of the License at
  *
- *      http://www.apache.org/licenses/LICENSE-2.0
+ * 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,
@@ -15,26 +15,54 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.commandline.baseline;
+package org.apache.ignite.internal.commandline;
 
 import org.apache.ignite.internal.commandline.argument.CommandArg;
 
 /**
- * {@link BaselineCommand#AUTO_ADJUST} command arguments.
+ * Transaction command arguments name.
  */
-public enum AutoAdjustCommandArg implements CommandArg {
-    /** Enable auto-adjust. */
-    ENABLE("enable"),
-    /** Disable auto-adjust. */
-    DISABLE("disable"),
-    /** Set soft timeout. */
-    TIMEOUT("timeout");
+public enum TxCommandArg implements CommandArg {
+    /** */
+    TX_LIMIT("--limit"),
+
+    /** */
+    TX_ORDER("--order"),
+
+    /** */
+    TX_SERVERS("--servers"),
+
+    /** */
+    TX_CLIENTS("--clients"),
+
+    /** */
+    TX_DURATION("--min-duration"),
+
+    /** */
+    TX_SIZE("--min-size"),
+
+    /** */
+    TX_LABEL("--label"),
+
+    /** */
+    TX_NODES("--nodes"),
+
+    /** */
+    TX_XID("--xid"),
+
+    /** */
+    TX_KILL("--kill"),
+
+    /** */
+    TX_INFO("--info");
 
     /** Option name. */
     private final String name;
 
-    /** */
-    AutoAdjustCommandArg(String name) {
+    /**
+     * @param name Argument name.
+     */
+    TxCommandArg(String name) {
         this.name = name;
     }
 
@@ -47,4 +75,4 @@ public enum AutoAdjustCommandArg implements CommandArg {
     @Override public String toString() {
         return name;
     }
-}
\ No newline at end of file
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/TxCommands.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/TxCommands.java
new file mode 100644
index 0000000..28c13a0
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/TxCommands.java
@@ -0,0 +1,582 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteNodeAttributes;
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.client.GridClientException;
+import org.apache.ignite.internal.client.GridClientNode;
+import org.apache.ignite.internal.commandline.argument.CommandArgUtils;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.visor.tx.FetchNearXidVersionTask;
+import org.apache.ignite.internal.visor.tx.TxKeyLockType;
+import org.apache.ignite.internal.visor.tx.TxMappingType;
+import org.apache.ignite.internal.visor.tx.TxVerboseId;
+import org.apache.ignite.internal.visor.tx.TxVerboseInfo;
+import org.apache.ignite.internal.visor.tx.TxVerboseKey;
+import org.apache.ignite.internal.visor.tx.VisorTxInfo;
+import org.apache.ignite.internal.visor.tx.VisorTxOperation;
+import org.apache.ignite.internal.visor.tx.VisorTxProjection;
+import org.apache.ignite.internal.visor.tx.VisorTxSortOrder;
+import org.apache.ignite.internal.visor.tx.VisorTxTask;
+import org.apache.ignite.internal.visor.tx.VisorTxTaskArg;
+import org.apache.ignite.internal.visor.tx.VisorTxTaskResult;
+import org.apache.ignite.transactions.TransactionState;
+
+import static org.apache.ignite.internal.commandline.CommandList.TX;
+import static org.apache.ignite.internal.commandline.CommandLogger.INDENT;
+import static org.apache.ignite.internal.commandline.CommandLogger.optional;
+import static org.apache.ignite.internal.commandline.CommandLogger.or;
+import static org.apache.ignite.internal.commandline.CommonArgParser.CMD_AUTO_CONFIRMATION;
+import static org.apache.ignite.internal.commandline.TaskExecutor.executeTask;
+import static org.apache.ignite.internal.commandline.TxCommandArg.TX_INFO;
+
+/**
+ * Transaction commands.
+ */
+public class TxCommands implements Command<VisorTxTaskArg> {
+    /** Double indent. */
+    private static final String DOUBLE_INDENT = INDENT + INDENT;
+
+    /** Arguments */
+    private VisorTxTaskArg args;
+
+    /** Logger. */
+    private CommandLogger logger;
+
+    /** {@inheritDoc} */
+    @Override public void printUsage(CommandLogger logger) {
+        Command.usage(logger, "List or kill transactions:", TX, getTxOptions());
+        Command.usage(logger, "Print detailed information (topology and key lock ownership) about specific transaction:",
+            TX, TX_INFO.argName(), or("<TX identifier as GridCacheVersion [topVer=..., order=..., nodeOrder=...] " +
+                "(can be found in logs)>", "<TX identifier as UUID (can be retrieved via --tx command)>"));
+
+    }
+
+    /**
+     * @return Transaction command options.
+     */
+    private String[] getTxOptions() {
+        List<String> list = new ArrayList<>();
+
+        list.add(optional(TxCommandArg.TX_XID, "XID"));
+        list.add(optional(TxCommandArg.TX_DURATION, "SECONDS"));
+        list.add(optional(TxCommandArg.TX_SIZE, "SIZE"));
+        list.add(optional(TxCommandArg.TX_LABEL, "PATTERN_REGEX"));
+        list.add(optional(or(TxCommandArg.TX_SERVERS, TxCommandArg.TX_CLIENTS)));
+        list.add(optional(TxCommandArg.TX_NODES, "consistentId1[,consistentId2,....,consistentIdN]"));
+        list.add(optional(TxCommandArg.TX_LIMIT, "NUMBER"));
+        list.add(optional(TxCommandArg.TX_ORDER, or(VisorTxSortOrder.values())));
+        list.add(optional(TxCommandArg.TX_KILL));
+        list.add(optional(TX_INFO));
+        list.add(optional(CMD_AUTO_CONFIRMATION));
+
+        return list.toArray(new String[list.size()]);
+    }
+
+    /** {@inheritDoc} */
+    @Override public VisorTxTaskArg arg() {
+        return args;
+    }
+
+    /**
+     * Dump transactions information.
+     *
+     * @param clientCfg Client configuration.
+     */
+    @Override public Object execute(GridClientConfiguration clientCfg, CommandLogger logger) throws Exception {
+        this.logger = logger;
+
+        try (GridClient client = Command.startClient(clientCfg)) {
+            if (args.getOperation() == VisorTxOperation.INFO)
+                return transactionInfo(client, clientCfg);
+
+            Map<ClusterNode, VisorTxTaskResult> res = executeTask(client, VisorTxTask.class, args, clientCfg);
+
+            if (res.isEmpty())
+                logger.log("Nothing found.");
+            else if (args.getOperation() == VisorTxOperation.KILL)
+                logger.log("Killed transactions:");
+            else
+                logger.log("Matching transactions:");
+
+            for (Map.Entry<ClusterNode, VisorTxTaskResult> entry : res.entrySet()) {
+                if (entry.getValue().getInfos().isEmpty())
+                    continue;
+
+                ClusterNode key = entry.getKey();
+
+                logger.log(key.getClass().getSimpleName() + " [id=" + key.id() +
+                    ", addrs=" + key.addresses() +
+                    ", order=" + key.order() +
+                    ", ver=" + key.version() +
+                    ", isClient=" + key.isClient() +
+                    ", consistentId=" + key.consistentId() +
+                    "]");
+
+                for (VisorTxInfo info : entry.getValue().getInfos())
+                    logger.log(info.toUserString());
+            }
+
+            return res;
+        }
+        catch (Throwable e) {
+            logger.error("Failed to perform operation.", e);
+
+            throw e;
+        }
+    }
+
+    /**
+     * Dump transactions information.
+     *
+     * @param client Client.
+     */
+    private void transactions(GridClient client, GridClientConfiguration conf) throws GridClientException {
+        try {
+            if (args.getOperation() == VisorTxOperation.INFO) {
+                transactionInfo(client, conf);
+
+                return;
+            }
+
+            Map<ClusterNode, VisorTxTaskResult> res = executeTask(client, VisorTxTask.class, args, conf);
+
+            for (Map.Entry<ClusterNode, VisorTxTaskResult> entry : res.entrySet()) {
+                if (entry.getValue().getInfos().isEmpty())
+                    continue;
+
+                ClusterNode key = entry.getKey();
+
+                logger.log(nodeDescription(key));
+
+                for (VisorTxInfo info : entry.getValue().getInfos())
+                    logger.log(info.toUserString());
+            }
+        }
+        catch (Throwable e) {
+            logger.log("Failed to perform operation.");
+
+            throw e;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String confirmationPrompt() {
+        if (args.getOperation() == VisorTxOperation.KILL)
+            return "Warning: the command will kill some transactions.";
+
+        return null;
+    }
+
+    /**
+     * @param argIter Argument iterator.
+     */
+    @Override public void parseArguments(CommandArgIterator argIter) {
+        VisorTxProjection proj = null;
+
+        Integer limit = null;
+
+        VisorTxSortOrder sortOrder = null;
+
+        Long duration = null;
+
+        Integer size = null;
+
+        String lbRegex = null;
+
+        List<String> consistentIds = null;
+
+        VisorTxOperation op = VisorTxOperation.LIST;
+
+        String xid = null;
+
+        TxVerboseId txVerboseId = null;
+
+        while (true) {
+            String str = argIter.peekNextArg();
+
+            if (str == null)
+                break;
+
+            TxCommandArg arg = CommandArgUtils.of(str, TxCommandArg.class);
+
+            if (arg == null)
+                break;
+
+            switch (arg) {
+                case TX_LIMIT:
+                    argIter.nextArg("");
+
+                    limit = (int)argIter.nextLongArg(TxCommandArg.TX_LIMIT.toString());
+
+                    break;
+
+                case TX_ORDER:
+                    argIter.nextArg("");
+
+                    sortOrder = VisorTxSortOrder.valueOf(argIter.nextArg(TxCommandArg.TX_ORDER.toString()).toUpperCase());
+
+                    break;
+
+                case TX_SERVERS:
+                    argIter.nextArg("");
+
+                    proj = VisorTxProjection.SERVER;
+                    break;
+
+                case TX_CLIENTS:
+                    argIter.nextArg("");
+
+                    proj = VisorTxProjection.CLIENT;
+                    break;
+
+                case TX_NODES:
+                    argIter.nextArg("");
+
+                    Set<String> ids = argIter.nextStringSet(TxCommandArg.TX_NODES.toString());
+
+                    if (ids.isEmpty()) {
+                        throw new IllegalArgumentException("Consistent id list is empty.");
+                    }
+
+                    consistentIds = new ArrayList<>(ids);
+                    break;
+
+                case TX_DURATION:
+                    argIter.nextArg("");
+
+                    duration = argIter.nextLongArg(TxCommandArg.TX_DURATION.toString()) * 1000L;
+                    break;
+
+                case TX_SIZE:
+                    argIter.nextArg("");
+
+                    size = (int)argIter.nextLongArg(TxCommandArg.TX_SIZE.toString());
+                    break;
+
+                case TX_LABEL:
+                    argIter.nextArg("");
+
+                    lbRegex = argIter.nextArg(TxCommandArg.TX_LABEL.toString());
+
+                    try {
+                        Pattern.compile(lbRegex);
+                    }
+                    catch (PatternSyntaxException ignored) {
+                        throw new IllegalArgumentException("Illegal regex syntax");
+                    }
+
+                    break;
+
+                case TX_XID:
+                    argIter.nextArg("");
+
+                    xid = argIter.nextArg(TxCommandArg.TX_XID.toString());
+                    break;
+
+                case TX_KILL:
+                    argIter.nextArg("");
+
+                    op = VisorTxOperation.KILL;
+                    break;
+
+                case TX_INFO:
+                    argIter.nextArg("");
+
+                    op = VisorTxOperation.INFO;
+
+                    txVerboseId = TxVerboseId.fromString(argIter.nextArg(TX_INFO.argName()));
+
+                    break;
+
+                default:
+                    throw new AssertionError();
+            }
+        }
+
+        if (proj != null && consistentIds != null)
+            throw new IllegalArgumentException("Projection can't be used together with list of consistent ids.");
+
+        this.args = new VisorTxTaskArg(op, limit, duration, size, null, proj,
+            consistentIds, xid, lbRegex, sortOrder, txVerboseId);
+    }
+
+    /**
+     * Provides text descrition of a cluster node.
+     *
+     * @param node Node.
+     */
+    private static String nodeDescription(ClusterNode node) {
+        return node.getClass().getSimpleName() + " [id=" + node.id() +
+            ", addrs=" + node.addresses() +
+            ", order=" + node.order() +
+            ", ver=" + node.version() +
+            ", isClient=" + node.isClient() +
+            ", consistentId=" + node.consistentId() +
+            "]";
+    }
+
+    /**
+     * Executes --tx --info command.
+     *
+     * @param client Client.
+     */
+    private Object transactionInfo(GridClient client, GridClientConfiguration conf) throws GridClientException {
+        checkFeatureSupportedByCluster(client, IgniteFeatures.TX_INFO_COMMAND, true);
+
+        GridCacheVersion nearXidVer = executeTask(client, FetchNearXidVersionTask.class, args.txInfoArgument(), conf);
+
+        boolean histMode = false;
+
+        if (nearXidVer != null) {
+            logger.log("Resolved transaction near XID version: " + nearXidVer);
+
+            args.txInfoArgument(new TxVerboseId(null, nearXidVer));
+        }
+        else {
+            logger.log("Active transactions not found.");
+
+            if (args.txInfoArgument().gridCacheVersion() != null) {
+                logger.log("Will try to peek history to find out whether transaction was committed / rolled back.");
+
+                histMode = true;
+            }
+            else {
+                logger.log("You can specify transaction in GridCacheVersion format in order to peek history " +
+                    "to find out whether transaction was committed / rolled back.");
+
+                return null;
+            }
+        }
+
+        Map<ClusterNode, VisorTxTaskResult> res = executeTask(client, VisorTxTask.class, args, conf);
+
+        if (histMode)
+            printTxInfoHistoricalResult(res);
+        else
+            printTxInfoResult(res);
+
+        return res;
+    }
+
+    /**
+     * Prints result of --tx --info command to output.
+     *
+     * @param res Response.
+     */
+    private void printTxInfoResult(Map<ClusterNode, VisorTxTaskResult> res) {
+        String lb = null;
+
+        Map<Integer, String> usedCaches = new HashMap<>();
+        Map<Integer, String> usedCacheGroups = new HashMap<>();
+        VisorTxInfo firstInfo = null;
+        TxVerboseInfo firstVerboseInfo = null;
+        Set<TransactionState> states = new HashSet<>();
+
+        for (Map.Entry<ClusterNode, VisorTxTaskResult> entry : res.entrySet()) {
+            for (VisorTxInfo info : entry.getValue().getInfos()) {
+                assert info.getTxVerboseInfo() != null;
+
+                if (lb == null)
+                    lb = info.getLabel();
+
+                if (firstInfo == null) {
+                    firstInfo = info;
+                    firstVerboseInfo = info.getTxVerboseInfo();
+                }
+
+                usedCaches.putAll(info.getTxVerboseInfo().usedCaches());
+                usedCacheGroups.putAll(info.getTxVerboseInfo().usedCacheGroups());
+                states.add(info.getState());
+            }
+        }
+
+        String indent = "";
+
+        logger.nl();
+        logger.log(indent + "Transaction detailed info:");
+
+        printTransactionDetailedInfo(
+            res, usedCaches, usedCacheGroups, firstInfo, firstVerboseInfo, states, indent + DOUBLE_INDENT);
+    }
+
+    /**
+     * Prints detailed info about transaction to output.
+     *
+     * @param res Response.
+     * @param usedCaches Used caches.
+     * @param usedCacheGroups Used cache groups.
+     * @param firstInfo First info.
+     * @param firstVerboseInfo First verbose info.
+     * @param states States.
+     * @param indent Indent.
+     */
+    private void printTransactionDetailedInfo(Map<ClusterNode, VisorTxTaskResult> res, Map<Integer, String> usedCaches,
+        Map<Integer, String> usedCacheGroups, VisorTxInfo firstInfo, TxVerboseInfo firstVerboseInfo,
+        Set<TransactionState> states, String indent) {
+        logger.log(indent + "Near XID version: " + firstVerboseInfo.nearXidVersion());
+        logger.log(indent + "Near XID version (UUID): " + firstInfo.getNearXid());
+        logger.log(indent + "Isolation: " + firstInfo.getIsolation());
+        logger.log(indent + "Concurrency: " + firstInfo.getConcurrency());
+        logger.log(indent + "Timeout: " + firstInfo.getTimeout());
+        logger.log(indent + "Initiator node: " + firstVerboseInfo.nearNodeId());
+        logger.log(indent + "Initiator node (consistent ID): " + firstVerboseInfo.nearNodeConsistentId());
+        logger.log(indent + "Label: " + firstInfo.getLabel());
+        logger.log(indent + "Topology version: " + firstInfo.getTopologyVersion());
+        logger.log(indent + "Used caches (ID to name): " + usedCaches);
+        logger.log(indent + "Used cache groups (ID to name): " + usedCacheGroups);
+        logger.log(indent + "States across the cluster: " + states);
+        logger.log(indent + "Transaction topology: ");
+
+        printTransactionTopology(res, indent + DOUBLE_INDENT);
+    }
+
+    /**
+     * Prints transaction topology to output.
+     *
+     * @param res Response.
+     * @param indent Indent.
+     */
+    private void printTransactionTopology(Map<ClusterNode, VisorTxTaskResult> res, String indent) {
+        for (Map.Entry<ClusterNode, VisorTxTaskResult> entry : res.entrySet()) {
+            logger.log(indent + nodeDescription(entry.getKey()) + ':');
+
+            printTransactionMappings(indent + DOUBLE_INDENT, entry);
+        }
+    }
+
+    /**
+     * Prints transaction mappings for specific cluster node to output.
+     *
+     * @param indent Indent.
+     * @param entry Entry.
+     */
+    private void printTransactionMappings(String indent, Map.Entry<ClusterNode, VisorTxTaskResult> entry) {
+        for (VisorTxInfo info : entry.getValue().getInfos()) {
+            TxVerboseInfo verboseInfo = info.getTxVerboseInfo();
+
+            if (verboseInfo != null) {
+                logger.log(indent + "Mapping [type=" + verboseInfo.txMappingType() + "]:");
+
+                printTransactionMapping(indent + DOUBLE_INDENT, info, verboseInfo);
+            }
+            else {
+                logger.log(indent + "Mapping [type=HISTORICAL]:");
+
+                logger.log(indent + DOUBLE_INDENT + "State: " + info.getState());
+            }
+        }
+    }
+
+    /**
+     * Prints specific transaction mapping to output.
+     *
+     * @param indent Indent.
+     * @param info Info.
+     * @param verboseInfo Verbose info.
+     */
+    private void printTransactionMapping(String indent, VisorTxInfo info, TxVerboseInfo verboseInfo) {
+        logger.log(indent + "XID version (UUID): " + info.getXid());
+        logger.log(indent + "State: " + info.getState());
+
+        if (verboseInfo.txMappingType() == TxMappingType.REMOTE) {
+            logger.log(indent + "Primary node: " + verboseInfo.dhtNodeId());
+            logger.log(indent + "Primary node (consistent ID): " + verboseInfo.dhtNodeConsistentId());
+        }
+
+        if (!F.isEmpty(verboseInfo.localTxKeys())) {
+            logger.log(indent + "Mapped keys:");
+
+            printTransactionKeys(indent + DOUBLE_INDENT, verboseInfo);
+        }
+    }
+
+    /**
+     * Prints keys of specific transaction mapping to output.
+     *
+     * @param indent Indent.
+     * @param verboseInfo Verbose info.
+     */
+    private void printTransactionKeys(String indent, TxVerboseInfo verboseInfo) {
+        for (TxVerboseKey txVerboseKey : verboseInfo.localTxKeys()) {
+            logger.log(indent + (txVerboseKey.read() ? "Read" : "Write") +
+                " [lock=" + txVerboseKey.lockType() + "]: " + txVerboseKey.txKey());
+
+            if (txVerboseKey.lockType() == TxKeyLockType.AWAITS_LOCK)
+                logger.log(indent + DOUBLE_INDENT + "Lock owner XID: " + txVerboseKey.ownerVersion());
+        }
+    }
+
+    /**
+     * Prints results of --tx --info to output in case requested transaction is not active.
+     *
+     * @param res Response.
+     */
+    private void printTxInfoHistoricalResult(Map<ClusterNode, VisorTxTaskResult> res) {
+        if (F.isEmpty(res))
+            logger.log("Transaction was not found in history across the cluster.");
+        else {
+            logger.log("Transaction was found in completed versions history of the following nodes:");
+
+            for (Map.Entry<ClusterNode, VisorTxTaskResult> entry : res.entrySet()) {
+                logger.log(DOUBLE_INDENT + nodeDescription(entry.getKey()) + ':');
+                logger.log(DOUBLE_INDENT + DOUBLE_INDENT + "State: " + entry.getValue().getInfos().get(0).getState());
+            }
+        }
+    }
+
+    /**
+     * Checks that all cluster nodes support specified feature.
+     *
+     * @param client Client.
+     * @param feature Feature.
+     * @param validateClientNodes Whether client nodes should be checked as well.
+     */
+    private static void checkFeatureSupportedByCluster(
+        GridClient client,
+        IgniteFeatures feature,
+        boolean validateClientNodes
+    ) throws GridClientException {
+        Collection<GridClientNode> nodes = validateClientNodes ?
+            client.compute().nodes() :
+            client.compute().nodes(GridClientNode::connectable);
+
+        for (GridClientNode node : nodes) {
+            byte[] featuresAttrBytes = node.attribute(IgniteNodeAttributes.ATTR_IGNITE_FEATURES);
+
+            if (!IgniteFeatures.nodeSupports(featuresAttrBytes, feature)) {
+                throw new IllegalStateException("Failed to execute command: cluster contains node that " +
+                    "doesn't support feature [nodeId=" + node.nodeId() + ", feature=" + feature + ']');
+            }
+        }
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/WalCommands.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/WalCommands.java
new file mode 100644
index 0000000..82c62a3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/WalCommands.java
@@ -0,0 +1,260 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.misc.VisorClusterNode;
+import org.apache.ignite.internal.visor.misc.VisorWalTask;
+import org.apache.ignite.internal.visor.misc.VisorWalTaskArg;
+import org.apache.ignite.internal.visor.misc.VisorWalTaskOperation;
+import org.apache.ignite.internal.visor.misc.VisorWalTaskResult;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_ENABLE_EXPERIMENTAL_COMMAND;
+import static org.apache.ignite.internal.commandline.CommandArgIterator.isCommandOrOption;
+import static org.apache.ignite.internal.commandline.CommandLogger.optional;
+import static org.apache.ignite.internal.commandline.CommandList.WAL;
+import static org.apache.ignite.internal.commandline.CommonArgParser.CMD_AUTO_CONFIRMATION;
+import static org.apache.ignite.internal.commandline.TaskExecutor.executeTask;
+
+/**
+ * Wal commands.
+ */
+public class WalCommands implements Command<T2<String, String>> {
+    /** */
+    static final String WAL_PRINT = "print";
+
+    /** */
+    static final String WAL_DELETE = "delete";
+
+    /** */
+    private CommandLogger logger;
+
+    /**
+     * Wal action.
+     */
+    private String walAct;
+
+    /**
+     * Wal arguments.
+     */
+    private String walArgs;
+
+    @Override public void printUsage(CommandLogger logger) {
+        if (IgniteSystemProperties.getBoolean(IGNITE_ENABLE_EXPERIMENTAL_COMMAND, false)) {
+            Command.usage(logger, "Print absolute paths of unused archived wal segments on each node:", WAL,
+                WAL_PRINT, "[consistentId1,consistentId2,....,consistentIdN]");
+            Command.usage(logger,"Delete unused archived wal segments on each node:", WAL, WAL_DELETE,
+                "[consistentId1,consistentId2,....,consistentIdN]", optional(CMD_AUTO_CONFIRMATION));
+        }
+    }
+
+    /**
+     * Execute WAL command.
+     *
+     * @param clientCfg Client configuration.
+     * @throws Exception If failed to execute wal action.
+     */
+    @Override public Object execute(GridClientConfiguration clientCfg, CommandLogger logger) throws Exception {
+        this.logger = logger;
+
+        try (GridClient client = Command.startClient(clientCfg)) {
+            switch (walAct) {
+                case WAL_DELETE:
+                    deleteUnusedWalSegments(client, walArgs, clientCfg);
+
+                    break;
+
+                case WAL_PRINT:
+                default:
+                    printUnusedWalSegments(client, walArgs, clientCfg);
+
+                    break;
+            }
+        }
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String confirmationPrompt() {
+        if (WAL_DELETE.equals(walAct))
+            return "Warning: the command will delete unused WAL segments.";
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void parseArguments(CommandArgIterator argIter) {
+        String str = argIter.nextArg("Expected arguments for " + WAL.text());
+
+        String walAct = str.toLowerCase();
+
+        if (WAL_PRINT.equals(walAct) || WAL_DELETE.equals(walAct)) {
+            String walArgs = (str = argIter.peekNextArg()) != null && !isCommandOrOption(str)
+                ? argIter.nextArg("Unexpected argument for " + WAL.text() + ": " + walAct)
+                : "";
+
+            this.walAct = walAct;
+            this.walArgs = walArgs;
+        }
+        else
+            throw new IllegalArgumentException("Unexpected action " + walAct + " for " + WAL.text());
+    }
+
+    /**
+     * @return Tuple where first string is wal action, second - wal arguments.
+     */
+    @Override public T2<String, String> arg() {
+        return new T2<>(walAct, walArgs);
+    }
+
+    /**
+     * Execute delete unused WAL segments task.
+     *  @param client Client.
+     * @param walArgs WAL args.
+     * @param clientCfg Client configuration.
+     */
+    private void deleteUnusedWalSegments(
+        GridClient client,
+        String walArgs,
+        GridClientConfiguration clientCfg
+    ) throws Exception {
+        VisorWalTaskResult res = executeTask(client, VisorWalTask.class,
+            walArg(VisorWalTaskOperation.DELETE_UNUSED_WAL_SEGMENTS, walArgs), clientCfg);
+        printDeleteWalSegments0(res);
+    }
+
+    /**
+     * Execute print unused WAL segments task.
+     *  @param client Client.
+     * @param walArgs Wal args.
+     * @param clientCfg Client configuration.
+     */
+    private void printUnusedWalSegments(
+        GridClient client,
+        String walArgs,
+        GridClientConfiguration clientCfg
+    ) throws Exception {
+        VisorWalTaskResult res = executeTask(client, VisorWalTask.class,
+            walArg(VisorWalTaskOperation.PRINT_UNUSED_WAL_SEGMENTS, walArgs), clientCfg);
+        printUnusedWalSegments0(res);
+    }
+
+    /**
+     * Prepare WAL task argument.
+     *
+     * @param op Operation.
+     * @param s Argument from command line.
+     * @return Task argument.
+     */
+    private VisorWalTaskArg walArg(VisorWalTaskOperation op, String s) {
+        List<String> consistentIds = null;
+
+        if (!F.isEmpty(s)) {
+            consistentIds = new ArrayList<>();
+
+            for (String consistentId : s.split(","))
+                consistentIds.add(consistentId.trim());
+        }
+
+        switch (op) {
+            case DELETE_UNUSED_WAL_SEGMENTS:
+            case PRINT_UNUSED_WAL_SEGMENTS:
+                return new VisorWalTaskArg(op, consistentIds);
+
+            default:
+                return new VisorWalTaskArg(VisorWalTaskOperation.PRINT_UNUSED_WAL_SEGMENTS, consistentIds);
+        }
+
+    }
+
+    /**
+     * Print list of unused wal segments.
+     *
+     * @param taskRes Task result with baseline topology.
+     */
+    private void printUnusedWalSegments0(VisorWalTaskResult taskRes) {
+        logger.log("Unused wal segments per node:");
+        logger.nl();
+
+        Map<String, Collection<String>> res = taskRes.results();
+        Map<String, Exception> failRes = taskRes.exceptions();
+        Map<String, VisorClusterNode> nodesInfo = taskRes.getNodesInfo();
+
+        for (Map.Entry<String, Collection<String>> entry : res.entrySet()) {
+            VisorClusterNode node = nodesInfo.get(entry.getKey());
+
+            logger.log("Node=" + node.getConsistentId());
+            logger.logWithIndent("addresses " + U.addressesAsString(node.getAddresses(), node.getHostNames()), 2);
+
+            for (String fileName : entry.getValue())
+                logger.logWithIndent(fileName);
+
+            logger.nl();
+        }
+
+        for (Map.Entry<String, Exception> entry : failRes.entrySet()) {
+            VisorClusterNode node = nodesInfo.get(entry.getKey());
+
+            logger.log("Node=" + node.getConsistentId());
+            logger.logWithIndent("addresses " + U.addressesAsString(node.getAddresses(), node.getHostNames()), 2);
+            logger.logWithIndent("failed with error: " + entry.getValue().getMessage());
+            logger.nl();
+        }
+    }
+
+    /**
+     * Print list of unused wal segments.
+     *
+     * @param taskRes Task result with baseline topology.
+     */
+    private void printDeleteWalSegments0(VisorWalTaskResult taskRes) {
+        logger.log("WAL segments deleted for nodes:");
+        logger.nl();
+
+        Map<String, Collection<String>> res = taskRes.results();
+        Map<String, Exception> errors = taskRes.exceptions();
+        Map<String, VisorClusterNode> nodesInfo = taskRes.getNodesInfo();
+
+        for (Map.Entry<String, Collection<String>> entry : res.entrySet()) {
+            VisorClusterNode node = nodesInfo.get(entry.getKey());
+
+            logger.log("Node=" + node.getConsistentId());
+            logger.logWithIndent("addresses " + U.addressesAsString(node.getAddresses(), node.getHostNames()), 2);
+            logger.nl();
+        }
+
+        for (Map.Entry<String, Exception> entry : errors.entrySet()) {
+            VisorClusterNode node = nodesInfo.get(entry.getKey());
+
+            logger.log("Node=" + node.getConsistentId());
+            logger.logWithIndent("addresses " + U.addressesAsString(node.getAddresses(), node.getHostNames()), 2);
+            logger.logWithIndent("failed with error: " + entry.getValue().getMessage());
+            logger.nl();
+        }
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/baseline/AutoAdjustCommandArg.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/baseline/AutoAdjustCommandArg.java
index 7e97d1d..c1f6adc3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/baseline/AutoAdjustCommandArg.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/baseline/AutoAdjustCommandArg.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.commandline.baseline;
 import org.apache.ignite.internal.commandline.argument.CommandArg;
 
 /**
- * {@link BaselineCommand#AUTO_ADJUST} command arguments.
+ * {@link BaselineSubcommands#AUTO_ADJUST} command arguments.
  */
 public enum AutoAdjustCommandArg implements CommandArg {
     /** Enable auto-adjust. */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/baseline/BaselineArguments.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/baseline/BaselineArguments.java
index 46a9c73..675b03b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/baseline/BaselineArguments.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/baseline/BaselineArguments.java
@@ -24,7 +24,7 @@ import java.util.List;
  */
 public class BaselineArguments {
     /** Command. */
-    private BaselineCommand cmd;
+    private BaselineSubcommands cmd;
     /**
      * {@code true} if auto-adjust should be enable, {@code false} if it should be disable, {@code null} if no operation
      * needed.
@@ -44,7 +44,7 @@ public class BaselineArguments {
      * @param topVer Requested topology version.
      * @param consistentIds List of consistent ids for operation.
      */
-    public BaselineArguments(BaselineCommand cmd, Boolean enableAutoAdjust, Long softBaselineTimeout,
+    public BaselineArguments(BaselineSubcommands cmd, Boolean enableAutoAdjust, Long softBaselineTimeout,
         long topVer, List<String> consistentIds) {
         this.cmd = cmd;
         this.enableAutoAdjust = enableAutoAdjust;
@@ -56,7 +56,7 @@ public class BaselineArguments {
     /**
      * @return Command.
      */
-    public BaselineCommand getCmd() {
+    public BaselineSubcommands getCmd() {
         return cmd;
     }
 
@@ -94,7 +94,7 @@ public class BaselineArguments {
      */
     public static class Builder {
         /** Command. */
-        private BaselineCommand cmd;
+        private BaselineSubcommands cmd;
         /**
          * {@code true} if auto-adjust should be enable, {@code false} if it should be disable, {@code null} if no
          * operation needed.
@@ -110,7 +110,7 @@ public class BaselineArguments {
         /**
          * @param cmd Command.
          */
-        public Builder(BaselineCommand cmd) {
+        public Builder(BaselineSubcommands cmd) {
             this.cmd = cmd;
         }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/baseline/BaselineCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/baseline/BaselineSubcommands.java
similarity index 63%
rename from modules/core/src/main/java/org/apache/ignite/internal/commandline/baseline/BaselineCommand.java
rename to modules/core/src/main/java/org/apache/ignite/internal/commandline/baseline/BaselineSubcommands.java
index 0b3744b..0426237 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/baseline/BaselineCommand.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/baseline/BaselineSubcommands.java
@@ -17,61 +17,67 @@
 
 package org.apache.ignite.internal.commandline.baseline;
 
+import org.apache.ignite.internal.visor.baseline.VisorBaselineOperation;
 import org.jetbrains.annotations.Nullable;
 
 /**
  * Set of baseline commands.
  */
-public enum BaselineCommand {
+public enum BaselineSubcommands {
     /**
      * Add nodes to baseline.
      */
-    ADD("add"),
+    ADD("add", VisorBaselineOperation.ADD),
 
     /**
      * Remove nodes from baseline.
      */
-    REMOVE("remove"),
+    REMOVE("remove", VisorBaselineOperation.REMOVE),
 
     /**
      * Collect information about baseline.
      */
-    COLLECT("collect"),
+    COLLECT("collect", VisorBaselineOperation.COLLECT),
 
     /**
      * Set new baseline.
      */
-    SET("set"),
+    SET("set", VisorBaselineOperation.SET),
 
     /**
      * Check current topology version.
      */
-    VERSION("version"),
+    VERSION("version", VisorBaselineOperation.VERSION),
 
     /**
      * Baseline auto-adjust configuration.
      */
-    AUTO_ADJUST("auto_adjust");
+    AUTO_ADJUST("auto_adjust", VisorBaselineOperation.AUTOADJUST);
 
     /** Enumerated values. */
-    private static final BaselineCommand[] VALS = values();
+    private static final BaselineSubcommands[] VALS = values();
 
     /** Name. */
     private final String name;
 
+    /** Corresponding visor baseline operation. */
+    private final VisorBaselineOperation visorBaselineOperation;
+
     /**
      * @param name Name.
+     * @param operation
      */
-    BaselineCommand(String name) {
+    BaselineSubcommands(String name, VisorBaselineOperation operation) {
         this.name = name;
+        visorBaselineOperation = operation;
     }
 
     /**
      * @param text Command text.
      * @return Command for the text.
      */
-    public static BaselineCommand of(String text) {
-        for (BaselineCommand cmd : BaselineCommand.values()) {
+    public static BaselineSubcommands of(String text) {
+        for (BaselineSubcommands cmd : BaselineSubcommands.values()) {
             if (cmd.text().equalsIgnoreCase(text))
                 return cmd;
         }
@@ -87,12 +93,19 @@ public enum BaselineCommand {
     }
 
     /**
+     * @return {@link VisorBaselineOperation} which is associated with baseline subcommand.
+     */
+    public VisorBaselineOperation visorBaselineOperation() {
+        return visorBaselineOperation;
+    }
+
+    /**
      * Efficiently gets enumerated value from its ordinal.
      *
      * @param ord Ordinal value.
      * @return Enumerated value or {@code null} if ordinal out of range.
      */
-    @Nullable public static BaselineCommand fromOrdinal(int ord) {
+    @Nullable public static BaselineSubcommands fromOrdinal(int ord) {
         return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheArguments.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheArguments.java
deleted file mode 100644
index cd29474..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheArguments.java
+++ /dev/null
@@ -1,361 +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.commandline.cache;
-
-import java.util.Set;
-import java.util.UUID;
-import org.apache.ignite.internal.commandline.OutputFormat;
-import org.apache.ignite.internal.visor.verify.CacheFilterEnum;
-import org.apache.ignite.internal.visor.verify.VisorViewCacheCmd;
-import org.jetbrains.annotations.Nullable;
-
-/**
- *
- */
-public class CacheArguments {
-    /** Command. */
-    private CacheCommand cmd;
-
-    /** Caches. */
-    private Set<String> caches;
-
-    /** Caches. */
-    private Set<String> groups;
-
-    /** Exclude caches or groups. */
-    private Set<String> excludeCaches;
-
-    /** Partition id. */
-    private int partId;
-
-    /** Regex. */
-    private String regex;
-
-    /** Node id. */
-    private UUID nodeId;
-
-    /** Min queue size. */
-    private int minQueueSize;
-
-    /** Max print. */
-    private int maxPrint;
-
-    /** validate_indexes 'checkFirst' argument */
-    private int checkFirst = -1;
-
-    /** validate_indexes 'checkThrough' argument */
-    private int checkThrough = -1;
-
-    /** Cache view command. */
-    @Nullable private VisorViewCacheCmd cacheCmd;
-
-    /** Calculate partition hash and print into standard output. */
-    private boolean dump;
-
-    /** Skip zeros partitions. */
-    private boolean skipZeros;
-
-    /** Additional user attributes in result. Set of attribute names whose values will be searched in ClusterNode.attributes(). */
-    private Set<String> userAttributes;
-
-    /** Output format. */
-    private OutputFormat outputFormat;
-
-    /** Full config flag. */
-    private boolean fullConfig;
-
-    /** Cache filter. */
-    private CacheFilterEnum cacheFilterEnum = CacheFilterEnum.DEFAULT;
-
-    /** Check CRC sum on idle verify. */
-    private boolean idleCheckCrc;
-
-    /** Delete garbage flag. */
-    private boolean delete;
-
-    /**
-     * @return Gets filter of caches, which will by checked.
-     */
-    public CacheFilterEnum getCacheFilterEnum() {
-        return cacheFilterEnum;
-    }
-
-    /**
-     * @param cacheFilterEnum Cache filter.
-     */
-    public void setCacheFilterEnum(CacheFilterEnum cacheFilterEnum) {
-        this.cacheFilterEnum = cacheFilterEnum;
-    }
-
-    /**
-     * @return Full config flag.
-     */
-    public boolean fullConfig(){ return fullConfig; }
-
-    /**
-     * @param fullConfig New full config flag.
-     */
-    public void fullConfig(boolean fullConfig) { this.fullConfig = fullConfig; }
-
-    /**
-     * @return Command.
-     */
-    public CacheCommand command() {
-        return cmd;
-    }
-
-    /**
-     * @return Cache view command.
-     */
-    @Nullable public VisorViewCacheCmd cacheCommand() {
-        return cacheCmd;
-    }
-
-    /**
-     * @param cmd Cache view command.
-     */
-    public void cacheCommand(VisorViewCacheCmd cmd) {
-        this.cacheCmd = cmd;
-    }
-
-    /**
-     * @param cmd New command.
-     */
-    public void command(CacheCommand cmd) {
-        this.cmd = cmd;
-    }
-
-    /**
-     * @return Caches.
-     */
-    public Set<String> caches() {
-        return caches;
-    }
-
-    /**
-     * @param caches New caches.
-     */
-    public void caches(Set<String> caches) {
-        this.caches = caches;
-    }
-
-    /**
-     * @return Exclude caches or groups.
-     */
-    public Set<String> excludeCaches() {
-        return excludeCaches;
-    }
-
-    /**
-     * @param excludeCaches Excluse caches or groups.
-     */
-    public void excludeCaches(Set<String> excludeCaches) {
-        this.excludeCaches = excludeCaches;
-    }
-
-    /**
-     * @return Partition id.
-     */
-    public int partitionId() {
-        return partId;
-    }
-
-    /**
-     * @param partId New partition id.
-     */
-    public void partitionId(int partId) {
-        this.partId = partId;
-    }
-
-    /**
-     * @return Regex.
-     */
-    public String regex() {
-        return regex;
-    }
-
-    /**
-     * @param regex New regex.
-     */
-    public void regex(String regex) {
-        this.regex = regex;
-    }
-
-    /**
-     * @return Node id.
-     */
-    public UUID nodeId() {
-        return nodeId;
-    }
-
-    /**
-     * @param nodeId New node id.
-     */
-    public void nodeId(UUID nodeId) {
-        this.nodeId = nodeId;
-    }
-
-    /**
-     * @return Min queue size.
-     */
-    public int minQueueSize() {
-        return minQueueSize;
-    }
-
-    /**
-     * @param minQueueSize New min queue size.
-     */
-    public void minQueueSize(int minQueueSize) {
-        this.minQueueSize = minQueueSize;
-    }
-
-    /**
-     * @return Max print.
-     */
-    public int maxPrint() {
-        return maxPrint;
-    }
-
-    /**
-     * @param maxPrint New max print.
-     */
-    public void maxPrint(int maxPrint) {
-        this.maxPrint = maxPrint;
-    }
-
-    /**
-     * @return Max number of entries to be checked.
-     */
-    public int checkFirst() {
-        return checkFirst;
-    }
-
-    /**
-     * @param checkFirst Max number of entries to be checked.
-     */
-    public void checkFirst(int checkFirst) {
-        this.checkFirst = checkFirst;
-    }
-
-    /**
-     * @return Number of entries to check through.
-     */
-    public int checkThrough() {
-        return checkThrough;
-    }
-
-    /**
-     * @param checkThrough Number of entries to check through.
-     */
-    public void checkThrough(int checkThrough) {
-        this.checkThrough = checkThrough;
-    }
-
-    /**
-     * @return Calculate partition hash and print into standard output.
-     */
-    public boolean dump() {
-        return dump;
-    }
-
-    /**
-     * @param dump Calculate partition hash and print into standard output.
-     */
-    public void dump(boolean dump) {
-        this.dump = dump;
-    }
-
-    /**
-     * @return Skip zeros partitions(size == 0) in result.
-     */
-    public boolean isSkipZeros() {
-        return skipZeros;
-    }
-
-    /**
-     * @param skipZeros Skip zeros partitions.
-     */
-    public void skipZeros(boolean skipZeros) {
-        this.skipZeros = skipZeros;
-    }
-
-    /**
-     * @return Additional user attributes in result. Set of attribute names whose values will be searched in ClusterNode.attributes().
-     */
-    public Set<String> getUserAttributes() {
-        return userAttributes;
-    }
-
-    /**
-     * @param userAttrs New additional user attributes in result.
-     */
-    public void setUserAttributes(Set<String> userAttrs) {
-        userAttributes = userAttrs;
-    }
-
-    /**
-     * @return Output format.
-     */
-    public OutputFormat outputFormat() { return outputFormat; }
-
-    /**
-     * @param outputFormat New output format.
-     */
-    public void outputFormat(OutputFormat outputFormat) { this.outputFormat = outputFormat; }
-
-    /**
-     * @return Check page CRC sum on idle verify flag.
-     */
-    public boolean idleCheckCrc() {
-        return idleCheckCrc;
-    }
-
-    /**
-     * @param idleCheckCrc Check page CRC sum on idle verify flag.
-     */
-    public void idleCheckCrc(boolean idleCheckCrc) {
-        this.idleCheckCrc = idleCheckCrc;
-    }
-
-    /**
-     * @return Set of groups to run operation on.
-     */
-    public Set<String> groups() {
-        return groups;
-    }
-
-    /**
-     * @param groups Set of groups to run operation on.
-     */
-    public void groups(Set<String> groups) {
-        this.groups = groups;
-    }
-
-    /**
-     * @return Delete found garbage or not.
-     */
-    public boolean delete() {
-        return delete;
-    }
-
-    /**
-     * @param delete Delete found garbage or not.
-     */
-    public void delete(boolean delete) {
-        this.delete = delete;
-    }
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommandList.java
similarity index 68%
copy from modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommand.java
copy to modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommandList.java
index 2ccaecb..7eb3b60 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommand.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommandList.java
@@ -17,72 +17,78 @@
 
 package org.apache.ignite.internal.commandline.cache;
 
+import org.apache.ignite.internal.commandline.Command;
 import org.jetbrains.annotations.Nullable;
 
 /**
  *
  */
-public enum CacheCommand {
+public enum CacheCommandList {
     /**
      * Prints out help for the cache command.
      */
-    HELP("help"),
+    HELP("help", null),
 
     /**
      * Checks consistency of primary and backup partitions assuming no concurrent updates are happening in the cluster.
      */
-    IDLE_VERIFY("idle_verify"),
+    IDLE_VERIFY("idle_verify", new IdleVerify()),
 
     /**
      * Prints info regarding caches, groups or sequences.
      */
-    LIST("list"),
+    LIST("list", new CacheViewer()),
 
     /**
      * Validates indexes attempting to read each indexed entry.
      */
-    VALIDATE_INDEXES("validate_indexes"),
+    VALIDATE_INDEXES("validate_indexes", new CacheValidateIndexes()),
 
     /**
      * Prints info about contended keys (the keys concurrently locked from multiple transactions).
      */
-    CONTENTION("contention"),
+    CONTENTION("contention", new CacheContention()),
 
     /**
      * Collect information on the distribution of partitions.
      */
-    DISTRIBUTION("distribution"),
+    DISTRIBUTION("distribution", new CacheDistribution()),
 
     /**
      * Reset lost partitions
      */
-    RESET_LOST_PARTITIONS("reset_lost_partitions"),
+    RESET_LOST_PARTITIONS("reset_lost_partitions", new ResetLostPartitions()),
 
     /**
      * Find and remove garbage.
      */
-    FIND_AND_DELETE_GARBAGE("find_garbage");
+    FIND_AND_DELETE_GARBAGE("find_garbage", new FindAndDeleteGarbage());
 
 
     /** Enumerated values. */
-    private static final CacheCommand[] VALS = values();
+    private static final CacheCommandList[] VALS = values();
 
     /** Name. */
     private final String name;
 
+    /** */
+    private final Command command;
+
     /**
      * @param name Name.
+     * @param command Command implementation.
      */
-    CacheCommand(String name) {
+    CacheCommandList(String name, Command command) {
         this.name = name;
+        this.command = command;
     }
 
     /**
      * @param text Command text.
      * @return Command for the text.
      */
-    public static CacheCommand of(String text) {
-        for (CacheCommand cmd : CacheCommand.values()) {
+    public static CacheCommandList of(String text) {
+        for (CacheCommandList cmd : CacheCommandList.values()) {
             if (cmd.text().equalsIgnoreCase(text))
                 return cmd;
         }
@@ -98,12 +104,19 @@ public enum CacheCommand {
     }
 
     /**
+     * @return Cache subcommand implementation.
+     */
+    public Command subcommand() {
+        return command;
+    }
+
+    /**
      * Efficiently gets enumerated value from its ordinal.
      *
      * @param ord Ordinal value.
      * @return Enumerated value or {@code null} if ordinal out of range.
      */
-    @Nullable public static CacheCommand fromOrdinal(int ord) {
+    @Nullable public static CacheCommandList fromOrdinal(int ord) {
         return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommands.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommands.java
new file mode 100644
index 0000000..fd525e7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommands.java
@@ -0,0 +1,221 @@
+/*
+ * 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.cache;
+
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.Map;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.commandline.Command;
+import org.apache.ignite.internal.commandline.CommandArgIterator;
+import org.apache.ignite.internal.commandline.CommandLogger;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.SB;
+
+import static org.apache.ignite.internal.commandline.CommandHandler.UTILITY_NAME;
+import static org.apache.ignite.internal.commandline.CommandList.CACHE;
+import static org.apache.ignite.internal.commandline.CommandLogger.optional;
+import static org.apache.ignite.internal.commandline.CommonArgParser.getCommonOptions;
+import static org.apache.ignite.internal.commandline.cache.CacheSubcommands.CONTENTION;
+import static org.apache.ignite.internal.commandline.cache.CacheSubcommands.FIND_AND_DELETE_GARBAGE;
+import static org.apache.ignite.internal.commandline.cache.CacheSubcommands.HELP;
+import static org.apache.ignite.internal.commandline.cache.CacheSubcommands.LIST;
+import static org.apache.ignite.internal.commandline.cache.CacheSubcommands.VALIDATE_INDEXES;
+import static org.apache.ignite.spi.discovery.tcp.ipfinder.sharedfs.TcpDiscoverySharedFsIpFinder.DELIM;
+
+/**
+ * High-level "cache" command implementation.
+ */
+public class CacheCommands implements Command<CacheSubcommands> {
+    /** */
+    protected static final String NODE_ID = "nodeId";
+
+    /** */
+    protected static final String OP_NODE_ID = optional(NODE_ID);
+
+    /** */
+    private CommandLogger logger;
+
+    /** */
+    private CacheSubcommands subcommand;
+
+    /** {@inheritDoc} */
+    @Override public void printUsage(CommandLogger logger) {
+        logger.logWithIndent("View caches information in a cluster. For more details type:");
+        logger.logWithIndent(CommandLogger.join(" ", UTILITY_NAME, CACHE, HELP), 2);
+        logger.nl();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object execute(GridClientConfiguration clientCfg, CommandLogger logger) throws Exception {
+        this.logger = logger;
+
+        if (subcommand == CacheSubcommands.HELP) {
+            printCacheHelp();
+
+            return null;
+        }
+
+        Command command = subcommand.subcommand();
+
+        if (command == null)
+            throw new IllegalStateException("Unknown command " + subcommand);
+
+        return command.execute(clientCfg, logger);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void parseArguments(CommandArgIterator argIter) {
+        if (!argIter.hasNextSubArg()) {
+            throw new IllegalArgumentException("Arguments are expected for --cache subcommand, " +
+                "run '--cache help' for more info.");
+        }
+
+        String str = argIter.nextArg("").toLowerCase();
+
+        CacheSubcommands cmd = CacheSubcommands.of(str);
+
+        if (cmd == null)
+            cmd = CacheSubcommands.HELP;
+
+        switch (cmd) {
+            case HELP:
+                break;
+
+            case RESET_LOST_PARTITIONS:
+            case LIST:
+            case IDLE_VERIFY:
+            case VALIDATE_INDEXES:
+            case FIND_AND_DELETE_GARBAGE:
+            case CONTENTION:
+            case DISTRIBUTION:
+                cmd.subcommand().parseArguments(argIter);
+
+                break;
+
+            default:
+                throw new IllegalArgumentException("Unknown --cache subcommand " + cmd);
+        }
+
+        if (argIter.hasNextSubArg())
+            throw new IllegalArgumentException("Unexpected argument of --cache subcommand: " + argIter.peekNextArg());
+
+        this.subcommand = cmd;
+    }
+
+
+    /** */
+    private void printCacheHelp() {
+        logger.logWithIndent("The '" + CACHE + " subcommand' is used to get information about and perform actions" +
+            " with caches. The command has the following syntax:");
+        logger.nl();
+        logger.logWithIndent(CommandLogger.join(" ", UTILITY_NAME, CommandLogger.join(" ", getCommonOptions())) + " " +
+            CACHE + " [subcommand] <subcommand_parameters>");
+        logger.nl();
+        logger.logWithIndent("The subcommands that take " + OP_NODE_ID + " as an argument ('" + LIST + "', '"
+            + FIND_AND_DELETE_GARBAGE+ "', '" + CONTENTION + "' and '" + VALIDATE_INDEXES +
+            "') will be executed on the given node or on all server nodes" +
+            " if the option is not specified. Other commands will run on a random server node.");
+        logger.nl();
+        logger.nl();
+        logger.logWithIndent("Subcommands:");
+
+        Arrays.stream(CacheCommandList.values()).forEach(c -> {
+            if (c.subcommand() != null) c.subcommand().printUsage(logger);
+        });
+
+        logger.nl();
+    }
+
+
+    /**
+     * Print cache command usage with default indention.
+     *
+     * @param cmd Cache command.
+     * @param description Command description.
+     * @param paramsDesc Parameter desciptors.
+     * @param args Cache command arguments.
+     */
+    protected static void usageCache(
+        CommandLogger logger,
+        CacheSubcommands cmd,
+        String description,
+        Map<String, String> paramsDesc,
+        String... args
+    ) {
+        int indentsNum = 1;
+
+        logger.logWithIndent(DELIM, indentsNum);
+        logger.nl();
+        logger.logWithIndent(CommandLogger.join(" ", CACHE, cmd, CommandLogger.join(" ", args)), indentsNum++);
+        logger.nl();
+        logger.logWithIndent(description, indentsNum);
+        logger.nl();
+
+        if (!F.isEmpty(paramsDesc)) {
+            logger.logWithIndent("Parameters:", indentsNum);
+
+            usageCacheParams(logger, paramsDesc, indentsNum + 1);
+
+            logger.nl();
+        }
+    }
+
+    /**
+     * Print cache command arguments usage.
+     *
+     * @param logger Command logger.
+     * @param paramsDesc Cache command arguments description.
+     * @param indentsNum Number of indents.
+     */
+    private static void usageCacheParams(CommandLogger logger, Map<String, String> paramsDesc, int indentsNum) {
+        int maxParamLen = paramsDesc.keySet().stream().max(Comparator.comparingInt(String::length)).get().length();
+
+        for (Map.Entry<String, String> param : paramsDesc.entrySet())
+            logger.logWithIndent(extendToLen(param.getKey(), maxParamLen) + "  " + "- " + param.getValue(), indentsNum);
+    }
+
+    /**
+     * Appends spaces to end of input string for extending to needed length.
+     *
+     * @param s Input string.
+     * @param targetLen Needed length.
+     * @return String with appended spaces on the end.
+     */
+    private static String extendToLen(String s, int targetLen) {
+        assert targetLen >= 0;
+        assert s.length() <= targetLen;
+
+        if (s.length() == targetLen)
+            return s;
+
+        SB sb = new SB(targetLen);
+
+        sb.a(s);
+
+        for (int i = 0; i < targetLen - s.length(); i++)
+            sb.a(" ");
+
+        return sb.toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheSubcommands arg() {
+        return subcommand;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheContention.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheContention.java
new file mode 100644
index 0000000..b4875f7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheContention.java
@@ -0,0 +1,139 @@
+/*
+ * 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.cache;
+
+import java.util.UUID;
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.commandline.Command;
+import org.apache.ignite.internal.commandline.CommandArgIterator;
+import org.apache.ignite.internal.commandline.CommandLogger;
+import org.apache.ignite.internal.processors.cache.verify.ContentionInfo;
+import org.apache.ignite.internal.visor.verify.VisorContentionTask;
+import org.apache.ignite.internal.visor.verify.VisorContentionTaskArg;
+import org.apache.ignite.internal.visor.verify.VisorContentionTaskResult;
+
+import static org.apache.ignite.internal.commandline.CommandLogger.optional;
+import static org.apache.ignite.internal.commandline.TaskExecutor.BROADCAST_UUID;
+import static org.apache.ignite.internal.commandline.TaskExecutor.executeTaskByNameOnNode;
+import static org.apache.ignite.internal.commandline.cache.CacheCommands.OP_NODE_ID;
+import static org.apache.ignite.internal.commandline.cache.CacheCommands.usageCache;
+import static org.apache.ignite.internal.commandline.cache.CacheSubcommands.CONTENTION;
+
+/**
+ * Cache contention detection subcommand.
+ */
+public class CacheContention implements Command<CacheContention.Arguments> {
+    /** {@inheritDoc} */
+    @Override public void printUsage(CommandLogger logger) {
+        String description = "Show the keys that are point of contention for multiple transactions.";
+
+        usageCache(logger, CONTENTION, description, null, "minQueueSize",
+            OP_NODE_ID, optional("maxPrint"));
+    }
+
+    /**
+     * Container for command arguments.
+     */
+    public class Arguments {
+        /** Node id. */
+        private UUID nodeId;
+
+        /** Min queue size. */
+        private int minQueueSize;
+
+        /** Max print. */
+        private int maxPrint;
+
+        /**
+         *
+         */
+        public Arguments(UUID nodeId, int minQueueSize, int maxPrint) {
+            this.nodeId = nodeId;
+            this.minQueueSize = minQueueSize;
+            this.maxPrint = maxPrint;
+        }
+
+        /**
+         * @return Node id.
+         */
+        public UUID nodeId() {
+            return nodeId;
+        }
+
+        /**
+         * @return Min queue size.
+         */
+        public int minQueueSize() {
+            return minQueueSize;
+        }
+        /**
+         * @return Max print.
+         */
+        public int maxPrint() {
+            return maxPrint;
+        }
+    }
+
+    /**
+     * Command parsed arguments.
+     */
+    private Arguments args;
+
+    /** {@inheritDoc} */
+    @Override public Arguments arg() {
+        return args;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object execute(GridClientConfiguration clientCfg, CommandLogger logger) throws Exception {
+        VisorContentionTaskArg taskArg = new VisorContentionTaskArg(args.minQueueSize(), args.maxPrint());
+
+        UUID nodeId = args.nodeId() == null ? BROADCAST_UUID : args.nodeId();
+
+        VisorContentionTaskResult res;
+
+        try (GridClient client = Command.startClient(clientCfg);) {
+            res = executeTaskByNameOnNode(client, VisorContentionTask.class.getName(), taskArg, nodeId, clientCfg);
+        }
+
+        logger.printErrors(res.exceptions(), "Contention check failed on nodes:");
+
+        for (ContentionInfo info : res.getInfos())
+            info.print();
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void parseArguments(CommandArgIterator argIter) {
+        int minQueueSize = Integer.parseInt(argIter.nextArg("Min queue size expected"));
+
+        UUID nodeId = null;
+
+        if (argIter.hasNextSubArg())
+            nodeId = UUID.fromString(argIter.nextArg(""));
+
+        int maxPrint = 10;
+
+        if (argIter.hasNextSubArg())
+            maxPrint = Integer.parseInt(argIter.nextArg(""));
+
+        args = new Arguments(nodeId, minQueueSize, maxPrint);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheDistribution.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheDistribution.java
new file mode 100644
index 0000000..5104240
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheDistribution.java
@@ -0,0 +1,163 @@
+/*
+ * 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.cache;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.commandline.Command;
+import org.apache.ignite.internal.commandline.CommandArgIterator;
+import org.apache.ignite.internal.commandline.CommandHandler;
+import org.apache.ignite.internal.commandline.CommandLogger;
+import org.apache.ignite.internal.commandline.argument.CommandArgUtils;
+import org.apache.ignite.internal.commandline.cache.argument.DistributionCommandArg;
+import org.apache.ignite.internal.commandline.cache.distribution.CacheDistributionTask;
+import org.apache.ignite.internal.commandline.cache.distribution.CacheDistributionTaskArg;
+import org.apache.ignite.internal.commandline.cache.distribution.CacheDistributionTaskResult;
+
+import static org.apache.ignite.internal.commandline.CommandHandler.NULL;
+import static org.apache.ignite.internal.commandline.CommandLogger.optional;
+import static org.apache.ignite.internal.commandline.CommandLogger.or;
+import static org.apache.ignite.internal.commandline.TaskExecutor.BROADCAST_UUID;
+import static org.apache.ignite.internal.commandline.TaskExecutor.executeTaskByNameOnNode;
+import static org.apache.ignite.internal.commandline.cache.CacheCommands.NODE_ID;
+import static org.apache.ignite.internal.commandline.cache.CacheCommands.usageCache;
+import static org.apache.ignite.internal.commandline.cache.CacheSubcommands.DISTRIBUTION;
+import static org.apache.ignite.internal.commandline.cache.argument.DistributionCommandArg.USER_ATTRIBUTES;
+
+/**
+ * Would collect and print info about how data is spread between nodes and partitions.
+ */
+public class CacheDistribution implements Command<CacheDistribution.Arguments> {
+    /** {@inheritDoc} */
+    @Override public void printUsage(CommandLogger logger) {
+        String CACHES = "cacheName1,...,cacheNameN";
+        String description = "Prints the information about partition distribution.";
+
+        usageCache(logger, DISTRIBUTION, description, null,
+            or(NODE_ID, CommandHandler.NULL), optional(CACHES), optional(USER_ATTRIBUTES, "attrName1,...,attrNameN"));
+    }
+
+    /**
+     * Container for command arguments.
+     */
+    public class Arguments {
+        /** Caches. */
+        private Set<String> caches;
+
+        /** Node id. */
+        private UUID nodeId;
+
+        /** Additional user attributes in result. Set of attribute names whose values will be searched in ClusterNode.attributes(). */
+        private Set<String> userAttributes;
+
+        /**
+         *
+         */
+        public Arguments(Set<String> caches, UUID nodeId, Set<String> userAttributes) {
+            this.caches = caches;
+            this.nodeId = nodeId;
+            this.userAttributes = userAttributes;
+        }
+
+        /**
+         * @return Caches.
+         */
+        public Set<String> caches() {
+            return caches;
+        }
+
+        /**
+         * @return Node id.
+         */
+        public UUID nodeId() {
+            return nodeId;
+        }
+
+        /**
+         * @return Additional user attributes in result. Set of attribute names whose values will be searched in ClusterNode.attributes().
+         */
+        public Set<String> getUserAttributes() {
+            return userAttributes;
+        }
+    }
+
+    /** Command parsed arguments */
+    private Arguments args;
+
+    /** {@inheritDoc} */
+    @Override public Arguments arg() {
+        return args;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object execute(GridClientConfiguration clientCfg, CommandLogger logger) throws Exception {
+        CacheDistributionTaskArg taskArg = new CacheDistributionTaskArg(args.caches(), args.getUserAttributes());
+
+        UUID nodeId = args.nodeId() == null ? BROADCAST_UUID : args.nodeId();
+
+        CacheDistributionTaskResult res;
+
+        try (GridClient client = Command.startClient(clientCfg)) {
+            res = executeTaskByNameOnNode(client, CacheDistributionTask.class.getName(), taskArg, nodeId, clientCfg);
+        }
+
+        logger.printErrors(res.exceptions(), "Cache distrubution task failed on nodes:");
+
+        res.print(System.out);
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void parseArguments(CommandArgIterator argIter) {
+        UUID nodeId = null;
+        Set<String> caches = null;
+        Set<String> userAttributes = null;
+
+        String nodeIdStr = argIter.nextArg("Node id expected or null");
+
+        if (!NULL.equals(nodeIdStr))
+            nodeId = UUID.fromString(nodeIdStr);
+
+        while (argIter.hasNextSubArg()) {
+            String nextArg = argIter.nextArg("");
+
+            DistributionCommandArg arg = CommandArgUtils.of(nextArg, DistributionCommandArg.class);
+
+            if (arg == USER_ATTRIBUTES) {
+                nextArg = argIter.nextArg("User attributes are expected to be separated by commas");
+
+                userAttributes = new HashSet<>();
+
+                for (String userAttribute : nextArg.split(","))
+                    userAttributes.add(userAttribute.trim());
+
+                nextArg = (argIter.hasNextSubArg()) ? argIter.nextArg("") : null;
+
+            }
+
+            if (nextArg != null)
+                caches = argIter.parseStringSet(nextArg);
+        }
+
+        args = new Arguments(caches, nodeId, userAttributes);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheSubcommands.java
similarity index 51%
rename from modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommand.java
rename to modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheSubcommands.java
index 2ccaecb..4543186 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommand.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheSubcommands.java
@@ -17,72 +17,92 @@
 
 package org.apache.ignite.internal.commandline.cache;
 
+import org.apache.ignite.internal.commandline.Command;
+import org.apache.ignite.internal.commandline.argument.CommandArg;
+import org.apache.ignite.internal.commandline.cache.argument.DistributionCommandArg;
+import org.apache.ignite.internal.commandline.cache.argument.FindAndDeleteGarbageArg;
+import org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCommandArg;
+import org.apache.ignite.internal.commandline.cache.argument.ListCommandArg;
+import org.apache.ignite.internal.commandline.cache.argument.ValidateIndexesCommandArg;
 import org.jetbrains.annotations.Nullable;
 
 /**
  *
  */
-public enum CacheCommand {
+public enum CacheSubcommands {
     /**
      * Prints out help for the cache command.
      */
-    HELP("help"),
+    HELP("help", null, null),
 
     /**
      * Checks consistency of primary and backup partitions assuming no concurrent updates are happening in the cluster.
      */
-    IDLE_VERIFY("idle_verify"),
+    IDLE_VERIFY("idle_verify", IdleVerifyCommandArg.class, new IdleVerify()),
 
     /**
      * Prints info regarding caches, groups or sequences.
      */
-    LIST("list"),
+    LIST("list", ListCommandArg.class, new CacheViewer()),
 
     /**
      * Validates indexes attempting to read each indexed entry.
      */
-    VALIDATE_INDEXES("validate_indexes"),
+    VALIDATE_INDEXES("validate_indexes", ValidateIndexesCommandArg.class, new CacheValidateIndexes()),
 
     /**
      * Prints info about contended keys (the keys concurrently locked from multiple transactions).
      */
-    CONTENTION("contention"),
+    CONTENTION("contention", null, new CacheContention()),
 
     /**
      * Collect information on the distribution of partitions.
      */
-    DISTRIBUTION("distribution"),
+    DISTRIBUTION("distribution", DistributionCommandArg.class, new CacheDistribution()),
 
     /**
      * Reset lost partitions
      */
-    RESET_LOST_PARTITIONS("reset_lost_partitions"),
+    RESET_LOST_PARTITIONS("reset_lost_partitions", null, new ResetLostPartitions()),
 
     /**
      * Find and remove garbage.
      */
-    FIND_AND_DELETE_GARBAGE("find_garbage");
+    FIND_AND_DELETE_GARBAGE("find_garbage", FindAndDeleteGarbageArg.class, new FindAndDeleteGarbage());
 
 
     /** Enumerated values. */
-    private static final CacheCommand[] VALS = values();
+    private static final CacheSubcommands[] VALS = values();
+
+    /** Enum class with argument list for command. */
+    private final Class<? extends Enum<? extends CommandArg>> commandArgs;
 
     /** Name. */
     private final String name;
 
+    /** Command instance for certain type. */
+    private final Command command;
+
     /**
      * @param name Name.
+     * @param command Command realization.
      */
-    CacheCommand(String name) {
+    CacheSubcommands(
+        String name,
+        Class<? extends Enum<? extends CommandArg>> commandArgs,
+        Command command
+    ) {
         this.name = name;
+        this.commandArgs = commandArgs;
+        this.command = command;
     }
 
     /**
      * @param text Command text.
      * @return Command for the text.
      */
-    public static CacheCommand of(String text) {
-        for (CacheCommand cmd : CacheCommand.values()) {
+    public static CacheSubcommands of(String text) {
+        for (CacheSubcommands cmd : CacheSubcommands.values()) {
             if (cmd.text().equalsIgnoreCase(text))
                 return cmd;
         }
@@ -98,12 +118,26 @@ public enum CacheCommand {
     }
 
     /**
+     * @return Subcommand realization.
+     */
+    public Command subcommand() {
+        return command;
+    }
+
+    /**
+     * @return Enum class with argument list for command.
+     */
+    public Class<? extends Enum<? extends CommandArg>> getCommandArgs() {
+        return commandArgs;
+    }
+
+    /**
      * Efficiently gets enumerated value from its ordinal.
      *
      * @param ord Ordinal value.
      * @return Enumerated value or {@code null} if ordinal out of range.
      */
-    @Nullable public static CacheCommand fromOrdinal(int ord) {
+    @Nullable public static CacheSubcommands fromOrdinal(int ord) {
         return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheValidateIndexes.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheValidateIndexes.java
new file mode 100644
index 0000000..5e2f473
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheValidateIndexes.java
@@ -0,0 +1,265 @@
+/*
+ * 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.cache;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.commandline.Command;
+import org.apache.ignite.internal.commandline.CommandArgIterator;
+import org.apache.ignite.internal.commandline.CommandLogger;
+import org.apache.ignite.internal.commandline.argument.CommandArgUtils;
+import org.apache.ignite.internal.commandline.cache.argument.ValidateIndexesCommandArg;
+import org.apache.ignite.internal.processors.cache.verify.PartitionKey;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.verify.IndexIntegrityCheckIssue;
+import org.apache.ignite.internal.visor.verify.IndexValidationIssue;
+import org.apache.ignite.internal.visor.verify.ValidateIndexesPartitionResult;
+import org.apache.ignite.internal.visor.verify.VisorValidateIndexesJobResult;
+import org.apache.ignite.internal.visor.verify.VisorValidateIndexesTaskArg;
+import org.apache.ignite.internal.visor.verify.VisorValidateIndexesTaskResult;
+
+import static org.apache.ignite.internal.commandline.CommandLogger.optional;
+import static org.apache.ignite.internal.commandline.CommandLogger.or;
+import static org.apache.ignite.internal.commandline.TaskExecutor.executeTaskByNameOnNode;
+import static org.apache.ignite.internal.commandline.cache.CacheCommandList.IDLE_VERIFY;
+import static org.apache.ignite.internal.commandline.cache.CacheCommands.OP_NODE_ID;
+import static org.apache.ignite.internal.commandline.cache.CacheCommands.usageCache;
+import static org.apache.ignite.internal.commandline.cache.CacheSubcommands.VALIDATE_INDEXES;
+import static org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCommandArg.CACHE_FILTER;
+import static org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCommandArg.EXCLUDE_CACHES;
+import static org.apache.ignite.internal.commandline.cache.argument.ValidateIndexesCommandArg.CHECK_FIRST;
+import static org.apache.ignite.internal.commandline.cache.argument.ValidateIndexesCommandArg.CHECK_THROUGH;
+
+/**
+ * Validate indexes command.
+ */
+public class CacheValidateIndexes implements Command<CacheValidateIndexes.Arguments> {
+    /** {@inheritDoc} */
+    @Override public void printUsage(CommandLogger logger) {
+        String CACHES = "cacheName1,...,cacheNameN";
+        String description = "Verify counters and hash sums of primary and backup partitions for the specified " +
+            "caches/cache groups on an idle cluster and print out the differences, if any. " +
+            "Cache filtering options configure the set of caches that will be processed by " + IDLE_VERIFY + " command. " +
+            "Default value for the set of cache names (or cache group names) is all cache groups. Default value for " +
+            EXCLUDE_CACHES + " is empty set. Default value for " + CACHE_FILTER + " is no filtering. Therefore, " +
+            "the set of all caches is sequently filtered by cache name " +
+            "regexps, by cache type and after all by exclude regexps.";
+
+        Map<String, String> map = U.newLinkedHashMap(16);
+
+        map.put(CHECK_FIRST + " N", "validate only the first N keys");
+        map.put(CHECK_THROUGH + " K", "validate every Kth key");
+
+        usageCache(logger, VALIDATE_INDEXES, description, map,
+            optional(CACHES), OP_NODE_ID, optional(or(CHECK_FIRST + " N", CHECK_THROUGH + " K")));
+    }
+
+    /**
+     * Container for command arguments.
+     */
+    public class Arguments {
+         /** Caches. */
+        private Set<String> caches;
+
+        /** Node id. */
+        private UUID nodeId;
+
+        /** Max number of entries to be checked. */
+        private int checkFirst = -1;
+
+        /** Number of entries to check through. */
+        private int checkThrough = -1;
+
+        /**
+         *
+         */
+        public Arguments(Set<String> caches, UUID nodeId, int checkFirst, int checkThrough) {
+            this.caches = caches;
+            this.nodeId = nodeId;
+            this.checkFirst = checkFirst;
+            this.checkThrough = checkThrough;
+        }
+
+        /**
+         * @return Caches.
+         */
+        public Set<String> caches() {
+            return caches;
+        }
+
+        /**
+         * @return Max number of entries to be checked.
+         */
+        public int checkFirst() {
+            return checkFirst;
+        }
+
+        /**
+         * @return Number of entries to check through.
+         */
+        public int checkThrough() {
+            return checkThrough;
+        }
+
+
+        /**
+         * @return Node id.
+         */
+        public UUID nodeId() {
+            return nodeId;
+        }
+    }
+
+    /** Command parsed arguments. */
+    private Arguments args;
+
+    /** {@inheritDoc} */
+    @Override public Arguments arg() {
+        return args;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object execute(GridClientConfiguration clientCfg, CommandLogger logger) throws Exception {
+        VisorValidateIndexesTaskArg taskArg = new VisorValidateIndexesTaskArg(
+            args.caches(),
+            args.nodeId() != null ? Collections.singleton(args.nodeId()) : null,
+            args.checkFirst(),
+            args.checkThrough()
+        );
+
+        try (GridClient client = Command.startClient(clientCfg)) {
+            VisorValidateIndexesTaskResult taskRes = executeTaskByNameOnNode(
+                client, "org.apache.ignite.internal.visor.verify.VisorValidateIndexesTask", taskArg, null, clientCfg);
+
+            boolean errors = logger.printErrors(taskRes.exceptions(), "Index validation failed on nodes:");
+
+            for (Map.Entry<UUID, VisorValidateIndexesJobResult> nodeEntry : taskRes.results().entrySet()) {
+                if (!nodeEntry.getValue().hasIssues())
+                    continue;
+
+                errors = true;
+
+                logger.log("Index issues found on node " + nodeEntry.getKey() + ":");
+
+                Collection<IndexIntegrityCheckIssue> integrityCheckFailures = nodeEntry.getValue().integrityCheckFailures();
+
+                if (!integrityCheckFailures.isEmpty()) {
+                    for (IndexIntegrityCheckIssue is : integrityCheckFailures)
+                        logger.logWithIndent(is);
+                }
+
+                Map<PartitionKey, ValidateIndexesPartitionResult> partRes = nodeEntry.getValue().partitionResult();
+
+                for (Map.Entry<PartitionKey, ValidateIndexesPartitionResult> e : partRes.entrySet()) {
+                    ValidateIndexesPartitionResult res = e.getValue();
+
+                    if (!res.issues().isEmpty()) {
+                        logger.logWithIndent(CommandLogger.join(" ", e.getKey(), e.getValue()));
+
+                        for (IndexValidationIssue is : res.issues())
+                            logger.logWithIndent(is, 2);
+                    }
+                }
+
+                Map<String, ValidateIndexesPartitionResult> idxRes = nodeEntry.getValue().indexResult();
+
+                for (Map.Entry<String, ValidateIndexesPartitionResult> e : idxRes.entrySet()) {
+                    ValidateIndexesPartitionResult res = e.getValue();
+
+                    if (!res.issues().isEmpty()) {
+                        logger.logWithIndent(CommandLogger.join(" ", "SQL Index", e.getKey(), e.getValue()));
+
+                        for (IndexValidationIssue is : res.issues())
+                            logger.logWithIndent(is, 2);
+                    }
+                }
+            }
+
+            if (!errors)
+                logger.log("no issues found.");
+            else
+                logger.log("issues found (listed above).");
+
+            logger.nl();
+
+            return taskRes;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void parseArguments(CommandArgIterator argIter) {
+        int checkFirst = -1;
+        int checkThrough = -1;
+        UUID nodeId = null;
+        Set<String> caches = null;
+
+        int argsCnt = 0;
+
+        while (argIter.hasNextSubArg() && argsCnt++ < 4) {
+            String nextArg = argIter.nextArg("");
+
+            ValidateIndexesCommandArg arg = CommandArgUtils.of(nextArg, ValidateIndexesCommandArg.class);
+
+            if (arg == CHECK_FIRST || arg == CHECK_THROUGH) {
+                if (!argIter.hasNextSubArg())
+                    throw new IllegalArgumentException("Numeric value for '" + nextArg + "' parameter expected.");
+
+                int numVal;
+
+                String numStr = argIter.nextArg("");
+
+                try {
+                    numVal = Integer.parseInt(numStr);
+                }
+                catch (IllegalArgumentException e) {
+                    throw new IllegalArgumentException(
+                        "Not numeric value was passed for '" + nextArg + "' parameter: " + numStr
+                    );
+                }
+
+                if (numVal <= 0)
+                    throw new IllegalArgumentException("Value for '" + nextArg + "' property should be positive.");
+
+                if (arg == CHECK_FIRST)
+                    checkFirst = numVal;
+                else
+                    checkThrough = numVal;
+
+                continue;
+            }
+
+            try {
+                nodeId = UUID.fromString(nextArg);
+
+                continue;
+            }
+            catch (IllegalArgumentException ignored) {
+                //No-op.
+            }
+
+            caches = argIter.parseStringSet(nextArg);
+        }
+
+        args = new Arguments(caches, nodeId, checkFirst, checkThrough);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheViewer.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheViewer.java
new file mode 100644
index 0000000..654d6f8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheViewer.java
@@ -0,0 +1,430 @@
+/*
+ * 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.cache;
+
+import java.util.Collection;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.client.GridClientException;
+import org.apache.ignite.internal.commandline.Command;
+import org.apache.ignite.internal.commandline.CommandArgIterator;
+import org.apache.ignite.internal.commandline.CommandLogger;
+import org.apache.ignite.internal.commandline.OutputFormat;
+import org.apache.ignite.internal.commandline.TaskExecutor;
+import org.apache.ignite.internal.commandline.argument.CommandArgUtils;
+import org.apache.ignite.internal.commandline.cache.argument.ListCommandArg;
+import org.apache.ignite.internal.processors.cache.verify.CacheInfo;
+import org.apache.ignite.internal.util.typedef.internal.SB;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.cache.VisorCacheAffinityConfiguration;
+import org.apache.ignite.internal.visor.cache.VisorCacheConfiguration;
+import org.apache.ignite.internal.visor.cache.VisorCacheConfigurationCollectorTask;
+import org.apache.ignite.internal.visor.cache.VisorCacheConfigurationCollectorTaskArg;
+import org.apache.ignite.internal.visor.cache.VisorCacheEvictionConfiguration;
+import org.apache.ignite.internal.visor.cache.VisorCacheNearConfiguration;
+import org.apache.ignite.internal.visor.cache.VisorCacheRebalanceConfiguration;
+import org.apache.ignite.internal.visor.cache.VisorCacheStoreConfiguration;
+import org.apache.ignite.internal.visor.query.VisorQueryConfiguration;
+import org.apache.ignite.internal.visor.verify.VisorViewCacheCmd;
+import org.apache.ignite.internal.visor.verify.VisorViewCacheTask;
+import org.apache.ignite.internal.visor.verify.VisorViewCacheTaskArg;
+import org.apache.ignite.internal.visor.verify.VisorViewCacheTaskResult;
+
+import static org.apache.ignite.internal.commandline.CommandLogger.optional;
+import static org.apache.ignite.internal.commandline.CommandLogger.or;
+import static org.apache.ignite.internal.commandline.OutputFormat.MULTI_LINE;
+import static org.apache.ignite.internal.commandline.OutputFormat.SINGLE_LINE;
+import static org.apache.ignite.internal.commandline.TaskExecutor.executeTaskByNameOnNode;
+import static org.apache.ignite.internal.commandline.cache.CacheCommands.OP_NODE_ID;
+import static org.apache.ignite.internal.commandline.cache.CacheCommands.usageCache;
+import static org.apache.ignite.internal.commandline.cache.CacheSubcommands.LIST;
+import static org.apache.ignite.internal.commandline.cache.argument.ListCommandArg.CONFIG;
+import static org.apache.ignite.internal.commandline.cache.argument.ListCommandArg.GROUP;
+import static org.apache.ignite.internal.commandline.cache.argument.ListCommandArg.OUTPUT_FORMAT;
+import static org.apache.ignite.internal.commandline.cache.argument.ListCommandArg.SEQUENCE;
+import static org.apache.ignite.internal.visor.verify.VisorViewCacheCmd.CACHES;
+import static org.apache.ignite.internal.visor.verify.VisorViewCacheCmd.GROUPS;
+import static org.apache.ignite.internal.visor.verify.VisorViewCacheCmd.SEQ;
+
+/**
+ * Command to show caches on cluster.
+ */
+public class CacheViewer implements Command<CacheViewer.Arguments> {
+    /** {@inheritDoc} */
+    @Override public void printUsage(CommandLogger logger) {
+        String description = "Show information about caches, groups or sequences that match a regular expression. " +
+            "When executed without parameters, this subcommand prints the list of caches.";
+
+        Map<String, String> map = U.newLinkedHashMap(16);
+
+        map.put(CONFIG.toString(), "print all configuration parameters for each cache.");
+        map.put(OUTPUT_FORMAT + " " + MULTI_LINE, "print configuration parameters per line. This option has effect only " +
+            "when used with " + CONFIG + " and without " + optional(or(GROUP, SEQUENCE)) + ".");
+        map.put(GROUP.toString(), "print information about groups.");
+        map.put(SEQUENCE.toString(), "print information about sequences.");
+
+        usageCache(logger, LIST, description, map, "regexPattern",
+            optional(or(GROUP, SEQUENCE)), OP_NODE_ID, optional(CONFIG), optional(OUTPUT_FORMAT, MULTI_LINE));
+    }
+
+    /**
+     * Container for command arguments.
+     */
+    public static class Arguments {
+        /** Regex. */
+        private String regex;
+
+        /** Full config flag. */
+        private boolean fullConfig;
+
+        /** Node id. */
+        private UUID nodeId;
+
+        /** Cache view command. */
+        private VisorViewCacheCmd cacheCmd;
+
+        /** Output format. */
+        private OutputFormat outputFormat;
+
+        /**
+         *
+         */
+        public Arguments(String regex, boolean fullConfig, UUID nodeId, VisorViewCacheCmd cacheCmd, OutputFormat outputFormat) {
+            this.regex = regex;
+            this.fullConfig = fullConfig;
+            this.nodeId = nodeId;
+            this.cacheCmd = cacheCmd;
+            this.outputFormat = outputFormat;
+        }
+
+        /**
+         * @return Regex.
+         */
+        public String regex() {
+            return regex;
+        }
+
+        /**
+         * @return Node id.
+         */
+        public UUID nodeId() {
+            return nodeId;
+        }
+
+        /**
+         * @return Output format.
+         */
+        public OutputFormat outputFormat() { return outputFormat; }
+
+        /**
+         * @return Cache view command.
+         */
+        public VisorViewCacheCmd cacheCommand() {
+            return cacheCmd;
+        }
+
+        /**
+         * @return Full config flag.
+         */
+        public boolean fullConfig(){ return fullConfig; }
+    }
+
+    /** Command parsed arguments */
+    private Arguments args;
+
+    /** {@inheritDoc} */
+    @Override public Arguments arg() {
+        return args;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object execute(GridClientConfiguration clientCfg, CommandLogger logger) throws Exception {
+        VisorViewCacheTaskArg taskArg = new VisorViewCacheTaskArg(args.regex(), args.cacheCommand());
+
+        VisorViewCacheTaskResult res;
+
+        try (GridClient client = Command.startClient(clientCfg)) {
+            res = TaskExecutor.executeTaskByNameOnNode(
+                client,
+                VisorViewCacheTask.class.getName(),
+                taskArg,
+                args.nodeId(),
+                clientCfg
+            );
+
+            if (args.fullConfig() && args.cacheCommand() == CACHES)
+                cachesConfig(client, args, res, clientCfg, logger);
+            else
+                printCacheInfos(res.cacheInfos(), args.cacheCommand(), logger);
+        }
+
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void parseArguments(CommandArgIterator argIter) {
+        String regex = argIter.nextArg("Regex is expected");
+        boolean fullConfig = false;
+        VisorViewCacheCmd cacheCmd = CACHES;
+        OutputFormat outputFormat = SINGLE_LINE;
+        UUID nodeId = null;
+
+        while (argIter.hasNextSubArg()) {
+            String nextArg = argIter.nextArg("").toLowerCase();
+
+            ListCommandArg arg = CommandArgUtils.of(nextArg, ListCommandArg.class);
+
+            if (arg != null) {
+                switch (arg) {
+                    case GROUP:
+                        cacheCmd = GROUPS;
+
+                        break;
+
+                    case SEQUENCE:
+                        cacheCmd = SEQ;
+
+                        break;
+
+                    case OUTPUT_FORMAT:
+                        String tmp2 = argIter.nextArg("output format must be defined!").toLowerCase();
+
+                        outputFormat = OutputFormat.fromConsoleName(tmp2);
+
+                        break;
+
+                    case CONFIG:
+                        fullConfig = true;
+
+                        break;
+                }
+            }
+            else
+                nodeId = UUID.fromString(nextArg);
+        }
+
+        args = new Arguments(regex, fullConfig, nodeId, cacheCmd, outputFormat);
+    }
+
+    /**
+     * Maps VisorCacheConfiguration to key-value pairs.
+     *
+     * @param cfg Visor cache configuration.
+     * @return map of key-value pairs.
+     */
+    private static Map<String, Object> mapToPairs(VisorCacheConfiguration cfg) {
+        Map<String, Object> params = new LinkedHashMap<>();
+
+        VisorCacheAffinityConfiguration affinityCfg = cfg.getAffinityConfiguration();
+        VisorCacheNearConfiguration nearCfg = cfg.getNearConfiguration();
+        VisorCacheRebalanceConfiguration rebalanceCfg = cfg.getRebalanceConfiguration();
+        VisorCacheEvictionConfiguration evictCfg = cfg.getEvictionConfiguration();
+        VisorCacheStoreConfiguration storeCfg = cfg.getStoreConfiguration();
+        VisorQueryConfiguration qryCfg = cfg.getQueryConfiguration();
+
+        params.put("Name", cfg.getName());
+        params.put("Group", cfg.getGroupName());
+        params.put("Dynamic Deployment ID", cfg.getDynamicDeploymentId());
+        params.put("System", cfg.isSystem());
+
+        params.put("Mode", cfg.getMode());
+        params.put("Atomicity Mode", cfg.getAtomicityMode());
+        params.put("Statistic Enabled", cfg.isStatisticsEnabled());
+        params.put("Management Enabled", cfg.isManagementEnabled());
+
+        params.put("On-heap cache enabled", cfg.isOnheapCacheEnabled());
+        params.put("Partition Loss Policy", cfg.getPartitionLossPolicy());
+        params.put("Query Parallelism", cfg.getQueryParallelism());
+        params.put("Copy On Read", cfg.isCopyOnRead());
+        params.put("Listener Configurations", cfg.getListenerConfigurations());
+        params.put("Load Previous Value", cfg.isLoadPreviousValue());
+        params.put("Memory Policy Name", cfg.getMemoryPolicyName());
+        params.put("Node Filter", cfg.getNodeFilter());
+        params.put("Read From Backup", cfg.isReadFromBackup());
+        params.put("Topology Validator", cfg.getTopologyValidator());
+
+        params.put("Time To Live Eager Flag", cfg.isEagerTtl());
+
+        params.put("Write Synchronization Mode", cfg.getWriteSynchronizationMode());
+        params.put("Invalidate", cfg.isInvalidate());
+
+        params.put("Affinity Function", affinityCfg.getFunction());
+        params.put("Affinity Backups", affinityCfg.getPartitionedBackups());
+        params.put("Affinity Partitions", affinityCfg.getPartitions());
+        params.put("Affinity Exclude Neighbors", affinityCfg.isExcludeNeighbors());
+        params.put("Affinity Mapper", affinityCfg.getMapper());
+
+        params.put("Rebalance Mode", rebalanceCfg.getMode());
+        params.put("Rebalance Batch Size", rebalanceCfg.getBatchSize());
+        params.put("Rebalance Timeout", rebalanceCfg.getTimeout());
+        params.put("Rebalance Delay", rebalanceCfg.getPartitionedDelay());
+        params.put("Time Between Rebalance Messages", rebalanceCfg.getThrottle());
+        params.put("Rebalance Batches Count", rebalanceCfg.getBatchesPrefetchCnt());
+        params.put("Rebalance Cache Order", rebalanceCfg.getRebalanceOrder());
+
+        params.put("Eviction Policy Enabled", (evictCfg.getPolicy() != null));
+        params.put("Eviction Policy Factory", evictCfg.getPolicy());
+        params.put("Eviction Policy Max Size", evictCfg.getPolicyMaxSize());
+        params.put("Eviction Filter", evictCfg.getFilter());
+
+        params.put("Near Cache Enabled", nearCfg.isNearEnabled());
+        params.put("Near Start Size", nearCfg.getNearStartSize());
+        params.put("Near Eviction Policy Factory", nearCfg.getNearEvictPolicy());
+        params.put("Near Eviction Policy Max Size", nearCfg.getNearEvictMaxSize());
+
+        params.put("Default Lock Timeout", cfg.getDefaultLockTimeout());
+        params.put("Query Entities", cfg.getQueryEntities());
+        params.put("Cache Interceptor", cfg.getInterceptor());
+
+        params.put("Store Enabled", storeCfg.isEnabled());
+        params.put("Store Class", storeCfg.getStore());
+        params.put("Store Factory Class", storeCfg.getStoreFactory());
+        params.put("Store Keep Binary", storeCfg.isStoreKeepBinary());
+        params.put("Store Read Through", storeCfg.isReadThrough());
+        params.put("Store Write Through", storeCfg.isWriteThrough());
+        params.put("Store Write Coalescing", storeCfg.getWriteBehindCoalescing());
+
+        params.put("Write-Behind Enabled", storeCfg.isWriteBehindEnabled());
+        params.put("Write-Behind Flush Size", storeCfg.getFlushSize());
+        params.put("Write-Behind Frequency", storeCfg.getFlushFrequency());
+        params.put("Write-Behind Flush Threads Count", storeCfg.getFlushThreadCount());
+        params.put("Write-Behind Batch Size", storeCfg.getBatchSize());
+
+        params.put("Concurrent Asynchronous Operations Number", cfg.getMaxConcurrentAsyncOperations());
+
+        params.put("Loader Factory Class Name", cfg.getLoaderFactory());
+        params.put("Writer Factory Class Name", cfg.getWriterFactory());
+        params.put("Expiry Policy Factory Class Name", cfg.getExpiryPolicyFactory());
+
+        params.put("Query Execution Time Threshold", qryCfg.getLongQueryWarningTimeout());
+        params.put("Query Escaped Names", qryCfg.isSqlEscapeAll());
+        params.put("Query SQL Schema", qryCfg.getSqlSchema());
+        params.put("Query SQL functions", qryCfg.getSqlFunctionClasses());
+        params.put("Query Indexed Types", qryCfg.getIndexedTypes());
+        params.put("Maximum payload size for offheap indexes", cfg.getSqlIndexMaxInlineSize());
+        params.put("Query Metrics History Size", cfg.getQueryDetailMetricsSize());
+
+        return params;
+    }
+
+    /**
+     * Prints caches config.
+     *
+     * @param caches Caches config.
+     * @param outputFormat Output format.
+     * @param cacheToMapped Map cache name to mapped.
+     */
+    private void printCachesConfig(
+        Map<String, VisorCacheConfiguration> caches,
+        OutputFormat outputFormat,
+        Map<String, Integer> cacheToMapped,
+        CommandLogger logger
+    ) {
+
+        for (Map.Entry<String, VisorCacheConfiguration> entry : caches.entrySet()) {
+            String cacheName = entry.getKey();
+
+            switch (outputFormat) {
+                case MULTI_LINE:
+                    Map<String, Object> params = mapToPairs(entry.getValue());
+
+                    params.put("Mapped", cacheToMapped.get(cacheName));
+
+                    logger.log("[cache = '%s']%n", cacheName);
+
+                    for (Map.Entry<String, Object> innerEntry : params.entrySet())
+                        logger.log("%s: %s%n", innerEntry.getKey(), innerEntry.getValue());
+
+                    logger.nl();
+
+                    break;
+
+                default:
+                    int mapped = cacheToMapped.get(cacheName);
+
+                    logger.log("%s: %s %s=%s%n", entry.getKey(), toString(entry.getValue()), "mapped", mapped);
+
+                    break;
+            }
+        }
+    }
+
+    /**
+     * Invokes toString() method and cuts class name from result string.
+     *
+     * @param cfg Visor cache configuration for invocation.
+     * @return String representation without class name in begin of string.
+     */
+    private String toString(VisorCacheConfiguration cfg) {
+        return cfg.toString().substring(cfg.getClass().getSimpleName().length() + 1);
+    }
+
+    /**
+     * @param client Client.
+     * @param cacheArgs Cache args.
+     * @param viewRes Cache view task result.
+     * @param clientCfg Client configuration.
+     */
+    private void cachesConfig(
+        GridClient client,
+        Arguments cacheArgs,
+        VisorViewCacheTaskResult viewRes,
+        GridClientConfiguration clientCfg,
+        CommandLogger logger
+    ) throws GridClientException {
+        VisorCacheConfigurationCollectorTaskArg taskArg = new VisorCacheConfigurationCollectorTaskArg(cacheArgs.regex());
+
+        UUID nodeId = cacheArgs.nodeId() == null ? TaskExecutor.BROADCAST_UUID : cacheArgs.nodeId();
+
+        Map<String, VisorCacheConfiguration> res =
+            executeTaskByNameOnNode(client, VisorCacheConfigurationCollectorTask.class.getName(), taskArg, nodeId, clientCfg);
+
+        Map<String, Integer> cacheToMapped =
+            viewRes.cacheInfos().stream().collect(Collectors.toMap(CacheInfo::getCacheName, CacheInfo::getMapped));
+
+        printCachesConfig(res, cacheArgs.outputFormat(), cacheToMapped, logger);
+    }
+
+    /**
+     * Prints caches info.
+     *
+     * @param infos Caches info.
+     * @param cmd Command.
+     */
+    private void printCacheInfos(Collection<CacheInfo> infos, VisorViewCacheCmd cmd, CommandLogger logger) {
+        for (CacheInfo info : infos) {
+            Map<String, Object> map = info.toMap(cmd);
+
+            SB sb = new SB("[");
+
+            for (Map.Entry<String, Object> e : map.entrySet())
+                sb.a(e.getKey()).a("=").a(e.getValue()).a(", ");
+
+            sb.setLength(sb.length() - 2);
+
+            sb.a("]");
+
+            logger.log(sb.toString());
+        }
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/FindAndDeleteGarbage.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/FindAndDeleteGarbage.java
new file mode 100644
index 0000000..f814a9e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/FindAndDeleteGarbage.java
@@ -0,0 +1,185 @@
+/*
+ * 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.cache;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.commandline.Command;
+import org.apache.ignite.internal.commandline.CommandArgIterator;
+import org.apache.ignite.internal.commandline.CommandLogger;
+import org.apache.ignite.internal.commandline.argument.CommandArgUtils;
+import org.apache.ignite.internal.commandline.cache.argument.FindAndDeleteGarbageArg;
+import org.apache.ignite.internal.visor.cache.VisorFindAndDeleteGarbageInPersistenceJobResult;
+import org.apache.ignite.internal.visor.cache.VisorFindAndDeleteGarbageInPersistenceTask;
+import org.apache.ignite.internal.visor.cache.VisorFindAndDeleteGarbageInPersistenceTaskArg;
+import org.apache.ignite.internal.visor.cache.VisorFindAndDeleteGarbageInPersistenceTaskResult;
+
+import static org.apache.ignite.internal.commandline.CommandLogger.optional;
+import static org.apache.ignite.internal.commandline.TaskExecutor.executeTask;
+import static org.apache.ignite.internal.commandline.cache.CacheCommands.OP_NODE_ID;
+import static org.apache.ignite.internal.commandline.cache.CacheCommands.usageCache;
+import static org.apache.ignite.internal.commandline.cache.CacheSubcommands.FIND_AND_DELETE_GARBAGE;
+
+/**
+ * Command to find and delete garbage which could left after destroying caches in shared group.
+ */
+public class FindAndDeleteGarbage implements Command<FindAndDeleteGarbage.Arguments> {
+    /** {@inheritDoc} */
+    @Override public void printUsage(CommandLogger logger) {
+        String GROUPS = "groupName1,...,groupNameN";
+        String description = "Find and optionally delete garbage from shared cache groups which could be left " +
+            "after cache destroy.";
+
+        usageCache(logger, FIND_AND_DELETE_GARBAGE, description, null,
+            optional(GROUPS), OP_NODE_ID, optional(FindAndDeleteGarbageArg.DELETE));
+    }
+
+    /**
+     * Container for command arguments.
+     */
+    public static class Arguments {
+        /** Groups. */
+        private Set<String> groups;
+
+        /** Node id. */
+        private UUID nodeId;
+
+        /** Delete garbage flag. */
+        private boolean delete;
+
+        /**
+         *
+         */
+        public Arguments(Set<String> groups, UUID nodeId, boolean delete) {
+            this.groups = groups;
+            this.nodeId = nodeId;
+            this.delete = delete;
+        }
+
+        /**
+         * @return Node id.
+         */
+        public UUID nodeId() {
+            return nodeId;
+        }
+
+        /**
+         * @return Cache group to scan for, null means scanning all groups.
+         */
+        public Set<String> groups() {
+            return groups;
+        }
+
+        /**
+         * @return True if it is needed to delete found garbage.
+         */
+        public boolean delete() {
+            return delete;
+        }
+    }
+
+    /** Command parsed arguments. */
+    private Arguments args;
+
+    /** {@inheritDoc} */
+    @Override public Arguments arg() {
+        return args;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object execute(GridClientConfiguration clientCfg, CommandLogger logger) throws Exception {
+        VisorFindAndDeleteGarbageInPersistenceTaskArg taskArg = new VisorFindAndDeleteGarbageInPersistenceTaskArg(
+            args.groups(),
+            args.delete(),
+            args.nodeId() != null ? Collections.singleton(args.nodeId()) : null
+        );
+
+        try (GridClient client = Command.startClient(clientCfg)) {
+            VisorFindAndDeleteGarbageInPersistenceTaskResult taskRes = executeTask(
+                client, VisorFindAndDeleteGarbageInPersistenceTask.class, taskArg, clientCfg);
+
+            logger.printErrors(taskRes.exceptions(), "Scanning for garbage failed on nodes:");
+
+            for (Map.Entry<UUID, VisorFindAndDeleteGarbageInPersistenceJobResult> nodeEntry : taskRes.result().entrySet()) {
+                if (!nodeEntry.getValue().hasGarbage()) {
+                    logger.log("Node " + nodeEntry.getKey() + " - garbage not found.");
+
+                    continue;
+                }
+
+                logger.log("Garbage found on node " + nodeEntry.getKey() + ":");
+
+                VisorFindAndDeleteGarbageInPersistenceJobResult value = nodeEntry.getValue();
+
+                Map<Integer, Map<Integer, Long>> grpPartErrorsCount = value.checkResult();
+
+                if (!grpPartErrorsCount.isEmpty()) {
+                    for (Map.Entry<Integer, Map<Integer, Long>> entry : grpPartErrorsCount.entrySet()) {
+                        for (Map.Entry<Integer, Long> e : entry.getValue().entrySet()) {
+                            logger.logWithIndent("Group=" + entry.getKey() +
+                                ", partition=" + e.getKey() +
+                                ", count of keys=" + e.getValue());
+                        }
+                    }
+                }
+
+                logger.nl();
+            }
+
+            return taskRes;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void parseArguments(CommandArgIterator argIter) {
+        boolean delete = false;
+        UUID nodeId = null;
+        Set<String> groups = null;
+
+        int argsCnt = 0;
+
+        while (argIter.hasNextSubArg() && argsCnt++ < 3) {
+            String nextArg = argIter.nextArg("");
+
+            FindAndDeleteGarbageArg arg = CommandArgUtils.of(nextArg, FindAndDeleteGarbageArg.class);
+
+            if (arg == FindAndDeleteGarbageArg.DELETE) {
+                delete = true;
+
+                continue;
+            }
+
+            try {
+                nodeId = UUID.fromString(nextArg);
+
+                continue;
+            }
+            catch (IllegalArgumentException ignored) {
+                //No-op.
+            }
+
+            groups = argIter.parseStringSet(nextArg);
+        }
+
+        args = new Arguments(groups, nodeId, delete);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/IdleVerify.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/IdleVerify.java
new file mode 100644
index 0000000..f249fc8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/IdleVerify.java
@@ -0,0 +1,354 @@
+/*
+ * 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.cache;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.IgniteNodeAttributes;
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.client.GridClientException;
+import org.apache.ignite.internal.client.GridClientNode;
+import org.apache.ignite.internal.commandline.Command;
+import org.apache.ignite.internal.commandline.CommandArgIterator;
+import org.apache.ignite.internal.commandline.CommandLogger;
+import org.apache.ignite.internal.commandline.argument.CommandArgUtils;
+import org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCommandArg;
+import org.apache.ignite.internal.processors.cache.verify.IdleVerifyResultV2;
+import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecord;
+import org.apache.ignite.internal.processors.cache.verify.PartitionKey;
+import org.apache.ignite.internal.processors.cache.verify.VerifyBackupPartitionsTaskV2;
+import org.apache.ignite.internal.visor.verify.CacheFilterEnum;
+import org.apache.ignite.internal.visor.verify.VisorIdleVerifyDumpTask;
+import org.apache.ignite.internal.visor.verify.VisorIdleVerifyDumpTaskArg;
+import org.apache.ignite.internal.visor.verify.VisorIdleVerifyTask;
+import org.apache.ignite.internal.visor.verify.VisorIdleVerifyTaskArg;
+import org.apache.ignite.internal.visor.verify.VisorIdleVerifyTaskResult;
+import org.apache.ignite.internal.visor.verify.VisorIdleVerifyTaskV2;
+import org.apache.ignite.lang.IgniteProductVersion;
+
+import static java.lang.String.format;
+import static org.apache.ignite.internal.commandline.CommandLogger.optional;
+import static org.apache.ignite.internal.commandline.CommandLogger.or;
+import static org.apache.ignite.internal.commandline.TaskExecutor.executeTask;
+import static org.apache.ignite.internal.commandline.cache.CacheCommands.usageCache;
+import static org.apache.ignite.internal.commandline.cache.CacheSubcommands.IDLE_VERIFY;
+import static org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCommandArg.CACHE_FILTER;
+import static org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCommandArg.CHECK_CRC;
+import static org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCommandArg.DUMP;
+import static org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCommandArg.EXCLUDE_CACHES;
+import static org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCommandArg.SKIP_ZEROS;
+
+/**
+ *
+ */
+public class IdleVerify implements Command<IdleVerify.Arguments> {
+    /** {@inheritDoc} */
+    @Override public void printUsage(CommandLogger logger) {
+        String CACHES = "cacheName1,...,cacheNameN";
+        String description = "Verify counters and hash sums of primary and backup partitions for the specified " +
+            "caches/cache groups on an idle cluster and print out the differences, if any. " +
+            "Cache filtering options configure the set of caches that will be processed by idle_verify command. " +
+            "Default value for the set of cache names (or cache group names) is all cache groups. Default value" +
+            " for " + EXCLUDE_CACHES + " is empty set. " +
+            "Default value for " + CACHE_FILTER + " is no filtering. Therefore, the set of all caches is sequently " +
+            "filtered by cache name " +
+            "regexps, by cache type and after all by exclude regexps.";
+
+        usageCache(logger,
+            IDLE_VERIFY,
+            description,
+            Collections.singletonMap(CHECK_CRC.toString(),
+                "check the CRC-sum of pages stored on disk before verifying data " +
+                    "consistency in partitions between primary and backup nodes."),
+            optional(DUMP), optional(SKIP_ZEROS), optional(CHECK_CRC), optional(EXCLUDE_CACHES, CACHES),
+                optional(CACHE_FILTER, or(CacheFilterEnum.values())), optional(CACHES));
+    }
+
+    /**
+     * Container for command arguments.
+     */
+    public static class Arguments {
+        /** Caches. */
+        private Set<String> caches;
+
+        /** Exclude caches or groups. */
+        private Set<String> excludeCaches;
+
+        /** Calculate partition hash and print into standard output. */
+        private boolean dump;
+
+        /** Skip zeros partitions. */
+        private boolean skipZeros;
+
+        /** Check CRC sum on idle verify. */
+        private boolean idleCheckCrc;
+
+        /** Cache filter. */
+        private CacheFilterEnum cacheFilterEnum;
+
+        /**
+         *
+         */
+        public Arguments(Set<String> caches, Set<String> excludeCaches, boolean dump, boolean skipZeros,
+            boolean idleCheckCrc,
+            CacheFilterEnum cacheFilterEnum) {
+            this.caches = caches;
+            this.excludeCaches = excludeCaches;
+            this.dump = dump;
+            this.skipZeros = skipZeros;
+            this.idleCheckCrc = idleCheckCrc;
+            this.cacheFilterEnum = cacheFilterEnum;
+        }
+
+        /**
+         * @return Gets filter of caches, which will by checked.
+         */
+        public CacheFilterEnum getCacheFilterEnum() {
+            return cacheFilterEnum;
+        }
+
+        /**
+         * @return Caches.
+         */
+        public Set<String> caches() {
+            return caches;
+        }
+
+        /**
+         * @return Exclude caches or groups.
+         */
+        public Set<String> excludeCaches() {
+            return excludeCaches;
+        }
+
+        /**
+         * @return Calculate partition hash and print into standard output.
+         */
+        public boolean dump() {
+            return dump;
+        }
+
+        /**
+         * @return Check page CRC sum on idle verify flag.
+         */
+        public boolean idleCheckCrc() {
+            return idleCheckCrc;
+        }
+
+
+        /**
+         * @return Skip zeros partitions(size == 0) in result.
+         */
+        public boolean isSkipZeros() {
+            return skipZeros;
+        }
+    }
+
+    /** Command parsed arguments. */
+    private Arguments args;
+
+    /** {@inheritDoc} */
+    @Override public Arguments arg() {
+        return args;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object execute(GridClientConfiguration clientCfg, CommandLogger logger) throws Exception {
+        try (GridClient client = Command.startClient(clientCfg)) {
+            Collection<GridClientNode> nodes = client.compute().nodes(GridClientNode::connectable);
+
+            boolean idleVerifyV2 = true;
+
+            for (GridClientNode node : nodes) {
+                String nodeVerStr = node.attribute(IgniteNodeAttributes.ATTR_BUILD_VER);
+
+                IgniteProductVersion nodeVer = IgniteProductVersion.fromString(nodeVerStr);
+
+                if (nodeVer.compareTo(VerifyBackupPartitionsTaskV2.V2_SINCE_VER) < 0) {
+                    idleVerifyV2 = false;
+
+                    break;
+                }
+            }
+
+            if (args.dump())
+                cacheIdleVerifyDump(client, clientCfg, logger);
+            else if (idleVerifyV2)
+                cacheIdleVerifyV2(client, clientCfg);
+            else
+                legacyCacheIdleVerify(client, clientCfg, logger);
+        }
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void parseArguments(CommandArgIterator argIter) {
+        Set<String> cacheNames = null;
+        boolean dump = false;
+        boolean skipZeros = false;
+        boolean idleCheckCrc = false;
+        CacheFilterEnum cacheFilterEnum = CacheFilterEnum.DEFAULT;
+        Set<String> excludeCaches = null;
+
+        int idleVerifyArgsCnt = 5;
+
+        while (argIter.hasNextSubArg() && idleVerifyArgsCnt-- > 0) {
+            String nextArg = argIter.nextArg("");
+
+            IdleVerifyCommandArg arg = CommandArgUtils.of(nextArg, IdleVerifyCommandArg.class);
+
+            if (arg == null) {
+                cacheNames = argIter.parseStringSet(nextArg);
+
+                validateRegexes(cacheNames);
+            }
+            else {
+                switch (arg) {
+                    case DUMP:
+                        dump = true;
+
+                        break;
+
+                    case SKIP_ZEROS:
+                        skipZeros = true;
+
+                        break;
+
+                    case CHECK_CRC:
+                        idleCheckCrc = true;
+
+                        break;
+
+                    case CACHE_FILTER:
+                        String filter = argIter.nextArg("The cache filter should be specified. The following " +
+                            "values can be used: " + Arrays.toString(CacheFilterEnum.values()) + '.');
+
+                        cacheFilterEnum = CacheFilterEnum.valueOf(filter.toUpperCase());
+
+                        break;
+
+                    case EXCLUDE_CACHES:
+                        excludeCaches = argIter.nextStringSet("caches, which will be excluded.");
+
+                        validateRegexes(excludeCaches);
+
+                        break;
+                }
+            }
+        }
+
+        args = new Arguments(cacheNames, excludeCaches, dump, skipZeros, idleCheckCrc, cacheFilterEnum);
+    }
+
+    /**
+     * @param string To validate that given name is valed regex.
+     */
+    private void validateRegexes(Set<String> string) {
+        string.forEach(s -> {
+            try {
+                Pattern.compile(s);
+            }
+            catch (PatternSyntaxException e) {
+                throw new IgniteException(format("Invalid cache name regexp '%s': %s", s, e.getMessage()));
+            }
+        });
+    }
+
+    /**
+     * @param client Client.
+     * @param clientCfg Client configuration.
+     */
+    private void cacheIdleVerifyDump(
+        GridClient client,
+        GridClientConfiguration clientCfg,
+        CommandLogger logger
+    ) throws GridClientException {
+        VisorIdleVerifyDumpTaskArg arg = new VisorIdleVerifyDumpTaskArg(
+            args.caches(),
+            args.excludeCaches(),
+            args.isSkipZeros(),
+            args.getCacheFilterEnum(),
+            args.idleCheckCrc()
+        );
+
+        String path = executeTask(client, VisorIdleVerifyDumpTask.class, arg, clientCfg);
+
+        logger.log("VisorIdleVerifyDumpTask successfully written output to '" + path + "'");
+    }
+
+
+    /**
+     * @param client Client.
+     * @param clientCfg Client configuration.
+     */
+    private void cacheIdleVerifyV2(
+        GridClient client,
+        GridClientConfiguration clientCfg
+    ) throws GridClientException {
+        IdleVerifyResultV2 res = executeTask(
+            client,
+            VisorIdleVerifyTaskV2.class,
+            new VisorIdleVerifyTaskArg(args.caches(), args.excludeCaches(), args.idleCheckCrc()),
+            clientCfg);
+
+        res.print(System.out::print);
+    }
+
+
+    /**
+     * @param client Client.
+     * @param clientCfg Client configuration.
+     */
+    private void legacyCacheIdleVerify(
+        GridClient client,
+        GridClientConfiguration clientCfg,
+        CommandLogger logger
+    ) throws GridClientException {
+        VisorIdleVerifyTaskResult res = executeTask(
+            client,
+            VisorIdleVerifyTask.class,
+            new VisorIdleVerifyTaskArg(args.caches(), args.excludeCaches(), args.idleCheckCrc()),
+            clientCfg);
+
+        Map<PartitionKey, List<PartitionHashRecord>> conflicts = res.getConflicts();
+
+        if (conflicts.isEmpty()) {
+            logger.log("idle_verify check has finished, no conflicts have been found.");
+            logger.nl();
+        }
+        else {
+            logger.log("idle_verify check has finished, found " + conflicts.size() + " conflict partitions.");
+            logger.nl();
+
+            for (Map.Entry<PartitionKey, List<PartitionHashRecord>> entry : conflicts.entrySet()) {
+                logger.log("Conflict partition: " + entry.getKey());
+
+                logger.log("Partition instances: " + entry.getValue());
+            }
+        }
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/ResetLostPartitions.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/ResetLostPartitions.java
new file mode 100644
index 0000000..8b4107e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/ResetLostPartitions.java
@@ -0,0 +1,74 @@
+/*
+ * 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.cache;
+
+import java.util.Set;
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.commandline.Command;
+import org.apache.ignite.internal.commandline.CommandArgIterator;
+import org.apache.ignite.internal.commandline.CommandLogger;
+import org.apache.ignite.internal.commandline.cache.reset_lost_partitions.CacheResetLostPartitionsTask;
+import org.apache.ignite.internal.commandline.cache.reset_lost_partitions.CacheResetLostPartitionsTaskArg;
+import org.apache.ignite.internal.commandline.cache.reset_lost_partitions.CacheResetLostPartitionsTaskResult;
+
+import static org.apache.ignite.internal.commandline.TaskExecutor.executeTaskByNameOnNode;
+import static org.apache.ignite.internal.commandline.cache.CacheCommands.usageCache;
+import static org.apache.ignite.internal.commandline.cache.CacheSubcommands.RESET_LOST_PARTITIONS;
+
+/**
+ * Command for reseting lost partition state.
+ */
+public class ResetLostPartitions implements Command<Set<String>> {
+    /** {@inheritDoc} */
+    @Override public void printUsage(CommandLogger logger) {
+        String CACHES = "cacheName1,...,cacheNameN";
+        String description = "Reset the state of lost partitions for the specified caches.";
+
+        usageCache(logger, RESET_LOST_PARTITIONS, description, null, CACHES);
+    }
+
+    /**
+     * Command argument. Caches which lost partitions should be reseted.
+     */
+    private Set<String> caches;
+
+    /** {@inheritDoc} */
+    @Override public Set<String> arg() {
+        return caches;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object execute(GridClientConfiguration clientCfg, CommandLogger logger) throws Exception {
+        CacheResetLostPartitionsTaskArg taskArg = new CacheResetLostPartitionsTaskArg(caches);
+
+        try (GridClient client = Command.startClient(clientCfg)) {
+            CacheResetLostPartitionsTaskResult res =
+                executeTaskByNameOnNode(client, CacheResetLostPartitionsTask.class.getName(), taskArg, null, clientCfg);
+
+            res.print(System.out);
+
+            return res;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void parseArguments(CommandArgIterator argIter) {
+        caches = argIter.nextStringSet("Cache names");
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/argument/DistributionCommandArg.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/argument/DistributionCommandArg.java
index 129f7a8..661ba9e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/argument/DistributionCommandArg.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/argument/DistributionCommandArg.java
@@ -18,10 +18,10 @@
 package org.apache.ignite.internal.commandline.cache.argument;
 
 import org.apache.ignite.internal.commandline.argument.CommandArg;
-import org.apache.ignite.internal.commandline.cache.CacheCommand;
+import org.apache.ignite.internal.commandline.cache.CacheSubcommands;
 
 /**
- * {@link CacheCommand#DISTRIBUTION} command arguments.
+ * {@link CacheSubcommands#DISTRIBUTION} command arguments.
  */
 public enum DistributionCommandArg implements CommandArg {
     /** User attributes. */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/argument/IdleVerifyCommandArg.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/argument/IdleVerifyCommandArg.java
index c435ed8..6c0c034 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/argument/IdleVerifyCommandArg.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/argument/IdleVerifyCommandArg.java
@@ -18,10 +18,10 @@
 package org.apache.ignite.internal.commandline.cache.argument;
 
 import org.apache.ignite.internal.commandline.argument.CommandArg;
-import org.apache.ignite.internal.commandline.cache.CacheCommand;
+import org.apache.ignite.internal.commandline.cache.CacheSubcommands;
 
 /**
- * {@link CacheCommand#IDLE_VERIFY} command arguments.
+ * {@link CacheSubcommands#IDLE_VERIFY} command arguments.
  */
 public enum IdleVerifyCommandArg implements CommandArg {
     /** Dump. */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/argument/ListCommandArg.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/argument/ListCommandArg.java
index be8166b..2ba4857 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/argument/ListCommandArg.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/argument/ListCommandArg.java
@@ -18,10 +18,10 @@
 package org.apache.ignite.internal.commandline.cache.argument;
 
 import org.apache.ignite.internal.commandline.argument.CommandArg;
-import org.apache.ignite.internal.commandline.cache.CacheCommand;
+import org.apache.ignite.internal.commandline.cache.CacheSubcommands;
 
 /**
- * {@link CacheCommand#LIST} command arguments.
+ * {@link CacheSubcommands#LIST} command arguments.
  */
 public enum ListCommandArg implements CommandArg {
     /** Group. */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/argument/ValidateIndexesCommandArg.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/argument/ValidateIndexesCommandArg.java
index 057d4d5..723b901 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/argument/ValidateIndexesCommandArg.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/argument/ValidateIndexesCommandArg.java
@@ -18,10 +18,10 @@
 package org.apache.ignite.internal.commandline.cache.argument;
 
 import org.apache.ignite.internal.commandline.argument.CommandArg;
-import org.apache.ignite.internal.commandline.cache.CacheCommand;
+import org.apache.ignite.internal.commandline.cache.CacheSubcommands;
 
 /**
- * {@link CacheCommand#VALIDATE_INDEXES} command arguments.
+ * {@link CacheSubcommands#VALIDATE_INDEXES} command arguments.
  */
 public enum ValidateIndexesCommandArg implements CommandArg {
     /** Check first. */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/IdleVerifyResultV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/IdleVerifyResultV2.java
index 7337376..745ec16 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/IdleVerifyResultV2.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/IdleVerifyResultV2.java
@@ -35,7 +35,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.visor.VisorDataTransferObject;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.internal.commandline.cache.CacheCommand.IDLE_VERIFY;
+import static org.apache.ignite.internal.commandline.cache.CacheSubcommands.IDLE_VERIFY;
 
 /**
  * Encapsulates result of {@link VerifyBackupPartitionsTaskV2}.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorFindAndDeleteGarbageInPersistenceClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorFindAndDeleteGarbageInPersistenceClosure.java
index 37af676..6990cd7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorFindAndDeleteGarbageInPersistenceClosure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorFindAndDeleteGarbageInPersistenceClosure.java
@@ -161,7 +161,7 @@ public class VisorFindAndDeleteGarbageInPersistenceClosure implements IgniteCall
     }
 
     /**
-     * By calling this method we would delete found garbarge in partitions and would try to
+     * By calling this method we would delete found garbage in partitions and would try to
      * cleanup indexes.
      *
      * @param grpIdToPartIdToGarbageCount GrpId -&gt; PartId -&gt; Garbage count.
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandArgFactory.java b/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandArgFactory.java
deleted file mode 100644
index b7cd1cc..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandArgFactory.java
+++ /dev/null
@@ -1,66 +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.commandline;
-
-import org.apache.ignite.internal.commandline.argument.CommandArg;
-import org.apache.ignite.internal.commandline.cache.CacheCommand;
-import org.apache.ignite.internal.commandline.cache.argument.DistributionCommandArg;
-import org.apache.ignite.internal.commandline.cache.argument.FindAndDeleteGarbageArg;
-import org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCommandArg;
-import org.apache.ignite.internal.commandline.cache.argument.ListCommandArg;
-import org.apache.ignite.internal.commandline.cache.argument.ValidateIndexesCommandArg;
-
-/**
- * Command arguments factory used for tests.
- */
-public class CommandArgFactory {
-    /**
-     * @param cmd Cache command.
-     * @return Possible arguments for passed cache command {@code cmd} or empty array, if arguments not defined for
-     * cache command.
-     */
-    public static CommandArg[] getArgs(CacheCommand cmd) {
-        switch (cmd) {
-            case RESET_LOST_PARTITIONS:
-            case CONTENTION:
-            case HELP:
-                return new CommandArg[0];
-
-            case DISTRIBUTION:
-                return DistributionCommandArg.values();
-            case IDLE_VERIFY:
-                return IdleVerifyCommandArg.values();
-
-            case LIST:
-                return ListCommandArg.values();
-
-            case VALIDATE_INDEXES:
-                return ValidateIndexesCommandArg.values();
-
-            case FIND_AND_DELETE_GARBAGE:
-                return FindAndDeleteGarbageArg.values();
-        }
-
-        throw new IllegalArgumentException("Unknown cache command " + cmd);
-    }
-
-    /** Private constructor */
-    private CommandArgFactory() {
-        /* No-op. */
-    }
-}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java
index 02453dc..566df03 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java
@@ -20,33 +20,38 @@ package org.apache.ignite.internal.commandline;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.UUID;
 import java.util.function.Predicate;
-import org.apache.ignite.internal.commandline.cache.CacheArguments;
+import org.apache.ignite.internal.commandline.baseline.BaselineArguments;
+import org.apache.ignite.internal.commandline.cache.CacheSubcommands;
+import org.apache.ignite.internal.commandline.cache.CacheCommands;
+import org.apache.ignite.internal.commandline.cache.CacheValidateIndexes;
+import org.apache.ignite.internal.commandline.cache.FindAndDeleteGarbage;
 import org.apache.ignite.internal.commandline.cache.argument.FindAndDeleteGarbageArg;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.visor.tx.VisorTxOperation;
 import org.apache.ignite.internal.visor.tx.VisorTxProjection;
 import org.apache.ignite.internal.visor.tx.VisorTxSortOrder;
 import org.apache.ignite.internal.visor.tx.VisorTxTaskArg;
-import org.junit.After;
-import org.junit.Before;
 import org.junit.Test;
 
 import static java.util.Arrays.asList;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_ENABLE_EXPERIMENTAL_COMMAND;
-import static org.apache.ignite.internal.commandline.Command.CACHE;
-import static org.apache.ignite.internal.commandline.Command.WAL;
-import static org.apache.ignite.internal.commandline.CommandHandler.DFLT_HOST;
-import static org.apache.ignite.internal.commandline.CommandHandler.DFLT_PORT;
-import static org.apache.ignite.internal.commandline.CommandHandler.WAL_DELETE;
-import static org.apache.ignite.internal.commandline.CommandHandler.WAL_PRINT;
-import static org.apache.ignite.internal.commandline.cache.CacheCommand.VALIDATE_INDEXES;
+import static org.apache.ignite.internal.commandline.CommandList.CACHE;
+import static org.apache.ignite.internal.commandline.CommandList.WAL;
+import static org.apache.ignite.internal.commandline.TaskExecutor.DFLT_HOST;
+import static org.apache.ignite.internal.commandline.TaskExecutor.DFLT_PORT;
+import static org.apache.ignite.internal.commandline.WalCommands.WAL_DELETE;
+import static org.apache.ignite.internal.commandline.WalCommands.WAL_PRINT;
+import static org.apache.ignite.internal.commandline.cache.CacheSubcommands.FIND_AND_DELETE_GARBAGE;
+import static org.apache.ignite.internal.commandline.cache.CacheSubcommands.VALIDATE_INDEXES;
 import static org.apache.ignite.internal.commandline.cache.argument.ValidateIndexesCommandArg.CHECK_FIRST;
 import static org.apache.ignite.internal.commandline.cache.argument.ValidateIndexesCommandArg.CHECK_THROUGH;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -55,47 +60,37 @@ import static org.junit.Assert.fail;
  * Tests Command Handler parsing arguments.
  */
 public class CommandHandlerParsingTest {
-    /** */
-    @Before
-    public void setUp() throws Exception {
-        System.setProperty(IGNITE_ENABLE_EXPERIMENTAL_COMMAND, "true");
-    }
-
-    /** */
-    @After
-    public void tearDown() throws Exception {
-        System.clearProperty(IGNITE_ENABLE_EXPERIMENTAL_COMMAND);
-    }
-
     /**
      * validate_indexes command arguments parsing and validation
      */
     @Test
     public void testValidateIndexArguments() {
-        CommandHandler hnd = new CommandHandler();
-
         //happy case for all parameters
         try {
             int expectedCheckFirst = 10;
             int expectedCheckThrough = 11;
             UUID nodeId = UUID.randomUUID();
 
-            CacheArguments args = hnd.parseAndValidate(
-                Arrays.asList(
-                    CACHE.text(),
-                    VALIDATE_INDEXES.text(),
-                    "cache1, cache2",
-                    nodeId.toString(),
-                    CHECK_FIRST.toString(),
-                    Integer.toString(expectedCheckFirst),
-                    CHECK_THROUGH.toString(),
-                    Integer.toString(expectedCheckThrough)
-                )
-            ).cacheArgs();
+            ConnectionAndSslParameters args = parseArgs(Arrays.asList(
+                CACHE.text(),
+                VALIDATE_INDEXES.text(),
+                "cache1, cache2",
+                nodeId.toString(),
+                CHECK_FIRST.toString(),
+                Integer.toString(expectedCheckFirst),
+                CHECK_THROUGH.toString(),
+                Integer.toString(expectedCheckThrough)
+            ));
+
+            assertTrue(args.command() instanceof CacheCommands);
 
-            assertEquals("nodeId parameter unexpected value", nodeId, args.nodeId());
-            assertEquals("checkFirst parameter unexpected value", expectedCheckFirst, args.checkFirst());
-            assertEquals("checkThrough parameter unexpected value", expectedCheckThrough, args.checkThrough());
+            CacheSubcommands subcommand = ((CacheCommands)args.command()).arg();
+
+            CacheValidateIndexes.Arguments arg = (CacheValidateIndexes.Arguments)subcommand.subcommand().arg();
+
+            assertEquals("nodeId parameter unexpected value", nodeId, arg.nodeId());
+            assertEquals("checkFirst parameter unexpected value", expectedCheckFirst, arg.checkFirst());
+            assertEquals("checkThrough parameter unexpected value", expectedCheckThrough, arg.checkThrough());
         }
         catch (IllegalArgumentException e) {
             fail("Unexpected exception: " + e);
@@ -105,27 +100,31 @@ public class CommandHandlerParsingTest {
             int expectedParam = 11;
             UUID nodeId = UUID.randomUUID();
 
-            CacheArguments args = hnd.parseAndValidate(
-                Arrays.asList(
+            ConnectionAndSslParameters args = parseArgs(Arrays.asList(
                     CACHE.text(),
                     VALIDATE_INDEXES.text(),
                     nodeId.toString(),
                     CHECK_THROUGH.toString(),
                     Integer.toString(expectedParam)
-                )
-            ).cacheArgs();
+                ));
+
+            assertTrue(args.command() instanceof CacheCommands);
+
+            CacheSubcommands subcommand = ((CacheCommands)args.command()).arg();
+
+            CacheValidateIndexes.Arguments arg = (CacheValidateIndexes.Arguments)subcommand.subcommand().arg();
 
-            assertNull("caches weren't specified, null value expected", args.caches());
-            assertEquals("nodeId parameter unexpected value", nodeId, args.nodeId());
-            assertEquals("checkFirst parameter unexpected value", -1, args.checkFirst());
-            assertEquals("checkThrough parameter unexpected value", expectedParam, args.checkThrough());
+            assertNull("caches weren't specified, null value expected", arg.caches());
+            assertEquals("nodeId parameter unexpected value", nodeId, arg.nodeId());
+            assertEquals("checkFirst parameter unexpected value", -1, arg.checkFirst());
+            assertEquals("checkThrough parameter unexpected value", expectedParam, arg.checkThrough());
         }
         catch (IllegalArgumentException e) {
             e.printStackTrace();
         }
 
         try {
-            hnd.parseAndValidate(
+            parseArgs(
                 Arrays.asList(
                     CACHE.text(),
                     VALIDATE_INDEXES.text(),
@@ -141,7 +140,7 @@ public class CommandHandlerParsingTest {
         }
 
         try {
-            hnd.parseAndValidate(Arrays.asList(CACHE.text(), VALIDATE_INDEXES.text(), CHECK_THROUGH.toString()));
+            parseArgs(Arrays.asList(CACHE.text(), VALIDATE_INDEXES.text(), CHECK_THROUGH.toString()));
 
             fail("Expected exception hasn't been thrown");
         }
@@ -151,16 +150,125 @@ public class CommandHandlerParsingTest {
     }
 
     /**
+     *
+     */
+    @Test
+    public void testFindAndDeleteGarbage() {
+        String nodeId = UUID.randomUUID().toString();
+        String delete = FindAndDeleteGarbageArg.DELETE.toString();
+        String groups = "group1,grpoup2,group3";
+
+        List<List<String>> lists = generateArgumentList(
+            FIND_AND_DELETE_GARBAGE.text(),
+            new T2<>(nodeId, false),
+            new T2<>(delete, false),
+            new T2<>(groups, false)
+        );
+
+        for (List<String> list : lists) {
+            ConnectionAndSslParameters args = parseArgs(list);
+
+            assertTrue(args.command() instanceof CacheCommands);
+
+            CacheSubcommands subcommand = ((CacheCommands)args.command()).arg();
+
+            FindAndDeleteGarbage.Arguments arg = (FindAndDeleteGarbage.Arguments)subcommand.subcommand().arg();
+
+            if (list.contains(nodeId)) {
+                assertEquals("nodeId parameter unexpected value", nodeId, arg.nodeId().toString());
+            }
+            else {
+                assertNull(arg.nodeId());
+            }
+
+            assertEquals(list.contains(delete), arg.delete());
+
+            if (list.contains(groups)) {
+                assertEquals(3, arg.groups().size());
+            }
+            else {
+                assertNull(arg.groups());
+            }
+        }
+    }
+
+    private List<List<String>> generateArgumentList(String subcommand, T2<String, Boolean>...optional) {
+        List<List<T2<String, Boolean>>> lists = generateAllCombinations(Arrays.asList(optional), (x) -> x.get2());
+
+        ArrayList<List<String>> res = new ArrayList<>();
+
+        ArrayList<String> empty = new ArrayList<>();
+
+        empty.add(CACHE.text());
+        empty.add(subcommand);
+
+        res.add(empty);
+
+        for (List<T2<String, Boolean>> list : lists) {
+            ArrayList<String> arg = new ArrayList<>(empty);
+
+            list.forEach(x -> arg.add(x.get1()));
+
+            res.add(arg);
+        }
+
+        return res;
+    }
+
+    private <T> List<List<T>> generateAllCombinations(List<T> source, Predicate<T> stopFunc) {
+        List<List<T>> res = new ArrayList<>();
+
+        for (int i = 0; i < source.size(); i++) {
+            List<T> sourceCopy = new ArrayList<>(source);
+
+            T removed = sourceCopy.remove(i);
+
+            generateAllCombinations(Collections.singletonList(removed), sourceCopy, stopFunc, res);
+        }
+
+        return res;
+    }
+
+
+    private <T> void generateAllCombinations(List<T> res, List<T> source, Predicate<T> stopFunc, List<List<T>> acc) {
+        acc.add(res);
+
+        if (stopFunc != null && stopFunc.test(res.get(res.size() - 1))) {
+            return;
+        }
+
+        if (source.size() == 1) {
+            ArrayList<T> list = new ArrayList<>(res);
+
+            list.add(source.get(0));
+
+            acc.add(list);
+
+            return;
+        }
+
+        for (int i = 0; i < source.size(); i++) {
+            ArrayList<T> res0 = new ArrayList<>(res);
+
+            List<T> sourceCopy = new ArrayList<>(source);
+
+            T removed = sourceCopy.remove(i);
+
+            res0.add(removed);
+
+            generateAllCombinations(res0, sourceCopy, stopFunc, acc);
+        }
+    }
+
+    /**
      * Test that experimental command (i.e. WAL command) is disabled by default.
      */
     @Test
     public void testExperimentalCommandIsDisabled() {
         System.clearProperty(IGNITE_ENABLE_EXPERIMENTAL_COMMAND);
 
-        CommandHandler hnd = new CommandHandler();
-
         try {
-            hnd.parseAndValidate(Arrays.asList(WAL.text(), WAL_PRINT));
+            parseArgs(Arrays.asList(WAL.text(), WAL_PRINT));
         }
         catch (Throwable e) {
             e.printStackTrace();
@@ -169,7 +277,7 @@ public class CommandHandlerParsingTest {
         }
 
         try {
-            hnd.parseAndValidate(Arrays.asList(WAL.text(), WAL_DELETE));
+            parseArgs(Arrays.asList(WAL.text(), WAL_DELETE));
         }
         catch (Throwable e) {
             e.printStackTrace();
@@ -183,15 +291,12 @@ public class CommandHandlerParsingTest {
      */
     @Test
     public void testParseAndValidateSSLArguments() {
-        CommandHandler hnd = new CommandHandler();
-
-        for (Command cmd : Command.values()) {
-
-            if (cmd == Command.CACHE || cmd == Command.WAL)
+        for (CommandList cmd : CommandList.values()) {
+            if (cmd == CommandList.CACHE || cmd == CommandList.WAL)
                 continue; // --cache subcommand requires its own specific arguments.
 
             try {
-                hnd.parseAndValidate(asList("--truststore"));
+                parseArgs(asList("--truststore"));
 
                 fail("expected exception: Expected truststore");
             }
@@ -199,7 +304,7 @@ public class CommandHandlerParsingTest {
                 e.printStackTrace();
             }
 
-            Arguments args = hnd.parseAndValidate(asList("--keystore", "testKeystore", "--keystore-password", "testKeystorePassword", "--keystore-type", "testKeystoreType",
+            ConnectionAndSslParameters args = parseArgs(asList("--keystore", "testKeystore", "--keystore-password", "testKeystorePassword", "--keystore-type", "testKeystoreType",
                 "--truststore", "testTruststore", "--truststore-password", "testTruststorePassword", "--truststore-type", "testTruststoreType",
                 "--ssl-key-algorithm", "testSSLKeyAlgorithm", "--ssl-protocol", "testSSLProtocol", cmd.text()));
 
@@ -212,7 +317,7 @@ public class CommandHandlerParsingTest {
             assertArrayEquals("testTruststorePassword".toCharArray(), args.sslTrustStorePassword());
             assertEquals("testTruststoreType", args.sslTrustStoreType());
 
-            assertEquals(cmd, args.command());
+            assertEquals(cmd.command(), args.command());
         }
     }
 
@@ -222,14 +327,12 @@ public class CommandHandlerParsingTest {
      */
     @Test
     public void testParseAndValidateUserAndPassword() {
-        CommandHandler hnd = new CommandHandler();
-
-        for (Command cmd : Command.values()) {
-            if (cmd == Command.CACHE || cmd == Command.WAL)
+        for (CommandList cmd : CommandList.values()) {
+            if (cmd == CommandList.CACHE || cmd == CommandList.WAL)
                 continue; // --cache subcommand requires its own specific arguments.
 
             try {
-                hnd.parseAndValidate(asList("--user"));
+                parseArgs(asList("--user"));
 
                 fail("expected exception: Expected user name");
             }
@@ -238,7 +341,7 @@ public class CommandHandlerParsingTest {
             }
 
             try {
-                hnd.parseAndValidate(asList("--password"));
+                parseArgs(asList("--password"));
 
                 fail("expected exception: Expected password");
             }
@@ -246,11 +349,11 @@ public class CommandHandlerParsingTest {
                 e.printStackTrace();
             }
 
-            Arguments args = hnd.parseAndValidate(asList("--user", "testUser", "--password", "testPass", cmd.text()));
+            ConnectionAndSslParameters args = parseArgs(asList("--user", "testUser", "--password", "testPass", cmd.text()));
 
-            assertEquals("testUser", args.getUserName());
-            assertEquals("testPass", args.getPassword());
-            assertEquals(cmd, args.command());
+            assertEquals("testUser", args.userName());
+            assertEquals("testPass", args.password());
+            assertEquals(cmd.command(), args.command());
         }
     }
 
@@ -259,24 +362,26 @@ public class CommandHandlerParsingTest {
      */
     @Test
     public void testParseAndValidateWalActions() {
-        CommandHandler hnd = new CommandHandler();
+        ConnectionAndSslParameters args = parseArgs(Arrays.asList(WAL.text(), WAL_PRINT));
 
-        Arguments args = hnd.parseAndValidate(Arrays.asList(WAL.text(), WAL_PRINT));
+        assertEquals(WAL.command(), args.command());
 
-        assertEquals(WAL, args.command());
+        T2<String, String> arg = ((WalCommands)args.command()).arg();
 
-        assertEquals(WAL_PRINT, args.walAction());
+        assertEquals(WAL_PRINT, arg.get1());
 
         String nodes = UUID.randomUUID().toString() + "," + UUID.randomUUID().toString();
 
-        args = hnd.parseAndValidate(Arrays.asList(WAL.text(), WAL_DELETE, nodes));
+        args = parseArgs(Arrays.asList(WAL.text(), WAL_DELETE, nodes));
 
-        assertEquals(WAL_DELETE, args.walAction());
+        arg = ((WalCommands)args.command()).arg();
 
-        assertEquals(nodes, args.walArguments());
+        assertEquals(WAL_DELETE, arg.get1());
+
+        assertEquals(nodes, arg.get2());
 
         try {
-            hnd.parseAndValidate(Collections.singletonList(WAL.text()));
+            parseArgs(Collections.singletonList(WAL.text()));
 
             fail("expected exception: invalid arguments for --wal command");
         }
@@ -285,7 +390,7 @@ public class CommandHandlerParsingTest {
         }
 
         try {
-            hnd.parseAndValidate(Arrays.asList(WAL.text(), UUID.randomUUID().toString()));
+            parseArgs(Arrays.asList(WAL.text(), UUID.randomUUID().toString()));
 
             fail("expected exception: invalid arguments for --wal command");
         }
@@ -299,57 +404,61 @@ public class CommandHandlerParsingTest {
      */
     @Test
     public void testParseAutoConfirmationFlag() {
-        CommandHandler hnd = new CommandHandler();
-
-        for (Command cmd : Command.values()) {
-            if (cmd != Command.DEACTIVATE
-                && cmd != Command.BASELINE
-                && cmd != Command.TX)
+        for (CommandList cmd : CommandList.values()) {
+            if (cmd != CommandList.DEACTIVATE
+                && cmd != CommandList.BASELINE
+                && cmd != CommandList.TX)
                 continue;
 
-            Arguments args = hnd.parseAndValidate(asList(cmd.text()));
+            ConnectionAndSslParameters args = parseArgs(asList(cmd.text()));
 
-            assertEquals(cmd, args.command());
+            assertEquals(cmd.command(), args.command());
             assertEquals(DFLT_HOST, args.host());
             assertEquals(DFLT_PORT, args.port());
-            assertEquals(false, args.autoConfirmation());
+            assertFalse(args.autoConfirmation());
 
             switch (cmd) {
                 case DEACTIVATE: {
-                    args = hnd.parseAndValidate(asList(cmd.text(), "--yes"));
+                    args = parseArgs(asList(cmd.text(), "--yes"));
 
-                    assertEquals(cmd, args.command());
+                    assertEquals(cmd.command(), args.command());
                     assertEquals(DFLT_HOST, args.host());
                     assertEquals(DFLT_PORT, args.port());
-                    assertEquals(true, args.autoConfirmation());
+                    assertTrue(args.autoConfirmation());
 
                     break;
                 }
                 case BASELINE: {
                     for (String baselineAct : asList("add", "remove", "set")) {
-                        args = hnd.parseAndValidate(asList(cmd.text(), baselineAct, "c_id1,c_id2", "--yes"));
+                        args = parseArgs(asList(cmd.text(), baselineAct, "c_id1,c_id2", "--yes"));
 
-                        assertEquals(cmd, args.command());
+                        assertEquals(cmd.command(), args.command());
                         assertEquals(DFLT_HOST, args.host());
                         assertEquals(DFLT_PORT, args.port());
-                        assertEquals(baselineAct, args.baselineArguments().getCmd().text());
-                        assertEquals(Arrays.asList("c_id1","c_id2"), args.baselineArguments().getConsistentIds());
-                        assertEquals(true, args.autoConfirmation());
+                        assertTrue(args.autoConfirmation());
+
+                        BaselineArguments arg = ((BaselineCommand)args.command()).arg();
+
+                        assertEquals(baselineAct, arg.getCmd().text());
+                        assertEquals(new HashSet<>(Arrays.asList("c_id1","c_id2")), new HashSet<>(arg.getConsistentIds()));
                     }
 
                     break;
                 }
+
                 case TX: {
-                    args = hnd.parseAndValidate(asList(cmd.text(), "--xid", "xid1", "--min-duration", "10", "--kill", "--yes"));
+                    args = parseArgs(asList(cmd.text(), "--xid", "xid1", "--min-duration", "10", "--kill", "--yes"));
 
-                    assertEquals(cmd, args.command());
+                    assertEquals(cmd.command(), args.command());
                     assertEquals(DFLT_HOST, args.host());
                     assertEquals(DFLT_PORT, args.port());
-                    assertEquals(true, args.autoConfirmation());
+                    assertTrue(args.autoConfirmation());
 
-                    assertEquals("xid1", args.transactionArguments().getXid());
-                    assertEquals(10_000, args.transactionArguments().getMinDuration().longValue());
-                    assertEquals(VisorTxOperation.KILL, args.transactionArguments().getOperation());
+                    VisorTxTaskArg txTaskArg = ((TxCommands)args.command()).arg();
+
+                    assertEquals("xid1", txTaskArg.getXid());
+                    assertEquals(10_000, txTaskArg.getMinDuration().longValue());
+                    assertEquals(VisorTxOperation.KILL, txTaskArg.getOperation());
                 }
             }
         }
@@ -361,29 +470,27 @@ public class CommandHandlerParsingTest {
      */
     @Test
     public void testConnectionSettings() {
-        CommandHandler hnd = new CommandHandler();
-
-        for (Command cmd : Command.values()) {
-            if (cmd == Command.CACHE || cmd == Command.WAL)
+        for (CommandList cmd : CommandList.values()) {
+            if (cmd == CommandList.CACHE || cmd == CommandList.WAL)
                 continue; // --cache subcommand requires its own specific arguments.
 
-            Arguments args = hnd.parseAndValidate(asList(cmd.text()));
+            ConnectionAndSslParameters args = parseArgs(asList(cmd.text()));
 
-            assertEquals(cmd, args.command());
+            assertEquals(cmd.command(), args.command());
             assertEquals(DFLT_HOST, args.host());
             assertEquals(DFLT_PORT, args.port());
 
-            args = hnd.parseAndValidate(asList("--port", "12345", "--host", "test-host", "--ping-interval", "5000",
+            args = parseArgs(asList("--port", "12345", "--host", "test-host", "--ping-interval", "5000",
                 "--ping-timeout", "40000", cmd.text()));
 
-            assertEquals(cmd, args.command());
+            assertEquals(cmd.command(), args.command());
             assertEquals("test-host", args.host());
             assertEquals("12345", args.port());
             assertEquals(5000, args.pingInterval());
             assertEquals(40000, args.pingTimeout());
 
             try {
-                hnd.parseAndValidate(asList("--port", "wrong-port", cmd.text()));
+                parseArgs(asList("--port", "wrong-port", cmd.text()));
 
                 fail("expected exception: Invalid value for port:");
             }
@@ -392,7 +499,7 @@ public class CommandHandlerParsingTest {
             }
 
             try {
-                hnd.parseAndValidate(asList("--ping-interval", "-10", cmd.text()));
+                parseArgs(asList("--ping-interval", "-10", cmd.text()));
 
                 fail("expected exception: Ping interval must be specified");
             }
@@ -401,7 +508,7 @@ public class CommandHandlerParsingTest {
             }
 
             try {
-                hnd.parseAndValidate(asList("--ping-timeout", "-20", cmd.text()));
+                parseArgs(asList("--ping-timeout", "-20", cmd.text()));
 
                 fail("expected exception: Ping timeout must be specified");
             }
@@ -416,12 +523,12 @@ public class CommandHandlerParsingTest {
      */
     @Test
     public void testTransactionArguments() {
-        CommandHandler hnd = new CommandHandler();
+        ConnectionAndSslParameters args;
 
-        hnd.parseAndValidate(asList("--tx"));
+        parseArgs(asList("--tx"));
 
         try {
-            hnd.parseAndValidate(asList("--tx", "minDuration"));
+            parseArgs(asList("--tx", "minDuration"));
 
             fail("Expected exception");
         }
@@ -429,7 +536,7 @@ public class CommandHandlerParsingTest {
         }
 
         try {
-            hnd.parseAndValidate(asList("--tx", "minDuration", "-1"));
+            parseArgs(asList("--tx", "minDuration", "-1"));
 
             fail("Expected exception");
         }
@@ -437,7 +544,7 @@ public class CommandHandlerParsingTest {
         }
 
         try {
-            hnd.parseAndValidate(asList("--tx", "minSize"));
+            parseArgs(asList("--tx", "minSize"));
 
             fail("Expected exception");
         }
@@ -445,7 +552,7 @@ public class CommandHandlerParsingTest {
         }
 
         try {
-            hnd.parseAndValidate(asList("--tx", "minSize", "-1"));
+            parseArgs(asList("--tx", "minSize", "-1"));
 
             fail("Expected exception");
         }
@@ -453,7 +560,7 @@ public class CommandHandlerParsingTest {
         }
 
         try {
-            hnd.parseAndValidate(asList("--tx", "label"));
+            parseArgs(asList("--tx", "label"));
 
             fail("Expected exception");
         }
@@ -461,7 +568,7 @@ public class CommandHandlerParsingTest {
         }
 
         try {
-            hnd.parseAndValidate(asList("--tx", "label", "tx123["));
+            parseArgs(asList("--tx", "label", "tx123["));
 
             fail("Expected exception");
         }
@@ -469,17 +576,16 @@ public class CommandHandlerParsingTest {
         }
 
         try {
-            hnd.parseAndValidate(asList("--tx", "servers", "nodes", "1,2,3"));
+            parseArgs(asList("--tx", "servers", "nodes", "1,2,3"));
 
             fail("Expected exception");
         }
         catch (IllegalArgumentException ignored) {
         }
 
-        Arguments args = hnd.parseAndValidate(asList("--tx", "--min-duration", "120", "--min-size", "10", "--limit", "100", "--order", "SIZE",
-            "--servers"));
+        args = parseArgs(asList("--tx", "--min-duration", "120", "--min-size", "10", "--limit", "100", "--order", "SIZE", "--servers"));
 
-        VisorTxTaskArg arg = args.transactionArguments();
+        VisorTxTaskArg arg = ((TxCommands)args.command()).arg();
 
         assertEquals(Long.valueOf(120 * 1000L), arg.getMinDuration());
         assertEquals(Integer.valueOf(10), arg.getMinSize());
@@ -487,10 +593,10 @@ public class CommandHandlerParsingTest {
         assertEquals(VisorTxSortOrder.SIZE, arg.getSortOrder());
         assertEquals(VisorTxProjection.SERVER, arg.getProjection());
 
-        args = hnd.parseAndValidate(asList("--tx", "--min-duration", "130", "--min-size", "1", "--limit", "60", "--order", "DURATION",
+        args = parseArgs(asList("--tx", "--min-duration", "130", "--min-size", "1", "--limit", "60", "--order", "DURATION",
             "--clients"));
 
-        arg = args.transactionArguments();
+        arg = ((TxCommands)args.command()).arg();
 
         assertEquals(Long.valueOf(130 * 1000L), arg.getMinDuration());
         assertEquals(Integer.valueOf(1), arg.getMinSize());
@@ -498,132 +604,20 @@ public class CommandHandlerParsingTest {
         assertEquals(VisorTxSortOrder.DURATION, arg.getSortOrder());
         assertEquals(VisorTxProjection.CLIENT, arg.getProjection());
 
-        args = hnd.parseAndValidate(asList("--tx", "--nodes", "1,2,3"));
+        args = parseArgs(asList("--tx", "--nodes", "1,2,3"));
 
-        arg = args.transactionArguments();
+        arg = ((TxCommands)args.command()).arg();
 
         assertNull(arg.getProjection());
         assertEquals(Arrays.asList("1", "2", "3"), arg.getConsistentIds());
     }
 
     /**
-     * Test parsing arguments by find_garbage command.
+     * @param args Raw arg list.
+     * @return Common parameters container object.
      */
-    @Test
-    public void testFindAndDeleteGarbage() {
-        CommandHandler hnd = new CommandHandler();
-
-        String nodeId = UUID.randomUUID().toString();
-        String delete = FindAndDeleteGarbageArg.DELETE.toString();
-        String groups = "group1,grpoup2,group3";
-
-        List<List<String>> lists = generateArgumentList(
-            "find_garbage",
-            new T2<>(nodeId, false),
-            new T2<>(delete, false),
-            new T2<>(groups, false)
-        );
-
-        for (List<String> list : lists) {
-            Arguments arg = hnd.parseAndValidate(list);
-
-            CacheArguments args = arg.cacheArgs();
-
-            if (list.contains(nodeId))
-                assertEquals("nodeId parameter unexpected value", nodeId, args.nodeId().toString());
-            else
-                assertNull(args.nodeId());
-
-            assertEquals(list.contains(delete), args.delete());
-
-            if (list.contains(groups))
-                assertEquals(3, args.groups().size());
-            else
-                assertNull(args.groups());
-        }
-    }
-
-    /**
... 208 lines suppressed ...