You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ir...@apache.org on 2019/04/10 13:32:48 UTC

[ignite] branch master updated: IGNITE-10896 Add ability to use simultaneous cache filtering options with control.sh --cache idle_verify - Fixes #6336.

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

irakov 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 3f0e85f  IGNITE-10896 Add ability to use simultaneous cache filtering options with control.sh --cache idle_verify - Fixes #6336.
3f0e85f is described below

commit 3f0e85f2e3d0a5abbc7fbf67c8d9938c90caae2c
Author: denis-chudov <dc...@gridgain.com>
AuthorDate: Wed Apr 10 16:31:02 2019 +0300

    IGNITE-10896 Add ability to use simultaneous cache filtering options with control.sh --cache idle_verify - Fixes #6336.
    
    Signed-off-by: Ivan Rakov <ir...@apache.org>
---
 .../internal/commandline/CommandHandler.java       |  36 ++--
 .../cache/verify/IdleVerifyResultV2.java           |  54 ++++--
 .../cache/verify/NoMatchingCachesException.java    |  28 +++
 .../verify/VerifyBackupPartitionsDumpTask.java     |  38 ++++
 .../cache/verify/VerifyBackupPartitionsTaskV2.java | 191 ++++++++++++-------
 .../apache/ignite/util/GridCommandHandlerTest.java | 207 ++++++++++++++++++---
 6 files changed, 432 insertions(+), 122 deletions(-)

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 f0c3ba3..1ddeee2 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
@@ -138,6 +138,7 @@ import org.apache.ignite.plugin.security.SecurityCredentialsProvider;
 import org.apache.ignite.ssl.SslContextFactory;
 
 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;
@@ -847,7 +848,7 @@ public class CommandHandler {
         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(or(g(EXCLUDE_CACHES, CACHES), g(CACHE_FILTER, or(CacheFilterEnum.values())), CACHES)));
