You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by se...@apache.org on 2021/04/15 08:07:35 UTC

[ignite] branch master updated: IGNITE-14384 No automatic page diagnostic for CorruptedTreeException, only metadata storing for further investigation - Fixes #8926.

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

sergeychugunov 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 3a07d4a  IGNITE-14384 No automatic page diagnostic for CorruptedTreeException, only metadata storing for further investigation - Fixes #8926.
3a07d4a is described below

commit 3a07d4aa63c28112b6b006d652ab835b0c588c62
Author: Kirill Tkalenko <tk...@yandex.ru>
AuthorDate: Thu Apr 15 10:55:57 2021 +0300

    IGNITE-14384 No automatic page diagnostic for CorruptedTreeException, only metadata storing for further investigation - Fixes #8926.
    
    Signed-off-by: Sergey Chugunov <se...@gmail.com>
---
 .../cache/persistence/wal/SegmentRouter.java       |   8 +-
 .../processors/diagnostic/DiagnosticProcessor.java | 222 ++++++-------
 .../diagnostic/PageHistoryDiagnoster.java          | 350 ---------------------
 .../processors/failure/FailureProcessor.java       |   2 +-
 .../CorruptedTreeFailureHandlingTest.java          |  40 ++-
 .../diagnostic/DiagnosticProcessorTest.java        | 313 +++++++++---------
 .../utils/IgniteWalConverterArgumentsTest.java     |  34 ++
 7 files changed, 327 insertions(+), 642 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SegmentRouter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SegmentRouter.java
