You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2019/01/09 11:28:19 UTC

[GitHub] antonovsergey93 closed pull request #5576: IGNITE-10507 idle_verify added CRC sum check and collecting exceptions from all nodes

antonovsergey93 closed pull request #5576: IGNITE-10507 idle_verify added CRC sum check and collecting exceptions from all nodes
URL: https://github.com/apache/ignite/pull/5576
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

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 fbc881949ef0..f779a1ca4296 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
@@ -361,6 +361,9 @@
     /** */
     private static final String CONFIG = "--config";
 
+    /** */
+    private static final String IDLE_CHECK_CRC = "--check-crc";
+
     /** Utility name. */
     private static final String UTILITY_NAME = "control.sh";
 
@@ -1031,7 +1034,10 @@ else if (idleVerifyV2)
      */
     private void legacyCacheIdleVerify(GridClient client, CacheArguments cacheArgs) throws GridClientException {
         VisorIdleVerifyTaskResult res = executeTask(
-            client, VisorIdleVerifyTask.class, new VisorIdleVerifyTaskArg(cacheArgs.caches()));
+            client,
+            VisorIdleVerifyTask.class,
+            new VisorIdleVerifyTaskArg(cacheArgs.caches(), cacheArgs.idleCheckCrc())
+        );
 
         Map<PartitionKey, List<PartitionHashRecord>> conflicts = res.getConflicts();
 
@@ -1180,7 +1186,12 @@ private void cacheIdleVerifyDump(GridClient client, CacheArguments cacheArgs) th
         String path = executeTask(
             client,
             VisorIdleVerifyDumpTask.class,
-            new VisorIdleVerifyDumpTaskArg(cacheArgs.caches(), cacheArgs.isSkipZeros(), cacheArgs.getCacheFilterEnum())
+            new VisorIdleVerifyDumpTaskArg(
+                cacheArgs.caches(),
+                cacheArgs.idleCheckCrc(),
+                cacheArgs.isSkipZeros(),
+                cacheArgs.getCacheFilterEnum()
+            )
         );
 
         log("VisorIdleVerifyDumpTask successfully written output to '" + path + "'");
@@ -1192,7 +1203,10 @@ private void cacheIdleVerifyDump(GridClient client, CacheArguments cacheArgs) th
      */
     private void cacheIdleVerifyV2(GridClient client, CacheArguments cacheArgs) throws GridClientException {
         IdleVerifyResultV2 res = executeTask(
-            client, VisorIdleVerifyTaskV2.class, new VisorIdleVerifyTaskArg(cacheArgs.caches()));
+            client,
+            VisorIdleVerifyTaskV2.class,
+            new VisorIdleVerifyTaskArg(cacheArgs.caches(), cacheArgs.idleCheckCrc())
+        );
 
         res.print(System.out::print);
     }