+            op(EXCLUDE_CACHES, CACHES), op(CACHE_FILTER, or(CacheFilterEnum.values())), 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);
@@ -1658,7 +1659,11 @@ public class CommandHandler {
                 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 on an idle cluster and print out the differences, if any.";
+                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. " +
+                    "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.";
 
             case VALIDATE_INDEXES:
                 return "Validate indexes on an idle cluster and print out the keys that are missing in the indexes.";
@@ -1954,7 +1959,7 @@ public class CommandHandler {
                     case CMD_PASSWORD:
                         pwd = nextArg("Expected password");
 
-                        log(String.format(pwdArgWarnFmt, CMD_PASSWORD, CMD_PASSWORD));
+                        log(format(pwdArgWarnFmt, CMD_PASSWORD, CMD_PASSWORD));
 
                         break;
 
@@ -1981,7 +1986,7 @@ public class CommandHandler {
                     case CMD_KEYSTORE_PASSWORD:
                         sslKeyStorePassword = nextArg("Expected SSL key store password").toCharArray();
 
-                        log(String.format(pwdArgWarnFmt, CMD_KEYSTORE_PASSWORD, CMD_KEYSTORE_PASSWORD));
+                        log(format(pwdArgWarnFmt, CMD_KEYSTORE_PASSWORD, CMD_KEYSTORE_PASSWORD));
 
                         break;
 
@@ -1998,7 +2003,7 @@ public class CommandHandler {
                     case CMD_TRUSTSTORE_PASSWORD:
                         sslTrustStorePassword = nextArg("Expected SSL trust store password").toCharArray();
 
-                        log(String.format(pwdArgWarnFmt, CMD_TRUSTSTORE_PASSWORD, CMD_TRUSTSTORE_PASSWORD));
+                        log(format(pwdArgWarnFmt, CMD_TRUSTSTORE_PASSWORD, CMD_TRUSTSTORE_PASSWORD));
 
                         break;
 
@@ -2115,19 +2120,15 @@ public class CommandHandler {
                 break;
 
             case IDLE_VERIFY:
-                int idleVerifyArgsCnt = 3;
+                int idleVerifyArgsCnt = 5;
 
                 while (hasNextSubArg() && idleVerifyArgsCnt-- > 0) {
                     String nextArg = nextArg("");
 
                     IdleVerifyCommandArg arg = CommandArgUtils.of(nextArg, IdleVerifyCommandArg.class);
 
-                    if (arg == null) {
-                        if (cacheArgs.excludeCaches() != null || cacheArgs.getCacheFilterEnum() != CacheFilterEnum.ALL)
-                            throw new IllegalArgumentException(ONE_CACHE_FILTER_OPT_SHOULD_USED_MSG);
-
+                    if (arg == null)
                         parseCacheNames(nextArg, cacheArgs);
-                    }
                     else {
                         switch (arg) {
                             case DUMP:
@@ -2146,9 +2147,6 @@ public class CommandHandler {
                                 break;
 
                             case CACHE_FILTER:
-                                if (cacheArgs.caches() != null || cacheArgs.excludeCaches() != null)
-                                    throw new IllegalArgumentException(ONE_CACHE_FILTER_OPT_SHOULD_USED_MSG);
-
                                 String filter = nextArg("The cache filter should be specified. The following " +
                                     "values can be used: " + Arrays.toString(CacheFilterEnum.values()) + '.');
 
@@ -2157,9 +2155,6 @@ public class CommandHandler {
                                 break;
 
                             case EXCLUDE_CACHES:
-                                if (cacheArgs.caches() != null || cacheArgs.getCacheFilterEnum() != CacheFilterEnum.ALL)
-                                    throw new IllegalArgumentException(ONE_CACHE_FILTER_OPT_SHOULD_USED_MSG);
-
                                 parseExcludeCacheNames(nextArg("Specify caches, which will be excluded."),
                                     cacheArgs);
 
@@ -2357,6 +2352,13 @@ public class CommandHandler {
             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());
         }
 
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 3600f69..1004c61 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
@@ -66,22 +66,28 @@ public class IdleVerifyResultV2 extends VisorDataTransferObject {
     @GridToStringInclude
     private Map<ClusterNode, Exception> exceptions;
 
+    /** Whether job succeeded or not. */
+    private boolean succeeded = true;
+
     /**
      * @param cntrConflicts Counter conflicts.
      * @param hashConflicts Hash conflicts.
      * @param movingPartitions Moving partitions.
      * @param exceptions Occured exceptions.
+     * @param succeeded Whether succeeded or not.
      */
     public IdleVerifyResultV2(
         Map<PartitionKeyV2, List<PartitionHashRecordV2>> cntrConflicts,
         Map<PartitionKeyV2, List<PartitionHashRecordV2>> hashConflicts,
         Map<PartitionKeyV2, List<PartitionHashRecordV2>> movingPartitions,
-        Map<ClusterNode, Exception> exceptions
+        Map<ClusterNode, Exception> exceptions,
+        boolean succeeded
     ) {
         this.cntrConflicts = cntrConflicts;
         this.hashConflicts = hashConflicts;
         this.movingPartitions = movingPartitions;
         this.exceptions = exceptions;
+        this.succeeded = succeeded;
     }
 
     /**
@@ -182,27 +188,45 @@ public class IdleVerifyResultV2 extends VisorDataTransferObject {
 
     /** */
     private void print(Consumer<String> printer, boolean printExceptionMessages) {
-        if (!F.isEmpty(exceptions)) {
-            int size = exceptions.size();
+        boolean noMatchingCaches = false;
 
-            printer.accept("idle_verify failed on " + size + " node" + (size == 1 ? "" : "s") + ".\n");
-        }
+        for (Exception e : exceptions.values())
+            if (e instanceof NoMatchingCachesException) {
+                noMatchingCaches = true;
+                succeeded = false;
 
-        if (!hasConflicts())
-            printer.accept("idle_verify check has finished, no conflicts have been found.\n");
-        else
-            printConflicts(printer);
+                break;
+            }
 
-        if (!F.isEmpty(movingPartitions())) {
-            printer.accept("Verification was skipped for " + movingPartitions().size() + " MOVING partitions:\n");
+        if (succeeded) {
+            if (!F.isEmpty(exceptions)) {
+                int size = exceptions.size();
 
-            for (Map.Entry<PartitionKeyV2, List<PartitionHashRecordV2>> entry : movingPartitions().entrySet()) {
-                printer.accept("Rebalancing partition: " + entry.getKey() + "\n");
+                printer.accept("idle_verify failed on " + size + " node" + (size == 1 ? "" : "s") + ".\n");
+            }
 
-                printer.accept("Partition instances: " + entry.getValue() + "\n");
+            if (!hasConflicts())
+                printer.accept("idle_verify check has finished, no conflicts have been found.\n");
+            else
+                printConflicts(printer);
+
+            if (!F.isEmpty(movingPartitions())) {
+                printer.accept("Verification was skipped for " + movingPartitions().size() + " MOVING partitions:\n");
+
+                for (Map.Entry<PartitionKeyV2, List<PartitionHashRecordV2>> entry : movingPartitions().entrySet()) {
+                    printer.accept("Rebalancing partition: " + entry.getKey() + "\n");
+
+                    printer.accept("Partition instances: " + entry.getValue() + "\n");
+                }
+
+                printer.accept("\n");
             }
+        }
+        else {
+            printer.accept("idle_verify failed.");
 
-            printer.accept("\n");
+            if (noMatchingCaches)
+                printer.accept("There are no caches matching given filter options.");
         }
 
         if (!F.isEmpty(exceptions())) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/NoMatchingCachesException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/NoMatchingCachesException.java
new file mode 100644
index 0000000..a073a42
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/NoMatchingCachesException.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.processors.cache.verify;
+
+import org.apache.ignite.IgniteException;
+
+/**
+ * Runtime exception that can be thrown in {@link VerifyBackupPartitionsTaskV2} when no caches matching given
+ * filter options can be found.
+ */
+public class NoMatchingCachesException extends IgniteException {
+    /** */
+    private static final long serialVersionUID = 0L;
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsDumpTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsDumpTask.java
index e80ab7e..f3037a8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsDumpTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsDumpTask.java
@@ -44,6 +44,9 @@ import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.resources.LoggerResource;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCommandArg.CACHE_FILTER;
+import static org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCommandArg.EXCLUDE_CACHES;
+
 /**
  * Task for collection checksums primary and backup partitions of specified caches. <br> Argument: Set of cache names,
  * 'null' will trigger verification for all caches. <br> Result: {@link IdleVerifyDumpResult} with all found partitions.
@@ -207,6 +210,7 @@ public class VerifyBackupPartitionsDumpTask extends ComputeTaskAdapter<VisorIdle
         }
 
         writer.write("idle_verify check has finished, found " + partitions.size() + " partitions\n");
+        writer.write("idle_verify task was executed with the following args: " + taskArgsAsCmd() + "\n");
 
         if (skippedRecords > 0)
             writer.write(skippedRecords + " partitions was skipped\n");
@@ -227,6 +231,40 @@ public class VerifyBackupPartitionsDumpTask extends ComputeTaskAdapter<VisorIdle
     }
 
     /**
+     * Method that builds command line string from the taskArg field.
+     *
+     * @return command line argument string
+     */
+    private String taskArgsAsCmd() {
+        StringBuilder result = new StringBuilder();
+
+        if (!F.isEmpty(taskArg.getCaches())) {
+            for (String cache : taskArg.getCaches()) {
+                result.append(cache);
+                result.append(" ");
+            }
+        }
+
+        if (taskArg.getCacheFilterEnum() != null) {
+            result.append(CACHE_FILTER);
+            result.append(" ");
+            result.append(taskArg.getCacheFilterEnum());
+            result.append(" ");
+        }
+
+        if (!F.isEmpty(taskArg.getExcludeCaches())) {
+            result.append(EXCLUDE_CACHES + " ");
+
+            for (String excluded : taskArg.getExcludeCaches()) {
+                result.append(excluded);
+                result.append(" ");
+            }
+        }
+
+        return result.toString();
+    }
+
+    /**
      * @return Comparator for {@link PartitionHashRecordV2}.
      */
     private Comparator<PartitionHashRecordV2> buildRecordComparator() {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTaskV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTaskV2.java
index 4b66444..9c389a5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTaskV2.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTaskV2.java
@@ -32,6 +32,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.regex.Pattern;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteInterruptedException;
@@ -64,6 +65,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.visor.verify.CacheFilterEnum;
 import org.apache.ignite.internal.visor.verify.VisorIdleVerifyDumpTaskArg;
 import org.apache.ignite.internal.visor.verify.VisorIdleVerifyTaskArg;
+import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.resources.LoggerResource;
@@ -115,7 +117,10 @@ public class VerifyBackupPartitionsTaskV2 extends ComputeTaskAdapter<VisorIdleVe
 
         reduceResults(results, clusterHashes, exceptions);
 
-        return checkConflicts(clusterHashes, exceptions);
+        if (results.size() != exceptions.size())
+            return checkConflicts(clusterHashes, exceptions);
+        else
+            return new IdleVerifyResultV2(new HashMap<>(), new HashMap<>(), new HashMap<>(), exceptions, false);
     }
 
     /** {@inheritDoc} */
@@ -183,7 +188,7 @@ public class VerifyBackupPartitionsTaskV2 extends ComputeTaskAdapter<VisorIdleVe
             }
         }
 
-        return new IdleVerifyResultV2(updateCntrConflicts, hashConflicts, movingParts, exceptions);
+        return new IdleVerifyResultV2(updateCntrConflicts, hashConflicts, movingParts, exceptions, true);
     }
 
     /** */
@@ -332,6 +337,48 @@ public class VerifyBackupPartitionsTaskV2 extends ComputeTaskAdapter<VisorIdleVe
             }
         }
 
+        /**
+         * Class that processes cache filtering chain.
+         */
+        private class CachesFiltering {
+            /**
+             * Initially all cache descriptors are included.
+             */
+            private final Set<CacheGroupContext> filteredCacheGroups;
+
+            /** */
+            public CachesFiltering(Collection<CacheGroupContext> cacheGroups) {
+                filteredCacheGroups = new HashSet<>(cacheGroups);
+            }
+
+            /**
+             * Applies filtering closure.
+             *
+             * @param closure filter
+             * @return this
+             */
+            public CachesFiltering filter(
+                    IgniteInClosure<Set<CacheGroupContext>> closure) {
+                closure.apply(filteredCacheGroups);
+
+                return this;
+            }
+
+            /**
+             * Returns result set of cache ids.
+             *
+             * @return set of filtered cache ids.
+             */
+            public Set<Integer> result() {
+                Set<Integer> res = new HashSet<>();
+
+                for (CacheGroupContext cacheGrp : filteredCacheGroups)
+                    res.add(cacheGrp.groupId());
+
+                return res;
+            }
+        }
+
         /** */
         private List<Future<Map<PartitionKeyV2, PartitionHashRecordV2>>> calcPartitionHashAsync(
             Set<Integer> grpIds,
@@ -356,71 +403,85 @@ public class VerifyBackupPartitionsTaskV2 extends ComputeTaskAdapter<VisorIdleVe
 
         /** */
         private Set<Integer> getGroupIds() {
-            Set<Integer> grpIds = new HashSet<>();
+            Collection<CacheGroupContext> cacheGroups = ignite.context().cache().cacheGroups();
 
-            if (arg.getCaches() != null && !arg.getCaches().isEmpty()) {
-                Set<String> missingCaches = new HashSet<>();
+            Set<Integer> grpIds = new CachesFiltering(cacheGroups)
+                .filter(this::filterByCacheNames)
+                .filter(this::filterByCacheFilter)
+                .filter(this::filterByExcludeCaches)
+                .result();
 
-                for (String cacheName : arg.getCaches()) {
-                    DynamicCacheDescriptor desc = ignite.context().cache().cacheDescriptor(cacheName);
+            if (F.isEmpty(grpIds))
+                throw new NoMatchingCachesException();
 
-                    if (desc == null || !isCacheMatchFilter(cacheName)) {
-                        missingCaches.add(cacheName);
+            return grpIds;
+        }
 
-                        continue;
-                    }
+        /**
+         * Filters cache groups by exclude regexps.
+         *
+         * @param cachesToFilter cache groups to filter
+         */
+        private void filterByExcludeCaches(Set<CacheGroupContext> cachesToFilter) {
+            if (!F.isEmpty(arg.getExcludeCaches())) {
+                Set<Pattern> excludedNamesPatterns = new HashSet<>();
 
-                    grpIds.add(desc.groupId());
-                }
+                for (String excluded : arg.getExcludeCaches())
+                    excludedNamesPatterns.add(Pattern.compile(excluded));
 
-                handlingMissedCaches(missingCaches);
-            }
-            else if (onlySpecificCaches()) {
-                for (DynamicCacheDescriptor desc : ignite.context().cache().cacheDescriptors().values()) {
-                    if (desc.cacheConfiguration().getCacheMode() != LOCAL && isCacheMatchFilter(desc.cacheName()))
-                        grpIds.add(desc.groupId());
-                }
+                cachesToFilter.removeIf(grp -> doesGrpMatchOneOfPatterns(grp, excludedNamesPatterns));
             }
-            else
-                grpIds = getCacheGroupIds();
-
-            return grpIds;
         }
 
         /**
-         * Gets filtered group ids.
+         * Filters cache groups by cache filter, also removes system (if not specified in filter option)
+         * and local caches.
+         *
+         * @param cachesToFilter cache groups to filter
          */
-        private Set<Integer> getCacheGroupIds() {
-            Collection<CacheGroupContext> groups = ignite.context().cache().cacheGroups();
+        private void filterByCacheFilter(Set<CacheGroupContext> cachesToFilter) {
+            if (onlySpecificCaches())
+                cachesToFilter.removeIf(grp -> !doesGrpMatchFilter(grp));
 
-            Set<Integer> grpIds = new HashSet<>();
+            boolean excludeSysCaches;
 
-            if (F.isEmpty(arg.getExcludeCaches())) {
-                for (CacheGroupContext grp : groups) {
-                    if (!grp.systemCache() && !grp.isLocal())
-                        grpIds.add(grp.groupId());
-                }
-
-                return grpIds;
-            }
+            if (arg instanceof VisorIdleVerifyDumpTaskArg) {
+                CacheFilterEnum filter = ((VisorIdleVerifyDumpTaskArg) arg).getCacheFilterEnum();
 
-            for (CacheGroupContext grp : groups) {
-                if (!grp.systemCache() && !grp.isLocal() && !isGrpExcluded(grp))
-                    grpIds.add(grp.groupId());
-            }
+                excludeSysCaches = !(filter == CacheFilterEnum.SYSTEM);
+            } else
+                excludeSysCaches = true;
 
-            return grpIds;
+            cachesToFilter.removeIf(grp -> (grp.systemCache() && excludeSysCaches) || grp.isLocal());
         }
 
         /**
-         * @param grp Group.
+         * Filters cache groups by whitelist of cache name regexps. By default, all cache groups are included.
+         *
+         * @param cachesToFilter cache groups to filter
          */
-        private boolean isGrpExcluded(CacheGroupContext grp) {
-            if (arg.getExcludeCaches().contains(grp.name()))
-                return true;
+        private void filterByCacheNames(Set<CacheGroupContext> cachesToFilter) {
+            if (arg.getCaches() != null && !arg.getCaches().isEmpty()) {
+                Set<Pattern> cacheNamesPatterns = new HashSet<>();
+
+                for (String cacheNameRegexp : arg.getCaches())
+                    cacheNamesPatterns.add(Pattern.compile(cacheNameRegexp));
+
+                cachesToFilter.removeIf(grp -> !doesGrpMatchOneOfPatterns(grp, cacheNamesPatterns));
+            }
+        }
 
+        /**
+         * Checks does the given group match filter.
+         *
+         * @param grp cache group.
+         * @return boolean result
+         */
+        private boolean doesGrpMatchFilter(CacheGroupContext grp) {
             for (GridCacheContext cacheCtx : grp.caches()) {
-                if (arg.getExcludeCaches().contains(cacheCtx.name()))
+                DynamicCacheDescriptor desc = ignite.context().cache().cacheDescriptor(cacheCtx.name());
+
+                if (desc.cacheConfiguration().getCacheMode() != LOCAL && isCacheMatchFilter(desc))
                     return true;
             }
 
@@ -428,30 +489,24 @@ public class VerifyBackupPartitionsTaskV2 extends ComputeTaskAdapter<VisorIdleVe
         }
 
         /**
-         * Checks and throw exception if caches was missed.
+         * Checks does the name of given cache group or some of the names of its caches
+         * match at least one of regexp from set.
          *
-         * @param missingCaches Missing caches.
+         * @param grp cache group
+         * @param patterns compiled regexp patterns
+         * @return boolean result
          */
-        private void handlingMissedCaches(Set<String> missingCaches) {
-            if (missingCaches.isEmpty())
-                return;
-
-            SB strBuilder = new SB("The following caches do not exist");
-
-            if (onlySpecificCaches()) {
-                VisorIdleVerifyDumpTaskArg vdta = (VisorIdleVerifyDumpTaskArg)arg;
+        private boolean doesGrpMatchOneOfPatterns(CacheGroupContext grp, Set<Pattern> patterns) {
+            for (Pattern pattern : patterns) {
+                if (grp.name() != null && pattern.matcher(grp.name()).matches())
+                    return true;
 
-                strBuilder.a(" or do not match to the given filter [").a(vdta.getCacheFilterEnum()).a("]: ");
+                for (GridCacheContext cacheCtx : grp.caches())
+                    if (cacheCtx.name() != null && pattern.matcher(cacheCtx.name()).matches())
+                        return true;
             }
-            else
-                strBuilder.a(": ");
-
-            for (String name : missingCaches)
-                strBuilder.a(name).a(", ");
 
-            strBuilder.d(strBuilder.length() - 2, strBuilder.length());
-
-            throw new IgniteException(strBuilder.toString());
+            return false;
         }
 
         /**
@@ -468,14 +523,12 @@ public class VerifyBackupPartitionsTaskV2 extends ComputeTaskAdapter<VisorIdleVe
         }
 
         /**
-         * @param cacheName Cache name.
+         * @param desc Cache descriptor.
          */
-        private boolean isCacheMatchFilter(String cacheName) {
+        private boolean isCacheMatchFilter(DynamicCacheDescriptor desc) {
             if (arg instanceof VisorIdleVerifyDumpTaskArg) {
                 DataStorageConfiguration dsCfg = ignite.context().config().getDataStorageConfiguration();
 
-                DynamicCacheDescriptor desc = ignite.context().cache().cacheDescriptor(cacheName);
-
                 CacheConfiguration cc = desc.cacheConfiguration();
 
                 VisorIdleVerifyDumpTaskArg vdta = (VisorIdleVerifyDumpTaskArg)arg;
diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
index 7356c7a..b1a792f 100644
--- a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
@@ -28,11 +28,11 @@ import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.TreeMap;
 import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
@@ -115,6 +115,7 @@ import org.junit.Test;
 
 import static java.nio.file.Files.delete;
 import static java.nio.file.Files.newDirectoryStream;
+import static java.util.Arrays.asList;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_ENABLE_EXPERIMENTAL_COMMAND;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -276,7 +277,7 @@ public class GridCommandHandlerTest extends GridCommonAbstractTest {
      * @return Result of execution.
      */
     protected int execute(String... args) {
-        return execute(new ArrayList<>(Arrays.asList(args)));
+        return execute(new ArrayList<>(asList(args)));
     }
 
     /**
@@ -310,7 +311,7 @@ public class GridCommandHandlerTest extends GridCommonAbstractTest {
      * @return Result of execution
      */
     protected int execute(CommandHandler hnd, String... args) {
-        ArrayList<String> args0 = new ArrayList<>(Arrays.asList(args));
+        ArrayList<String> args0 = new ArrayList<>(asList(args));
 
         // Add force to avoid interactive confirmation
         args0.add(CMD_AUTO_CONFIRMATION);
@@ -441,9 +442,9 @@ public class GridCommandHandlerTest extends GridCommonAbstractTest {
 
         assertTrue(i != -1);
 
-        String crdStr = outStr.substring(i);
+        String crdStr = outStr.substring(i).trim();
 
-        return crdStr.substring(0, crdStr.indexOf('\n'));
+        return crdStr.substring(0, crdStr.indexOf('\n')).trim();
     }
 
     /**
@@ -1147,10 +1148,9 @@ public class GridCommandHandlerTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @throws Exception If failed.
      */
     @Test
-    public void testCacheHelp() throws Exception {
+    public void testCacheHelp() {
         injectTestSystemOut();
 
         assertEquals(EXIT_CODE_OK, execute("--cache", "help"));
@@ -1169,10 +1169,9 @@ public class GridCommandHandlerTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @throws Exception If failed.
      */
     @Test
-    public void testCorrectCacheOptionsNaming() throws Exception {
+    public void testCorrectCacheOptionsNaming() {
         Pattern p = Pattern.compile("^--([a-z]+(-)?)+([a-z]+)");
 
         for (CacheCommand cmd : CacheCommand.values()) {
@@ -1182,10 +1181,9 @@ public class GridCommandHandlerTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @throws Exception If failed.
      */
     @Test
-    public void testHelp() throws Exception {
+    public void testHelp() {
         injectTestSystemOut();
 
         assertEquals(EXIT_CODE_OK, execute("--help"));
@@ -1199,7 +1197,7 @@ public class GridCommandHandlerTest extends GridCommonAbstractTest {
      */
     @Test
     public void testCacheIdleVerify() throws Exception {
-        IgniteEx ignite = (IgniteEx)startGrids(2);
+        IgniteEx ignite = startGrids(2);
 
         ignite.cluster().active(true);
 
@@ -1209,15 +1207,15 @@ public class GridCommandHandlerTest extends GridCommonAbstractTest {
 
         assertEquals(EXIT_CODE_OK, execute("--cache", "idle_verify"));
 
-        assertTrue(testOut.toString().contains("no conflicts have been found"));
+        assertContains(testOut.toString(), "no conflicts have been found");
 
-        HashSet<Integer> clearKeys = new HashSet<>(Arrays.asList(1, 2, 3, 4, 5, 6));
+        HashSet<Integer> clearKeys = new HashSet<>(asList(1, 2, 3, 4, 5, 6));
 
         ignite.context().cache().cache(DEFAULT_CACHE_NAME).clearLocallyAll(clearKeys, true, true, true);
 
         assertEquals(EXIT_CODE_OK, execute("--cache", "idle_verify"));
 
-        assertTrue(testOut.toString().contains("conflict partitions"));
+        assertContains(testOut.toString(), "conflict partitions");
     }
 
     /**
@@ -1227,7 +1225,7 @@ public class GridCommandHandlerTest extends GridCommonAbstractTest {
      */
     @Test
     public void testCacheIdleVerifyTwoConflictTypes() throws Exception {
-        IgniteEx ignite = (IgniteEx)startGrids(2);
+        IgniteEx ignite = startGrids(2);
 
         ignite.cluster().active(true);
 
@@ -1257,7 +1255,7 @@ public class GridCommandHandlerTest extends GridCommonAbstractTest {
      */
     @Test
     public void testCacheIdleVerifyDumpSkipZerosUpdateCounters() throws Exception {
-        IgniteEx ignite = (IgniteEx)startGrids(2);
+        IgniteEx ignite = startGrids(2);
 
         ignite.cluster().active(true);
 
@@ -1314,7 +1312,7 @@ public class GridCommandHandlerTest extends GridCommonAbstractTest {
      */
     @Test
     public void testCacheIdleVerifyDump() throws Exception {
-        IgniteEx ignite = (IgniteEx)startGrids(3);
+        IgniteEx ignite = startGrids(3);
 
         ignite.cluster().active(true);
 
@@ -1366,6 +1364,171 @@ public class GridCommandHandlerTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Common method for idle_verify tests with multiple options.
+     *
+     * @throws Exception if failed
+     */
+    @Test
+    public void testCacheIdleVerifyMultipleCacheFilterOptions()
+            throws Exception {
+        IgniteEx ignite = startGrids(2);
+
+        ignite.cluster().active(true);
+
+        ignite.createCache(new CacheConfiguration<>()
+                .setAffinity(new RendezvousAffinityFunction(false, 32))
+                .setGroupName("shared_grp")
+                .setBackups(1)
+                .setName(DEFAULT_CACHE_NAME));
+
+        ignite.createCache(new CacheConfiguration<>()
+                .setAffinity(new RendezvousAffinityFunction(false, 32))
+                .setGroupName("shared_grp")
+                .setBackups(1)
+                .setName(DEFAULT_CACHE_NAME + "_second"));
+
+        ignite.createCache(new CacheConfiguration<>()
+                .setAffinity(new RendezvousAffinityFunction(false, 64))
+                .setBackups(1)
+                .setName(DEFAULT_CACHE_NAME + "_third"));
+
+        ignite.createCache(new CacheConfiguration<>()
+                .setAffinity(new RendezvousAffinityFunction(false, 128))
+                .setBackups(1)
+                .setName("wrong_cache"));
+
+        injectTestSystemOut();
+
+        testCacheIdleVerifyMultipleCacheFilterOptionsCommon(
+            true,
+            "idle_verify check has finished",
+            "idle_verify task was executed with the following args: --cache-filter SYSTEM --exclude-caches wrong.* ",
+            "--cache", "idle_verify", "--dump", "--cache-filter", "SYSTEM", "--exclude-caches", "wrong.*"
+        );
+        testCacheIdleVerifyMultipleCacheFilterOptionsCommon(
+            true,
+            "idle_verify check has finished, found 96 partitions",
+            null,
+            "--cache", "idle_verify", "--dump", ".*", "--exclude-caches", "wrong.*"
+        );
+        testCacheIdleVerifyMultipleCacheFilterOptionsCommon(
+            true,
+            "idle_verify check has finished, found 32 partitions",
+            null,
+            "--cache", "idle_verify", "--dump", "shared.*", "--cache-filter", "ALL"
+        );
+        testCacheIdleVerifyMultipleCacheFilterOptionsCommon(
+            true,
+            "idle_verify check has finished, found 160 partitions",
+            null,
+            "--cache", "idle_verify", "--dump", "shared.*,wrong.*", "--cache-filter", "ALL"
+        );
+        testCacheIdleVerifyMultipleCacheFilterOptionsCommon(
+            true,
+            "idle_verify check has finished, found 160 partitions",
+            null,
+            "--cache", "idle_verify", "--dump", "shared.*,wrong.*", "--cache-filter", "ALL"
+        );
+        testCacheIdleVerifyMultipleCacheFilterOptionsCommon(
+            true,
+            "idle_verify check has finished, found 160 partitions",
+            null,
+            "--cache", "idle_verify", "--dump", "shared.*,wrong.*", "--cache-filter", "ALL"
+        );
+        testCacheIdleVerifyMultipleCacheFilterOptionsCommon(
+            true,
+            "There are no caches matching given filter options.",
+            null,
+            "--cache", "idle_verify", "--exclude-caches", ".*"
+        );
+        testCacheIdleVerifyMultipleCacheFilterOptionsCommon(
+            false,
+            "Invalid cache name regexp",
+            null,
+            "--cache", "idle_verify", "--dump", "--exclude-caches", "["
+        );
+        testCacheIdleVerifyMultipleCacheFilterOptionsCommon(
+            true,
+            "idle_verify check has finished, no conflicts have been found.",
+            null,
+            "--cache", "idle_verify", ".*", "--exclude-caches", "wrong-.*"
+        );
+        testCacheIdleVerifyMultipleCacheFilterOptionsCommon(
+            true,
+            "idle_verify check has finished, no conflicts have been found.",
+            null,
+            "--cache", "idle_verify", "--dump", ".*", "--cache-filter", "PERSISTENT"
+        );
+    }
+
+    /**
+     * Runs idle_verify with specified arguments and checks the dump if dump option was present.
+     *
+     * @param exitOk whether CommandHandler should exit without errors
+     * @param outputExp expected dump output
+     * @param cmdExp expected command built from command line arguments
+     * @param args command handler arguments
+     * @throws IOException if some of file operations failed
+     */
+    private void testCacheIdleVerifyMultipleCacheFilterOptionsCommon(
+        boolean exitOk,
+        String outputExp,
+        String cmdExp,
+        String... args
+    ) throws IOException {
+        testOut.reset();
+
+        Set<String> argsSet = new HashSet<>(asList(args));
+
+        int exitCode = execute(args);
+
+        assertEquals(exitOk, EXIT_CODE_OK == exitCode);
+
+        if (exitCode == EXIT_CODE_OK) {
+            Matcher fileNameMatcher = dumpFileNameMatcher();
+
+            if (fileNameMatcher.find()) {
+                assertTrue(argsSet.contains("--dump"));
+
+                Path filePath = Paths.get(fileNameMatcher.group(1));
+
+                String dump = new String(Files.readAllBytes(filePath));
+
+                Files.delete(filePath);
+
+                assertContains(dump, outputExp);
+
+                if (cmdExp != null)
+                    assertContains(dump, cmdExp);
+            }
+            else {
+                assertFalse(argsSet.contains("--dump"));
+
+                assertContains(testOut.toString(), outputExp);
+            }
+        } else
+            assertContains(testOut.toString(), outputExp);
+    }
+
+    /**
+     * Checks that string {@param str} contains substring {@param substr}. Logs both strings
+     * and throws {@link java.lang.AssertionError}, if not.
+     *
+     * @param str string
+     * @param substr substring
+     */
+    private void assertContains(String str, String substr) {
+        try {
+            assertTrue(str.contains(substr));
+        } catch (AssertionError e) {
+            log.warning(String.format("String does not contain substring: '%s':", substr));
+            log.warning("String:");
+            log.warning(str);
+            throw e;
+        }
+    }
+
+    /**
      * Checking sorting of partitions.
      *
      * @param expectedPartsCount Expected parts count.
@@ -1395,7 +1558,7 @@ public class GridCommandHandlerTest extends GridCommonAbstractTest {
      */
     @Test
     public void testCacheIdleVerifyDumpForCorruptedData() throws Exception {
-        IgniteEx ignite = (IgniteEx)startGrids(3);
+        IgniteEx ignite = startGrids(3);
 
         ignite.cluster().active(true);
 
@@ -1426,6 +1589,8 @@ public class GridCommandHandlerTest extends GridCommonAbstractTest {
         if (fileNameMatcher.find()) {
             String dumpWithConflicts = new String(Files.readAllBytes(Paths.get(fileNameMatcher.group(1))));
 
+            log.info(dumpWithConflicts);
+
             assertTrue(dumpWithConflicts.contains("found 2 conflict partitions: [counterConflicts=1, hashConflicts=1]"));
         }
         else
@@ -1882,7 +2047,7 @@ public class GridCommandHandlerTest extends GridCommonAbstractTest {
 
         assertEquals(EXIT_CODE_OK, execute("--cache", "idle_verify"));
 
-        assertTrue(testOut.toString().contains("no conflicts have been found"));
+        assertContains(testOut.toString(), "no conflicts have been found");
 
         startGrid(2);
 
@@ -1890,7 +2055,7 @@ public class GridCommandHandlerTest extends GridCommonAbstractTest {
 
         assertEquals(EXIT_CODE_OK, execute("--cache", "idle_verify"));
 
-        assertTrue(testOut.toString().contains("MOVING partitions"));
+        assertContains(testOut.toString(), "MOVING partitions");
     }
 
     /**