index ce14a75..63a6d6d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SegmentRouter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SegmentRouter.java
@@ -32,16 +32,16 @@ public class SegmentRouter {
     public static final String ZIP_SUFFIX = ".zip";
 
     /** */
-    private File walWorkDir;
+    private final File walWorkDir;
 
     /** WAL archive directory (including consistent ID as subfolder) */
-    private File walArchiveDir;
+    private final File walArchiveDir;
 
     /** Holder of actual information of latest manipulation on WAL segments. */
-    private SegmentAware segmentAware;
+    private final SegmentAware segmentAware;
 
     /** */
-    private DataStorageConfiguration dsCfg;
+    private final DataStorageConfiguration dsCfg;
 
     /**
      * @param walWorkDir WAL work directory.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/diagnostic/DiagnosticProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/diagnostic/DiagnosticProcessor.java
index f8f2563..ea31ec7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/diagnostic/DiagnosticProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/diagnostic/DiagnosticProcessor.java
@@ -18,27 +18,32 @@
 package org.apache.ignite.internal.processors.diagnostic;
 
 import java.io.File;
+import java.io.IOException;
 import java.nio.file.Path;
 import java.time.LocalDateTime;
 import java.time.format.DateTimeFormatter;
-import org.apache.ignite.Ignite;
+import java.util.Arrays;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.failure.FailureContext;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
 import org.apache.ignite.internal.processors.cache.persistence.tree.CorruptedTreeException;
+import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentRouter;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.X;
-import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_DUMP_PAGE_LOCK_ON_FAILURE;
-import static org.apache.ignite.internal.processors.diagnostic.DiagnosticProcessor.DiagnosticAction.PRINT_TO_FILE;
-import static org.apache.ignite.internal.processors.diagnostic.DiagnosticProcessor.DiagnosticAction.PRINT_TO_LOG;
-import static org.apache.ignite.internal.processors.diagnostic.DiagnosticProcessor.DiagnosticAction.PRINT_TO_RAW_FILE;
-import static org.apache.ignite.internal.util.IgniteStopwatch.logTime;
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.stream.Collectors.joining;
+import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_PAGE_SIZE;
+import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isPersistenceEnabled;
 
 /**
  * Processor which contained helper methods for different diagnostic cases.
@@ -53,164 +58,145 @@ public class DiagnosticProcessor extends GridProcessorAdapter {
             DFLT_DUMP_PAGE_LOCK_ON_FAILURE);
 
     /** Time formatter for dump file name. */
-    private static final DateTimeFormatter TIME_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH-mm-ss_SSS");
+    private static final DateTimeFormatter TIME_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd'_'HH-mm-ss_SSS");
 
     /** Folder name for store diagnostic info. **/
     public static final String DEFAULT_TARGET_FOLDER = "diagnostic";
 
-    /** File format. */
-    static final String FILE_FORMAT = ".txt";
-
-    /** Raw file format. */
-    static final String RAW_FILE_FORMAT = ".raw";
-
     /** Full path for store dubug info. */
     private final Path diagnosticPath;
 
-    /** */
-    private final PageHistoryDiagnoster pageHistoryDiagnoster;
+    /** File I/O factory. */
+    @Nullable private final FileIOFactory fileIOFactory;
 
     /**
+     * Constructor.
+     *
      * @param ctx Kernal context.
      */
     public DiagnosticProcessor(GridKernalContext ctx) throws IgniteCheckedException {
         super(ctx);
 
-        diagnosticPath = U.resolveWorkDirectory(ctx.config().getWorkDirectory(), DEFAULT_TARGET_FOLDER, false).toPath();
-
-        pageHistoryDiagnoster = new PageHistoryDiagnoster(ctx, this::diagnosticFile);
-
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onKernalStart(boolean active) throws IgniteCheckedException {
-        super.onKernalStart(active);
-
-        pageHistoryDiagnoster.onStart();
-    }
+        diagnosticPath = U.resolveWorkDirectory(ctx.config().getWorkDirectory(), DEFAULT_TARGET_FOLDER, false)
+            .toPath();
 
-    /**
-     * Dump all history caches of given page.
-     *
-     * @param builder Parameters of dumping.
-     * @throws IgniteCheckedException If scanning was failed.
-     */
-    public void dumpPageHistory(
-        @NotNull PageHistoryDiagnoster.DiagnosticPageBuilder builder
-    ) throws IgniteCheckedException {
-        logTime(log, "DiagnosticPageHistory", () -> pageHistoryDiagnoster.dumpPageHistory(builder));
+        fileIOFactory = isPersistenceEnabled(ctx.config()) ?
+            ctx.config().getDataStorageConfiguration().getFileIOFactory() : null;
     }
 
     /**
      * Print diagnostic info about failure occurred on {@code ignite} instance.
      * Failure details is contained in {@code failureCtx}.
      *
-     * @param ignite Ignite instance.
      * @param failureCtx Failure context.
      */
-    public void onFailure(Ignite ignite, FailureContext failureCtx) {
+    public void onFailure(FailureContext failureCtx) {
         // Dump data structures page locks.
         if (IGNITE_DUMP_PAGE_LOCK_ON_FAILURE)
             ctx.cache().context().diagnostic().pageLockTracker().dumpLocksToLog();
 
-        // If we have some corruption in data structure,
-        // we should scan WAL and print to log and save to file all pages related to corruption for
-        // future investigation.
-        if (X.hasCause(failureCtx.error(), CorruptedTreeException.class)) {
-            CorruptedTreeException corruptedTreeException = X.cause(failureCtx.error(), CorruptedTreeException.class);
-
-            T2<Integer, Long>[] pageIds = corruptedTreeException.pages();
-
-            try {
-                dumpPageHistory(
-                    new PageHistoryDiagnoster.DiagnosticPageBuilder()
-                        .pageIds(pageIds)
-                        .addAction(PRINT_TO_LOG)
-                        .addAction(PRINT_TO_FILE)
-                        .addAction(PRINT_TO_RAW_FILE)
-                );
-            }
-            catch (IgniteCheckedException e) {
-                SB sb = new SB();
-                sb.a("[");
-
-                for (int i = 0; i < pageIds.length; i++)
-                    sb.a("(").a(pageIds[i].get1()).a(",").a(pageIds[i].get2()).a(")");
+        CorruptedTreeException corruptedTreeE = X.cause(failureCtx.error(), CorruptedTreeException.class);
 
-                sb.a("]");
+        if (corruptedTreeE != null && !F.isEmpty(corruptedTreeE.pages()) && fileIOFactory != null) {
+            File[] walDirs = walDirs(ctx);
 
-                ignite.log().error(
-                    "Failed to dump diagnostic info on tree corruption. PageIds=" + sb, e);
+            if (F.isEmpty(walDirs)) {
+                if (log.isInfoEnabled())
+                    log.info("Skipping dump diagnostic info due to WAL not configured");
+            }
+            else {
+                try {
+                    File corruptedPagesFile = corruptedPagesFile(
+                        diagnosticPath,
+                        fileIOFactory,
+                        corruptedTreeE.pages()
+                    );
+
+                    String walDirsStr = Arrays.stream(walDirs).map(File::getAbsolutePath)
+                        .collect(joining(", ", "[", "]"));
+
+                    String args = "walDir=" + walDirs[0].getAbsolutePath() + (walDirs.length == 1 ? "" :
+                        " walArchiveDir=" + walDirs[1].getAbsolutePath());
+
+                    if (ctx.config().getDataStorageConfiguration().getPageSize() != DFLT_PAGE_SIZE)
+                        args += " pageSize=" + ctx.config().getDataStorageConfiguration().getPageSize();
+
+                    args += " pages=" + corruptedPagesFile.getAbsolutePath();
+
+                    log.warning(corruptedTreeE.getClass().getSimpleName() + " has occurred. " +
+                        "To diagnose it, make a backup of the following directories: " + walDirsStr + ". " +
+                        "Then, run the following command: java -cp <classpath> " +
+                        "org.apache.ignite.development.utils.IgniteWalConverter " + args);
+                }
+                catch (Throwable t) {
+                    String pages = Arrays.stream(corruptedTreeE.pages())
+                        .map(t2 -> "(" + t2.get1() + ',' + t2.get2() + ')').collect(joining("", "[", "]"));
+
+                    log.error("Failed to dump diagnostic info on tree corruption. PageIds=" + pages, t);
+                }
             }
         }
     }
 
     /**
-     * Resolve file to store diagnostic info.
+     * Creation and filling of a file with pages that can be corrupted.
+     * Pages are written on each line in format "grpId:pageId".
+     * File name format "corruptedPages_yyyy-MM-dd'_'HH-mm-ss_SSS.txt".
      *
-     * @param customFile Custom file if customized.
-     * @param writeMode Diagnostic file write mode.
-     * @return File to store diagnostic info.
+     * @param dirPath Path to the directory where the file will be created.
+     * @param ioFactory File I/O factory.
+     * @param pages Pages that could be corrupted. Mapping: cache group id -> page id.
+     * @return Created and filled file.
+     * @throws IOException If an I/O error occurs.
      */
-    private File diagnosticFile(File customFile, DiagnosticFileWriteMode writeMode) {
-        if (customFile == null)
-            return finalizeFile(diagnosticPath, writeMode);
+    public static File corruptedPagesFile(
+        Path dirPath,
+        FileIOFactory ioFactory,
+        T2<Integer, Long>... pages
+    ) throws IOException {
+        dirPath.toFile().mkdirs();
 
-        if (customFile.isAbsolute())
-            return finalizeFile(customFile.toPath(), writeMode);
+        File f = dirPath.resolve("corruptedPages_" + LocalDateTime.now().format(TIME_FORMATTER) + ".txt").toFile();
 
-        return finalizeFile(diagnosticPath.resolve(customFile.toPath()), writeMode);
-    }
+        assert !f.exists();
 
-    /**
-     * @param diagnosticPath Path to diagnostic file.
-     * @param writeMode Diagnostic file write mode.
-     * @return File to store diagnostic info.
-     */
-    private static File finalizeFile(Path diagnosticPath, DiagnosticFileWriteMode writeMode) {
-        diagnosticPath.toFile().mkdirs();
+        try (FileIO fileIO = ioFactory.create(f)) {
+            for (T2<Integer, Long> p : pages) {
+                byte[] bytes = (p.get1().toString() + ':' + p.get2().toString() + U.nl()).getBytes(UTF_8);
+
+                int left = bytes.length;
+
+                while ((left - fileIO.writeFully(bytes, bytes.length - left, left)) > 0)
+                    ;
+            }
+
+            fileIO.force();
+        }
 
-        return diagnosticPath.resolve(LocalDateTime.now().format(TIME_FORMATTER) + getFileExtension(writeMode)).toFile();
+        return f;
     }
 
     /**
-     * Get file format for given write mode.
+     * Getting the WAL directories.
+     * Note:
+     * Index 0: WAL working directory.
+     * Index 1: WAL archive directory (may be absent).
      *
-     * @param writeMode Diagnostic file write mode.
-     * @return File extention with dot.
+     * @param ctx Kernal context.
+     * @return WAL directories.
      */
-    private static String getFileExtension(DiagnosticFileWriteMode writeMode) {
-        switch (writeMode) {
-            case HUMAN_READABLE:
-                return FILE_FORMAT;
+    @Nullable static File[] walDirs(GridKernalContext ctx) {
+        IgniteWriteAheadLogManager walMgr = ctx.cache().context().wal();
 
-            case RAW:
-                return RAW_FILE_FORMAT;
+        if (walMgr instanceof FileWriteAheadLogManager) {
+            SegmentRouter sr = ((FileWriteAheadLogManager)walMgr).getSegmentRouter();
 
-            default:
-                throw new IllegalArgumentException("writeMode=" + writeMode);
+            if (sr != null) {
+                File workDir = sr.getWalWorkDir();
+                return sr.hasArchive() ? F.asArray(workDir, sr.getWalArchiveDir()) : F.asArray(workDir);
+            }
         }
-    }
-
-    /**
-     * Possible action after WAL scanning.
-     */
-    public enum DiagnosticAction {
-        /** Print result to log. */
-        PRINT_TO_LOG,
-        /** Print result to file. */
-        PRINT_TO_FILE,
-        /** Print result to file in raw format. */
-        PRINT_TO_RAW_FILE
-    }
 
-    /**
-     * Mode of diagnostic dump file.
-     */
-    public enum DiagnosticFileWriteMode {
-        /** Use humanly readable data representation. */
-        HUMAN_READABLE,
-        /** Use raw data format. */
-        RAW
+        return null;
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/diagnostic/PageHistoryDiagnoster.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/diagnostic/PageHistoryDiagnoster.java
deleted file mode 100644
index 6f84e07..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/diagnostic/PageHistoryDiagnoster.java
+++ /dev/null
@@ -1,350 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.diagnostic;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.EnumSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import java.util.function.BiFunction;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
-import org.apache.ignite.internal.processors.cache.persistence.wal.FileDescriptor;
-import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
-import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentRouter;
-import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer;
-import org.apache.ignite.internal.processors.cache.persistence.wal.reader.IgniteWalIteratorFactory;
-import org.apache.ignite.internal.processors.cache.persistence.wal.reader.IgniteWalIteratorFactory.IteratorParametersBuilder;
-import org.apache.ignite.internal.processors.cache.persistence.wal.scanner.ScannerHandler;
-import org.apache.ignite.internal.processors.cache.persistence.wal.scanner.WalScanner.ScanTerminateStep;
-import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializer;
-import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactoryImpl;
-import org.apache.ignite.internal.processors.diagnostic.DiagnosticProcessor.DiagnosticFileWriteMode;
-import org.apache.ignite.internal.util.tostring.GridToStringExclude;
-import org.apache.ignite.internal.util.typedef.T2;
-import org.jetbrains.annotations.NotNull;
-
-import static java.util.Objects.requireNonNull;
-import static org.apache.ignite.internal.processors.cache.persistence.wal.reader.IgniteWalIteratorFactory.IteratorParametersBuilder.withIteratorParameters;
-import static org.apache.ignite.internal.processors.cache.persistence.wal.scanner.ScannerHandlers.printRawToFile;
-import static org.apache.ignite.internal.processors.cache.persistence.wal.scanner.ScannerHandlers.printToFile;
-import static org.apache.ignite.internal.processors.cache.persistence.wal.scanner.ScannerHandlers.printToLog;
-import static org.apache.ignite.internal.processors.cache.persistence.wal.scanner.WalScanner.buildWalScanner;
-
-/**
- * Diagnostic WAL page history.
- */
-public class PageHistoryDiagnoster {
-    /** Kernal context. */
-    @GridToStringExclude
-    protected final GridKernalContext ctx;
-
-    /** Diagnostic logger. */
-    @GridToStringExclude
-    protected final IgniteLogger log;
-
-    /** Wal folders to scan. */
-    private File[] walFolders;
-
-    /** Function to provide target end file to store diagnostic info. */
-    private final BiFunction<File, DiagnosticFileWriteMode, File> targetFileSupplier;
-
-    private final IgniteWalIteratorFactory iteratorFactory = new IgniteWalIteratorFactory();
-
-    /** */
-    private volatile FileWriteAheadLogManager wal;
-
-    /**
-     * @param ctx Kernal context.
-     * @param supplier Function to provide target end file to store diagnostic info.
-     */
-    public PageHistoryDiagnoster(GridKernalContext ctx, BiFunction<File, DiagnosticFileWriteMode, File> supplier) {
-        log = ctx.log(getClass());
-        this.ctx = ctx;
-        targetFileSupplier = supplier;
-    }
-
-    /**
-     * Do action on start.
-     */
-    public void onStart() {
-        FileWriteAheadLogManager wal = (FileWriteAheadLogManager)ctx.cache().context().wal();
-
-        if (wal == null)
-            return;
-
-        this.wal = wal;
-
-        SegmentRouter segmentRouter = wal.getSegmentRouter();
-
-        if (segmentRouter.hasArchive())
-            walFolders = new File[] {segmentRouter.getWalArchiveDir(), segmentRouter.getWalWorkDir()};
-        else
-            walFolders = new File[] {segmentRouter.getWalWorkDir()};
-    }
-
-    /**
-     * Dump all history caches of given page.
-     *
-     * @param builder Parameters of dumping.
-     * @throws IgniteCheckedException If scanning was failed.
-     */
-    public void dumpPageHistory(
-        @NotNull PageHistoryDiagnoster.DiagnosticPageBuilder builder
-    ) throws IgniteCheckedException {
-        if (walFolders == null) {
-            if (log.isInfoEnabled())
-                log.info("Skipping dump page history due to WAL not configured");
-
-            return;
-        }
-
-        ScannerHandler action = null;
-
-        for (DiagnosticProcessor.DiagnosticAction act : builder.actions) {
-            if (action == null)
-                action = toHandler(act, builder.dumpFolder);
-            else
-                action = action.andThen(toHandler(act, builder.dumpFolder));
-        }
-
-        requireNonNull(action, "Should be configured at least one action");
-
-        IteratorParametersBuilder params = withIteratorParameters()
-            .log(log)
-            .filesOrDirs(walFolders);
-
-        // Resolve available WAL segment files.
-        List<FileDescriptor> descs = iteratorFactory.resolveWalFiles(params);
-
-        int descIdx = -1;
-        WALPointer reserved = null;
-
-        for (int i = 0; i < descs.size(); i++) {
-            // Try resever minimal available segment.
-            if (wal.reserve(reserved = new WALPointer(descs.get(i).idx(), 0, 0))) {
-                descIdx = i;
-
-                break;
-            }
-        }
-
-        if (descIdx == -1) {
-            if (log.isInfoEnabled())
-                log.info("Skipping dump page history due to can not reserve WAL segments: " + descToString(descs));
-
-            return;
-        }
-
-        if (log.isDebugEnabled())
-            log.debug("Reserverd WAL segment idx: " + reserved.index());
-
-        // Check gaps in the reserved interval.
-        List<T2<Long, Long>> gaps = iteratorFactory.hasGaps(descs.subList(descIdx, descs.size()));
-
-        if (!gaps.isEmpty())
-            log.warning("Potentialy missed record because WAL has gaps: " + gapsToString(gaps));
-
-        try {
-            scan(builder, params, action, reserved);
-        }
-        finally {
-            assert reserved != null;
-
-            wal.release(reserved);
-
-            if (log.isDebugEnabled())
-                log.debug("Release WAL segment idx:" + reserved.index());
-        }
-    }
-
-    /**
-     * @param builder Diagnostic parameter builder.
-     * @param params Iterator parameter builder.
-     * @param action Action.
-     * @param from Pointer from replay.
-     */
-    private void scan(
-        PageHistoryDiagnoster.DiagnosticPageBuilder builder,
-        IteratorParametersBuilder params,
-        ScannerHandler action,
-        WALPointer from
-    ) throws IgniteCheckedException {
-        // Try scan via WAL manager. More safety way on working node.
-        try {
-            buildWalScanner(wal.replay(from))
-                .findAllRecordsFor(builder.pageIds)
-                .forEach(action);
-
-            return;
-
-        }
-        catch (IgniteCheckedException e) {
-            log.warning("Failed to diagnosric scan via WAL manager", e);
-        }
-
-        // Try scan via stand alone iterator is not safety if wal still generated and moving to archive.
-        // Build scanner for pageIds from reserved pointer.
-        ScanTerminateStep scanner = buildWalScanner(params.from(from)).findAllRecordsFor(builder.pageIds);
-
-        scanner.forEach(action);
-    }
-
-    /**
-     * @param descs WAL file descriptors.
-     * @return String representation.
-     */
-    private String descToString(List<FileDescriptor> descs) {
-        StringBuilder sb = new StringBuilder();
-
-        sb.append("[");
-
-        Iterator<FileDescriptor> iter = descs.iterator();
-
-        while (iter.hasNext()) {
-            FileDescriptor desc = iter.next();
-
-            sb.append(desc.idx());
-
-            if (!iter.hasNext())
-                sb.append(", ");
-        }
-
-        sb.append("]");
-
-        return sb.toString();
-    }
-
-    /**
-     * @param gaps WAL file gaps.
-     * @return String representation.
-     */
-    private String gapsToString(Collection<T2<Long, Long>> gaps) {
-        StringBuilder sb = new StringBuilder();
-
-        sb.append("[");
-
-        Iterator<T2<Long, Long>> iter = gaps.iterator();
-
-        while (iter.hasNext()) {
-            T2<Long, Long> gap = iter.next();
-
-            sb.append("(").append(gap.get1()).append("..").append(gap.get2()).append(")");
-
-            if (!iter.hasNext())
-                sb.append(", ");
-        }
-
-        sb.append("]");
-
-        return sb.toString();
-    }
-
-    /**
-     * @param action Action for converting.
-     * @param customFile File to store diagnostic info.
-     * @return {@link ScannerHandler} for handle records.
-     */
-    private ScannerHandler toHandler(DiagnosticProcessor.DiagnosticAction action, File customFile) {
-        switch (action) {
-            case PRINT_TO_LOG:
-                return printToLog(log);
-
-            case PRINT_TO_FILE:
-                return printToFile(targetFileSupplier.apply(customFile, DiagnosticFileWriteMode.HUMAN_READABLE));
-
-            case PRINT_TO_RAW_FILE:
-                return printRawToFile(targetFileSupplier.apply(customFile, DiagnosticFileWriteMode.RAW), serializer());
-
-            default:
-                throw new IllegalArgumentException("Unknown diagnostic action : " + action);
-        }
-    }
-
-    /**
-     * @return WAL records serializer.
-     * @throws IgniteException If serializer initialization failed for some reason.
-     */
-    private RecordSerializer serializer() {
-        GridCacheSharedContext<?, ?> cctx = ctx.cache().context();
-
-        int serializerVer = cctx.wal().serializerVersion();
-
-        try {
-            return new RecordSerializerFactoryImpl(cctx).createSerializer(serializerVer);
-        }
-        catch (IgniteCheckedException e) {
-            log.error(
-                "Failed to create WAL records serializer for diagnostic purposes [serializerVer=" + serializerVer + "]"
-            );
-
-            throw new IgniteException(e);
-        }
-    }
-
-    /**
-     * Parameters for diagnostic pages.
-     */
-    public static class DiagnosticPageBuilder {
-        /** Pages for searching in WAL. */
-        List<T2<Integer, Long>> pageIds = new ArrayList<>();
-
-        /** Action after which should be executed after WAL scanning . */
-        Set<DiagnosticProcessor.DiagnosticAction> actions = EnumSet.noneOf(DiagnosticProcessor.DiagnosticAction.class);
-
-        /** Folder for dump diagnostic info. */
-        File dumpFolder;
-
-        /**
-         * @param pageIds Pages for searching in WAL.
-         * @return This instance for chaining.
-         */
-        public DiagnosticPageBuilder pageIds(T2<Integer, Long>... pageIds) {
-            this.pageIds.addAll(Arrays.asList(pageIds));
-
-            return this;
-        }
-
-        /**
-         * @param action Action after which should be executed after WAL scanning .
-         * @return This instance for chaining.
-         */
-        public DiagnosticPageBuilder addAction(@NotNull DiagnosticProcessor.DiagnosticAction action) {
-            this.actions.add(action);
-
-            return this;
-        }
-
-        /**
-         * @param file Folder for dump diagnostic info.
-         * @return This instance for chaining.
-         */
-        public DiagnosticPageBuilder folderForDump(@NotNull File file) {
-            this.dumpFolder = file;
-
-            return this;
-        }
-    }
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java
index 2eab2c5..10a61b3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java
@@ -194,7 +194,7 @@ public class FailureProcessor extends GridProcessorAdapter {
         DiagnosticProcessor diagnosticProcessor = ctx.diagnostic();
 
         if (diagnosticProcessor != null)
-            diagnosticProcessor.onFailure(ignite, failureCtx);
+            diagnosticProcessor.onFailure(failureCtx);
 
         boolean invalidated = hnd.onFailure(ignite, failureCtx);
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/CorruptedTreeFailureHandlingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/CorruptedTreeFailureHandlingTest.java
index 40c0ca9..ed09e53 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/CorruptedTreeFailureHandlingTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/CorruptedTreeFailureHandlingTest.java
@@ -25,10 +25,12 @@ import java.nio.file.OpenOption;
 import java.nio.file.StandardOpenOption;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Pattern;
 import org.apache.commons.io.FileUtils;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -51,9 +53,12 @@ import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
 import org.apache.ignite.internal.processors.cache.tree.DataLeafIO;
 import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataLeafIO;
 import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.LogListener;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.After;
 import org.junit.Before;
@@ -116,7 +121,12 @@ public class CorruptedTreeFailureHandlingTest extends GridCommonAbstractTest imp
         return new StopNodeFailureHandler();
     }
 
-    /** */
+    /**
+     * Check that if a corrupted page exists, an {@link CorruptedTreeException}
+     * will be thrown and a diagnostic file will be generated.
+     *
+     * @throws Exception If failed.
+     */
     @Test
     public void testCorruptedPage() throws Exception {
         IgniteEx srv = startGrid(0);
@@ -125,7 +135,7 @@ public class CorruptedTreeFailureHandlingTest extends GridCommonAbstractTest imp
 
         FileUtils.deleteDirectory(diagnosticDir);
 
-        srv.cluster().active(true);
+        srv.cluster().state(ClusterState.ACTIVE);
 
         IgniteCache<Integer, Integer> cache = srv.getOrCreateCache(DEFAULT_CACHE_NAME);
 
@@ -174,7 +184,12 @@ public class CorruptedTreeFailureHandlingTest extends GridCommonAbstractTest imp
             fileIO.writeFully(pageBuf);
         }
 
-        srv = startGrid(0);
+        LogListener logLsnr = LogListener.matches("CorruptedTreeException has occurred. " +
+            "To diagnose it, make a backup of the following directories: ").build();
+
+        srv = startGrid(0, cfg -> {
+            cfg.setGridLogger(new ListeningTestLogger(cfg.getGridLogger(), logLsnr));
+        });
 
         // Add modified page to WAL so it won't be restored to previous (valid) state.
         pageBuf.rewind();
@@ -203,13 +218,13 @@ public class CorruptedTreeFailureHandlingTest extends GridCommonAbstractTest imp
         assertTrue(diagnosticDir.exists());
         assertTrue(diagnosticDir.isDirectory());
 
-        File[] txtFiles = diagnosticDir.listFiles((dir, name) -> name.endsWith(".txt"));
-
-        assertTrue(txtFiles != null && txtFiles.length == 1);
+        Pattern corruptedPagesFileNamePtrn = corruptedPagesFileNamePattern();
+        File[] txtFiles = diagnosticDir.listFiles((dir, name) -> corruptedPagesFileNamePtrn.matcher(name).matches());
 
-        File[] rawFiles = diagnosticDir.listFiles((dir, name) -> name.endsWith(".raw"));
+        assertFalse(F.isEmpty(txtFiles));
+        assertEquals(1, txtFiles.length);
 
-        assertTrue(rawFiles != null && rawFiles.length == 1);
+        assertTrue(logLsnr.check());
     }
 
     /** */
@@ -253,4 +268,13 @@ public class CorruptedTreeFailureHandlingTest extends GridCommonAbstractTest imp
             };
         }
     }
+
+    /**
+     * Getting pattern corrupted pages file name.
+     *
+     * @return Pattern.
+     */
+    private Pattern corruptedPagesFileNamePattern() {
+        return Pattern.compile("corruptedPages_\\d{4}-\\d{2}-\\d{2}_\\d{2}-\\d{2}-\\d{2}_\\d{3}\\.txt");
+    }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/diagnostic/DiagnosticProcessorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/diagnostic/DiagnosticProcessorTest.java
index 40bbddd..62b23e2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/diagnostic/DiagnosticProcessorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/diagnostic/DiagnosticProcessorTest.java
@@ -17,256 +17,247 @@
 
 package org.apache.ignite.internal.processors.diagnostic;
 
+import java.io.BufferedReader;
 import java.io.File;
-import java.nio.file.Files;
+import java.io.FileReader;
 import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.nio.file.StandardOpenOption;
+import java.util.Arrays;
 import java.util.List;
+import java.util.function.Consumer;
+import java.util.regex.Pattern;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.FailureContext;
+import org.apache.ignite.failure.FailureType;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.pagemem.wal.WALIterator;
 import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot;
 import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
-import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIO;
-import org.apache.ignite.internal.processors.cache.persistence.wal.io.SegmentIO;
-import org.apache.ignite.internal.processors.cache.persistence.wal.io.SimpleSegmentFileInputFactory;
-import org.apache.ignite.internal.processors.cache.persistence.wal.reader.IgniteWalIteratorFactory;
-import org.apache.ignite.internal.processors.cache.persistence.wal.reader.IgniteWalIteratorFactory.IteratorParametersBuilder;
-import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactory;
-import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer;
-import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.SegmentHeader;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.tree.CorruptedTreeException;
+import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentRouter;
+import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.LogListener;
+import org.apache.ignite.testframework.junits.GridAbstractTest;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.Nullable;
 import org.junit.Test;
 
+import static java.util.stream.Collectors.toList;
+import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_WAL_PATH;
 import static org.apache.ignite.internal.processors.diagnostic.DiagnosticProcessor.DEFAULT_TARGET_FOLDER;
-import static org.apache.ignite.internal.processors.diagnostic.DiagnosticProcessor.DiagnosticAction.PRINT_TO_FILE;
-import static org.apache.ignite.internal.processors.diagnostic.DiagnosticProcessor.DiagnosticAction.PRINT_TO_LOG;
-import static org.apache.ignite.internal.processors.diagnostic.DiagnosticProcessor.DiagnosticAction.PRINT_TO_RAW_FILE;
-import static org.apache.ignite.internal.processors.diagnostic.DiagnosticProcessor.FILE_FORMAT;
-import static org.apache.ignite.internal.processors.diagnostic.DiagnosticProcessor.RAW_FILE_FORMAT;
+import static org.apache.ignite.internal.processors.diagnostic.DiagnosticProcessor.corruptedPagesFile;
+import static org.apache.ignite.internal.processors.diagnostic.DiagnosticProcessor.walDirs;
+import static org.apache.ignite.testframework.GridTestUtils.getFieldValue;
 
 /**
- *
+ * Class for testing diagnostics.
  */
 public class DiagnosticProcessorTest extends GridCommonAbstractTest {
-    /** Cache name. */
-    private static final String CACHE_NAME = "cache0";
-
-    /** Test directory for dump. */
-    private static final String TEST_DUMP_PAGE_FILE = "testDumpPage";
-
-    /** One time configured diagnosticProcessor. */
-    private static DiagnosticProcessor diagnosticProcessor;
-
-    /** One time configured page id for searching. */
-    private static T2<Integer, Long> expectedPageId;
-
     /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
 
-        DataStorageConfiguration dsCfg = new DataStorageConfiguration()
-            .setDefaultDataRegionConfiguration(
-                new DataRegionConfiguration()
-                    .setMaxSize(1024L * 1024 * 1024)
-                    .setPersistenceEnabled(true));
-
-        cfg.setDataStorageConfiguration(dsCfg);
-
-        return cfg;
+        stopAllGrids();
+        cleanPersistenceDir();
     }
 
     /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
 
         stopAllGrids();
-
         cleanPersistenceDir();
-
-        try {
-            IgniteEx ignite = startGrid("node0");
-
-            ignite.cluster().active(true);
-
-            ignite.createCache(CACHE_NAME);
-            try (IgniteDataStreamer<Integer, Integer> st = ignite.dataStreamer(CACHE_NAME)) {
-                st.allowOverwrite(true);
-
-                for (int i = 0; i < 10_000; i++)
-                    st.addData(i, i);
-            }
-
-            diagnosticProcessor = ignite.context().diagnostic();
-            expectedPageId = findAnyPageId();
-        }
-        finally {
-            stopAllGrids();
-        }
     }
 
     /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        super.afterTest();
-
-        U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), DEFAULT_TARGET_FOLDER, false));
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        return super.getConfiguration(gridName)
+            .setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME))
+            .setDataStorageConfiguration(
+                new DataStorageConfiguration()
+                    .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setPersistenceEnabled(true))
+            );
     }
 
     /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        super.afterTestsStopped();
