You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sk...@apache.org on 2020/03/20 09:00:31 UTC

[ignite] branch master updated: IGNITE-12761 Added ability to disable check crc sums of stored pages due to invalidate_indexes.

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

sk0x50 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 4a94ac2  IGNITE-12761 Added ability to disable check crc sums of stored pages due to invalidate_indexes.
4a94ac2 is described below

commit 4a94ac2125cc0e233b59c4dd70db97fd88b84c3c
Author: Philipp Masharov <fm...@gridgain.com>
AuthorDate: Fri Mar 20 11:59:38 2020 +0300

    IGNITE-12761 Added ability to disable check crc sums of stored pages due to invalidate_indexes.
    
    Signed-off-by: Slava Koptilin <sl...@gmail.com>
---
 .../commandline/cache/CacheValidateIndexes.java    | 37 +++++++++++----
 .../cache/argument/ValidateIndexesCommandArg.java  |  3 ++
 .../internal/dto/IgniteDataTransferObject.java     |  3 ++
 .../visor/verify/VisorValidateIndexesTaskArg.java  | 34 +++++++++++--
 ...mandHandlerClusterByClassTest_cache_help.output |  1 +
 ...dlerClusterByClassWithSSLTest_cache_help.output |  1 +
 .../visor/verify/ValidateIndexesClosure.java       | 37 +++++++++------
 .../visor/verify/VisorValidateIndexesTask.java     |  7 ++-
 ...xingMultithreadedLoadContinuousRestartTest.java |  2 +-
 .../db/LongDestroyDurableBackgroundTaskTest.java   |  2 +-
 .../RebuildIndexWithHistoricalRebalanceTest.java   |  2 +-
 .../query/h2/GridIndexFullRebuildTest.java         |  2 +-
 ...idCommandHandlerIndexingClusterByClassTest.java | 12 +++++
 .../util/GridCommandHandlerIndexingTest.java       | 55 +++++++++++++++++-----
 14 files changed, 155 insertions(+), 43 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheValidateIndexes.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheValidateIndexes.java
index f8661c4..c98f28a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheValidateIndexes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheValidateIndexes.java
@@ -52,6 +52,7 @@ import static org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCo
 import static org.apache.ignite.internal.commandline.cache.argument.IdleVerifyCommandArg.EXCLUDE_CACHES;
 import static org.apache.ignite.internal.commandline.cache.argument.ValidateIndexesCommandArg.CHECK_FIRST;
 import static org.apache.ignite.internal.commandline.cache.argument.ValidateIndexesCommandArg.CHECK_THROUGH;
+import static org.apache.ignite.internal.commandline.cache.argument.ValidateIndexesCommandArg.CHECK_CRC;
 
 /**
  * Validate indexes command.
@@ -72,6 +73,7 @@ public class CacheValidateIndexes implements Command<CacheValidateIndexes.Argume
 
         map.put(CHECK_FIRST + " N", "validate only the first N keys");
         map.put(CHECK_THROUGH + " K", "validate every Kth key");
+        map.put(CHECK_CRC.toString(), "check the CRC-sum of pages stored on disk");
 
         usageCache(logger, VALIDATE_INDEXES, description, map,
             optional(CACHES), OP_NODE_ID, optional(or(CHECK_FIRST + " N", CHECK_THROUGH + " K")));
@@ -93,14 +95,22 @@ public class CacheValidateIndexes implements Command<CacheValidateIndexes.Argume
         /** Number of entries to check through. */
         private int checkThrough = -1;
 
+        /** Check CRC */
+        private boolean checkCrc;
+
         /**
-         *
+         * @param caches Caches to validate.
+         * @param nodeId Node Id.
+         * @param checkFirst Max number of entries to be checked..
+         * @param checkThrough Number of entries to check through.
+         * @param checkCrc Check CRC.
          */
