You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sd...@apache.org on 2022/06/29 13:02:56 UTC

[ignite] branch master updated: IGNITE-17002 Control.sh command to schedule index rebuild in Maintenance Mode (#10042)

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

sdanilov 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 1428fda2774 IGNITE-17002 Control.sh command to schedule index rebuild in Maintenance Mode (#10042)
1428fda2774 is described below

commit 1428fda2774aaa679f44cb2d3aaaa9717bf12b75
Author: Semyon Danilov <sa...@yandex.ru>
AuthorDate: Wed Jun 29 16:02:45 2022 +0300

    IGNITE-17002 Control.sh command to schedule index rebuild in Maintenance Mode (#10042)
---
 .../commandline/cache/CacheCommandList.java        |   7 +-
 .../cache/CacheScheduleIndexesRebuild.java         | 333 ++++++++++++
 .../commandline/cache/CacheSubcommands.java        |   6 +
 .../cache/argument/IndexRebuildCommandArg.java     |  55 ++
 .../commandline/CommandHandlerParsingTest.java     | 143 +++++
 .../testsuites/IgniteControlUtilityTestSuite.java  |   2 +
 .../util/GridCommandHandlerIndexingUtils.java      |   3 +-
 ...GridCommandHandlerScheduleIndexRebuildTest.java | 596 +++++++++++++++++++++
 .../maintenance/MaintenanceRebuildIndexTarget.java |   6 +
 .../maintenance/MaintenanceRebuildIndexUtils.java  |  36 ++
 .../cache/index/ScheduleIndexRebuildJobRes.java    | 117 ++++
 .../cache/index/ScheduleIndexRebuildTaskArg.java   |  84 +++
 .../cache/index/ScheduleIndexRebuildTaskRes.java   |  70 +++
 .../main/resources/META-INF/classnames.properties  |   4 +
 ...mandHandlerClusterByClassTest_cache_help.output |   8 +
 ...dlerClusterByClassWithSSLTest_cache_help.output |   8 +
 .../query/h2/maintenance/RebuildIndexAction.java   |   7 +-
 .../cache/index/ScheduleIndexRebuildTask.java      | 215 ++++++++
 .../MaintenanceRebuildIndexUtilsSelfTest.java      |  35 +-
 19 files changed, 1725 insertions(+), 10 deletions(-)

diff --git a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommandList.java b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommandList.java
index 5eed85e63c8..1e0420749bd 100644
--- a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommandList.java
+++ b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommandList.java
@@ -92,7 +92,12 @@ public enum CacheCommandList {
     /**
      * Enable, disable or show status for cache metrics.
      */
-    METRICS("metrics", new CacheMetrics());
+    METRICS("metrics", new CacheMetrics()),
+
+    /**
+     * Schedule index rebuild via the maintenance mode.
+     */
+    INDEX_REBUILD("schedule_indexes_rebuild", new CacheScheduleIndexesRebuild());
 
     /** Enumerated values. */
     private static final CacheCommandList[] VALS = values();
diff --git a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/cache/CacheScheduleIndexesRebuild.java b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/cache/CacheScheduleIndexesRebuild.java
new file mode 100644
index 00000000000..31bf521c3fe
--- /dev/null
+++ b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/cache/CacheScheduleIndexesRebuild.java
@@ -0,0 +1,333 @@
+/*
+ * 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.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.logging.Logger;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.commandline.AbstractCommand;
+import org.apache.ignite.internal.commandline.Command;
+import org.apache.ignite.internal.commandline.CommandArgIterator;
+import org.apache.ignite.internal.commandline.TaskExecutor;
+import org.apache.ignite.internal.commandline.argument.CommandArgUtils;
+import org.apache.ignite.internal.commandline.cache.argument.IndexRebuildCommandArg;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.visor.cache.index.ScheduleIndexRebuildTaskArg;
+import org.apache.ignite.internal.visor.cache.index.ScheduleIndexRebuildTaskRes;
+import org.jetbrains.annotations.Nullable;
+
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.commandline.CommandLogger.INDENT;
+import static org.apache.ignite.internal.commandline.cache.argument.IndexRebuildCommandArg.CACHE_GROUPS_TARGET;
+import static org.apache.ignite.internal.commandline.cache.argument.IndexRebuildCommandArg.CACHE_NAMES_TARGET;
+import static org.apache.ignite.internal.commandline.cache.argument.IndexRebuildCommandArg.NODE_ID;
+
+/**
+ * Cache subcommand that schedules indexes rebuild via the maintenance mode.
+ */
+public class CacheScheduleIndexesRebuild extends AbstractCommand<CacheScheduleIndexesRebuild.Arguments> {
+    /** --cache-names parameter format. */
+    private static final String CACHE_NAMES_FORMAT = "cacheName[index1,...indexN],cacheName2,cacheName3[index1]";
+
+    /** --group-names parameter format. */
+    private static final String CACHE_GROUPS_FORMAT = "groupName1,groupName2,...groupNameN";
+
+    /** Command's parsed arguments. */
+    private Arguments args;
+
+    /** {@inheritDoc} */
+    @Override public void printUsage(Logger logger) {
+        String desc = "Schedules rebuild of the indexes for specified caches via the Maintenance Mode. Schedules rebuild of specified "
+            + "caches and cache-groups";
+
+        Map<String, String> map = new LinkedHashMap<>(2);
+
+        map.put(NODE_ID.argName(), "(Optional) Specify node for indexes rebuild. If not specified, schedules rebuild on all nodes.");
+
+        map.put(
+            CACHE_NAMES_TARGET.argName(),
+            "Comma-separated list of cache names with optionally specified indexes. If indexes are not specified then all indexes "
+            + "of the cache will be scheduled for the rebuild operation. Can be used simultaneously with cache group names."
+        );
+
+        map.put(CACHE_GROUPS_TARGET.argName(), "Comma-separated list of cache group names for which indexes should be scheduled for the "
+            + "rebuild. Can be used simultaneously with cache names.");
+
+        usageCache(
+            logger,
+            CacheSubcommands.INDEX_REBUILD,
+            desc,
+            map,
+            NODE_ID.argName() + " nodeId",
+            CACHE_NAMES_TARGET + " " + CACHE_NAMES_FORMAT,
+            CACHE_GROUPS_TARGET + " " + CACHE_GROUPS_FORMAT
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object execute(GridClientConfiguration clientCfg, Logger logger) throws Exception {
+        ScheduleIndexRebuildTaskRes taskRes;
+
+        try (GridClient client = Command.startClient(clientCfg)) {
+            UUID nodeId = args.nodeId;
+
+            if (nodeId == null)
+                nodeId = TaskExecutor.BROADCAST_UUID;
+
+            taskRes = TaskExecutor.executeTaskByNameOnNode(
+                client,
+                "org.apache.ignite.internal.visor.cache.index.ScheduleIndexRebuildTask",
+                new ScheduleIndexRebuildTaskArg(args.cacheToIndexes, args.cacheGroups),
+                nodeId,
+                clientCfg
+            );
+        }
+
+        printResult(taskRes, logger);
+
+        return taskRes;
+    }
+
+    /**
+     * @param taskRes Rebuild task result.
+     * @param logger Logger to print to.
+     */
+    private void printResult(ScheduleIndexRebuildTaskRes taskRes, Logger logger) {
+        taskRes.results().forEach((nodeId, res) -> {
+            printMissed(logger, "WARNING: These caches were not found:", res.notFoundCacheNames());
+            printMissed(logger, "WARNING: These cache groups were not found:", res.notFoundGroupNames());
+
+            if (!F.isEmpty(res.notFoundIndexes()) && hasAtLeastOneIndex(res.notFoundIndexes())) {
+                String warning = "WARNING: These indexes were not found:";
+
+                logger.info(warning);
+
+                printCachesAndIndexes(res.notFoundIndexes(), logger);
+            }
+
+            if (!F.isEmpty(res.cacheToIndexes()) && hasAtLeastOneIndex(res.cacheToIndexes())) {
+                logger.info("Indexes rebuild was scheduled for these caches:");
+
+                printCachesAndIndexes(res.cacheToIndexes(), logger);
+            }
+            else
+                logger.info("WARNING: Indexes rebuild was not scheduled for any cache. Check command input.");
+
+            logger.info("");
+        });
+    }
+
+    /**
+     * Prints missed caches' or cache groups' names.
+     *
+     * @param logger Logger.
+     * @param message Message.
+     * @param missed Missed caches or cache groups' names.
+     */
+    private void printMissed(Logger logger, String message, Set<String> missed) {
+        if (!F.isEmpty(missed)) {
+            logger.info(message);
+
+            missed.stream()
+                .sorted()
+                .forEach(name -> logger.info(INDENT + name));
+
+            logger.info("");
+        }
+    }
+
+    /**
+     * Prints caches and their indexes.
+     *
+     * @param cachesToIndexes Cache -> indexes map.
+     * @param logger Logger.
+     */
+    private static void printCachesAndIndexes(Map<String, Set<String>> cachesToIndexes, Logger logger) {
+        cachesToIndexes.forEach((cacheName, indexes) -> {
+            logger.info(INDENT + cacheName + ":");
+            indexes.forEach(index -> logger.info(INDENT + INDENT + index));
+        });
+    }
+
+    /**
+     * @param cacheToIndexes Cache name -> indexes map.
+     * @return {@code true} if has at least one index in the map, {@code false} otherwise.
+     */
+    private static boolean hasAtLeastOneIndex(Map<String, Set<String>> cacheToIndexes) {
+        return cacheToIndexes.values().stream()
+            .anyMatch(indexes -> !indexes.isEmpty());
+    }
+
+    /** {@inheritDoc} */
+    @Override public Arguments arg() {
+        return args;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String name() {
+        return CacheSubcommands.INDEX_REBUILD.text().toUpperCase();
+    }
+
+    /**
+     * Container for command arguments.
+     */
+    public static class Arguments {
+        /** Node id. */
+        @Nullable
+        private final UUID nodeId;
+
+        /** Cache name -> indexes. */
+        @Nullable
+        private final Map<String, Set<String>> cacheToIndexes;
+
+        /** Cache groups' names. */
+        @Nullable
+        private final Set<String> cacheGroups;
+
+        /** */
+        private Arguments(@Nullable UUID nodeId, @Nullable Map<String, Set<String>> cacheToIndexes, @Nullable Set<String> cacheGroups) {
+            this.nodeId = nodeId;
+            this.cacheToIndexes = cacheToIndexes;
+            this.cacheGroups = cacheGroups;
+        }
+
+        /**
+         * @return Cache -> indexes map.
+         */
+        @Nullable
+        public Map<String, Set<String>> cacheToIndexes() {
+            return cacheToIndexes;
+        }
+
+        /**
+         * @return Cache groups.
+         */
+        @Nullable
+        public Set<String> cacheGroups() {
+            return cacheGroups;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(Arguments.class, this);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void parseArguments(CommandArgIterator argIterator) {
+        UUID nodeId = null;
+        Map<String, Set<String>> cacheToIndexes = null;
+        Set<String> cacheGroups = null;
+
+        while (argIterator.hasNextSubArg()) {
+            String nextArg = argIterator.nextArg("");
+
+            IndexRebuildCommandArg arg = CommandArgUtils.of(nextArg, IndexRebuildCommandArg.class);
+
+            if (arg == null)
+                throw new IllegalArgumentException("Unknown argument: " + nextArg);
+
+            switch (arg) {
+                case NODE_ID:
+                    if (nodeId != null)
+                        throw new IllegalArgumentException(arg.argName() + " arg specified twice.");
+
+                    nodeId = UUID.fromString(argIterator.nextArg("Failed to read node id."));
+
+                    break;
+
+                case CACHE_NAMES_TARGET:
+                    if (cacheToIndexes != null)
+                        throw new IllegalArgumentException(arg.argName() + " arg specified twice.");
+
+                    cacheToIndexes = new HashMap<>();
+
+                    String cacheNamesArg = argIterator.nextArg("Expected a comma-separated cache names (and optionally a"
+                        + " comma-separated list of index names in square brackets).");
+
+                    Pattern cacheNamesPattern = Pattern.compile("([^,\\[\\]]+)(\\[(.*?)])?");
+                    Matcher matcher = cacheNamesPattern.matcher(cacheNamesArg);
+
+                    boolean found = false;
+
+                    while (matcher.find()) {
+                        found = true;
+
+                        String cacheName = matcher.group(1);
+                        boolean specifiedIndexes = matcher.group(2) != null;
+                        String commaSeparatedIndexes = matcher.group(3);
+
+                        if (!specifiedIndexes) {
+                            cacheToIndexes.put(cacheName, Collections.emptySet());
+
+                            continue;
+                        }
+
+                        if (F.isEmpty(commaSeparatedIndexes)) {
+                            throw new IllegalArgumentException("Square brackets must contain comma-separated indexes or not be used "
+                                + "at all.");
+                        }
+
+                        Set<String> indexes = Arrays.stream(commaSeparatedIndexes.split(",")).collect(toSet());
+                        cacheToIndexes.put(cacheName, indexes);
+                    }
+
+                    if (!found)
+                        throw new IllegalArgumentException("Wrong format for --cache-names, should be: " + CACHE_NAMES_FORMAT);
+
+                    break;
+
+                case CACHE_GROUPS_TARGET:
+                    if (cacheGroups != null)
+                        throw new IllegalArgumentException(arg.argName() + " arg specified twice.");
+
+                    String cacheGroupsArg = argIterator.nextArg("Expected comma-separated cache group names");
+
+                    cacheGroups = Arrays.stream(cacheGroupsArg.split(",")).collect(toSet());
+
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Unknown argument: " + arg.argName());
+            }
+        }
+
+        args = new Arguments(nodeId, cacheToIndexes, cacheGroups);
+
+        validateArguments();
+    }
+
+    /** */
+    private void validateArguments() {
+        Set<String> cacheGroups = args.cacheGroups;
+        Map<String, Set<String>> cacheToIndexes = args.cacheToIndexes;
+
+        if ((cacheGroups == null || cacheGroups.isEmpty()) && (cacheToIndexes == null || cacheToIndexes.isEmpty()))
+            throw new IllegalArgumentException(CACHE_NAMES_TARGET + " or " + CACHE_GROUPS_TARGET + " must be specified.");
+    }
+}
diff --git a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/cache/CacheSubcommands.java b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/cache/CacheSubcommands.java
index 1565dafc95e..e0b62e6253c 100644
--- a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/cache/CacheSubcommands.java
+++ b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/cache/CacheSubcommands.java
@@ -24,6 +24,7 @@ import org.apache.ignite.internal.commandline.cache.argument.FindAndDeleteGarbag
 import org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCommandArg;
 import org.apache.ignite.internal.commandline.cache.argument.IndexForceRebuildCommandArg;
 import org.apache.ignite.internal.commandline.cache.argument.IndexListCommandArg;
+import org.apache.ignite.internal.commandline.cache.argument.IndexRebuildCommandArg;
 import org.apache.ignite.internal.commandline.cache.argument.IndexRebuildStatusArg;
 import org.apache.ignite.internal.commandline.cache.argument.ListCommandArg;
 import org.apache.ignite.internal.commandline.cache.argument.ValidateIndexesCommandArg;
@@ -88,6 +89,11 @@ public enum CacheSubcommands {
      */
     INDEX_FORCE_REBUILD("indexes_force_rebuild", IndexForceRebuildCommandArg.class, new CacheIndexesForceRebuild()),
 
+    /**
+     * Index rebuild via the maintenance mode.
+     */
+    INDEX_REBUILD("schedule_indexes_rebuild", IndexRebuildCommandArg.class, new CacheScheduleIndexesRebuild()),
+
     /**
      * Check secondary indexes inline size.
      */
diff --git a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/cache/argument/IndexRebuildCommandArg.java b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/cache/argument/IndexRebuildCommandArg.java
new file mode 100644
index 00000000000..59adc7c039e
--- /dev/null
+++ b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/cache/argument/IndexRebuildCommandArg.java
@@ -0,0 +1,55 @@
+/*
+ * 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.argument;
+
+import org.apache.ignite.internal.commandline.argument.CommandArg;
+import org.apache.ignite.internal.commandline.cache.CacheScheduleIndexesRebuild;
+
+/**
+ * Arguments for {@link CacheScheduleIndexesRebuild} command.
+ */
+public enum IndexRebuildCommandArg implements CommandArg {
+    /** Node id. */
+    NODE_ID("--node-id"),
+
+    /** Target cache and index names. Format: cacheName[indexName],cacheName2,cacheName3[idx1,idx2]. */
+    CACHE_NAMES_TARGET("--cache-names"),
+
+    /** Target cache groups' names. */
+    CACHE_GROUPS_TARGET("--group-names");
+
+    /** Argument name. */
+    private final String name;
+
+    /**
+     * @param name Argument name.
+     */
+    IndexRebuildCommandArg(String name) {
+        this.name = name;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String argName() {
+        return name;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return name;
+    }
+}
diff --git a/modules/control-utility/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java b/modules/control-utility/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java
index 675f17785ad..a8010781088 100644
--- a/modules/control-utility/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java
+++ b/modules/control-utility/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java
@@ -19,16 +19,24 @@ package org.apache.ignite.internal.commandline;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
 import java.util.UUID;
 import java.util.function.Predicate;
 import java.util.logging.Level;
 import java.util.logging.Logger;
+import java.util.stream.Collectors;
 import org.apache.ignite.ShutdownPolicy;
 import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.internal.commandline.baseline.BaselineArguments;
 import org.apache.ignite.internal.commandline.cache.CacheCommands;
+import org.apache.ignite.internal.commandline.cache.CacheScheduleIndexesRebuild;
+import org.apache.ignite.internal.commandline.cache.CacheScheduleIndexesRebuild.Arguments;
 import org.apache.ignite.internal.commandline.cache.CacheSubcommands;
 import org.apache.ignite.internal.commandline.cache.CacheValidateIndexes;
 import org.apache.ignite.internal.commandline.cache.FindAndDeleteGarbage;
@@ -49,6 +57,7 @@ import org.junit.Test;
 import org.junit.rules.TestRule;
 
 import static java.util.Arrays.asList;
+import static java.util.Collections.emptySet;
 import static java.util.Collections.singletonList;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_ENABLE_EXPERIMENTAL_COMMAND;
 import static org.apache.ignite.internal.QueryMXBeanImpl.EXPECTED_GLOBAL_QRY_ID_FORMAT;
@@ -934,6 +943,140 @@ public class CommandHandlerParsingTest {
         );
     }
 
+    /** */
+    @Test
+    public void testScheduleIndexRebuildWrongArgs() {
+        GridTestUtils.assertThrows(
+            null,
+            () -> parseArgs(asList("--cache", "schedule_indexes_rebuild", "--node-id")),
+            IllegalArgumentException.class,
+            "Failed to read node id."
+        );
+
+        String nodeId = UUID.randomUUID().toString();
+
+        GridTestUtils.assertThrows(
+            null,
+            () -> parseArgs(asList("--cache", "schedule_indexes_rebuild", "--node-id", nodeId, "--cache-names")),
+            IllegalArgumentException.class,
+            "Expected a comma-separated cache names (and optionally a comma-separated list of index names in square brackets)."
+        );
+
+        GridTestUtils.assertThrows(
+            null,
+            () -> parseArgs(asList("--cache", "schedule_indexes_rebuild", "--node-id", nodeId, "--node-id", nodeId)),
+            IllegalArgumentException.class,
+            "--node-id arg specified twice."
+        );
+
+        GridTestUtils.assertThrows(
+            null,
+            () -> parseArgs(asList("--cache", "schedule_indexes_rebuild", "--node-id", nodeId, "--cache-names", "a",
+                "--cache-names", "b")),
+            IllegalArgumentException.class,
+            "--cache-names arg specified twice."
+        );
+
+        GridTestUtils.assertThrows(
+            null,
+            () -> parseArgs(asList("--cache", "schedule_indexes_rebuild", "--node-id", nodeId, "--group-names", "a",
+                "--group-names", "b")),
+            IllegalArgumentException.class,
+            "--group-names arg specified twice."
+        );
+
+        GridTestUtils.assertThrows(
+            null,
+            () -> parseArgs(asList("--cache", "schedule_indexes_rebuild")),
+            IllegalArgumentException.class,
+            "--cache-names or --group-names must be specified."
+        );
+
+        GridTestUtils.assertThrows(
+            null,
+            () -> parseArgs(asList("--cache", "schedule_indexes_rebuild", "--cache-names", "foo[]")),
+            IllegalArgumentException.class,
+            "Square brackets must contain comma-separated indexes or not be used at all."
+        );
+    }
+
+    /** */
+    @Test
+    public void testScheduleIndexRebuildArgs() {
+        UUID nodeId = UUID.randomUUID();
+
+        Map<String, Set<String>> params1 = new HashMap<>();
+        params1.put("cache1", new HashSet<>(Arrays.asList("foo", "bar")));
+        params1.put("cache2", null);
+        params1.put("foocache", new HashSet<>(Arrays.asList("idx", "bar")));
+        params1.put("bar", Collections.singleton("foo"));
+
+        CacheCommands cacheCommand1 = (CacheCommands)parseArgs(asList("--cache", "schedule_indexes_rebuild", "--node-id", nodeId.toString(),
+            "--cache-names", buildScheduleIndexRebuildCacheNames(params1))
+        ).command();
+
+        CacheScheduleIndexesRebuild.Arguments arg1 = (Arguments)cacheCommand1.arg().subcommand().arg();
+        assertEquals(normalizeScheduleIndexRebuildCacheNamesMap(params1), arg1.cacheToIndexes());
+        assertEquals(null, arg1.cacheGroups());
+
+        Map<String, Set<String>> params2 = new HashMap<>();
+        params2.put("cache1", new HashSet<>(Arrays.asList("foo", "bar")));
+        params2.put("cache2", null);
+        params2.put("foocache", new HashSet<>(Arrays.asList("idx", "bar")));
+        params2.put("bar", Collections.singleton("foo"));
+
+        CacheCommands cacheCommand2 = (CacheCommands)parseArgs(asList("--cache", "schedule_indexes_rebuild", "--node-id", nodeId.toString(),
+            "--cache-names", buildScheduleIndexRebuildCacheNames(params2), "--group-names", "foocache,someGrp")
+        ).command();
+
+        Map<String, Set<String>> normalized = normalizeScheduleIndexRebuildCacheNamesMap(params2);
+
+        CacheScheduleIndexesRebuild.Arguments arg2 = (Arguments)cacheCommand2.arg().subcommand().arg();
+        assertEquals(normalized, arg2.cacheToIndexes());
+        assertEquals(new HashSet<>(Arrays.asList("foocache", "someGrp")), arg2.cacheGroups());
+    }
+
+    /**
+     * Builds a new --cache-names parameters map replacing nulls with empty set so it should be the same as
+     * the parsed argument of the {@link CacheScheduleIndexesRebuild.Arguments#cacheToIndexes()}.
+     *
+     * @param paramsMap Cache -> indexes map.
+     * @return New map with nulls replaced with empty set.
+     */
+    private static Map<String, Set<String>> normalizeScheduleIndexRebuildCacheNamesMap(Map<String, Set<String>> paramsMap) {
+        return paramsMap.entrySet().stream()
+            .collect(Collectors.toMap(Entry::getKey, e -> e.getValue() != null ? e.getValue() : emptySet()));
+    }
+
+    /**
+     * Builds a --cache-names parameter string for the schedule_indexes_rebuild command from the
+     * cache -> indexes map.
+     * Example: {foo: [], bar: null, test: [idx1, idx2]} will be converted into the "foo[],bar,test[idx1,idx2]" string.
+     *
+     * @param paramsMap Cache -> indexes map.
+     * @return --cache-names parameter string.
+     */
+    private String buildScheduleIndexRebuildCacheNames(Map<String, Set<String>> paramsMap) {
+        return paramsMap.entrySet().stream().map(e -> {
+            StringBuilder sb = new StringBuilder();
+
+            String cacheName = e.getKey();
+            Set<String> indexes = e.getValue();
+
+            sb.append(cacheName);
+
+            if (indexes != null) {
+                sb.append("[");
+
+                sb.append(String.join(",", indexes));
+
+                sb.append("]");
+            }
+
+            return sb.toString();
+        }).collect(Collectors.joining(","));
+    }
+
     /** */
     @SuppressWarnings("ThrowableNotThrown")
     @Test
diff --git a/modules/control-utility/src/test/java/org/apache/ignite/testsuites/IgniteControlUtilityTestSuite.java b/modules/control-utility/src/test/java/org/apache/ignite/testsuites/IgniteControlUtilityTestSuite.java
index 02206c76b8b..efbd4ce577d 100644
--- a/modules/control-utility/src/test/java/org/apache/ignite/testsuites/IgniteControlUtilityTestSuite.java
+++ b/modules/control-utility/src/test/java/org/apache/ignite/testsuites/IgniteControlUtilityTestSuite.java
@@ -44,6 +44,7 @@ import org.apache.ignite.util.GridCommandHandlerIndexingWithSSLTest;
 import org.apache.ignite.util.GridCommandHandlerInterruptCommandTest;
 import org.apache.ignite.util.GridCommandHandlerMetadataTest;
 import org.apache.ignite.util.GridCommandHandlerPropertiesTest;
+import org.apache.ignite.util.GridCommandHandlerScheduleIndexRebuildTest;
 import org.apache.ignite.util.GridCommandHandlerSslTest;
 import org.apache.ignite.util.GridCommandHandlerTest;
 import org.apache.ignite.util.GridCommandHandlerTracingConfigurationTest;
@@ -88,6 +89,7 @@ import org.junit.runners.Suite;
     GridCommandHandlerIndexForceRebuildTest.class,
     GridCommandHandlerIndexListTest.class,
     GridCommandHandlerIndexRebuildStatusTest.class,
+    GridCommandHandlerScheduleIndexRebuildTest.class,
 
     GridCommandHandlerTracingConfigurationTest.class,
 
diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingUtils.java b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingUtils.java
index 13c8bbed2a9..d097c60fd46 100644
--- a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingUtils.java
+++ b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingUtils.java
@@ -149,14 +149,13 @@ public class GridCommandHandlerIndexingUtils {
     public static void createAndFillCache(
         Ignite ignite,
         String cacheName,
-        String grpName,
+        @Nullable String grpName,
         @Nullable String dataRegionName,
         Map<QueryEntity, Function<Random, Object>> qryEntities,
         int cnt
     ) {
         requireNonNull(ignite);
         requireNonNull(cacheName);
-        requireNonNull(grpName);
         requireNonNull(qryEntities);
 
         ignite.createCache(new CacheConfiguration<>()
diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerScheduleIndexRebuildTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerScheduleIndexRebuildTest.java
new file mode 100644
index 00000000000..c2d558249e8
--- /dev/null
+++ b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerScheduleIndexRebuildTest.java
@@ -0,0 +1,596 @@
+/*
+ * 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.util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.maintenance.MaintenanceRebuildIndexTarget;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndexBase;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.maintenance.MaintenanceTask;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.ListeningTestLogger;
+import org.h2.index.Index;
+import org.junit.Test;
+
+import static java.util.Collections.singletonMap;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Collectors.mapping;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.internal.cache.query.index.sorted.maintenance.MaintenanceRebuildIndexUtils.parseMaintenanceTaskParameters;
+import static org.apache.ignite.internal.commandline.CommandHandler.EXIT_CODE_OK;
+import static org.apache.ignite.internal.commandline.CommandLogger.INDENT;
+import static org.apache.ignite.internal.util.IgniteUtils.max;
+import static org.apache.ignite.testframework.GridTestUtils.assertContains;
+import static org.apache.ignite.util.GridCommandHandlerIndexingUtils.breakSqlIndex;
+import static org.apache.ignite.util.GridCommandHandlerIndexingUtils.complexIndexEntity;
+import static org.apache.ignite.util.GridCommandHandlerIndexingUtils.createAndFillCache;
+import static org.apache.ignite.util.GridCommandHandlerIndexingUtils.createAndFillThreeFieldsEntryCache;
+
+/**
+ * Tests for --cache schedule_indexes_rebuild command. Uses single cluster per suite.
+ */
+public class GridCommandHandlerScheduleIndexRebuildTest extends GridCommandHandlerAbstractTest {
+    /** */
+    private static final String INDEX_REBUILD_MNTC_TASK = "indexRebuildMaintenanceTask";
+
+    /** */
+    private static final String CACHE_NAME_1_1 = "cache_1_1";
+
+    /** */
+    private static final String CACHE_NAME_1_2 = "cache_1_2";
+
+    /** */
+    private static final String CACHE_NAME_2_1 = "cache_2_1";
+
+    /** */
+    private static final String CACHE_NAME_NO_GRP = "cache_no_group";
+
+    /** */
+    private static final String CACHE_NAME_NON_EXISTING = "non_existing_cache";
+
+    /** */
+    private static final String GROUP_NAME_NON_EXISTING = "non_existing_group";
+
+    /** */
+    private static final String GRP_NAME_1 = "group_1";
+
+    /** */
+    private static final String GRP_NAME_2 = "group_2";
+
+    /** */
+    private static final int GRIDS_NUM = 3;
+
+    /** */
+    private static final int LAST_NODE_NUM = GRIDS_NUM - 1;
+
+    /** */
+    private static final int REBUILD_TIMEOUT = 30_000;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setGridLogger(new ListeningTestLogger(log));
+
+        cfg.setBuildIndexThreadPoolSize(max(2, cfg.getBuildIndexThreadPoolSize()));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+
+        startupTestCluster();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        G.allGrids().forEach(ignite -> assertFalse(((IgniteEx)ignite).context().maintenanceRegistry().isMaintenanceMode()));
+    }
+
+    /** */
+    private void startupTestCluster() throws Exception {
+        for (int i = 0; i < GRIDS_NUM; i++)
+            startGrid(i);
+
+        IgniteEx ignite = grid(0);
+
+        ignite.cluster().state(ClusterState.ACTIVE);
+
+        awaitPartitionMapExchange();
+
+        createAndFillCache(ignite, CACHE_NAME_1_1, GRP_NAME_1);
+        createAndFillCache(ignite, CACHE_NAME_1_2, GRP_NAME_1);
+        createAndFillCache(ignite, CACHE_NAME_2_1, GRP_NAME_2);
+
+        createAndFillThreeFieldsEntryCache(ignite, CACHE_NAME_NO_GRP, null, Collections.singletonList(complexIndexEntity()));
+
+        // Flush indexes rebuild status (it happens only on checkpoint).
+        forceCheckpoint();
+    }
+
+    /**
+     * Checks error messages when trying to rebuild indexes for non-existent cache, when trying
+     * to rebuild non-existent indexes or when not specifying any indexes inside the square brackets.
+     */
+    @Test
+    public void testErrors() {
+        injectTestSystemOut();
+
+        IgniteEx lastNode = grid(LAST_NODE_NUM);
+
+        // Tests non-existing cache name.
+        assertEquals(EXIT_CODE_OK, execute("--cache", "schedule_indexes_rebuild",
+            "--node-id", lastNode.localNode().id().toString(),
+            "--cache-names", CACHE_NAME_NON_EXISTING));
+
+        String notExistingCacheOutputStr = testOut.toString();
+
+        assertTrue(notExistingCacheOutputStr.contains("WARNING: Indexes rebuild was not scheduled for any cache. Check command input."));
+        assertTrue(notExistingCacheOutputStr.contains(
+            "WARNING: These caches were not found:" + System.lineSeparator()
+            + INDENT + CACHE_NAME_NON_EXISTING
+        ));
+
+        // Test non-existing cache group name.
+        assertEquals(EXIT_CODE_OK, execute("--cache", "schedule_indexes_rebuild",
+            "--node-id", lastNode.localNode().id().toString(),
+            "--group-names", GROUP_NAME_NON_EXISTING));
+
+        String notExistingGroupOutputStr = testOut.toString();
+
+        assertTrue(notExistingGroupOutputStr.contains("WARNING: Indexes rebuild was not scheduled for any cache. Check command input."));
+        assertTrue(notExistingGroupOutputStr.contains(
+            "WARNING: These cache groups were not found:" + System.lineSeparator()
+            + INDENT + GROUP_NAME_NON_EXISTING
+        ));
+
+        testOut.reset();
+
+        // Tests non-existing index name.
+        assertEquals(EXIT_CODE_OK, execute("--cache", "schedule_indexes_rebuild",
+            "--node-id", grid(LAST_NODE_NUM).localNode().id().toString(),
+            "--cache-names", CACHE_NAME_1_1 + "[non-existing-index]"));
+
+        String notExistingIndexOutputStr = testOut.toString();
+
+        assertTrue(notExistingIndexOutputStr.contains("WARNING: Indexes rebuild was not scheduled for any cache. Check command input."));
+
+        assertTrue(notExistingIndexOutputStr.contains(
+            "WARNING: These indexes were not found:" + System.lineSeparator()
+            + INDENT + CACHE_NAME_1_1 + ":" + System.lineSeparator()
+            + INDENT + INDENT + "non-existing-index")
+        );
+    }
+
+    /**
+     * Checks that index is rebuilt correctly.
+     */
+    @Test
+    public void testRebuild() throws Exception {
+        IgniteEx node = grid(LAST_NODE_NUM);
+
+        assertEquals(EXIT_CODE_OK, execute("--cache", "schedule_indexes_rebuild",
+            "--node-id", node.localNode().id().toString(),
+            "--cache-names", CACHE_NAME_NO_GRP));
+
+        checkIndexesRebuildScheduled(node, singletonMap(CU.cacheId(CACHE_NAME_NO_GRP), indexes(node, CACHE_NAME_NO_GRP)));
+
+        node.close();
+
+        node = startGrid(LAST_NODE_NUM);
+
+        assertTrue(node.context().maintenanceRegistry().isMaintenanceMode());
+
+        assertTrue(waitForIndexesRebuild(grid(LAST_NODE_NUM)));
+
+        node.close();
+
+        node = startGrid(LAST_NODE_NUM);
+
+        assertFalse(node.context().maintenanceRegistry().isMaintenanceMode());
+
+        checkIndexes(CACHE_NAME_NO_GRP);
+    }
+
+    /**
+     * Checks that corrupted index is successfully rebuilt by the command.
+     */
+    @Test
+    public void testCorruptedIndexRebuildCache() throws Exception {
+        testCorruptedIndexRebuild(false, true);
+    }
+
+    /**
+     * Checks that corrupted index is successfully rebuilt by the command.
+     */
+    @Test
+    public void testCorruptedIndexRebuildCacheWithGroup() throws Exception {
+        testCorruptedIndexRebuild(true, true);
+    }
+
+    /**
+     * Checks that corrupted index is successfully rebuilt by the command.
+     */
+    @Test
+    public void testCorruptedIndexRebuildCacheOnAllNodes() throws Exception {
+        testCorruptedIndexRebuild(false, false);
+    }
+
+    /**
+     * Checks that corrupted index is successfully rebuilt by the command.
+     */
+    @Test
+    public void testCorruptedIndexRebuildCacheWithGroupOnAllNodes() throws Exception {
+        testCorruptedIndexRebuild(true, false);
+    }
+
+    /**
+     * Checks that corrupted index is successfully rebuilt by the command.
+     *
+     * @param withCacheGroup If {@code true} creates a cache with a cache group.
+     * @param specifyNodeId If {@code true} then execute rebuild only on one node.
+     */
+    private void testCorruptedIndexRebuild(boolean withCacheGroup, boolean specifyNodeId) throws Exception {
+        IgniteEx firstNode = grid(0);
+
+        String cacheName = "tmpCache";
+
+        try {
+            createAndFillCache(firstNode, cacheName, withCacheGroup ? "tmpGrp" : null);
+
+            breakSqlIndex(firstNode.cachex(cacheName), 1, null);
+
+            injectTestSystemOut();
+
+            assertEquals(EXIT_CODE_OK, execute("--cache", "validate_indexes", "--check-crc", "--check-sizes"));
+
+            assertContains(log, testOut.toString(), "issues found (listed above)");
+
+            testOut.reset();
+
+            List<String> args = new ArrayList<>();
+            args.add("--cache");
+            args.add("schedule_indexes_rebuild");
+            if (specifyNodeId) {
+                args.add("--node-id");
+                args.add(firstNode.localNode().id().toString());
+            }
+            args.add("--cache-names");
+            args.add(cacheName);
+
+            assertEquals(EXIT_CODE_OK, execute(args.toArray(new String[0])));
+
+            int nodeCount = specifyNodeId ? 1 : GRIDS_NUM;
+
+            for (int i = 0; i < nodeCount; i++) {
+                IgniteEx grid = grid(i);
+
+                checkIndexesRebuildScheduled(grid, singletonMap(CU.cacheId(cacheName), indexes(grid, cacheName)));
+
+                grid.close();
+
+                grid = startGrid(i);
+
+                assertTrue(grid.context().maintenanceRegistry().isMaintenanceMode());
+
+                assertTrue(waitForIndexesRebuild(grid));
+
+                grid.close();
+
+                startGrid(i);
+            }
+
+            checkIndexes(cacheName);
+        }
+        finally {
+            grid(0).destroyCache(cacheName);
+        }
+    }
+
+    /**
+     * Checks that command can be executed multiple times and all specified indexes will be rebuilt.
+     */
+    @Test
+    public void testConsecutiveCommandInvocations() throws Exception {
+        IgniteEx ignite = grid(0);
+
+        breakAndCheckBroken(ignite, CACHE_NAME_1_1);
+        breakAndCheckBroken(ignite, CACHE_NAME_1_2);
+        breakAndCheckBroken(ignite, CACHE_NAME_2_1);
+        breakAndCheckBroken(ignite, CACHE_NAME_NO_GRP);
+
+        assertEquals(EXIT_CODE_OK, execute("--cache", "schedule_indexes_rebuild",
+            "--node-id", ignite.localNode().id().toString(),
+            "--cache-names", CACHE_NAME_1_1 + "," + CACHE_NAME_1_2));
+
+        assertEquals(EXIT_CODE_OK, execute("--cache", "schedule_indexes_rebuild",
+            "--node-id", ignite.localNode().id().toString(),
+            "--cache-names", CACHE_NAME_2_1 + "," + CACHE_NAME_NO_GRP));
+
+        Map<Integer, Set<String>> cacheToIndexes = new HashMap<>();
+        cacheToIndexes.put(CU.cacheId(CACHE_NAME_1_1), indexes(ignite, CACHE_NAME_1_1));
+        cacheToIndexes.put(CU.cacheId(CACHE_NAME_1_2), indexes(ignite, CACHE_NAME_1_2));
+        cacheToIndexes.put(CU.cacheId(CACHE_NAME_2_1), indexes(ignite, CACHE_NAME_2_1));
+        cacheToIndexes.put(CU.cacheId(CACHE_NAME_NO_GRP), indexes(ignite, CACHE_NAME_NO_GRP));
+
+        checkIndexesRebuildScheduled(ignite, cacheToIndexes);
+
+        ignite.close();
+
+        ignite = startGrid(0);
+
+        assertTrue(waitForIndexesRebuild(ignite));
+
+        checkIndexes(CACHE_NAME_1_1);
+        checkIndexes(CACHE_NAME_1_2);
+        checkIndexes(CACHE_NAME_2_1);
+        checkIndexes(CACHE_NAME_NO_GRP);
+
+        ignite.close();
+
+        startGrid(0);
+
+        checkIndexes(CACHE_NAME_1_1);
+        checkIndexes(CACHE_NAME_1_2);
+        checkIndexes(CACHE_NAME_2_1);
+        checkIndexes(CACHE_NAME_NO_GRP);
+    }
+
+    /**
+     * Checks that specific indexes can be passed to the schedule rebuild command.
+     */
+    @Test
+    public void testSpecificIndexes() throws Exception {
+        IgniteEx ignite = grid(0);
+
+        assertEquals(EXIT_CODE_OK, execute("--cache", "schedule_indexes_rebuild",
+            "--node-id", ignite.localNode().id().toString(),
+            "--cache-names", CACHE_NAME_1_1 + "[_key_PK]," + CACHE_NAME_1_2 + "[PERSON_ORGID_ASC_IDX]"));
+
+        Map<Integer, Set<String>> cacheToIndexes = new HashMap<>();
+        cacheToIndexes.put(CU.cacheId(CACHE_NAME_1_1), Collections.singleton("_key_PK"));
+        cacheToIndexes.put(CU.cacheId(CACHE_NAME_1_2), Collections.singleton("PERSON_ORGID_ASC_IDX"));
+
+        checkIndexesRebuildScheduled(ignite, cacheToIndexes);
+
+        ignite.close();
+
+        ignite = startGrid(0);
+
+        assertTrue(waitForIndexesRebuild(ignite));
+
+        checkIndexes(CACHE_NAME_1_1);
+        checkIndexes(CACHE_NAME_1_2);
+
+        ignite.close();
+
+        startGrid(0);
+
+        checkIndexes(CACHE_NAME_1_1);
+        checkIndexes(CACHE_NAME_1_2);
+    }
+
+    /**
+     * Checks that cache groups can be passed to the schedule rebuild command.
+     */
+    @Test
+    public void testCacheGroupParameter() throws Exception {
+        testCacheGroupsParameter(false);
+    }
+
+    /**
+     * Checks that cache groups can be passed to the schedule rebuild command along with cache names parameter.
+     */
+    @Test
+    public void testCacheGroupParameterWithCacheNames() throws Exception {
+        testCacheGroupsParameter(true);
+    }
+
+    /**
+     * Checks that cache groups can be passed to the schedule rebuild command
+     * along with cache names parameter if {@code withCacheNames} is {@code true}.
+     *
+     * @param withCacheNames Pass --cache-names parameter along with --group-names.
+     * @throws Exception If failed.
+     */
+    private void testCacheGroupsParameter(boolean withCacheNames) throws Exception {
+        IgniteEx ignite = grid(0);
+
+        List<String> cmd = new ArrayList<>();
+
+        cmd.add("--cache");
+        cmd.add("schedule_indexes_rebuild");
+        cmd.add("--node-id");
+        cmd.add(ignite.localNode().id().toString());
+        cmd.add("--group-names");
+        cmd.add(GRP_NAME_1);
+
+        if (withCacheNames) {
+            cmd.add("--cache-names");
+            cmd.add(CACHE_NAME_2_1 + "[PERSON_ORGID_ASC_IDX]");
+        }
+
+        assertEquals(EXIT_CODE_OK, execute(cmd));
+
+        HashSet<String> allIndexes = new HashSet<>(Arrays.asList("_key_PK", "PERSON_ORGID_ASC_IDX", "PERSON_NAME_ASC_IDX"));
+
+        Map<Integer, Set<String>> cacheToIndexes = new HashMap<>();
+        cacheToIndexes.put(CU.cacheId(CACHE_NAME_1_1), allIndexes);
+        cacheToIndexes.put(CU.cacheId(CACHE_NAME_1_2), allIndexes);
+
+        if (withCacheNames)
+            cacheToIndexes.put(CU.cacheId(CACHE_NAME_2_1), Collections.singleton("PERSON_ORGID_ASC_IDX"));
+
+        checkIndexesRebuildScheduled(ignite, cacheToIndexes);
+
+        ignite.close();
+
+        ignite = startGrid(0);
+
+        assertTrue(waitForIndexesRebuild(ignite));
+
+        checkIndexes(CACHE_NAME_1_1);
+        checkIndexes(CACHE_NAME_1_2);
+
+        if (withCacheNames)
+            checkIndexes(CACHE_NAME_2_1);
+
+        ignite.close();
+
+        startGrid(0);
+
+        checkIndexes(CACHE_NAME_1_1);
+        checkIndexes(CACHE_NAME_1_2);
+
+        if (withCacheNames)
+            checkIndexes(CACHE_NAME_2_1);
+    }
+
+    /**
+     * Breaks sql index and checks that it is broken.
+     *
+     * @param ignite Node.
+     * @param cacheName Cache name.
+     * @throws Exception If failed.
+     */
+    private void breakAndCheckBroken(IgniteEx ignite, String cacheName) throws Exception {
+        injectTestSystemOut();
+
+        breakSqlIndex(ignite.cachex(cacheName), 1, null);
+
+        assertEquals(EXIT_CODE_OK, execute("--cache", "validate_indexes", "--check-crc", cacheName));
+
+        assertContains(log, testOut.toString(), "issues found (listed above)");
+
+        testOut.reset();
+    }
+
+    /**
+     * Checks that indexes are valid.
+     *
+     * @param cacheName Cache name.
+     */
+    private void checkIndexes(String cacheName) {
+        injectTestSystemOut();
+
+        assertEquals(EXIT_CODE_OK, execute("--cache", "validate_indexes", "--check-crc", cacheName));
+
+        assertContains(log, testOut.toString(), "no issues found.");
+
+        testOut.reset();
+    }
+
+    /**
+     * Waits for the rebuild of the indexes.
+     *
+     * @param ignite Ignite instance.
+     * @return {@code True} if index rebuild was completed before {@code timeout} was reached.
+     * @throws IgniteInterruptedCheckedException if failed.
+     */
+    private boolean waitForIndexesRebuild(IgniteEx ignite) throws IgniteInterruptedCheckedException {
+        return GridTestUtils.waitForCondition(
+            () -> ignite.context().cache().publicCaches()
+                .stream()
+                .allMatch(c -> c.indexReadyFuture().isDone()),
+            REBUILD_TIMEOUT);
+    }
+
+    /**
+     * Checks that given indexes are scheduled for the rebuild.
+     *
+     * @param node Node.
+     * @param cacheToIndexes Map of caches to indexes.
+     */
+    private void checkIndexesRebuildScheduled(IgniteEx node, Map<Integer, Set<String>> cacheToIndexes) {
+        MaintenanceTask maintenanceTask = node.context().maintenanceRegistry().requestedTask(INDEX_REBUILD_MNTC_TASK);
+
+        assertNotNull(maintenanceTask);
+
+        List<MaintenanceRebuildIndexTarget> targets = parseMaintenanceTaskParameters(maintenanceTask.parameters());
+
+        Map<Integer, Set<String>> result = targets.stream().collect(groupingBy(
+            MaintenanceRebuildIndexTarget::cacheId,
+            mapping(MaintenanceRebuildIndexTarget::idxName, toSet())
+        ));
+
+        assertEqualsMaps(cacheToIndexes, result);
+    }
+
+    /**
+     * Returns indexes' names of the given cache.
+     *
+     * @param node Node.
+     * @param cache Cache name.
+     * @return Indexes of the cache.
+     */
+    private Set<String> indexes(IgniteEx node, String cache) {
+        GridQueryProcessor qry = node.context().query();
+
+        IgniteH2Indexing indexing = (IgniteH2Indexing)qry.getIndexing();
+
+        Set<String> indexes = new HashSet<>();
+
+        for (GridQueryTypeDescriptor type : qry.types(cache)) {
+            GridH2Table gridH2Tbl = indexing.schemaManager().dataTable(type.schemaName(), type.tableName());
+
+            if (gridH2Tbl == null)
+                continue;
+
+            for (Index idx : gridH2Tbl.getIndexes()) {
+                // We need only indexes that can be rebuilt.
+                if (idx instanceof H2TreeIndexBase)
+                    indexes.add(idx.getName());
+            }
+        }
+
+        return indexes;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/maintenance/MaintenanceRebuildIndexTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/maintenance/MaintenanceRebuildIndexTarget.java
index d766502a825..53604d9bb34 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/maintenance/MaintenanceRebuildIndexTarget.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/maintenance/MaintenanceRebuildIndexTarget.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.cache.query.index.sorted.maintenance;
 
 import java.util.Objects;
+import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
  * RebuildIndexAction's parameters.
@@ -71,4 +72,9 @@ public class MaintenanceRebuildIndexTarget {
     @Override public int hashCode() {
         return Objects.hash(cacheId, idxName);
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(MaintenanceRebuildIndexTarget.class, this);
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/maintenance/MaintenanceRebuildIndexUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/maintenance/MaintenanceRebuildIndexUtils.java
index c21c466232b..a6a493512be 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/maintenance/MaintenanceRebuildIndexUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/maintenance/MaintenanceRebuildIndexUtils.java
@@ -22,10 +22,14 @@ import java.util.ArrayList;
 import java.util.Base64;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.Objects;
+import java.util.Set;
 import org.apache.ignite.maintenance.MaintenanceTask;
 import org.jetbrains.annotations.Nullable;
 
+import static java.util.stream.Collectors.joining;
+
 /** Utility methods for the index rebuild maintenance task. */
 public class MaintenanceRebuildIndexUtils {
     /** Index rebuild maintenance task name. */
@@ -96,6 +100,38 @@ public class MaintenanceRebuildIndexUtils {
         );
     }
 
+    /**
+     * Constructs an index rebuild maintenance task based on a map cacheId -> indexes.
+     * For example:
+     * <pre>
+     * {@code
+     * Map<Integer, Set<String>> cacheToIndexes = new HashMap<>();
+     * cacheToIndexes.put(CU.cacheId("some-cache"), singleton("some-index"));
+     * MaintenanceTask task = toMaintenanceTask(cacheToIndexes);
+     * }
+     * </pre>
+     *
+     * @param cacheToIndexes cacheId -> indexes map.
+     * @return Maintenance task.
+     */
+    public static MaintenanceTask toMaintenanceTask(Map<Integer, Set<String>> cacheToIndexes) {
+        String parameters = cacheToIndexes.entrySet().stream().flatMap(entry -> {
+            Integer cacheId = entry.getKey();
+            Set<String> indexes = entry.getValue();
+            return indexes.stream().map(index -> {
+                String encodedIdxName = ENCODER.encodeToString(index.getBytes(StandardCharsets.UTF_8));
+
+                return cacheId + INDEX_REBUILD_PARAMETER_SEPARATOR + encodedIdxName;
+            });
+        }).collect(joining(INDEX_REBUILD_PARAMETER_SEPARATOR));
+
+        return new MaintenanceTask(
+            INDEX_REBUILD_MNTC_TASK_NAME,
+            TASK_DESCRIPTION,
+            parameters
+        );
+    }
+
     /**
      * Merges two index rebuild maintenance tasks concatenating their parameters.
      *
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/index/ScheduleIndexRebuildJobRes.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/index/ScheduleIndexRebuildJobRes.java
new file mode 100644
index 00000000000..eb799c8491c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/index/ScheduleIndexRebuildJobRes.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.cache.index;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Map;
+import java.util.Set;
+import org.apache.ignite.internal.dto.IgniteDataTransferObject;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Result of the ScheduleIndexRebuildJob.
+ */
+public class ScheduleIndexRebuildJobRes extends IgniteDataTransferObject {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
+    /** Map cache names -> indexes scheduled for the rebuild. */
+    private Map<String, Set<String>> cacheToIndexes;
+
+    /** Names of cache indexes that were not found (cache -> set of indexes). */
+    private Map<String, Set<String>> notFoundIndexes;
+
+    /** Names of caches that were not found. */
+    private Set<String> notFoundCacheNames;
+
+    /** Names of cache groups that were not found. */
+    private Set<String> notFoundGroupNames;
+
+    /**
+     * Empty constructor required for Serializable.
+     */
+    public ScheduleIndexRebuildJobRes() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param cacheToIndexes Map cache names -> indexes scheduled for the rebuild.
+     * @param notFoundIndexes Names of cache indexes that were not found (cache -> set of indexes).
+     * @param notFoundCacheNames Names of caches that were not found.
+     * @param notFoundGroupNames Names of cache groups that were not found.
+     */
+    public ScheduleIndexRebuildJobRes(
+        Map<String, Set<String>> cacheToIndexes,
+        Map<String, Set<String>> notFoundIndexes,
+        Set<String> notFoundCacheNames,
+        Set<String> notFoundGroupNames
+    ) {
+        this.cacheToIndexes = cacheToIndexes;
+        this.notFoundIndexes = notFoundIndexes;
+        this.notFoundCacheNames = notFoundCacheNames;
+        this.notFoundGroupNames = notFoundGroupNames;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        U.writeMap(out, cacheToIndexes);
+        U.writeMap(out, notFoundIndexes);
+        U.writeCollection(out, notFoundCacheNames);
+        U.writeCollection(out, notFoundGroupNames);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        cacheToIndexes = U.readMap(in);
+        notFoundIndexes = U.readMap(in);
+        notFoundCacheNames = U.readSet(in);
+        notFoundGroupNames = U.readSet(in);
+    }
+
+    /**
+     * @return Map cache names -> indexes scheduled for the rebuild.
+     */
+    public Map<String, Set<String>> cacheToIndexes() {
+        return cacheToIndexes;
+    }
+
+    /**
+     * @return Names of caches that were not found.
+     */
+    public Set<String> notFoundCacheNames() {
+        return notFoundCacheNames;
+    }
+
+    /**
+     * @return Names of cache indexes that were not found (cache -> set of indexes).
+     */
+    public Map<String, Set<String>> notFoundIndexes() {
+        return notFoundIndexes;
+    }
+
+    /**
+     * @return Names of cache groups that were not found.
+     */
+    public Set<String> notFoundGroupNames() {
+        return notFoundGroupNames;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/index/ScheduleIndexRebuildTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/index/ScheduleIndexRebuildTaskArg.java
new file mode 100644
index 00000000000..11da21ef17f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/index/ScheduleIndexRebuildTaskArg.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.cache.index;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Map;
+import java.util.Set;
+import org.apache.ignite.internal.dto.IgniteDataTransferObject;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Argument for the ScheduleIndexRebuildTask.
+ */
+public class ScheduleIndexRebuildTaskArg extends IgniteDataTransferObject {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
+    /** Map cache names -> indexes scheduled for the rebuild. */
+    private Map<String, Set<String>> cacheToIndexes;
+
+    /** Set of cache group names scheduled for the rebuild. */
+    private Set<String> cacheGroups;
+
+    /**
+     * Empty constructor required for Serializable.
+     */
+    public ScheduleIndexRebuildTaskArg() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param cacheToIndexes Map cache names -> indexes scheduled for the rebuild.
+     * @param cacheGroups Cache group names scheduled for the rebuild.
+     */
+    public ScheduleIndexRebuildTaskArg(Map<String, Set<String>> cacheToIndexes, Set<String> cacheGroups) {
+        this.cacheToIndexes = cacheToIndexes;
+        this.cacheGroups = cacheGroups;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        U.writeMap(out, cacheToIndexes);
+        U.writeCollection(out, cacheGroups);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        cacheToIndexes = U.readMap(in);
+        cacheGroups = U.readSet(in);
+    }
+
+    /**
+     * @return Map cache names -> indexes scheduled for the rebuild.
+     */
+    public Map<String, Set<String>> cacheToIndexes() {
+        return cacheToIndexes;
+    }
+
+    /**
+     * @return Set of cache group names scheduled for the rebuild.
+     */
+    public Set<String> cacheGroups() {
+        return cacheGroups;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/index/ScheduleIndexRebuildTaskRes.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/index/ScheduleIndexRebuildTaskRes.java
new file mode 100644
index 00000000000..fb6100d3f2f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/index/ScheduleIndexRebuildTaskRes.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.cache.index;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.internal.dto.IgniteDataTransferObject;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Result of the ScheduleIndexRebuildTask.
+ */
+public class ScheduleIndexRebuildTaskRes extends IgniteDataTransferObject {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
+    /** Map node id -> rebuild command result. */
+    private Map<UUID, ScheduleIndexRebuildJobRes> results;
+
+    /**
+     * Empty constructor required for Serializable.
+     */
+    public ScheduleIndexRebuildTaskRes() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param results Map node id -> rebuild command result.
+     */
+    public ScheduleIndexRebuildTaskRes(Map<UUID, ScheduleIndexRebuildJobRes> results) {
+        this.results = results;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        U.writeMap(out, results);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        results = U.readMap(in);
+    }
+
+    /**
+     * @return Map node id -> rebuild command result.
+     */
+    public Map<UUID, ScheduleIndexRebuildJobRes> results() {
+        return results;
+    }
+}
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index f91f5af964a..17490a8e7ae 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -2014,6 +2014,10 @@ org.apache.ignite.internal.visor.binary.VisorBinaryMetadataField
 org.apache.ignite.internal.visor.cache.index.IndexListInfoContainer
 org.apache.ignite.internal.visor.cache.index.IndexListTask
 org.apache.ignite.internal.visor.cache.index.IndexListTaskArg
+org.apache.ignite.internal.visor.cache.index.ScheduleIndexRebuildTask
+org.apache.ignite.internal.visor.cache.index.ScheduleIndexRebuildTaskArg
+org.apache.ignite.internal.visor.cache.index.ScheduleIndexRebuildTaskRes
+org.apache.ignite.internal.visor.cache.index.ScheduleIndexRebuildJobRes
 org.apache.ignite.internal.visor.cache.index.IndexForceRebuildTask
 org.apache.ignite.internal.visor.cache.index.IndexForceRebuildTaskArg
 org.apache.ignite.internal.visor.cache.index.IndexForceRebuildTaskRes
diff --git a/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassTest_cache_help.output b/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassTest_cache_help.output
index 585370b4963..3b634f9658e 100644
--- a/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassTest_cache_help.output
+++ b/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassTest_cache_help.output
@@ -90,6 +90,14 @@ Arguments: --cache help --yes
       --caches cache1[,...,cacheN]  - specifies a comma-separated list of cache names to which operation should be applied.
       --all-caches                  - applies operation to all user caches.
 
+  --cache schedule_indexes_rebuild --node-id nodeId --cache-names cacheName[index1,...indexN],cacheName2,cacheName3[index1] --group-names groupName1,groupName2,...groupNameN
+    Schedules rebuild of the indexes for specified caches via the Maintenance Mode. Schedules rebuild of specified caches and cache-groups
+
+    Parameters:
+      --node-id      - (Optional) Specify node for indexes rebuild. If not specified, schedules rebuild on all nodes.
+      --cache-names  - Comma-separated list of cache names with optionally specified indexes. If indexes are not specified then all indexes of the cache will be scheduled for the rebuild operation. Can be used simultaneously with cache group names.
+      --group-names  - Comma-separated list of cache group names for which indexes should be scheduled for the rebuild. Can be used simultaneously with cache names.
+
 Command [CACHE] finished with code: 0
 Control utility has completed execution at: <!any!>
 Execution time: <!any!>
diff --git a/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassWithSSLTest_cache_help.output b/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassWithSSLTest_cache_help.output
index 585370b4963..3b634f9658e 100644
--- a/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassWithSSLTest_cache_help.output
+++ b/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassWithSSLTest_cache_help.output
@@ -90,6 +90,14 @@ Arguments: --cache help --yes
       --caches cache1[,...,cacheN]  - specifies a comma-separated list of cache names to which operation should be applied.
       --all-caches                  - applies operation to all user caches.
 
+  --cache schedule_indexes_rebuild --node-id nodeId --cache-names cacheName[index1,...indexN],cacheName2,cacheName3[index1] --group-names groupName1,groupName2,...groupNameN
+    Schedules rebuild of the indexes for specified caches via the Maintenance Mode. Schedules rebuild of specified caches and cache-groups
+
+    Parameters:
+      --node-id      - (Optional) Specify node for indexes rebuild. If not specified, schedules rebuild on all nodes.
+      --cache-names  - Comma-separated list of cache names with optionally specified indexes. If indexes are not specified then all indexes of the cache will be scheduled for the rebuild operation. Can be used simultaneously with cache group names.
+      --group-names  - Comma-separated list of cache group names for which indexes should be scheduled for the rebuild. Can be used simultaneously with cache names.
+
 Command [CACHE] finished with code: 0
 Control utility has completed execution at: <!any!>
 Execution time: <!any!>
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/maintenance/RebuildIndexAction.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/maintenance/RebuildIndexAction.java
index 46d0fd7c3d0..73afacf0aac 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/maintenance/RebuildIndexAction.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/maintenance/RebuildIndexAction.java
@@ -22,7 +22,6 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.cache.query.index.IndexDefinition;
 import org.apache.ignite.internal.cache.query.index.IndexProcessor;
 import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
 import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexFactory;
@@ -31,7 +30,6 @@ import org.apache.ignite.internal.cache.query.index.sorted.maintenance.Maintenan
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
-import org.apache.ignite.internal.processors.query.GridQueryIndexing;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.processors.query.aware.IndexBuildStatusStorage;
 import org.apache.ignite.internal.processors.query.h2.H2TableDescriptor;
@@ -139,7 +137,6 @@ public class RebuildIndexAction implements MaintenanceAction<Boolean> {
 
         String cacheName = context.name();
 
-        GridQueryIndexing idx;
         SchemaManager schemaManager = indexing.schemaManager();
 
         H2TreeIndex targetIndex = findIndex(cacheName, idxName, schemaManager);
@@ -203,14 +200,14 @@ public class RebuildIndexAction implements MaintenanceAction<Boolean> {
 
         IndexProcessor indexProcessor = context.kernalContext().indexProcessor();
 
-        IndexDefinition definition = oldIndex.index().indexDefinition();
+        SortedIndexDefinition definition = oldIndex.index().indexDefinition();
 
         org.apache.ignite.internal.cache.query.index.Index newIndex = indexProcessor.createIndexDynamically(
             targetTable.cacheContext(), InlineIndexFactory.INSTANCE, definition, visitor);
 
         InlineIndexImpl queryIndex = newIndex.unwrap(InlineIndexImpl.class);
 
-        H2TreeIndex newIdx = oldIndex.createCopy(queryIndex, (SortedIndexDefinition)definition);
+        H2TreeIndex newIdx = oldIndex.createCopy(queryIndex, definition);
 
         schemaManager.createIndex(
             targetTable.getSchema().getName(),
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/visor/cache/index/ScheduleIndexRebuildTask.java b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/cache/index/ScheduleIndexRebuildTask.java
new file mode 100644
index 00000000000..4760e2e2878
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/cache/index/ScheduleIndexRebuildTask.java
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.cache.index;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndexBase;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.visor.VisorJob;
+import org.apache.ignite.internal.visor.VisorMultiNodeTask;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+import org.apache.ignite.maintenance.MaintenanceTask;
+import org.h2.index.Index;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.cache.query.index.sorted.maintenance.MaintenanceRebuildIndexUtils.mergeTasks;
+import static org.apache.ignite.internal.cache.query.index.sorted.maintenance.MaintenanceRebuildIndexUtils.toMaintenanceTask;
+
+/**
+ * Task that schedules indexes rebuild for specified caches via the maintenance mode.
+ */
+@GridInternal
+public class ScheduleIndexRebuildTask
+    extends VisorMultiNodeTask<ScheduleIndexRebuildTaskArg, ScheduleIndexRebuildTaskRes, ScheduleIndexRebuildJobRes> {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override protected ScheduleIndexRebuildJob job(ScheduleIndexRebuildTaskArg arg) {
+        return new ScheduleIndexRebuildJob(arg, debug);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected @Nullable ScheduleIndexRebuildTaskRes reduce0(List<ComputeJobResult> results) throws IgniteException {
+        Map<UUID, ScheduleIndexRebuildJobRes> taskResultMap = results.stream()
+            .collect(Collectors.toMap(res -> res.getNode().id(), ComputeJobResult::getData));
+
+        return new ScheduleIndexRebuildTaskRes(taskResultMap);
+    }
+
+    /** Job that schedules index rebuild (via maintenance mode) on a specific node. */
+    private static class ScheduleIndexRebuildJob extends VisorJob<ScheduleIndexRebuildTaskArg, ScheduleIndexRebuildJobRes> {
+        /** Serial version uid. */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         * Create job with specified argument.
+         *
+         * @param arg Job argument.
+         * @param debug Flag indicating whether debug information should be printed into node log.
+         */
+        protected ScheduleIndexRebuildJob(@Nullable ScheduleIndexRebuildTaskArg arg, boolean debug) {
+            super(arg, debug);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected ScheduleIndexRebuildJobRes run(@Nullable ScheduleIndexRebuildTaskArg arg) throws IgniteException {
+            Set<String> argCacheGroups = arg.cacheGroups();
+
+            assert (arg.cacheToIndexes() != null && !arg.cacheToIndexes().isEmpty())
+                || (argCacheGroups != null && !argCacheGroups.isEmpty()) : "Cache to indexes map or cache groups must be specified.";
+
+            Map<String, Set<String>> argCacheToIndexes = arg.cacheToIndexes() != null ? arg.cacheToIndexes() : new HashMap<>();
+
+            Set<String> notFoundCaches = new HashSet<>();
+            Set<String> notFoundGroups = new HashSet<>();
+
+            GridCacheProcessor cacheProcessor = ignite.context().cache();
+
+            Map<String, Set<String>> cacheToIndexes = new HashMap<>();
+            Map<String, Set<String>> cacheToMissedIndexes = new HashMap<>();
+
+            if (argCacheGroups != null) {
+                argCacheGroups.forEach(groupName -> {
+                    CacheGroupContext grpCtx = cacheProcessor.cacheGroup(CU.cacheId(groupName));
+
+                    if (grpCtx == null) {
+                        notFoundGroups.add(groupName);
+                        return;
+                    }
+
+                    grpCtx.caches().stream().map(GridCacheContext::name).forEach(cache -> {
+                        argCacheToIndexes.put(cache, Collections.emptySet());
+                    });
+                });
+            }
+
+            for (Entry<String, Set<String>> indexesByCache : argCacheToIndexes.entrySet()) {
+                String cache = indexesByCache.getKey();
+                Set<String> indexesArg = indexesByCache.getValue();
+                int cacheId = CU.cacheId(cache);
+
+                GridCacheContext<?, ?> cacheCtx = cacheProcessor.context().cacheContext(cacheId);
+
+                if (cacheCtx == null) {
+                    notFoundCaches.add(cache);
+                    continue;
+                }
+
+                Set<String> existingIndexes = indexes(cache);
+                Set<String> indexesToRebuild = cacheToIndexes.computeIfAbsent(cache, s -> new HashSet<>());
+                Set<String> missedIndexes = cacheToMissedIndexes.computeIfAbsent(cache, s -> new HashSet<>());
+
+                if (indexesArg.isEmpty())
+                    indexesToRebuild.addAll(existingIndexes);
+                else {
+                    indexesArg.forEach(index -> {
+                        if (!existingIndexes.contains(index)) {
+                            missedIndexes.add(index);
+                            return;
+                        }
+
+                        indexesToRebuild.add(index);
+                    });
+                }
+            }
+
+            if (hasAtLeastOneIndex(cacheToIndexes)) {
+                MaintenanceRegistry maintenanceRegistry = ignite.context().maintenanceRegistry();
+
+                MaintenanceTask task = toMaintenanceTask(
+                    cacheToIndexes.entrySet().stream().collect(Collectors.toMap(
+                        (e) -> CU.cacheId(e.getKey()),
+                        Entry::getValue
+                    ))
+                );
+
+                try {
+                    maintenanceRegistry.registerMaintenanceTask(
+                        task,
+                        oldTask -> mergeTasks(oldTask, task)
+                    );
+                }
+                catch (IgniteCheckedException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+
+            return new ScheduleIndexRebuildJobRes(
+                cacheToIndexes,
+                cacheToMissedIndexes,
+                notFoundCaches,
+                notFoundGroups
+            );
+        }
+
+        /**
+         * @param cache Cache name.
+         * @return Indexes of the cache.
+         */
+        private Set<String> indexes(String cache) {
+            GridQueryProcessor qry = ignite.context().query();
+
+            IgniteH2Indexing indexing = (IgniteH2Indexing)qry.getIndexing();
+
+            Set<String> indexes = new HashSet<>();
+
+            for (GridQueryTypeDescriptor type : qry.types(cache)) {
+                GridH2Table gridH2Tbl = indexing.schemaManager().dataTable(type.schemaName(), type.tableName());
+
+                if (gridH2Tbl == null)
+                    continue;
+
+                for (Index idx : gridH2Tbl.getIndexes()) {
+                    if (idx instanceof H2TreeIndexBase)
+                        indexes.add(idx.getName());
+                }
+            }
+
+            return indexes;
+        }
+    }
+
+    /**
+     * @param cacheToIndexes Cache name -> indexes map.
+     * @return {@code true} if has at least one index in the map, {@code false} otherwise.
+     */
+    private static boolean hasAtLeastOneIndex(Map<String, Set<String>> cacheToIndexes) {
+        return cacheToIndexes.values().stream()
+            .anyMatch(indexes -> !indexes.isEmpty());
+    }
+}
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/maintenance/MaintenanceRebuildIndexUtilsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/maintenance/MaintenanceRebuildIndexUtilsSelfTest.java
index 66c8ac8ec7d..06d32282259 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/maintenance/MaintenanceRebuildIndexUtilsSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/maintenance/MaintenanceRebuildIndexUtilsSelfTest.java
@@ -17,23 +17,31 @@
 
 package org.apache.ignite.internal.processors.query.h2.maintenance;
 
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
 import java.util.stream.IntStream;
 import org.apache.ignite.internal.cache.query.index.sorted.maintenance.MaintenanceRebuildIndexTarget;
 import org.apache.ignite.internal.cache.query.index.sorted.maintenance.MaintenanceRebuildIndexUtils;
 import org.apache.ignite.maintenance.MaintenanceTask;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
 
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Collectors.mapping;
 import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toSet;
 import static org.apache.ignite.internal.cache.query.index.sorted.maintenance.MaintenanceRebuildIndexUtils.INDEX_REBUILD_MNTC_TASK_NAME;
 import static org.apache.ignite.internal.cache.query.index.sorted.maintenance.MaintenanceRebuildIndexUtils.INDEX_REBUILD_PARAMETER_SEPARATOR;
 import static org.apache.ignite.internal.cache.query.index.sorted.maintenance.MaintenanceRebuildIndexUtils.mergeTasks;
 import static org.apache.ignite.internal.cache.query.index.sorted.maintenance.MaintenanceRebuildIndexUtils.parseMaintenanceTaskParameters;
 import static org.apache.ignite.internal.cache.query.index.sorted.maintenance.MaintenanceRebuildIndexUtils.toMaintenanceTask;
-import static org.junit.Assert.assertEquals;
 
 /** Tests for {@link MaintenanceRebuildIndexTarget}. */
-public class MaintenanceRebuildIndexUtilsSelfTest {
+public class MaintenanceRebuildIndexUtilsSelfTest extends GridCommonAbstractTest {
     /**
      * Tests that maintenance task's parameters can be stringified and parsed back.
      */
@@ -117,4 +125,27 @@ public class MaintenanceRebuildIndexUtilsSelfTest {
         assertEquals(cacheId, target.cacheId());
         assertEquals(idxName, target.idxName());
     }
+
+    /**
+     * Tests that maintenance task can be constructed from a map.
+     */
+    @Test
+    public void testConstructFromMap() {
+        Map<Integer, Set<String>> cacheToIndexes = new HashMap<>();
+        cacheToIndexes.put(1, new HashSet<>(Arrays.asList("foo", "bar")));
+        cacheToIndexes.put(2, new HashSet<>(Arrays.asList("foo1", "bar1")));
+
+        MaintenanceTask task = toMaintenanceTask(cacheToIndexes);
+
+        List<MaintenanceRebuildIndexTarget> targets = parseMaintenanceTaskParameters(task.parameters());
+
+        assertEquals(4, targets.size());
+
+        Map<Integer, Set<String>> result = targets.stream().collect(groupingBy(
+            MaintenanceRebuildIndexTarget::cacheId,
+            mapping(MaintenanceRebuildIndexTarget::idxName, toSet())
+        ));
+
+        assertEqualsMaps(cacheToIndexes, result);
+    }
 }