+    @Override protected void cleanPersistenceDir() throws Exception {
+        super.cleanPersistenceDir();
 
-        diagnosticProcessor = null;
+        U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), DEFAULT_TARGET_FOLDER, false));
     }
 
     /**
+     * Checks the correctness of the {@link DiagnosticProcessor#corruptedPagesFile}.
+     *
      * @throws Exception If failed.
      */
     @Test
-    public void dumpPageHistoryToDefaultDir() throws Exception {
-        diagnosticProcessor.dumpPageHistory(new PageHistoryDiagnoster.DiagnosticPageBuilder()
-            .pageIds(expectedPageId)
-            .addAction(PRINT_TO_LOG)
-            .addAction(PRINT_TO_FILE)
-        );
+    public void testCorruptedPagesFile() throws Exception {
+        File tmpDir = new File(System.getProperty("java.io.tmpdir"), getName());
 
-        Path path = Paths.get(U.defaultWorkDirectory(), DEFAULT_TARGET_FOLDER);
-        File dumpFile = path.toFile().listFiles((dir, name) -> name.endsWith(FILE_FORMAT))[0];
+        try {
+            T2<Integer, Long>[] pages = new T2[] {new T2<>(10, 20L), new T2<>(30, 40L)};
+
+            File f = corruptedPagesFile(tmpDir.toPath(), new RandomAccessFileIOFactory(), pages);
 
-        List<String> records = Files.readAllLines(dumpFile.toPath());
+            assertTrue(f.exists());
+            assertTrue(f.isFile());
+            assertTrue(f.length() > 0);
+            assertTrue(Arrays.asList(tmpDir.listFiles()).contains(f));
+            assertTrue(corruptedPagesFileNamePattern().matcher(f.getName()).matches());
 
-        assertTrue(!records.isEmpty());
+            try (BufferedReader br = new BufferedReader(new FileReader(f))) {
+                List<String> lines = br.lines().collect(toList());
+                List<String> pageStrs = Arrays.stream(pages).map(t2 -> t2.get1() + ":" + t2.get2()).collect(toList());
 
-        assertTrue(records.stream().anyMatch(line -> line.contains("CheckpointRecord")));
+                assertEqualsCollections(lines, pageStrs);
+            }
+        }
+        finally {
+            if (tmpDir.exists())
+                assertTrue(U.delete(tmpDir));
+        }
     }
 
     /**
+     * Checks the correctness of the {@link DiagnosticProcessor#walDirs}.
+     *
      * @throws Exception If failed.
      */
     @Test
-    public void dumpRawPageHistoryToDefaultDir() throws Exception {
-        diagnosticProcessor.dumpPageHistory(new PageHistoryDiagnoster.DiagnosticPageBuilder()
-            .pageIds(expectedPageId)
-            .addAction(PRINT_TO_RAW_FILE)
-        );
-
-        Path path = Paths.get(U.defaultWorkDirectory(), DEFAULT_TARGET_FOLDER);
-        File dumpFile = path.toFile().listFiles((dir, name) -> name.endsWith(RAW_FILE_FORMAT))[0];
+    public void testWalDirs() throws Exception {
+        IgniteEx n = startGrid(0);
 
-        try (SegmentIO io = new SegmentIO(0L, new RandomAccessFileIO(dumpFile, StandardOpenOption.READ))) {
-            SegmentHeader hdr = RecordV1Serializer.readSegmentHeader(io, new SimpleSegmentFileInputFactory());
+        // Work + archive dirs.
+        File[] expWalDirs = expWalDirs(n);
+        assertEquals(2, expWalDirs.length);
+        assertEqualsCollections(F.asList(expWalDirs), F.asList(walDirs(n.context())));
 
-            assertFalse(hdr.isCompacted());
-
-            assertEquals(RecordSerializerFactory.LATEST_SERIALIZER_VERSION, hdr.getSerializerVersion());
-        }
-    }
+        stopAllGrids();
+        cleanPersistenceDir();
 
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void dumpPageHistoryToCustomAbsoluteDir() throws Exception {
-        Path path = Paths.get(U.defaultWorkDirectory(), TEST_DUMP_PAGE_FILE);
-        try {
-            diagnosticProcessor.dumpPageHistory(new PageHistoryDiagnoster.DiagnosticPageBuilder()
-                .pageIds(expectedPageId)
-                .folderForDump(path.toFile())
-                .addAction(PRINT_TO_FILE)
-            );
+        n = startGrid(0,
+            (Consumer<IgniteConfiguration>)cfg -> cfg.getDataStorageConfiguration().setWalArchivePath(DFLT_WAL_PATH));
 
-            File dumpFile = path.toFile().listFiles((dir, name) -> name.endsWith(FILE_FORMAT))[0];
+        // Only work dir.
+        expWalDirs = expWalDirs(n);
+        assertEquals(1, expWalDirs.length);
+        assertEqualsCollections(F.asList(expWalDirs), F.asList(walDirs(n.context())));
 
-            List<String> records = Files.readAllLines(dumpFile.toPath());
+        stopAllGrids();
+        cleanPersistenceDir();
 
-            assertTrue(!records.isEmpty());
+        n = startGrid(0,
+            (Consumer<IgniteConfiguration>)cfg -> cfg.setDataStorageConfiguration(new DataStorageConfiguration()));
 
-            assertTrue(records.stream().anyMatch(line -> line.contains("CheckpointRecord")));
-        }
-        finally {
-            U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), TEST_DUMP_PAGE_FILE, false));
-        }
+        // No wal dirs.
+        assertNull(expWalDirs(n));
+        assertNull(walDirs(n.context()));
     }
 
     /**
+     * Check that when an CorruptedTreeException is thrown, a "corruptedPages_TIMESTAMP.txt"
+     * will be created and a warning will be in the log.
+     *
      * @throws Exception If failed.
      */
     @Test