@@ -2127,6 +2141,8 @@ private CacheArguments parseAndValidateCacheArgs() {
                         cacheArgs.dump(true);
                     else if (CMD_SKIP_ZEROS.equals(nextArg))
                         cacheArgs.skipZeros(true);
+                    else if (IDLE_CHECK_CRC.equals(nextArg))
+                        cacheArgs.idleCheckCrc(true);
                     else if (CACHE_FILTER.equals(nextArg)) {
                         String filter = nextArg("The cache filter should be specified. The following values can be " +
                             "used: " + Arrays.toString(CacheFilterEnum.values()) + '.');
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheArguments.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheArguments.java
index 9372391397db..61a1f2e72b7f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheArguments.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheArguments.java
@@ -75,6 +75,9 @@
     /** Cache filter. */
     private CacheFilterEnum cacheFilterEnum = CacheFilterEnum.ALL;
 
+    /** Check CRC sum on idle verify. */
+    private boolean idleCheckCrc;
+
     /**
      * @return Gets filter of caches, which will by checked.
      */
@@ -290,4 +293,15 @@ public void setUserAttributes(Set<String> userAttrs) {
      * @param outputFormat New output format.
      */
     public void outputFormat(OutputFormat outputFormat) { this.outputFormat = outputFormat; }
+
+    /**
+     * @return Check page CRC sum on idle verify flag.
+     */
+    public boolean idleCheckCrc() { return idleCheckCrc; }
+
+    /**
+     * @param idleCheckCrc Check page CRC sum on idle verify flag.
+     */
+    public void idleCheckCrc(boolean idleCheckCrc) { this.idleCheckCrc = idleCheckCrc; }
+
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
index a8fae08d332a..907e53efcae3 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
@@ -661,6 +661,13 @@ private void reinit(FileIO fileIO) throws IOException {
         }
     }
 
+    /**
+     * @return File absolute path.
+     */
+    public String getFileAbsolutePath() {
+        return cfgFile.getAbsolutePath();
+    }
+
     /**
      * @param pageBuf Page buffer.
      * @param pageSize Page size.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/GridNotIdleException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/GridNotIdleException.java
new file mode 100644
index 000000000000..1124c4b8b24b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/GridNotIdleException.java
@@ -0,0 +1,68 @@
+/*
+ * 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;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * This exception defines not idle cluster state, when idle state expected.
+ */
+public class GridNotIdleException extends IgniteException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Create empty exception.
+     */
+    public GridNotIdleException() {
+        // No-op.
+    }
+
+    /**
+     * Creates new exception with given error message.
+     *
+     * @param msg Error message.
+     */
+    public GridNotIdleException(String msg) {
+        super(msg);
+    }
+
+    /**
+     * Creates new exception with given throwable as a cause and source of error message.
+     *
+     * @param cause Non-null throwable cause.
+     */
+    public GridNotIdleException(Throwable cause) {
+        super(cause);
+    }
+
+    /**
+     * Creates new exception with given error message and optional nested exception.
+     *
+     * @param msg Error message.
+     * @param cause Optional nested exception (can be {@code null}).
+     */
+    public GridNotIdleException(String msg, @Nullable Throwable cause) {
+        super(msg, cause);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return getClass() + ": " + getMessage();
+    }
+}
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 a153063d8998..f78a87abefc4 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
@@ -19,10 +19,12 @@
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import java.util.function.Consumer;
+import java.util.stream.Collectors;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -47,12 +49,30 @@
     /** Exceptions. */
     private Map<UUID, Exception> exceptions;
 
+    public IdleVerifyResultV2(Map<UUID, Exception> exceptions) {
+        this(new HashMap<>(), new HashMap<>(), new HashMap<>(), exceptions);
+    }
+
     /**
      * @param cntrConflicts Counter conflicts.
      * @param hashConflicts Hash conflicts.
      * @param movingPartitions Moving partitions.
      */
     public IdleVerifyResultV2(
+        Map<PartitionKeyV2, List<PartitionHashRecordV2>> cntrConflicts,
+        Map<PartitionKeyV2, List<PartitionHashRecordV2>> hashConflicts,
+        Map<PartitionKeyV2, List<PartitionHashRecordV2>> movingPartitions
+    ) {
+        this(cntrConflicts, hashConflicts, movingPartitions, new HashMap<>());
+    }
+
+    /**
+     * @param cntrConflicts Counter conflicts.
+     * @param hashConflicts Hash conflicts.
+     * @param movingPartitions Moving partitions.
+     * @param exceptions Occurred exceptions.
+     */
+    private IdleVerifyResultV2(
         Map<PartitionKeyV2, List<PartitionHashRecordV2>> cntrConflicts,
         Map<PartitionKeyV2, List<PartitionHashRecordV2>> hashConflicts,
         Map<PartitionKeyV2, List<PartitionHashRecordV2>> movingPartitions,
@@ -135,7 +155,22 @@ public boolean hasConflicts() {
      * @param printer Consumer for handle formatted result.
      */
     public void print(Consumer<String> printer) {
-        if (!hasConflicts())
+        if(!F.isEmpty(exceptions)){
+            Map<UUID, Exception> notIdleExceptions = exceptions.entrySet().stream()
+                .filter(e -> e.getValue() instanceof GridNotIdleException)
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+            if(!F.isEmpty(notIdleExceptions))
+                printer.accept("idle_verify check has finished, cluster not idle.\n");
+            else {
+                printer.accept("idle_verify check has finished, from " + exceptions.size() + " nodes were got errors.\n");
+
+                printer.accept("nodes with errors are following:\n");
+
+                for(Map.Entry<UUID, Exception> e : exceptions.entrySet())
+                    printer.accept("Node ID:" + e.getKey() +", error message: " + e.getValue().getMessage() + "\n");
+            }
+        } else if (!hasConflicts())
             printer.accept("idle_verify check has finished, no conflicts have been found.\n");
         else {
             int cntrConflictsSize = counterConflicts().size();
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 c0fd36ae5a6b..22b44146db20 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
@@ -20,6 +20,7 @@
 import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
+import java.io.PrintWriter;
 import java.time.LocalDateTime;
 import java.time.format.DateTimeFormatter;
 import java.util.ArrayList;
@@ -28,6 +29,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
+import java.util.UUID;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
@@ -77,8 +79,10 @@
     private IgniteLogger log;
 
     /** {@inheritDoc} */
-    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(
-        List<ClusterNode> subgrid, VisorIdleVerifyTaskArg arg) throws IgniteException {
+    @Override public @Nullable Map<? extends ComputeJob, ClusterNode> map(
+        List<ClusterNode> subgrid,
+        VisorIdleVerifyTaskArg arg
+    ) throws IgniteException {
         if (arg instanceof VisorIdleVerifyDumpTaskArg)
             taskArg = (VisorIdleVerifyDumpTaskArg)arg;
 
@@ -86,13 +90,17 @@
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public String reduce(List<ComputeJobResult> results)
-        throws IgniteException {
+    @Override public @Nullable String reduce(List<ComputeJobResult> results) throws IgniteException {
+        boolean hasExceptions = false;
+
         Map<PartitionKeyV2, List<PartitionHashRecordV2>> clusterHashes = new TreeMap<>(buildPartitionKeyComparator());
 
         for (ComputeJobResult res : results) {
-            if (res.getException() != null)
-                continue;
+            if (res.getException() != null) {
+                hasExceptions = true;
+
+                break;
+            }
 
             Map<PartitionKeyV2, PartitionHashRecordV2> nodeHashes = res.getData();
 
@@ -109,33 +117,27 @@
 
         int skippedRecords = 0;
 
-        for (Map.Entry<PartitionKeyV2, List<PartitionHashRecordV2>> entry : clusterHashes.entrySet()) {
-            if (needToAdd(entry.getValue())) {
-                entry.getValue().sort(recordComp);
+        if (!hasExceptions) {
+            for (Map.Entry<PartitionKeyV2, List<PartitionHashRecordV2>> entry : clusterHashes.entrySet()) {
+                if (needToAdd(entry.getValue())) {
+                    entry.getValue().sort(recordComp);
 
-                partitions.put(entry.getKey(), entry.getValue());
+                    partitions.put(entry.getKey(), entry.getValue());
+                }
+                else
+                    skippedRecords++;
             }
-            else
-                skippedRecords++;
         }
 
         return writeHashes(partitions, delegate.reduce(results), skippedRecords);
     }
 
     /** {@inheritDoc} */
-    @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws
-        IgniteException {
-        ComputeJobResultPolicy superRes = super.result(res, rcvd);
-
-        // Deny failover.
-        if (superRes == ComputeJobResultPolicy.FAILOVER) {
-            superRes = ComputeJobResultPolicy.WAIT;
-
-            log.warning("VerifyBackupPartitionsJobV2 failed on node " +
-                "[consistentId=" + res.getNode().consistentId() + "]", res.getException());
-        }
-
-        return superRes;
+    @Override public ComputeJobResultPolicy result(
+        ComputeJobResult res,
+        List<ComputeJobResult> rcvd
+    ) throws IgniteException {
+        return delegate.result(res, rcvd);
     }
 
     /**
@@ -177,62 +179,76 @@ private String writeHashes(
         IdleVerifyResultV2 conflictRes,
         int skippedRecords
     ) throws IgniteException {
-        File workDir = ignite.configuration().getWorkDirectory() == null
-            ? new File("/tmp")
-            : new File(ignite.configuration().getWorkDirectory());
+        String wd = ignite.configuration().getWorkDirectory();
+
+        File workDir = wd == null ? new File("/tmp") : new File(wd);
 
         File out = new File(workDir, IDLE_DUMP_FILE_PREFIX + LocalDateTime.now().format(TIME_FORMATTER) + ".txt");
 
         ignite.log().info("IdleVerifyDumpTask will write output to " + out.getAbsolutePath());
 
-        try (BufferedWriter writer = new BufferedWriter(new FileWriter(out))) {
-            try {
+        try (PrintWriter writer = new PrintWriter(new FileWriter(out))) {
+            if (!F.isEmpty(conflictRes.exceptions()))
+                writeExceptions(conflictRes.exceptions(), writer);
+            else
+                writeResult(partitions, conflictRes, skippedRecords, writer);
+
+            writer.flush();
+
+            ignite.log().info("IdleVerifyDumpTask successfully written dump to '" + out.getAbsolutePath() + "'");
+        }
+        catch (IOException | IgniteException e) {
+            ignite.log().error("Failed to write dump file: " + out.getAbsolutePath(), e);
 
-                writer.write("idle_verify check has finished, found " + partitions.size() + " partitions\n");
+            throw new IgniteException(e);
+        }
 
-                if (skippedRecords > 0)
-                    writer.write(skippedRecords + " partitions was skipped\n");
+        return out.getAbsolutePath();
+    }
 
-                if (!F.isEmpty(partitions)) {
-                    writer.write("Cluster partitions:\n");
+    /** */
+    private void writeResult(
+        Map<PartitionKeyV2, List<PartitionHashRecordV2>> partitions,
+        IdleVerifyResultV2 conflictRes,
+        int skippedRecords,
+        PrintWriter writer
+    ) {
+        writer.write("idle_verify check has finished, found " + partitions.size() + " partitions\n");
 
-                    for (Map.Entry<PartitionKeyV2, List<PartitionHashRecordV2>> entry : partitions.entrySet()) {
-                        writer.write("Partition: " + entry.getKey() + "\n");
+        if (skippedRecords > 0)
+            writer.write(skippedRecords + " partitions was skipped\n");
 
-                        writer.write("Partition instances: " + entry.getValue() + "\n");
-                    }
+        if (!F.isEmpty(partitions)) {
+            writer.write("Cluster partitions:\n");
 
-                    writer.write("\n\n-----------------------------------\n\n");
+            for (Map.Entry<PartitionKeyV2, List<PartitionHashRecordV2>> entry : partitions.entrySet()) {
+                writer.write("Partition: " + entry.getKey() + "\n");
 
-                    conflictRes.print(str -> {
-                        try {
-                            writer.write(str);
-                        }
-                        catch (IOException e) {
-                            throw new IgniteException("Failed to write partitions conflict.", e);
-                        }
-                    });
-                }
-            }
-            finally {
-                writer.flush();
+                writer.write("Partition instances: " + entry.getValue() + "\n");
             }
 
-            ignite.log().info("IdleVerifyDumpTask successfully written dump to '" + out.getAbsolutePath() + "'");
-        }
-        catch (IOException | IgniteException e) {
-            ignite.log().error("Failed to write dump file: " + out.getAbsolutePath(), e);
+            writer.write("\n\n-----------------------------------\n\n");
 
-            throw new IgniteException(e);
+            conflictRes.print(writer::write);
         }
+    }
 
-        return out.getAbsolutePath();
+    /** */
+    private void writeExceptions(Map<UUID, Exception> exceptions, PrintWriter writer) {
+        writer.write("idle_verify check has finished, " + exceptions.size() + " nodes return error\n");
+
+        for (Map.Entry<UUID, Exception> entry : exceptions.entrySet()) {
+            writer.write("Node ID: " + entry.getKey() + "\n");
+
+            entry.getValue().printStackTrace(writer);
+        }
     }
 
+
     /**
      * @return Comparator for {@link PartitionHashRecordV2}.
      */
-    @NotNull private Comparator<PartitionHashRecordV2> buildRecordComparator() {
+    private Comparator<PartitionHashRecordV2> buildRecordComparator() {
         return (o1, o2) -> {
             int compare = Boolean.compare(o1.isPrimary(), o2.isPrimary());
 
@@ -246,7 +262,7 @@ private String writeHashes(
     /**
      * @return Comparator for {@link PartitionKeyV2}.
      */
-    @NotNull private Comparator<PartitionKeyV2> buildPartitionKeyComparator() {
+    private Comparator<PartitionKeyV2> buildPartitionKeyComparator() {
         return (o1, o2) -> {
             int compare = Integer.compare(o1.groupId(), o2.groupId());
 
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 5e872eb7c612..de48add25912 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
@@ -16,6 +16,8 @@
 */
 package org.apache.ignite.internal.processors.cache.verify;
 
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -26,7 +28,6 @@
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
-import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ForkJoinPool;
 import java.util.concurrent.Future;
@@ -47,14 +48,21 @@
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
 import org.apache.ignite.internal.processors.cache.GridCacheUtils;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStore;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
 import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.util.lang.GridIterator;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.SB;
 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;
@@ -64,6 +72,8 @@
 import org.apache.ignite.resources.LoggerResource;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.internal.processors.cache.verify.PartitionHashRecordV2.MOVING_PARTITION_SIZE;
+
 /**
  * Task for comparing update counters and checksums between primary and backup partitions of specified caches.
  * <br>
@@ -87,8 +97,10 @@
     private static final long serialVersionUID = 0L;
 
     /** {@inheritDoc} */
-    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(
-        List<ClusterNode> subgrid, VisorIdleVerifyTaskArg arg) throws IgniteException {
+    @Override public @Nullable Map<? extends ComputeJob, ClusterNode> map(
+        List<ClusterNode> subgrid,
+        VisorIdleVerifyTaskArg arg
+    ) throws IgniteException {
         Map<ComputeJob, ClusterNode> jobs = new HashMap<>();
 
         for (ClusterNode node : subgrid)
@@ -98,31 +110,23 @@
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IdleVerifyResultV2 reduce(List<ComputeJobResult> results)
-        throws IgniteException {
+    @Override public @Nullable IdleVerifyResultV2 reduce(List<ComputeJobResult> results) throws IgniteException {
         Map<PartitionKeyV2, List<PartitionHashRecordV2>> clusterHashes = new HashMap<>();
-        Map<UUID, Exception> exceptions = new HashMap<>();
-
-        for (ComputeJobResult res : results) {
-            if (res.getException() != null) {
-                exceptions.put(res.getNode().id(), res.getException());
 
-                continue;
-            }
+        Map<UUID, Exception> exceptions = new HashMap<>();
 
-            Map<PartitionKeyV2, PartitionHashRecordV2> nodeHashes = res.getData();
+        reduce(results, clusterHashes, exceptions);
 
-            for (Map.Entry<PartitionKeyV2, PartitionHashRecordV2> e : nodeHashes.entrySet()) {
-                List<PartitionHashRecordV2> records = clusterHashes.computeIfAbsent(e.getKey(), k -> new ArrayList<>());
+        if(!F.isEmpty(exceptions))
+            return new IdleVerifyResultV2(exceptions);
 
-                records.add(e.getValue());
-            }
-        }
+        return checkConflicts(clusterHashes);
+    }
 
+    /** */
+    private IdleVerifyResultV2 checkConflicts(Map<PartitionKeyV2, List<PartitionHashRecordV2>> clusterHashes) {
         Map<PartitionKeyV2, List<PartitionHashRecordV2>> hashConflicts = new HashMap<>();
-
         Map<PartitionKeyV2, List<PartitionHashRecordV2>> updateCntrConflicts = new HashMap<>();
-
         Map<PartitionKeyV2, List<PartitionHashRecordV2>> movingParts = new HashMap<>();
 
         for (Map.Entry<PartitionKeyV2, List<PartitionHashRecordV2>> e : clusterHashes.entrySet()) {
@@ -130,7 +134,7 @@
             Long updateCntr = null;
 
             for (PartitionHashRecordV2 record : e.getValue()) {
-                if (record.size() == PartitionHashRecordV2.MOVING_PARTITION_SIZE) {
+                if (record.size() == MOVING_PARTITION_SIZE) {
                     List<PartitionHashRecordV2> records = movingParts.computeIfAbsent(
                         e.getKey(), k -> new ArrayList<>());
 
@@ -154,23 +158,52 @@
             }
         }
 
-        return new IdleVerifyResultV2(updateCntrConflicts, hashConflicts, movingParts, exceptions);
+        return new IdleVerifyResultV2(updateCntrConflicts, hashConflicts, movingParts);
     }
 
-    /** {@inheritDoc} */
-    @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws
-        IgniteException {
-        ComputeJobResultPolicy superRes = super.result(res, rcvd);
+    /** */
+    private void reduce(
+        List<ComputeJobResult> results,
+        Map<PartitionKeyV2, List<PartitionHashRecordV2>> clusterHashes,
+        Map<UUID, Exception> exceptions
+    ) {
+        for (ComputeJobResult res : results) {
+            if (res.getException() != null)
+                exceptions.put(res.getNode().id(), res.getException());
+
+            if(F.isEmpty(exceptions)) {
+                Map<PartitionKeyV2, PartitionHashRecordV2> nodeHashes = res.getData();
 
-        // Deny failover.
-        if (superRes == ComputeJobResultPolicy.FAILOVER) {
-            superRes = ComputeJobResultPolicy.WAIT;
+                for (Map.Entry<PartitionKeyV2, PartitionHashRecordV2> e : nodeHashes.entrySet()) {
+                    List<PartitionHashRecordV2> r = clusterHashes.computeIfAbsent(e.getKey(), k -> new ArrayList<>());
 
-            log.warning("VerifyBackupPartitionsJobV2 failed on node " +
-                "[consistentId=" + res.getNode().consistentId() + "]", res.getException());
+                    r.add(e.getValue());
+                }
+            }
         }
+    }
 
-        return superRes;
+    /** {@inheritDoc} */
+    @Override public ComputeJobResultPolicy result(
+        ComputeJobResult res,
+        List<ComputeJobResult> rcvd
+    ) throws IgniteException {
+        try {
+            ComputeJobResultPolicy superRes = super.result(res, rcvd);
+
+            // Deny failover.
+            if (superRes == ComputeJobResultPolicy.FAILOVER) {
+                superRes = ComputeJobResultPolicy.WAIT;
+
+                log.warning("VerifyBackupPartitionsJobV2 failed on node " +
+                    "[consistentId=" + res.getNode().consistentId() + "]", res.getException());
+            }
+
+            return superRes;
+        }
+        catch (IgniteException e) {
+            return ComputeJobResultPolicy.WAIT;
+        }
     }
 
     /**
@@ -203,44 +236,53 @@ public VerifyBackupPartitionsJobV2(VisorIdleVerifyTaskArg arg) {
 
         /** {@inheritDoc} */
         @Override public Map<PartitionKeyV2, PartitionHashRecordV2> execute() throws IgniteException {
-            Set<Integer> grpIds = new HashSet<>();
+            Set<Integer> grpIds = getGroupIds();
 
-            Set<String> missingCaches = new HashSet<>();
+            completionCntr.set(0);
 
-            if (arg.getCaches() != null) {
-                for (String cacheName : arg.getCaches()) {
-                    DynamicCacheDescriptor desc = ignite.context().cache().cacheDescriptor(cacheName);
+            List<Future<Map<PartitionKeyV2, PartitionHashRecordV2>>> partHashCalcFuts = calcPartitionHashAsync(grpIds);
 
-                    if (desc == null || !isCacheMatchFilter(cacheName)) {
-                        missingCaches.add(cacheName);
+            Map<PartitionKeyV2, PartitionHashRecordV2> res = new HashMap<>();
 
-                        continue;
-                    }
+            long lastProgressLogTs = U.currentTimeMillis();
 
-                    grpIds.add(desc.groupId());
+            for (int i = 0; i < partHashCalcFuts.size(); ) {
+                Future<Map<PartitionKeyV2, PartitionHashRecordV2>> fut = partHashCalcFuts.get(i);
+
+                try {
+                    Map<PartitionKeyV2, PartitionHashRecordV2> partHash = fut.get(100, TimeUnit.MILLISECONDS);
+
+                    res.putAll(partHash);
+
+                    i++;
                 }
+                catch (InterruptedException | ExecutionException e) {
+                    for (int j = i + 1; j < partHashCalcFuts.size(); j++)
+                        partHashCalcFuts.get(j).cancel(false);
 
-                handlingMissedCaches(missingCaches);
-            }
-            else if (onlySpecificCaches()) {
-                for (DynamicCacheDescriptor desc : ignite.context().cache().cacheDescriptors().values()) {
-                    if (desc.cacheConfiguration().getCacheMode() != CacheMode.LOCAL
-                        && isCacheMatchFilter(desc.cacheName()))
-                        grpIds.add(desc.groupId());
+                    if (e instanceof InterruptedException)
+                        throw new IgniteInterruptedException((InterruptedException)e);
+                    else if (e.getCause() instanceof IgniteException)
+                        throw (IgniteException)e.getCause();
+                    else
+                        throw new IgniteException(e.getCause());
                 }
-            }
-            else {
-                Collection<CacheGroupContext> groups = ignite.context().cache().cacheGroups();
+                catch (TimeoutException ignored) {
+                    if (U.currentTimeMillis() - lastProgressLogTs > 3 * 60 * 1000L) {
+                        lastProgressLogTs = U.currentTimeMillis();
 
-                for (CacheGroupContext grp : groups) {
-                    if (!grp.systemCache() && !grp.isLocal())
-                        grpIds.add(grp.groupId());
+                        log.warning("idle_verify is still running, processed " + completionCntr.get() + " of " +
+                            partHashCalcFuts.size() + " local partitions");
+                    }
                 }
             }
 
-            List<Future<Map<PartitionKeyV2, PartitionHashRecordV2>>> partHashCalcFutures = new ArrayList<>();
+            return res;
+        }
 
-            completionCntr.set(0);
+        /** */
+        private List<Future<Map<PartitionKeyV2, PartitionHashRecordV2>>> calcPartitionHashAsync(Set<Integer> grpIds) {
+            List<Future<Map<PartitionKeyV2, PartitionHashRecordV2>>> partHashCalcFutures = new ArrayList<>();
 
             for (Integer grpId : grpIds) {
                 CacheGroupContext grpCtx = ignite.context().cache().cacheGroup(grpId);
@@ -254,44 +296,50 @@ else if (onlySpecificCaches()) {
                     partHashCalcFutures.add(calculatePartitionHashAsync(grpCtx, part));
             }
 
-            Map<PartitionKeyV2, PartitionHashRecordV2> res = new HashMap<>();
+            return partHashCalcFutures;
+        }
 
-            long lastProgressLogTs = U.currentTimeMillis();
+        /** */
+        private Set<Integer> getGroupIds() {
+            Set<Integer> grpIds = new HashSet<>();
 
-            for (int i = 0; i < partHashCalcFutures.size(); ) {
-                Future<Map<PartitionKeyV2, PartitionHashRecordV2>> fut = partHashCalcFutures.get(i);
+            Set<String> missingCaches = new HashSet<>();
 
-                try {
-                    Map<PartitionKeyV2, PartitionHashRecordV2> partHash = fut.get(100, TimeUnit.MILLISECONDS);
+            if (arg.getCaches() != null) {
+                for (String cacheName : arg.getCaches()) {
+                    DynamicCacheDescriptor desc = ignite.context().cache().cacheDescriptor(cacheName);
 
-                    res.putAll(partHash);
+                    if (desc == null || !isCacheMatchFilter(cacheName)) {
+                        missingCaches.add(cacheName);
 
-                    i++;
+                        continue;
+                    }
+
+                    grpIds.add(desc.groupId());
                 }
-                catch (InterruptedException | ExecutionException e) {
-                    for (int j = i + 1; j < partHashCalcFutures.size(); j++)
-                        partHashCalcFutures.get(j).cancel(false);
 
-                    if (e instanceof InterruptedException)
-                        throw new IgniteInterruptedException((InterruptedException)e);
-                    else if (e.getCause() instanceof IgniteException)
-                        throw (IgniteException)e.getCause();
-                    else
-                        throw new IgniteException(e.getCause());
+                handlingMissedCaches(missingCaches);
+            }
+            else if (onlySpecificCaches()) {
+                for (DynamicCacheDescriptor desc : ignite.context().cache().cacheDescriptors().values()) {
+                    if (desc.cacheConfiguration().getCacheMode() != CacheMode.LOCAL
+                        && isCacheMatchFilter(desc.cacheName()))
+                        grpIds.add(desc.groupId());
                 }
-                catch (TimeoutException ignored) {
-                    if (U.currentTimeMillis() - lastProgressLogTs > 3 * 60 * 1000L) {
-                        lastProgressLogTs = U.currentTimeMillis();
+            }
+            else {
+                Collection<CacheGroupContext> groups = ignite.context().cache().cacheGroups();
 
-                        log.warning("idle_verify is still running, processed " + completionCntr.get() + " of " +
-                            partHashCalcFutures.size() + " local partitions");
-                    }
+                for (CacheGroupContext grp : groups) {
+                    if (!grp.systemCache() && !grp.isLocal())
+                        grpIds.add(grp.groupId());
                 }
             }
 
-            return res;
+            return grpIds;
         }
 
+
         /**
          *  Checks and throw exception if caches was missed.
          *
@@ -301,22 +349,20 @@ private void handlingMissedCaches(Set<String> missingCaches) {
             if (missingCaches.isEmpty())
                 return;
 
-            StringBuilder strBuilder = new StringBuilder("The following caches do not exist");
+            SB strBuilder = new SB("The following caches do not exist");
 
             if (onlySpecificCaches()) {
                 VisorIdleVerifyDumpTaskArg vdta = (VisorIdleVerifyDumpTaskArg)arg;
 
-                strBuilder.append(" or do not match to the given filter [")
-                    .append(vdta.getCacheFilterEnum())
-                    .append("]: ");
+                strBuilder.a(" or do not match to the given filter [").a(vdta.getCacheFilterEnum()).a("]: ");
             }
             else
-                strBuilder.append(": ");
+                strBuilder.a(": ");
 
             for (String name : missingCaches)
-                strBuilder.append(name).append(", ");
+                strBuilder.a(name).a(", ");
 
-            strBuilder.delete(strBuilder.length() - 2, strBuilder.length());
+            strBuilder.setLength(strBuilder.length() - 2);
 
             throw new IgniteException(strBuilder.toString());
         }
@@ -373,11 +419,7 @@ private boolean isCacheMatchFilter(String cacheName) {
             final CacheGroupContext grpCtx,
             final GridDhtLocalPartition part
         ) {
-            return ForkJoinPool.commonPool().submit(new Callable<Map<PartitionKeyV2, PartitionHashRecordV2>>() {
-                @Override public Map<PartitionKeyV2, PartitionHashRecordV2> call() throws Exception {
-                    return calculatePartitionHash(grpCtx, part);
-                }
-            });
+            return ForkJoinPool.commonPool().submit(() -> calculatePartitionHash(grpCtx, part));
         }
 
 
@@ -405,7 +447,7 @@ private boolean isCacheMatchFilter(String cacheName) {
             try {
                 if (part.state() == GridDhtPartitionState.MOVING) {
                     PartitionHashRecordV2 movingHashRecord = new PartitionHashRecordV2(partKey, isPrimary, consId,
-                        partHash, updateCntrBefore, PartitionHashRecordV2.MOVING_PARTITION_SIZE);
+                        partHash, updateCntrBefore, MOVING_PARTITION_SIZE);
 
                     return Collections.singletonMap(partKey, movingHashRecord);
                 }
@@ -414,6 +456,10 @@ else if (part.state() != GridDhtPartitionState.OWNING)
 
                 partSize = part.dataStore().fullSize();
 
+                if(arg.isCheckCrc())
+                    checkPartitionCrc(grpCtx, part);
+
+
                 GridIterator<CacheDataRow> it = grpCtx.offheap().partitionIterator(part.id());
 
                 while (it.hasNextX()) {
@@ -449,5 +495,64 @@ else if (part.state() != GridDhtPartitionState.OWNING)
 
             return Collections.singletonMap(partKey, partRec);
         }
+
+        /**
+         * @return {@code True} if checkpoint is now, {@code False} otherwise.
+         */
+        private boolean isCheckpointNow() {
+            GridCacheDatabaseSharedManager db =
+                (GridCacheDatabaseSharedManager)ignite.context().cache().context().database();
+
+            return db.writtenPagesCounter() != null;
+        }
+
+        /**
+         * Checks correct CRC sum for given partition and cache group.
+         *
+         * @param grpCtx Cache group context
+         * @param part partition.
+         */
+        private void checkPartitionCrc(CacheGroupContext grpCtx, GridDhtLocalPartition part) {
+            if (grpCtx.persistenceEnabled()) {
+                FilePageStore pageStore = null;
+
+                try {
+                    FilePageStoreManager pageStoreMgr =
+                        (FilePageStoreManager)ignite.context().cache().context().pageStore();
+
+                    if (pageStoreMgr == null)
+                        return;
+
+                    pageStore = (FilePageStore)pageStoreMgr.getStore(grpCtx.groupId(), part.id());
+
+                    long pageId = PageIdUtils.pageId(part.id(), PageIdAllocator.FLAG_DATA, 0);
+
+                    ByteBuffer buf = ByteBuffer.allocateDirect(grpCtx.dataRegion().pageMemory().pageSize());
+
+                    buf.order(ByteOrder.nativeOrder());
+
+                    for (int pageNo = 0; pageNo < pageStore.pages(); pageId++, pageNo++) {
+                        buf.clear();
+
+                        if(isCheckpointNow())
+                            throw new GridNotIdleException("Checkpoint is now! Cluster isn't idle.");
+
+                        pageStore.read(pageId, buf, true);
+                    }
+                }
+                catch(GridNotIdleException e){
+                    throw e;
+                }
+                catch (Exception | AssertionError e) {
+                    String msg = new SB("CRC check of partition: ").a(part.id()).a(", for cache group ")
+                        .a(grpCtx.cacheOrGroupName()).a(" failed.")
+                        .a(pageStore != null ? " file: " + pageStore.getFileAbsolutePath() : "").toString();
+
+                    log.error(msg, e);
+
+                    throw new IgniteException(msg, e);
+                }
+            }
+        }
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyDumpTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyDumpTaskArg.java
index 29dfb5b7a45b..dd504c2c4cc2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyDumpTaskArg.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyDumpTaskArg.java
@@ -45,11 +45,17 @@ public VisorIdleVerifyDumpTaskArg() {
 
     /**
      * @param caches Caches.
+     * @param checkCrc Check CRC.
      * @param skipZeros Skip zeros partitions.
      * @param cacheFilterEnum Cache kind.
      */
-    public VisorIdleVerifyDumpTaskArg(Set<String> caches, boolean skipZeros, CacheFilterEnum cacheFilterEnum) {
-        super(caches);
+    public VisorIdleVerifyDumpTaskArg(
+        Set<String> caches,
+        boolean checkCrc,
+        boolean skipZeros,
+        CacheFilterEnum cacheFilterEnum
+    ) {
+        super(caches, checkCrc);
         this.skipZeros = skipZeros;
         this.cacheFilterEnum = cacheFilterEnum;
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyJob.java
index a8dc697f0771..83b1c33ede66 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyJob.java
@@ -63,11 +63,7 @@
             if (!fut.isDone()) {
                 jobCtx.holdcc();
 
-                fut.listen(new IgniteInClosure<IgniteFuture<ResultT>>() {
-                    @Override public void apply(IgniteFuture<ResultT> f) {
-                        jobCtx.callcc();
-                    }
-                });
+                fut.listen((IgniteInClosure<IgniteFuture<ResultT>>)f -> jobCtx.callcc());
 
                 return null;
             }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskArg.java
index c82af5878ebb..da0873b69930 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskArg.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskArg.java
@@ -35,6 +35,9 @@
     /** Caches. */
     private Set<String> caches;
 
+    /** Check CRC */
+    private boolean checkCrc;
+
     /**
      * Default constructor.
      */
@@ -44,9 +47,11 @@ public VisorIdleVerifyTaskArg() {
 
     /**
      * @param caches Caches.
+     * @param checkCrc Check CRC.
      */
-    public VisorIdleVerifyTaskArg(Set<String> caches) {
+    public VisorIdleVerifyTaskArg(Set<String> caches, boolean checkCrc) {
         this.caches = caches;
+        this.checkCrc = checkCrc;
     }
 
 
@@ -57,14 +62,28 @@ public VisorIdleVerifyTaskArg(Set<String> caches) {
         return caches;
     }
 
+    /** */
+    public boolean isCheckCrc() {
+        return checkCrc;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte getProtocolVersion() {
+        return V2;
+    }
+
     /** {@inheritDoc} */
     @Override protected void writeExternalData(ObjectOutput out) throws IOException {
         U.writeCollection(out, caches);
+        out.writeBoolean(checkCrc);
     }
 
     /** {@inheritDoc} */
     @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
         caches = U.readSet(in);
+
+        if(protoVer >= V2)
+            checkCrc = in.readBoolean();
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskV2.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskV2.java
index b9250ef28c54..31f4adaf5243 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskV2.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskV2.java
@@ -40,55 +40,6 @@
 
     /** {@inheritDoc} */
     @Override protected VisorJob<VisorIdleVerifyTaskArg, IdleVerifyResultV2> job(VisorIdleVerifyTaskArg arg) {
-        return new VisorIdleVerifyJobV2(arg, debug);
-    }
-
-    /**
-     *
-     */
-    private static class VisorIdleVerifyJobV2 extends VisorJob<VisorIdleVerifyTaskArg, IdleVerifyResultV2> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** */
-        private ComputeTaskFuture<IdleVerifyResultV2> fut;
-
-        /** Auto-inject job context. */
-        @JobContextResource
-        protected transient ComputeJobContext jobCtx;
-
-        /**
-         * @param arg Argument.
-         * @param debug Debug.
-         */
-        private VisorIdleVerifyJobV2(VisorIdleVerifyTaskArg arg, boolean debug) {
-            super(arg, debug);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected IdleVerifyResultV2 run(VisorIdleVerifyTaskArg arg) throws IgniteException {
-            if (fut == null) {
-                fut = ignite.compute().executeAsync(VerifyBackupPartitionsTaskV2.class, arg);
-
-                if (!fut.isDone()) {
-                    jobCtx.holdcc();
-
-                    fut.listen(new IgniteInClosure<IgniteFuture<IdleVerifyResultV2>>() {
-                        @Override public void apply(IgniteFuture<IdleVerifyResultV2> f) {
-                            jobCtx.callcc();
-                        }
-                    });
-
-                    return null;
-                }
-            }
-
-            return fut.get();
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(VisorIdleVerifyJobV2.class, this);
-        }
+        return new VisorIdleVerifyJob<>(arg, debug, VerifyBackupPartitionsTaskV2.class);
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index 4344f15c5dcc..e7daa6e7a12d 100755
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -2004,7 +2004,7 @@ protected IdleVerifyResultV2 idleVerify(Ignite ig, String... caches) {
         if (node == null)
             throw new IgniteException("None server node for verification.");
 
-        VisorIdleVerifyTaskArg taskArg = new VisorIdleVerifyTaskArg(cacheNames);
+        VisorIdleVerifyTaskArg taskArg = new VisorIdleVerifyTaskArg(cacheNames, false);
 
         return ig.compute().execute(
             VisorIdleVerifyTaskV2.class.getName(),
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 57ae628d06a8..3d57bd1220a9 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
@@ -17,10 +17,14 @@
 
 package org.apache.ignite.util;
 
+import java.io.BufferedReader;
 import java.io.ByteArrayOutputStream;
 import java.io.File;
+import java.io.FileReader;
 import java.io.IOException;
 import java.io.PrintStream;
+import java.io.RandomAccessFile;
+import java.io.Serializable;
 import java.nio.file.DirectoryStream;
 import java.nio.file.Files;
 import java.nio.file.Path;
@@ -36,6 +40,7 @@
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.LongAdder;
@@ -1190,6 +1195,86 @@ public void testCacheIdleVerifyDumpWhenSeveralNodesFailing() throws Exception {
             checkExceptionMessageOnReport(unstableId);
     }
 
+    @Test
+    public void testCacheIdleVerifyWithCorruptedPartition() throws Exception {
+        testCacheIdleVerifyWithCorruptedPartition("--cache", "idle_verify", "--check-crc");
+
+        String out = testOut.toString();
+
+        assertTrue(out.contains("idle_verify check has finished, from 1 nodes were got errors."));
+        assertTrue(out.contains("CRC check of partition"));
+        assertTrue(out.contains("for cache group default failed"));
+
+        testOut.reset();
+
+        log.error("output: " + out);
+    }
+
+    @Test
+    public void testCacheIdleVerifyDumpWithCorruptedPartition() throws Exception {
+        testCacheIdleVerifyWithCorruptedPartition("--cache", "idle_verify", "--dump", "--check-crc");
+
+        String parts[] = testOut.toString().split("VisorIdleVerifyDumpTask successfully written output to '");
+
+        assertEquals(2, parts.length);
+
+        String dumpFile = parts[1].split("\\.")[0]+".txt";
+
+        try(BufferedReader br = new BufferedReader(new FileReader(dumpFile))){
+            assertEquals("idle_verify check has finished, 1 nodes return error", br.readLine());
+            assertTrue(br.readLine().contains("Node ID"));
+            assertTrue(br.readLine().contains("CRC check of partition"));
+        }
+
+        testOut.reset();
+    }
+
+    /** */
+    private void corruptPartition(File partitionsDir) throws IOException {
+        ThreadLocalRandom rand = ThreadLocalRandom.current();
+
+        for(File partFile : partitionsDir.listFiles((d, n) -> n.startsWith("part"))) {
+            try (RandomAccessFile raf = new RandomAccessFile(partFile, "rw")) {
+                byte[] buf = new byte[1024];
+
+                rand.nextBytes(buf);
+
+                raf.seek(4096 * 2 + 1);
+
+                raf.write(buf);
+            }
+        }
+    }
+
+    /** */
+    private void testCacheIdleVerifyWithCorruptedPartition(String... args) throws Exception {
+        Ignite ignite = startGrids(2);
+
+        ignite.cluster().active(true);
+
+        createCacheAndPreload(ignite, 1000);
+
+        Serializable consistId = ignite.configuration().getConsistentId();
+
+        File partitionsDir = U.resolveWorkDirectory(
+            ignite.configuration().getWorkDirectory(),
+            "db/" + consistId + "/cache-" + DEFAULT_CACHE_NAME,
+            false
+        );
+
+        stopGrid(0);
+
+        corruptPartition(partitionsDir);
+
+        startGrid(0);
+
+        awaitPartitionMapExchange();
+
+        injectTestSystemOut();
+
+        assertEquals(EXIT_CODE_OK, execute(args));
+    }
+
     /**
      * Creates default cache and preload some data entries.
      *


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services