-        public Arguments(Set<String> caches, UUID nodeId, int checkFirst, int checkThrough) {
+        public Arguments(Set<String> caches, UUID nodeId, int checkFirst, int checkThrough, boolean checkCrc) {
             this.caches = caches;
             this.nodeId = nodeId;
             this.checkFirst = checkFirst;
             this.checkThrough = checkThrough;
+            this.checkCrc = checkCrc;
         }
 
         /**
@@ -118,6 +128,13 @@ public class CacheValidateIndexes implements Command<CacheValidateIndexes.Argume
         }
 
         /**
+         * @return Check CRC
+         */
+        public boolean checkCrc() {
+            return checkCrc;
+        }
+
+        /**
          * @return Number of entries to check through.
          */
         public int checkThrough() {
@@ -146,7 +163,8 @@ public class CacheValidateIndexes implements Command<CacheValidateIndexes.Argume
             args.caches(),
             args.nodeId() != null ? Collections.singleton(args.nodeId()) : null,
             args.checkFirst(),
-            args.checkThrough()
+            args.checkThrough(),
+            args.checkCrc()
         );
 
         try (GridClient client = Command.startClient(clientCfg)) {
@@ -200,7 +218,7 @@ public class CacheValidateIndexes implements Command<CacheValidateIndexes.Argume
             if (!errors)
                 logger.info("no issues found.");
             else
-                logger.info("issues found (listed above).");
+                logger.severe("issues found (listed above).");
 
             logger.info("");
 
@@ -212,15 +230,18 @@ public class CacheValidateIndexes implements Command<CacheValidateIndexes.Argume
     @Override public void parseArguments(CommandArgIterator argIter) {
         int checkFirst = -1;
         int checkThrough = -1;
+        boolean checkCrc = false;
         UUID nodeId = null;
         Set<String> caches = null;
 
-        int argsCnt = 0;
-
-        while (argIter.hasNextSubArg() && argsCnt++ < 4) {
+        while (argIter.hasNextSubArg()) {
             String nextArg = argIter.nextArg("");
 
             ValidateIndexesCommandArg arg = CommandArgUtils.of(nextArg, ValidateIndexesCommandArg.class);
+            if (arg == CHECK_CRC) {
+                checkCrc = true;
+                continue;
+            }
 
             if (arg == CHECK_FIRST || arg == CHECK_THROUGH) {
                 if (!argIter.hasNextSubArg())
@@ -262,7 +283,7 @@ public class CacheValidateIndexes implements Command<CacheValidateIndexes.Argume
             caches = argIter.parseStringSet(nextArg);
         }
 
-        args = new Arguments(caches, nodeId, checkFirst, checkThrough);
+        args = new Arguments(caches, nodeId, checkFirst, checkThrough, checkCrc);
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/argument/ValidateIndexesCommandArg.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/argument/ValidateIndexesCommandArg.java
index 723b901..f3b4284 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/argument/ValidateIndexesCommandArg.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/argument/ValidateIndexesCommandArg.java
@@ -27,6 +27,9 @@ public enum ValidateIndexesCommandArg implements CommandArg {
     /** Check first. */
     CHECK_FIRST("--check-first"),
 
+    /** Check crc. */
+    CHECK_CRC("--check-crc"),
+
     /** Check through. */
     CHECK_THROUGH("--check-through");
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/dto/IgniteDataTransferObject.java b/modules/core/src/main/java/org/apache/ignite/internal/dto/IgniteDataTransferObject.java
index 3441742..bdfe7c2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/dto/IgniteDataTransferObject.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/dto/IgniteDataTransferObject.java
@@ -53,6 +53,9 @@ public abstract class IgniteDataTransferObject implements Externalizable {
     /** Version 5. */
     protected static final byte V5 = 5;
 
+    /** Version 6. */
+    protected static final byte V6 = 6;
+
     /**
      * @param col Source collection.
      * @param <T> Collection type.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTaskArg.java
index 44cbccf..0bddb9c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTaskArg.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTaskArg.java
@@ -22,14 +22,14 @@ import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.util.Set;
 import java.util.UUID;
+import org.apache.ignite.internal.dto.IgniteDataTransferObject;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.internal.visor.VisorDataTransferObject;
 
 /**
  *
  */
-public class VisorValidateIndexesTaskArg extends VisorDataTransferObject {
+public class VisorValidateIndexesTaskArg extends IgniteDataTransferObject {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -39,6 +39,9 @@ public class VisorValidateIndexesTaskArg extends VisorDataTransferObject {
     /** Check first K elements. */
     private int checkFirst;
 
+    /** Check CRC */
+    private boolean checkCrc;
+
     /** Check through K element (skip K-1, check Kth). */
     private int checkThrough;
 
@@ -55,11 +58,18 @@ public class VisorValidateIndexesTaskArg extends VisorDataTransferObject {
     /**
      * @param caches Caches.
      */
-    public VisorValidateIndexesTaskArg(Set<String> caches, Set<UUID> nodes, int checkFirst, int checkThrough) {
+    public VisorValidateIndexesTaskArg(
+        Set<String> caches,
+        Set<UUID> nodes,
+        int checkFirst,
+        int checkThrough,
+        boolean checkCrc
+    ) {
         this.caches = caches;
         this.checkFirst = checkFirst;
         this.checkThrough = checkThrough;
         this.nodes = nodes;
+        this.checkCrc = checkCrc;
     }
 
     /**
@@ -84,6 +94,13 @@ public class VisorValidateIndexesTaskArg extends VisorDataTransferObject {
     }
 
     /**
+     * @return checkCrc.
+     */
+    public boolean сheckCrc() {
+        return checkCrc;
+    }
+
+    /**
      * @return checkThrough.
      */
     public int getCheckThrough() {
@@ -96,6 +113,7 @@ public class VisorValidateIndexesTaskArg extends VisorDataTransferObject {
         out.writeInt(checkFirst);
         out.writeInt(checkThrough);
         U.writeCollection(out, nodes);
+        out.writeBoolean(checkCrc);
     }
 
     /** {@inheritDoc} */
@@ -113,11 +131,19 @@ public class VisorValidateIndexesTaskArg extends VisorDataTransferObject {
 
         if (protoVer > V2)
             nodes = U.readSet(in);
+
+        if (protoVer >= V6)
+            checkCrc = in.readBoolean();
+    }
+
+    /** Set checkCrc */
+    protected void checkCrc(boolean checkCrc) {
+        this.checkCrc = checkCrc;
     }
 
     /** {@inheritDoc} */
     @Override public byte getProtocolVersion() {
-        return V3;
+        return V6;
     }
 
     /** {@inheritDoc} */
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 2d70ab8..1b2c8d7 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
@@ -35,6 +35,7 @@ Arguments: --cache help --yes
     Parameters:
       --check-first N    - validate only the first N keys
       --check-through K  - validate every Kth key
+      --check-crc        - check the CRC-sum of pages stored on disk
 
   --cache contention minQueueSize [nodeId] [maxPrint]
     Show the keys that are point of contention for multiple transactions.
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 2d70ab8..1b2c8d7 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
@@ -35,6 +35,7 @@ Arguments: --cache help --yes
     Parameters:
       --check-first N    - validate only the first N keys
       --check-through K  - validate every Kth key
+      --check-crc        - check the CRC-sum of pages stored on disk
 
   --cache contention minQueueSize [nodeId] [maxPrint]
     Show the keys that are point of contention for multiple transactions.
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java
index 8f1d53f..9a69d62 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java
@@ -111,6 +111,9 @@ public class ValidateIndexesClosure implements IgniteCallable<VisorValidateIndex
     /** If provided only each Kth element will be validated. */
     private final int checkThrough;
 
+    /** Check CRC */
+    private boolean checkCrc;
+
     /** Counter of processed partitions. */
     private final AtomicInteger processedPartitions = new AtomicInteger(0);
 
@@ -139,11 +142,13 @@ public class ValidateIndexesClosure implements IgniteCallable<VisorValidateIndex
      * @param cacheNames Cache names.
      * @param checkFirst If positive only first K elements will be validated.
      * @param checkThrough If positive only each Kth element will be validated.
+     * @param checkCrc Check CRC sum on stored pages on disk.
      */
-    public ValidateIndexesClosure(Set<String> cacheNames, int checkFirst, int checkThrough) {
+    public ValidateIndexesClosure(Set<String> cacheNames, int checkFirst, int checkThrough, boolean checkCrc) {
         this.cacheNames = cacheNames;
         this.checkFirst = checkFirst;
         this.checkThrough = checkThrough;
+        this.checkCrc = checkCrc;
     }
 
     /** {@inheritDoc} */
@@ -337,25 +342,27 @@ public class ValidateIndexesClosure implements IgniteCallable<VisorValidateIndex
                     throw new GridNotIdleException(IdleVerifyUtility.CLUSTER_NOT_IDLE_MSG);
             }
 
-            for (Integer grpId: grpIds) {
-                final CacheGroupContext grpCtx = ignite.context().cache().cacheGroup(grpId);
+            if (checkCrc) {
+                for (Integer grpId: grpIds) {
+                    final CacheGroupContext grpCtx = ignite.context().cache().cacheGroup(grpId);
 
-                if (grpCtx == null || !grpCtx.persistenceEnabled()) {
-                    integrityCheckedIndexes.incrementAndGet();
+                    if (grpCtx == null || !grpCtx.persistenceEnabled()) {
+                        integrityCheckedIndexes.incrementAndGet();
 
-                    continue;
-                }
+                        continue;
+                    }
 
-                Future<T2<Integer, IndexIntegrityCheckIssue>> checkFut =
-                        calcExecutor.submit(new Callable<T2<Integer, IndexIntegrityCheckIssue>>() {
-                            @Override public T2<Integer, IndexIntegrityCheckIssue> call() throws Exception {
-                                IndexIntegrityCheckIssue issue = integrityCheckIndexPartition(grpCtx, cpFlag);
+                    Future<T2<Integer, IndexIntegrityCheckIssue>> checkFut =
+                            calcExecutor.submit(new Callable<T2<Integer, IndexIntegrityCheckIssue>>() {
+                                @Override public T2<Integer, IndexIntegrityCheckIssue> call() throws Exception {
+                                    IndexIntegrityCheckIssue issue = integrityCheckIndexPartition(grpCtx, cpFlag);
 
-                                return new T2<>(grpCtx.groupId(), issue);
-                            }
-                        });
+                                    return new T2<>(grpCtx.groupId(), issue);
+                                }
+                            });
 
-                integrityCheckFutures.add(checkFut);
+                    integrityCheckFutures.add(checkFut);
+                }
             }
 
             for (Future<T2<Integer, IndexIntegrityCheckIssue>> fut : integrityCheckFutures) {
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTask.java b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTask.java
index 922c53e..dc5972f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTask.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTask.java
@@ -105,7 +105,12 @@ public class VisorValidateIndexesTask extends VisorMultiNodeTask<VisorValidateIn
         /** {@inheritDoc} */
         @Override protected VisorValidateIndexesJobResult run(@Nullable VisorValidateIndexesTaskArg arg) throws IgniteException {
             try {
-                ValidateIndexesClosure clo = new ValidateIndexesClosure(arg.getCaches(), arg.getCheckFirst(), arg.getCheckThrough());
+                ValidateIndexesClosure clo = new ValidateIndexesClosure(
+                    arg.getCaches(),
+                    arg.getCheckFirst(),
+                    arg.getCheckThrough(),
+                    arg.сheckCrc()
+                );
 
                 ignite.context().resource().injectGeneric(clo);
 
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IndexingMultithreadedLoadContinuousRestartTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IndexingMultithreadedLoadContinuousRestartTest.java
index 2b5d882..7db43f5 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IndexingMultithreadedLoadContinuousRestartTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IndexingMultithreadedLoadContinuousRestartTest.java
@@ -156,7 +156,7 @@ public class IndexingMultithreadedLoadContinuousRestartTest extends GridCommonAb
             forceCheckpoint();
 
             // Validate indexes on start.
-            ValidateIndexesClosure clo = new ValidateIndexesClosure(Collections.singleton(CACHE_NAME), 0, 0);
+            ValidateIndexesClosure clo = new ValidateIndexesClosure(Collections.singleton(CACHE_NAME), 0, 0, false);
             ignite.context().resource().injectGeneric(clo);
             VisorValidateIndexesJobResult res = clo.call();
 
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/LongDestroyDurableBackgroundTaskTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/LongDestroyDurableBackgroundTaskTest.java
index ba7ea74..97f6925 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/LongDestroyDurableBackgroundTaskTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/LongDestroyDurableBackgroundTaskTest.java
@@ -322,7 +322,7 @@ public class LongDestroyDurableBackgroundTaskTest extends GridCommonAbstractTest
         log.info("Doing indexes validation.");
 
         VisorValidateIndexesTaskArg taskArg =
-            new VisorValidateIndexesTaskArg(Collections.singleton("SQL_PUBLIC_T"), nodeIds, 0, 1);
+            new VisorValidateIndexesTaskArg(Collections.singleton("SQL_PUBLIC_T"), nodeIds, 0, 1, true);
 
         VisorValidateIndexesTaskResult taskRes =
             ignite.compute().execute(VisorValidateIndexesTask.class.getName(), new VisorTaskArgument<>(nodeIds, taskArg, false));
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/RebuildIndexWithHistoricalRebalanceTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/RebuildIndexWithHistoricalRebalanceTest.java
index 03a3a6b..a7e07e3 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/RebuildIndexWithHistoricalRebalanceTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/RebuildIndexWithHistoricalRebalanceTest.java
@@ -229,7 +229,7 @@ public class RebuildIndexWithHistoricalRebalanceTest extends GridCommonAbstractT
 
         awaitPartitionMapExchange();
 
-        ValidateIndexesClosure clo = new ValidateIndexesClosure(Collections.singleton(CACHE_NAME), 0, 0);
+        ValidateIndexesClosure clo = new ValidateIndexesClosure(Collections.singleton(CACHE_NAME), 0, 0, false);
         node2.context().resource().injectGeneric(clo);
         VisorValidateIndexesJobResult res = clo.call();
 
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexFullRebuildTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexFullRebuildTest.java
index 9f4b558..2baa378 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexFullRebuildTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexFullRebuildTest.java
@@ -206,7 +206,7 @@ public class GridIndexFullRebuildTest extends GridCommonAbstractTest {
             ((IgniteProcessProxy)grid(2)).getId());
 
         VisorValidateIndexesTaskArg arg = new VisorValidateIndexesTaskArg(null,
-            null, 10000, 1);
+            null, 10000, 1, true);
 
         VisorTaskArgument<VisorValidateIndexesTaskArg> argument = new VisorTaskArgument<>(nodes, arg, true);
 
diff --git a/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingClusterByClassTest.java b/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingClusterByClassTest.java
index 8833fbc..66addd2 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingClusterByClassTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingClusterByClassTest.java
@@ -69,6 +69,18 @@ public class GridCommandHandlerIndexingClusterByClassTest extends GridCommandHan
     }
 
     /**
+     * Tests that validation with CRC checking doesn't fail if nothing is broken.
+     */
+    @Test
+    public void testValidateIndexesWithCrcNoErrors() {
+        injectTestSystemOut();
+
+        assertEquals(EXIT_CODE_OK, execute("--cache", "validate_indexes", "--check-crc", CACHE_NAME));
+
+        assertContains(log, testOut.toString(), "no issues found");
+    }
+
+    /**
      * Test verifies that validate_indexes command finishes successfully when no cache names are specified.
      */
     @Test
diff --git a/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java b/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java
index 822d51a..95b5f38 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java
@@ -31,9 +31,10 @@ import org.junit.Test;
 import static org.apache.ignite.internal.commandline.CommandHandler.EXIT_CODE_OK;
 import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
 import static org.apache.ignite.testframework.GridTestUtils.assertContains;
+import static org.apache.ignite.testframework.GridTestUtils.assertNotContains;
+import static org.apache.ignite.util.GridCommandHandlerIndexingUtils.createAndFillCache;
 import static org.apache.ignite.util.GridCommandHandlerIndexingUtils.CACHE_NAME;
 import static org.apache.ignite.util.GridCommandHandlerIndexingUtils.GROUP_NAME;
-import static org.apache.ignite.util.GridCommandHandlerIndexingUtils.createAndFillCache;
 
 /**
  * If you not necessary create nodes for each test you can try use
@@ -41,6 +42,9 @@ import static org.apache.ignite.util.GridCommandHandlerIndexingUtils.createAndFi
  */
 public class GridCommandHandlerIndexingTest extends GridCommandHandlerClusterPerMethodAbstractTest {
     /** */
+    public static final int GRID_CNT = 2;
+
+    /** */
     @Test
     public void testValidateIndexesFailedOnNotIdleCluster() throws Exception {
         checkpointFreq = 100L;
@@ -71,7 +75,7 @@ public class GridCommandHandlerIndexingTest extends GridCommandHandlerClusterPer
 
             injectTestSystemOut();
 
-            assertEquals(EXIT_CODE_OK, execute("--cache", "validate_indexes", CACHE_NAME));
+            assertEquals(EXIT_CODE_OK, execute("--cache", "validate_indexes", "--check-crc", CACHE_NAME));
         }
         finally {
             stopFlag.set(true);
@@ -86,10 +90,10 @@ public class GridCommandHandlerIndexingTest extends GridCommandHandlerClusterPer
     }
 
     /**
-     * Tests that corrupted pages in the index partition are detected.
+     * Tests with checkCrc=true that corrupted pages in the index partition are detected.
      */
     @Test
-    public void testCorruptedIndexPartitionShouldFailValidation() throws Exception {
+    public void testCorruptedIndexPartitionShouldFailValidationWithCrc() throws Exception {
         Ignite ignite = prepareGridForTest();
 
         forceCheckpoint();
@@ -98,17 +102,46 @@ public class GridCommandHandlerIndexingTest extends GridCommandHandlerClusterPer
 
         stopAllGrids();
 
-        corruptIndexPartition(idxPath);
+        corruptIndexPartition(idxPath, 1024, 4096);
 
-        startGrids(2);
+        startGrids(GRID_CNT);
 
         awaitPartitionMapExchange();
 
         injectTestSystemOut();
 
-        assertEquals(EXIT_CODE_OK, execute("--cache", "validate_indexes", CACHE_NAME));
+        assertEquals(EXIT_CODE_OK, execute("--cache", "validate_indexes", "--check-crc", CACHE_NAME));
 
         assertContains(log, testOut.toString(), "issues found (listed above)");
+        assertContains(log, testOut.toString(), "CRC validation failed");
+        assertNotContains(log, testOut.toString(), "Runtime failure on bounds");
+    }
+
+    /**
+     * Tests with that corrupted pages in the index partition are detected.
+     */
+    @Test
+    public void testCorruptedIndexPartitionShouldFailValidationWithoutCrc() throws Exception {
+        Ignite ignite = prepareGridForTest();
+
+        forceCheckpoint();
+
+        stopAllGrids();
+
+        File idxPath = indexPartition(ignite, GROUP_NAME);
+
+        corruptIndexPartition(idxPath, 6, 47746);
+
+        startGrids(GRID_CNT);
+
+        awaitPartitionMapExchange();
+
+        injectTestSystemOut();
+
+        assertEquals(EXIT_CODE_OK, execute("--cache", "validate_indexes", CACHE_NAME));
+
+        assertContains(log, testOut.toString(), "Runtime failure on bounds");
+        assertNotContains(log, testOut.toString(), "CRC validation failed");
     }
 
     /**
@@ -117,7 +150,7 @@ public class GridCommandHandlerIndexingTest extends GridCommandHandlerClusterPer
      * @throws Exception
      */
     private Ignite prepareGridForTest() throws Exception{
-        Ignite ignite = startGrids(2);
+        Ignite ignite = startGrids(GRID_CNT);
 
         ignite.cluster().active(true);
 
@@ -142,17 +175,17 @@ public class GridCommandHandlerIndexingTest extends GridCommandHandlerClusterPer
     /**
      * Write some random trash in index partition.
      */
-    private void corruptIndexPartition(File path) throws IOException {
+    private void corruptIndexPartition(File path, int size, int offset) throws IOException {
         assertTrue(path.exists());
 
         ThreadLocalRandom rand = ThreadLocalRandom.current();
 
         try (RandomAccessFile idx = new RandomAccessFile(path, "rw")) {
-            byte[] trash = new byte[1024];
+            byte[] trash = new byte[size];
 
             rand.nextBytes(trash);
 
-            idx.seek(4096);
+            idx.seek(offset);
 
             idx.write(trash);
         }