-    public void dumpPageHistoryToCustomRelativeDir() throws Exception {
-        Path path = Paths.get(U.defaultWorkDirectory(), DEFAULT_TARGET_FOLDER, TEST_DUMP_PAGE_FILE);
+    public void testOutputDiagnosticCorruptedPagesInfo() throws Exception {
+        ListeningTestLogger listeningTestLog = new ListeningTestLogger(GridAbstractTest.log);
 
-        diagnosticProcessor.dumpPageHistory(new PageHistoryDiagnoster.DiagnosticPageBuilder()
-            .pageIds(expectedPageId)
-            .folderForDump(new File(TEST_DUMP_PAGE_FILE))
-            .addAction(PRINT_TO_FILE)
-        );
+        IgniteEx n = startGrid(0, cfg -> {
+            cfg.setGridLogger(listeningTestLog);
+        });
 
-        File dumpFile = path.toFile().listFiles((dir, name) -> name.endsWith(FILE_FORMAT))[0];
+        n.cluster().state(ClusterState.ACTIVE);
+        awaitPartitionMapExchange();
 
-        List<String> records = Files.readAllLines(dumpFile.toPath());
+        for (int i = 0; i < 10_000; i++)
+            n.cache(DEFAULT_CACHE_NAME).put(i, "val_" + i);
 
-        assertTrue(!records.isEmpty());
+        assertNotNull(n.context().diagnostic());
 
-        assertTrue(records.stream().anyMatch(line -> line.contains("CheckpointRecord")));
-    }
+        T2<Integer, Long> anyPageId = findAnyPageId(n);
+        assertNotNull(anyPageId);
 
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void dumpOnlyCheckpointRecordBecausePageIdNotSet() throws Exception {
-        diagnosticProcessor.dumpPageHistory(new PageHistoryDiagnoster.DiagnosticPageBuilder()
-            .addAction(PRINT_TO_LOG)
-            .addAction(PRINT_TO_FILE)
-        );
+        LogListener logLsnr = LogListener.matches("CorruptedTreeException has occurred. " +
+            "To diagnose it, make a backup of the following directories: ").build();
 
-        Path path = Paths.get(U.defaultWorkDirectory(), DEFAULT_TARGET_FOLDER);
+        listeningTestLog.registerListener(logLsnr);
 
-        File dumpFile = path.toFile().listFiles((dir, name) -> name.endsWith(FILE_FORMAT))[0];
+        n.context().failure().process(new FailureContext(FailureType.CRITICAL_ERROR,
+            new CorruptedTreeException("Test ex", null, anyPageId.get1(), DEFAULT_CACHE_NAME, anyPageId.get2())));
 
-        List<String> records = Files.readAllLines(dumpFile.toPath());
+        assertTrue(logLsnr.check());
 
-        assertTrue(records.stream().allMatch(line -> line.contains("CheckpointRecord")));
-    }
+        Path diagnosticPath = getFieldValue(n.context().diagnostic(), "diagnosticPath");
 
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    @Test(expected = NullPointerException.class)
-    public void throwExceptionBecauseNotAnyActionsWasSet() throws IgniteCheckedException {
-        diagnosticProcessor.dumpPageHistory(new PageHistoryDiagnoster.DiagnosticPageBuilder()
-            .pageIds(expectedPageId)
-        );
+        List<File> corruptedPagesFiles = Arrays.stream(diagnosticPath.toFile().listFiles())
+            .filter(f -> corruptedPagesFileNamePattern().matcher(f.getName()).matches()).collect(toList());
+
+        assertEquals(1, corruptedPagesFiles.size());
+        assertTrue(corruptedPagesFiles.get(0).length() > 0);
     }
 
     /**
      * Find first any page id for test.
      *
+     * @param n Node.
      * @return Page id in WAL.
-     * @throws org.apache.ignite.IgniteCheckedException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    private T2<Integer, Long> findAnyPageId() throws org.apache.ignite.IgniteCheckedException {
-        IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory();
+    @Nullable private T2<Integer, Long> findAnyPageId(IgniteEx n) throws IgniteCheckedException {
+        try (WALIterator walIter = n.context().cache().context().wal().replay(new WALPointer(0, 0, 0))) {
+            while (walIter.hasNextX()) {
+                WALRecord walRecord = walIter.nextX().get2();
 
-        try (WALIterator it = factory.iterator(new IteratorParametersBuilder().filesOrDirs(U.defaultWorkDirectory()))) {
-            while (it.hasNext()) {
-                WALRecord record = it.next().get2();
-
-                if (record instanceof PageSnapshot) {
-                    PageSnapshot rec = (PageSnapshot)record;
+                if (walRecord instanceof PageSnapshot) {
+                    PageSnapshot rec = (PageSnapshot)walRecord;
 
                     return new T2<>(rec.groupId(), rec.fullPageId().pageId());
                 }
             }
         }
 
-        throw new IgniteCheckedException();
+        return null;
+    }
+
+    /**
+     * Getting expected WAL directories.
+     *
+     * @param n Node.
+     * @return WAL directories.
+     */
+    @Nullable private File[] expWalDirs(IgniteEx n) {
+        FileWriteAheadLogManager walMgr = walMgr(n);
+
+        if (walMgr != null) {
+            SegmentRouter sr = walMgr.getSegmentRouter();
+            assertNotNull(sr);
+
+            File workDir = sr.getWalWorkDir();
+            return sr.hasArchive() ? F.asArray(workDir, sr.getWalArchiveDir()) : F.asArray(workDir);
+        }
+
+        return null;
+    }
+
+    /**
+     * Getting pattern corrupted pages file name.
+     *
+     * @return Pattern.
+     */
+    private Pattern corruptedPagesFileNamePattern() {
+        return Pattern.compile("corruptedPages_\\d{4}-\\d{2}-\\d{2}_\\d{2}-\\d{2}-\\d{2}_\\d{3}\\.txt");
     }
 }
diff --git a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterArgumentsTest.java b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterArgumentsTest.java
index b5c6976..52c42c7 100644
--- a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterArgumentsTest.java
+++ b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterArgumentsTest.java
@@ -25,6 +25,8 @@ import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.diagnostic.DiagnosticProcessor;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -533,4 +535,36 @@ public class IgniteWalConverterArgumentsTest extends GridCommonAbstractTest {
             assertTrue(U.delete(walDir));
         }
     }
+
+    /**
+     * Checks that the file generated by the diagnostics is correct for the "pages" argument.
+     *
+     * @throws IOException If failed.
+     */
+    @Test
+    public void testCorruptedPagesFile() throws IOException {
+        File tmpDir = new File(System.getProperty("java.io.tmpdir"), getName());
+
+        try {
+            T2<Integer, Long>[] pages = new T2[] {new T2<>(10, 20L), new T2(30, 40L)};
+
+            File f = DiagnosticProcessor.corruptedPagesFile(tmpDir.toPath(), new RandomAccessFileIOFactory(), pages);
+            assertTrue(f.exists());
+            assertTrue(f.isFile());
+            assertTrue(f.length() > 0);
+
+            ByteArrayOutputStream baos = new ByteArrayOutputStream();
+            PrintStream ps = new PrintStream(baos);
+
+            IgniteWalConverterArguments args =
+                parse(ps, "walDir=" + tmpDir.getAbsolutePath(), "pages=" + f.getAbsolutePath());
+
+            assertNotNull(args.getPages());
+            assertEqualsCollections(F.asList(pages), args.getPages());
+        }
+        finally {
+            if (tmpDir.exists())
+                assertTrue(U.delete(tmpDir));
+        }
+    }
 }