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

[ignite] branch ignite-cdc updated: IGNITE-14435 Refactor PdsConsistentIdProcessor for reusage (#8946)

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

nizhikov pushed a commit to branch ignite-cdc
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/ignite-cdc by this push:
     new ab7bf2b  IGNITE-14435 Refactor PdsConsistentIdProcessor for reusage (#8946)
ab7bf2b is described below

commit ab7bf2b7848b2e431aaa092c640ea337cc0fdbc6
Author: Nikolay <ni...@apache.org>
AuthorDate: Wed Mar 31 12:25:12 2021 +0300

    IGNITE-14435 Refactor PdsConsistentIdProcessor for reusage (#8946)
---
 .../filename/PdsConsistentIdProcessor.java         | 453 +--------------------
 ...tentIdProcessor.java => PdsFolderResolver.java} | 205 ++++------
 .../persistence/filename/PdsFolderSettings.java    |  10 +-
 .../persistence/filename/PdsFoldersResolver.java   |   3 +-
 .../snapshot/IgniteSnapshotManager.java            |   2 +-
 .../IgniteUidAsConsistentIdMigrationTest.java      |  14 +-
 .../db/wal/reader/IgniteWalReaderTest.java         |   2 +-
 .../persistence/db/wal/IgniteWalRecoveryTest.java  |   4 +-
 8 files changed, 118 insertions(+), 575 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java
index 35f06dd..56fe06b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java
@@ -18,89 +18,32 @@
 package org.apache.ignite.internal.processors.cache.persistence.filename;
 
 import java.io.File;
-import java.io.FileFilter;
-import java.io.IOException;
 import java.io.Serializable;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.UUID;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.NodeFileLockHolder;
-import org.apache.ignite.internal.util.typedef.internal.A;
-import org.apache.ignite.internal.util.typedef.internal.CU;
-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_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID;
-import static org.apache.ignite.IgniteSystemProperties.getBoolean;
-
 /**
  * Component for resolving PDS storage file names, also used for generating consistent ID for case PDS mode is enabled
  */
 public class PdsConsistentIdProcessor extends GridProcessorAdapter implements PdsFoldersResolver {
-    /** Database subfolders constant prefix. */
-    private static final String DB_FOLDER_PREFIX = "node";
-
-    /** Node index and uid separator in subfolders name. */
-    private static final String NODEIDX_UID_SEPARATOR = "-";
-
-    /** Constant node subfolder prefix and node index pattern (nodeII, where II - node index as decimal integer) */
-    private static final String NODE_PATTERN = DB_FOLDER_PREFIX + "[0-9]*" + NODEIDX_UID_SEPARATOR;
-
-    /** Uuid as string pattern. */
-    private static final String UUID_STR_PATTERN = "[a-fA-F0-9]{8}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{12}";
-
-    /**
-     * Subdir (nodeII-UID, where II - node index as decimal integer, UID - string representation of consistent ID)
-     * pattern.
-     */
-    private static final String SUBDIR_PATTERN = NODE_PATTERN + UUID_STR_PATTERN;
-
-    /** Database subfolders for new style filter. */
-    public static final FileFilter DB_SUBFOLDERS_NEW_STYLE_FILTER = new FileFilter() {
-        @Override public boolean accept(File pathname) {
-            return pathname.isDirectory() && pathname.getName().matches(SUBDIR_PATTERN);
-        }
-    };
-
-    /** Database subfolders for old style filter. */
-    private static final FileFilter DB_SUBFOLDERS_OLD_STYLE_FILTER = new FileFilter() {
-        @Override public boolean accept(File pathname) {
-            String path = pathname.toString();
-            return pathname.isDirectory()
-                && !"wal".equals(pathname.getName())
-                && !path.contains(DataStorageConfiguration.DFLT_BINARY_METADATA_PATH)
-                && !path.contains(DataStorageConfiguration.DFLT_MARSHALLER_PATH)
-                && !pathname.getName().matches(SUBDIR_PATTERN);
-        }
-    };
-
-    /** Database default folder. */
-    public static final String DB_DEFAULT_FOLDER = "db";
-
     /** Config. */
-    private IgniteConfiguration cfg;
+    private final IgniteConfiguration cfg;
 
     /** Logger. */
-    private IgniteLogger log;
+    private final IgniteLogger log;
 
     /** Context. */
-    private GridKernalContext ctx;
+    private final GridKernalContext ctx;
 
     /** Cached folder settings. */
-    private PdsFolderSettings settings;
+    private PdsFolderSettings<NodeFileLockHolder> settings;
 
     /**
      * Creates folders resolver
@@ -122,22 +65,29 @@ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements Pd
      * @param consistentId compatibility consistent ID
      * @return PDS folder settings compatible with previous versions.
      */
-    private PdsFolderSettings compatibleResolve(
+    private PdsFolderSettings<NodeFileLockHolder> compatibleResolve(
         @Nullable final File pstStoreBasePath,
         @NotNull final Serializable consistentId) {
 
         if (cfg.getConsistentId() != null) {
             // compatible mode from configuration is used fot this case, no locking, no consitent id change
-            return new PdsFolderSettings(pstStoreBasePath, cfg.getConsistentId());
+            return new PdsFolderSettings<>(pstStoreBasePath, cfg.getConsistentId());
         }
 
-        return new PdsFolderSettings(pstStoreBasePath, consistentId);
+        return new PdsFolderSettings<>(pstStoreBasePath, consistentId);
     }
 
     /** {@inheritDoc} */
-    @Override public PdsFolderSettings resolveFolders() throws IgniteCheckedException {
+    @Override public PdsFolderSettings<NodeFileLockHolder> resolveFolders() throws IgniteCheckedException {
         if (settings == null) {
-            settings = prepareNewSettings();
+            //here deprecated method is used to get compatible version of consistentId
+            PdsFolderResolver<NodeFileLockHolder> resolver =
+                new PdsFolderResolver<>(cfg, log, ctx.discovery().consistentId(), this::tryLock);
+
+            settings = resolver.resolve();
+
+            if (settings == null)
+                settings = resolver.generateNew();
 
             if (!settings.isCompatible()) {
                 if (log.isInfoEnabled())
@@ -151,262 +101,6 @@ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements Pd
     }
 
     /**
-     * Creates new settings when we don't have cached one.
-     *
-     * @return new settings with prelocked directory (if appropriate).
-     * @throws IgniteCheckedException if IO failed.
-     */
-    private PdsFolderSettings prepareNewSettings() throws IgniteCheckedException {
-        final File pstStoreBasePath = resolvePersistentStoreBasePath();
-        //here deprecated method is used to get compatible version of consistentId
-        final Serializable consistentId = ctx.discovery().consistentId();
-
-        if (!CU.isPersistenceEnabled(cfg))
-            return compatibleResolve(pstStoreBasePath, consistentId);
-
-        if (ctx.clientNode())
-            return new PdsFolderSettings(pstStoreBasePath, UUID.randomUUID());
-
-        if (getBoolean(IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID, false))
-            return compatibleResolve(pstStoreBasePath, consistentId);
-
-        // compatible mode from configuration is used fot this case
-        if (cfg.getConsistentId() != null) {
-            // compatible mode from configuration is used fot this case, no locking, no consistent id change
-            return new PdsFolderSettings(pstStoreBasePath, cfg.getConsistentId());
-        }
-        // The node scans the work directory and checks if there is a folder matching the consistent ID.
-        // If such a folder exists, we start up with this ID (compatibility mode)
-        final String subFolder = U.maskForFileName(consistentId.toString());
-
-        final NodeFileLockHolder oldStyleFolderLockHolder = tryLock(new File(pstStoreBasePath, subFolder));
-
-        if (oldStyleFolderLockHolder != null)
-            return new PdsFolderSettings(pstStoreBasePath,
-                subFolder,
-                consistentId,
-                oldStyleFolderLockHolder,
-                true);
-
-        final File[] oldStyleFolders = pstStoreBasePath.listFiles(DB_SUBFOLDERS_OLD_STYLE_FILTER);
-
-        if (oldStyleFolders != null && oldStyleFolders.length != 0) {
-            for (File folder : oldStyleFolders) {
-                final String path = getPathDisplayableInfo(folder);
-
-                U.warn(log, "There is other non-empty storage folder under storage base directory [" + path + "]");
-            }
-        }
-
-        for (FolderCandidate next : getNodeIndexSortedCandidates(pstStoreBasePath)) {
-            final NodeFileLockHolder fileLockHolder = tryLock(next.subFolderFile());
-
-            if (fileLockHolder != null) {
-                if (log.isInfoEnabled())
-                    log.info("Successfully locked persistence storage folder [" + next.subFolderFile() + "]");
-
-                return new PdsFolderSettings(pstStoreBasePath,
-                    next.subFolderFile().getName(),
-                    next.uuid(),
-                    fileLockHolder,
-                    false);
-            }
-        }
-
-        // was not able to find free slot, allocating new
-        try (final NodeFileLockHolder rootDirLock = lockRootDirectory(pstStoreBasePath)) {
-            final List<FolderCandidate> sortedCandidates = getNodeIndexSortedCandidates(pstStoreBasePath);
-            final int nodeIdx = sortedCandidates.isEmpty() ? 0 : (sortedCandidates.get(sortedCandidates.size() - 1).nodeIndex() + 1);
-
-            return generateAndLockNewDbStorage(pstStoreBasePath, nodeIdx);
-        }
-    }
-
-    /**
-     * Calculate overall folder size.
-     *
-     * @param dir directory to scan.
-     * @return total size in bytes.
-     */
-    private static FolderParams folderSize(File dir) {
-        final FolderParams params = new FolderParams();
-
-        visitFolder(dir, params);
-
-        return params;
-    }
-
-    /**
-     * Scans provided directory and its sub dirs, collects found metrics.
-     *
-     * @param dir directory to start scan from.
-     * @param params input/output.
-     */
-    private static void visitFolder(final File dir, final FolderParams params) {
-        for (File file : dir.listFiles()) {
-            if (file.isDirectory())
-                visitFolder(file, params);
-            else {
-                params.size += file.length();
-                params.lastModified = Math.max(params.lastModified, dir.lastModified());
-            }
-        }
-    }
-
-    /**
-     * @param folder folder to scan.
-     * @return folder displayable information.
-     */
-    @NotNull private String getPathDisplayableInfo(final File folder) {
-        final SB res = new SB();
-
-        res.a(getCanonicalPath(folder));
-        res.a(", ");
-        final FolderParams params = folderSize(folder);
-
-        res.a(params.size);
-        res.a(" bytes, modified ");
-        final SimpleDateFormat simpleDateFormat = new SimpleDateFormat("MM/dd/yyyy hh:mm a");
-
-        res.a(simpleDateFormat.format(params.lastModified));
-        res.a(" ");
-
-        return res.toString();
-    }
-
-    /**
-     * Returns the canonical pathname string of this abstract pathname.
-     *
-     * @param file path to convert.
-     * @return canonical pathname or at leas absolute if convert to canonical failed.
-     */
-    @NotNull private String getCanonicalPath(final File file) {
-        try {
-            return file.getCanonicalPath();
-        }
-        catch (IOException ignored) {
-            return file.getAbsolutePath();
-        }
-    }
-
-    /**
-     * Pad start of string with provided character.
-     *
-     * @param str sting to pad.
-     * @param minLength expected length.
-     * @param padChar padding character.
-     * @return padded string.
-     */
-    private static String padStart(String str, int minLength, char padChar) {
-        A.notNull(str, "String should not be empty");
-        if (str.length() >= minLength)
-            return str;
-
-        final SB sb = new SB(minLength);
-
-        for (int i = str.length(); i < minLength; ++i)
-            sb.a(padChar);
-
-        sb.a(str);
-
-        return sb.toString();
-
-    }
-
-    /**
-     * Creates new DB storage folder.
-     *
-     * @param pstStoreBasePath DB root path.
-     * @param nodeIdx next node index to use in folder name.
-     * @return new settings to be used in this node.
-     * @throws IgniteCheckedException if failed.
-     */
-    @NotNull private PdsFolderSettings generateAndLockNewDbStorage(final File pstStoreBasePath,
-        final int nodeIdx) throws IgniteCheckedException {
-
-        final UUID uuid = UUID.randomUUID();
-        final String consIdBasedFolder = genNewStyleSubfolderName(nodeIdx, uuid);
-        final File newRandomFolder = U.resolveWorkDirectory(pstStoreBasePath.getAbsolutePath(), consIdBasedFolder, false); //mkdir here
-        final NodeFileLockHolder fileLockHolder = tryLock(newRandomFolder);
-
-        if (fileLockHolder != null) {
-            if (log.isInfoEnabled())
-                log.info("Successfully created new persistent storage folder [" + newRandomFolder + "]");
-
-            return new PdsFolderSettings(pstStoreBasePath, consIdBasedFolder, uuid, fileLockHolder, false);
-        }
-        throw new IgniteCheckedException("Unable to lock file generated randomly [" + newRandomFolder + "]");
-    }
-
-    /**
-     * Generates DB subfolder name for provided node index (local) and UUID (consistent ID)
-     *
-     * @param nodeIdx node index.
-     * @param uuid consistent ID.
-     * @return folder file name
-     */
-    @NotNull public static String genNewStyleSubfolderName(final int nodeIdx, final UUID uuid) {
-        final String uuidAsStr = uuid.toString();
-
-        assert uuidAsStr.matches(UUID_STR_PATTERN);
-
-        final String nodeIdxPadded = padStart(Integer.toString(nodeIdx), 2, '0');
-
-        return DB_FOLDER_PREFIX + nodeIdxPadded + NODEIDX_UID_SEPARATOR + uuidAsStr;
-    }
-
-    /**
-     * Acquires lock to root storage directory, used to lock root directory in case creating new files is required.
-     *
-     * @param pstStoreBasePath rood DB dir to lock
-     * @return locked directory, should be released and closed later
-     * @throws IgniteCheckedException if failed
-     */
-    @NotNull private NodeFileLockHolder lockRootDirectory(File pstStoreBasePath)
-        throws IgniteCheckedException {
-
-        NodeFileLockHolder rootDirLock;
-        int retry = 0;
-
-        while ((rootDirLock = tryLock(pstStoreBasePath)) == null) {
-            if (retry > 600)
-                throw new IgniteCheckedException("Unable to start under DB storage path [" + pstStoreBasePath + "]" +
-                    ". Lock is being held to root directory");
-            retry++;
-        }
-
-        return rootDirLock;
-    }
-
-    /**
-     * @param pstStoreBasePath root storage folder to scan.
-     * @return empty list if there is no files in folder to test. Non null value is returned for folder having
-     * applicable new style files. Collection is sorted ascending according to node ID, 0 node index is coming first.
-     */
-    @Nullable private List<FolderCandidate> getNodeIndexSortedCandidates(File pstStoreBasePath) {
-        final File[] files = pstStoreBasePath.listFiles(DB_SUBFOLDERS_NEW_STYLE_FILTER);
-
-        if (files == null)
-            return Collections.emptyList();
-
-        final List<FolderCandidate> res = new ArrayList<>();
-
-        for (File file : files) {
-            final FolderCandidate candidate = parseFileName(file);
-
-            if (candidate != null)
-                res.add(candidate);
-        }
-        Collections.sort(res, new Comparator<FolderCandidate>() {
-            @Override public int compare(FolderCandidate c1, FolderCandidate c2) {
-                return Integer.compare(c1.nodeIndex(), c2.nodeIndex());
-            }
-        });
-
-        return res;
-    }
-
-    /**
      * Tries to lock subfolder within storage root folder.
      *
      * @param dbStoreDirWithSubdirectory DB store directory, is to be absolute and should include consistent ID based
@@ -437,66 +131,6 @@ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements Pd
         }
     }
 
-    /**
-     * @return DB storage absolute root path resolved as 'db' folder in Ignite work dir (by default) or using persistent
-     * store configuration. Null if persistence is not enabled. Returned folder is created automatically.
-     * @throws IgniteCheckedException if I/O failed.
-     */
-    @Nullable private File resolvePersistentStoreBasePath() throws IgniteCheckedException {
-        final DataStorageConfiguration dsCfg = cfg.getDataStorageConfiguration();
-
-        if (dsCfg == null)
-            return null;
-
-        final String pstPath = dsCfg.getStoragePath();
-
-        return U.resolveWorkDirectory(
-            cfg.getWorkDirectory(),
-            pstPath != null ? pstPath : DB_DEFAULT_FOLDER,
-            false
-        );
-
-    }
-
-    /**
-     * @param subFolderFile new style folder name to parse
-     * @return Pair of UUID and node index
-     */
-    private FolderCandidate parseFileName(@NotNull final File subFolderFile) {
-        return parseSubFolderName(subFolderFile, log);
-    }
-
-    /**
-     * @param subFolderFile new style file to parse.
-     * @param log Logger.
-     * @return Pair of UUID and node index.
-     */
-    @Nullable public static FolderCandidate parseSubFolderName(
-        @NotNull final File subFolderFile,
-        @NotNull final IgniteLogger log) {
-
-        final String fileName = subFolderFile.getName();
-        final Matcher matcher = Pattern.compile(NODE_PATTERN).matcher(fileName);
-        if (!matcher.find())
-            return null;
-
-        int uidStart = matcher.end();
-
-        try {
-            final String uid = fileName.substring(uidStart);
-            final UUID uuid = UUID.fromString(uid);
-            final String substring = fileName.substring(DB_FOLDER_PREFIX.length(), uidStart - NODEIDX_UID_SEPARATOR.length());
-            final int idx = Integer.parseInt(substring);
-
-            return new FolderCandidate(subFolderFile, idx, uuid);
-        }
-        catch (Exception e) {
-            U.warn(log, "Unable to parse new style file format from [" + subFolderFile.getAbsolutePath() + "]: " + e);
-
-            return null;
-        }
-    }
-
     /** {@inheritDoc} */
     @Override public void stop(boolean cancel) throws IgniteCheckedException {
         if (settings != null) {
@@ -508,61 +142,6 @@ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements Pd
 
         super.stop(cancel);
     }
-
-    /** Path metrics */
-    private static class FolderParams {
-        /** Overall size in bytes. */
-        private long size;
-
-        /** Last modified. */
-        private long lastModified;
-    }
-
-    /**
-     * Represents parsed new style file and encoded parameters in this file name
-     */
-    public static class FolderCandidate {
-        /** Absolute file path pointing to DB subfolder within DB storage root folder. */
-        private final File subFolderFile;
-
-        /** Node index (local, usually 0 if multiple nodes are not started at local PC). */
-        private final int nodeIdx;
-
-        /** Uuid contained in file name, is to be set as consistent ID. */
-        private final UUID uuid;
-
-        /**
-         * @param subFolderFile Absolute file path pointing to DB subfolder.
-         * @param nodeIdx Node index.
-         * @param uuid Uuid.
-         */
-        public FolderCandidate(File subFolderFile, int nodeIdx, UUID uuid) {
-            this.subFolderFile = subFolderFile;
-            this.nodeIdx = nodeIdx;
-            this.uuid = uuid;
-        }
-
-        /**
-         * @return Node index (local, usually 0 if multiple nodes are not started at local PC).
-         */
-        public int nodeIndex() {
-            return nodeIdx;
-        }
-
-        /**
-         * @return Uuid contained in file name, is to be set as consistent ID.
-         */
-        public Serializable uuid() {
-            return uuid;
-        }
-
-        /**
-         * @return Absolute file path pointing to DB subfolder within DB storage root folder.
-         */
-        public File subFolderFile() {
-            return subFolderFile;
-        }
-    }
 }
 
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java
similarity index 74%
copy from modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java
copy to modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java
index 35f06dd..225a973 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java
@@ -1,12 +1,12 @@
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
+ * 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
+ * the License.  You may obtain a copy of the License at
  *
- * http://www.apache.org/licenses/LICENSE-2.0
+ *      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,
@@ -27,15 +27,14 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
 import java.util.UUID;
+import java.util.function.Function;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.processors.GridProcessorAdapter;
-import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.NodeFileLockHolder;
+import org.apache.ignite.internal.processors.cache.persistence.FileLockHolder;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.SB;
@@ -47,9 +46,12 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_DATA_STORAGE_FOLDE
 import static org.apache.ignite.IgniteSystemProperties.getBoolean;
 
 /**
- * Component for resolving PDS storage file names, also used for generating consistent ID for case PDS mode is enabled
+ * This class contains logic to resolve and possibly lock PDS folder
+ * based on provided {@link IgniteConfiguration} and {@link #consistentId}.
+ *
+ * @param <L> Type of the lock holder.
  */
-public class PdsConsistentIdProcessor extends GridProcessorAdapter implements PdsFoldersResolver {
+public class PdsFolderResolver<L extends FileLockHolder> {
     /** Database subfolders constant prefix. */
     private static final String DB_FOLDER_PREFIX = "node";
 
@@ -90,29 +92,34 @@ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements Pd
     /** Database default folder. */
     public static final String DB_DEFAULT_FOLDER = "db";
 
-    /** Config. */
-    private IgniteConfiguration cfg;
+    /** */
+    private final IgniteConfiguration cfg;
 
-    /** Logger. */
-    private IgniteLogger log;
+    /** */
+    private final IgniteLogger log;
 
-    /** Context. */
-    private GridKernalContext ctx;
+    /** */
+    private final @Nullable Serializable consistentId;
 
-    /** Cached folder settings. */
-    private PdsFolderSettings settings;
+    /** */
+    private final Function<File, L> tryLock;
 
     /**
-     * Creates folders resolver
-     *
-     * @param ctx Context.
+     * @param cfg Ignite configuration.
+     * @param log Logger.
+     * @param consistentId Constent id.
+     * @param tryLock Lock function.
      */
-    public PdsConsistentIdProcessor(final GridKernalContext ctx) {
-        super(ctx);
-
-        this.cfg = ctx.config();
-        this.log = ctx.log(PdsFoldersResolver.class);
-        this.ctx = ctx;
+    public PdsFolderResolver(
+        IgniteConfiguration cfg,
+        IgniteLogger log,
+        @Nullable Serializable consistentId,
+        Function<File, L> tryLock
+    ) {
+        this.cfg = cfg;
+        this.log = log;
+        this.consistentId = consistentId;
+        this.tryLock = tryLock;
     }
 
     /**
@@ -122,50 +129,35 @@ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements Pd
      * @param consistentId compatibility consistent ID
      * @return PDS folder settings compatible with previous versions.
      */
-    private PdsFolderSettings compatibleResolve(
+    private PdsFolderSettings<L> compatibleResolve(
         @Nullable final File pstStoreBasePath,
-        @NotNull final Serializable consistentId) {
+        @Nullable final Serializable consistentId) {
 
         if (cfg.getConsistentId() != null) {
             // compatible mode from configuration is used fot this case, no locking, no consitent id change
-            return new PdsFolderSettings(pstStoreBasePath, cfg.getConsistentId());
+            return new PdsFolderSettings<>(pstStoreBasePath, cfg.getConsistentId());
         }
 
-        return new PdsFolderSettings(pstStoreBasePath, consistentId);
-    }
-
-    /** {@inheritDoc} */
-    @Override public PdsFolderSettings resolveFolders() throws IgniteCheckedException {
-        if (settings == null) {
-            settings = prepareNewSettings();
+        if (consistentId == null)
+            return new PdsFolderSettings<>(pstStoreBasePath, consistentId);
 
-            if (!settings.isCompatible()) {
-                if (log.isInfoEnabled())
-                    log.info("Consistent ID used for local node is [" + settings.consistentId() + "] " +
-                        "according to persistence data storage folders");
-
-                ctx.discovery().consistentId(settings.consistentId());
-            }
-        }
-        return settings;
+        return null;
     }
 
     /**
-     * Creates new settings when we don't have cached one.
+     * Resolves {@link PdsFolderSettings} according to specified {@link IgniteConfiguration}, {@link #consistentId}.
      *
-     * @return new settings with prelocked directory (if appropriate).
+     * @return new settings with prelocked directory (if appropriate) or null.
      * @throws IgniteCheckedException if IO failed.
      */
-    private PdsFolderSettings prepareNewSettings() throws IgniteCheckedException {
+    public PdsFolderSettings<L> resolve() throws IgniteCheckedException {
         final File pstStoreBasePath = resolvePersistentStoreBasePath();
-        //here deprecated method is used to get compatible version of consistentId
-        final Serializable consistentId = ctx.discovery().consistentId();
 
         if (!CU.isPersistenceEnabled(cfg))
             return compatibleResolve(pstStoreBasePath, consistentId);
 
-        if (ctx.clientNode())
-            return new PdsFolderSettings(pstStoreBasePath, UUID.randomUUID());
+        if (cfg.isClientMode() || cfg.isDaemon())
+            return new PdsFolderSettings<>(pstStoreBasePath, UUID.randomUUID());
 
         if (getBoolean(IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID, false))
             return compatibleResolve(pstStoreBasePath, consistentId);
@@ -173,20 +165,24 @@ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements Pd
         // compatible mode from configuration is used fot this case
         if (cfg.getConsistentId() != null) {
             // compatible mode from configuration is used fot this case, no locking, no consistent id change
-            return new PdsFolderSettings(pstStoreBasePath, cfg.getConsistentId());
+            return new PdsFolderSettings<>(pstStoreBasePath, cfg.getConsistentId());
         }
-        // The node scans the work directory and checks if there is a folder matching the consistent ID.
-        // If such a folder exists, we start up with this ID (compatibility mode)
-        final String subFolder = U.maskForFileName(consistentId.toString());
 
-        final NodeFileLockHolder oldStyleFolderLockHolder = tryLock(new File(pstStoreBasePath, subFolder));
+        if (consistentId != null) {
+            // The node scans the work directory and checks if there is a folder matching the consistent ID.
+            // If such a folder exists, we start up with this ID (compatibility mode)
+            final String subFolder = U.maskForFileName(consistentId.toString());
+
+            final L oldStyleFolderLockHolder = tryLock.apply(new File(pstStoreBasePath, subFolder));
 
-        if (oldStyleFolderLockHolder != null)
-            return new PdsFolderSettings(pstStoreBasePath,
-                subFolder,
-                consistentId,
-                oldStyleFolderLockHolder,
-                true);
+            if (oldStyleFolderLockHolder != null) {
+                return new PdsFolderSettings<>(pstStoreBasePath,
+                    subFolder,
+                    consistentId,
+                    oldStyleFolderLockHolder,
+                    true);
+            }
+        }
 
         final File[] oldStyleFolders = pstStoreBasePath.listFiles(DB_SUBFOLDERS_OLD_STYLE_FILTER);
 
@@ -199,13 +195,13 @@ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements Pd
         }
 
         for (FolderCandidate next : getNodeIndexSortedCandidates(pstStoreBasePath)) {
-            final NodeFileLockHolder fileLockHolder = tryLock(next.subFolderFile());
+            final L fileLockHolder = tryLock.apply(next.subFolderFile());
 
             if (fileLockHolder != null) {
                 if (log.isInfoEnabled())
                     log.info("Successfully locked persistence storage folder [" + next.subFolderFile() + "]");
 
-                return new PdsFolderSettings(pstStoreBasePath,
+                return new PdsFolderSettings<>(pstStoreBasePath,
                     next.subFolderFile().getName(),
                     next.uuid(),
                     fileLockHolder,
@@ -213,8 +209,18 @@ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements Pd
             }
         }
 
+        return null;
+    }
+
+    /**
+     * @return New PDS folder.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public PdsFolderSettings<L> generateNew() throws IgniteCheckedException {
+        final File pstStoreBasePath = resolvePersistentStoreBasePath();
+
         // was not able to find free slot, allocating new
-        try (final NodeFileLockHolder rootDirLock = lockRootDirectory(pstStoreBasePath)) {
+        try (final L rootDirLock = lockRootDirectory(pstStoreBasePath)) {
             final List<FolderCandidate> sortedCandidates = getNodeIndexSortedCandidates(pstStoreBasePath);
             final int nodeIdx = sortedCandidates.isEmpty() ? 0 : (sortedCandidates.get(sortedCandidates.size() - 1).nodeIndex() + 1);
 
@@ -310,7 +316,6 @@ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements Pd
         sb.a(str);
 
         return sb.toString();
-
     }
 
     /**
@@ -321,20 +326,22 @@ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements Pd
      * @return new settings to be used in this node.
      * @throws IgniteCheckedException if failed.
      */
-    @NotNull private PdsFolderSettings generateAndLockNewDbStorage(final File pstStoreBasePath,
-        final int nodeIdx) throws IgniteCheckedException {
-
+    @NotNull private PdsFolderSettings<L> generateAndLockNewDbStorage(
+        final File pstStoreBasePath,
+        final int nodeIdx
+    ) throws IgniteCheckedException {
         final UUID uuid = UUID.randomUUID();
         final String consIdBasedFolder = genNewStyleSubfolderName(nodeIdx, uuid);
         final File newRandomFolder = U.resolveWorkDirectory(pstStoreBasePath.getAbsolutePath(), consIdBasedFolder, false); //mkdir here
-        final NodeFileLockHolder fileLockHolder = tryLock(newRandomFolder);
+        final L fileLockHolder = tryLock.apply(newRandomFolder);
 
         if (fileLockHolder != null) {
             if (log.isInfoEnabled())
                 log.info("Successfully created new persistent storage folder [" + newRandomFolder + "]");
 
-            return new PdsFolderSettings(pstStoreBasePath, consIdBasedFolder, uuid, fileLockHolder, false);
+            return new PdsFolderSettings<>(pstStoreBasePath, consIdBasedFolder, uuid, fileLockHolder, false);
         }
+
         throw new IgniteCheckedException("Unable to lock file generated randomly [" + newRandomFolder + "]");
     }
 
@@ -362,13 +369,13 @@ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements Pd
      * @return locked directory, should be released and closed later
      * @throws IgniteCheckedException if failed
      */
-    @NotNull private NodeFileLockHolder lockRootDirectory(File pstStoreBasePath)
+    @NotNull private L lockRootDirectory(File pstStoreBasePath)
         throws IgniteCheckedException {
 
-        NodeFileLockHolder rootDirLock;
+        L rootDirLock;
         int retry = 0;
 
-        while ((rootDirLock = tryLock(pstStoreBasePath)) == null) {
+        while ((rootDirLock = tryLock.apply(pstStoreBasePath)) == null) {
             if (retry > 600)
                 throw new IgniteCheckedException("Unable to start under DB storage path [" + pstStoreBasePath + "]" +
                     ". Lock is being held to root directory");
@@ -397,8 +404,10 @@ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements Pd
             if (candidate != null)
                 res.add(candidate);
         }
+
         Collections.sort(res, new Comparator<FolderCandidate>() {
-            @Override public int compare(FolderCandidate c1, FolderCandidate c2) {
+            @Override public int compare(
+                FolderCandidate c1, FolderCandidate c2) {
                 return Integer.compare(c1.nodeIndex(), c2.nodeIndex());
             }
         });
@@ -407,37 +416,6 @@ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements Pd
     }
 
     /**
-     * Tries to lock subfolder within storage root folder.
-     *
-     * @param dbStoreDirWithSubdirectory DB store directory, is to be absolute and should include consistent ID based
-     * sub folder.
-     * @return non null holder if lock was successful, null in case lock failed. If directory does not exist method will
-     * always fail to lock.
-     */
-    private NodeFileLockHolder tryLock(File dbStoreDirWithSubdirectory) {
-        if (!dbStoreDirWithSubdirectory.exists())
-            return null;
-
-        final String path = dbStoreDirWithSubdirectory.getAbsolutePath();
-        final NodeFileLockHolder fileLockHolder
-            = new NodeFileLockHolder(path, ctx, log);
-
-        try {
-            fileLockHolder.tryLock(1000);
-
-            return fileLockHolder;
-        }
-        catch (IgniteCheckedException e) {
-            U.closeQuiet(fileLockHolder);
-
-            if (log.isInfoEnabled())
-                log.info("Unable to acquire lock to file [" + path + "], reason: " + e.getMessage());
-
-            return null;
-        }
-    }
-
-    /**
      * @return DB storage absolute root path resolved as 'db' folder in Ignite work dir (by default) or using persistent
      * store configuration. Null if persistence is not enabled. Returned folder is created automatically.
      * @throws IgniteCheckedException if I/O failed.
@@ -455,7 +433,6 @@ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements Pd
             pstPath != null ? pstPath : DB_DEFAULT_FOLDER,
             false
         );
-
     }
 
     /**
@@ -497,18 +474,6 @@ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements Pd
         }
     }
 
-    /** {@inheritDoc} */
-    @Override public void stop(boolean cancel) throws IgniteCheckedException {
-        if (settings != null) {
-            final NodeFileLockHolder fileLockHolder = settings.getLockedFileLockHolder();
-
-            if (fileLockHolder != null)
-                fileLockHolder.close();
-        }
-
-        super.stop(cancel);
-    }
-
     /** Path metrics */
     private static class FolderParams {
         /** Overall size in bytes. */
@@ -564,5 +529,3 @@ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements Pd
         }
     }
 }
-
-
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderSettings.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderSettings.java
index 72e0720..28400c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderSettings.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderSettings.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.cache.persistence.filename;
 
 import java.io.File;
 import java.io.Serializable;
-import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.NodeFileLockHolder;
+import org.apache.ignite.internal.processors.cache.persistence.FileLockHolder;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.NotNull;
@@ -28,7 +28,7 @@ import org.jetbrains.annotations.Nullable;
 /**
  * Class holds information required for folder generation for ignite persistent store
  */
-public class PdsFolderSettings {
+public class PdsFolderSettings<L extends FileLockHolder> {
     /**
      * DB storage absolute root path resolved as 'db' folder in Ignite work dir (by default) or using persistent store
      * configuration. <br>
@@ -48,7 +48,7 @@ public class PdsFolderSettings {
      * directory. This value is to be used at activate instead of locking. <br> May be null in case preconfigured
      * consistent ID is used or in case lock holder was already taken by other processor.
      */
-    @Nullable private final NodeFileLockHolder fileLockHolder;
+    @Nullable private final L fileLockHolder;
 
     /**
      * Indicates if compatible mode is enabled, in that case all sub folders are generated from consistent ID without
@@ -68,7 +68,7 @@ public class PdsFolderSettings {
     public PdsFolderSettings(@Nullable final File persistentStoreRootPath,
         final String folderName,
         final Serializable consistentId,
-        @Nullable final NodeFileLockHolder fileLockHolder,
+        @Nullable final L fileLockHolder,
         final boolean compatible) {
 
         this.consistentId = consistentId;
@@ -125,7 +125,7 @@ public class PdsFolderSettings {
      *
      * @return File lock holder with prelocked db directory.
      */
-    @Nullable public NodeFileLockHolder getLockedFileLockHolder() {
+    @Nullable public L getLockedFileLockHolder() {
         return fileLockHolder;
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFoldersResolver.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFoldersResolver.java
index cefaa04..6122e28 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFoldersResolver.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFoldersResolver.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache.persistence.filename;
 
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.NodeFileLockHolder;
 
 /**
  * Resolves folders for PDS mode, may have side effect as setting random UUID as local node consistent ID.
@@ -29,5 +30,5 @@ public interface PdsFoldersResolver {
      * @return PDS folder settings, consistentID and prelocked DB file lock.
      * @throws IgniteCheckedException if failed.
      */
-    public PdsFolderSettings resolveFolders() throws IgniteCheckedException;
+    public PdsFolderSettings<NodeFileLockHolder> resolveFolders() throws IgniteCheckedException;
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
index d23b584..10da397 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
@@ -165,7 +165,7 @@ import static org.apache.ignite.internal.processors.cache.persistence.file.FileP
 import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
 import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.cacheDirectories;
 import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.getPartitionFile;
-import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.DB_DEFAULT_FOLDER;
+import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver.DB_DEFAULT_FOLDER;
 import static org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId.getTypeByPartId;
 import static org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO.T_DATA;
 import static org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO.getPageIO;
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java
index 96a32f8..9240892 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java
@@ -31,7 +31,7 @@ import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
-import org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridStringLogger;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -40,7 +40,7 @@ import org.junit.Test;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_CONSISTENT_ID_BY_HOST_WITHOUT_PORT;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID;
-import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.parseSubFolderName;
+import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver.parseSubFolderName;
 
 /**
  * Test for new and old style persistent storage folders generation
@@ -365,7 +365,7 @@ public class IgniteUidAsConsistentIdMigrationTest extends GridCommonAbstractTest
                 " but actual class is " + (consistentId == null ? null : consistentId.getClass()),
             consistentId instanceof UUID);
 
-        return PdsConsistentIdProcessor.genNewStyleSubfolderName(nodeIdx, (UUID)consistentId);
+        return PdsFolderResolver.genNewStyleSubfolderName(nodeIdx, (UUID)consistentId);
     }
 
     /**
@@ -655,12 +655,12 @@ public class IgniteUidAsConsistentIdMigrationTest extends GridCommonAbstractTest
      * @throws IgniteCheckedException if failed.
      */
     @NotNull private Set<Integer> getAllNodeIndexesInFolder() throws IgniteCheckedException {
-        final File curFolder = new File(U.defaultWorkDirectory(), PdsConsistentIdProcessor.DB_DEFAULT_FOLDER);
+        final File curFolder = new File(U.defaultWorkDirectory(), PdsFolderResolver.DB_DEFAULT_FOLDER);
         final Set<Integer> indexes = new TreeSet<>();
-        final File[] files = curFolder.listFiles(PdsConsistentIdProcessor.DB_SUBFOLDERS_NEW_STYLE_FILTER);
+        final File[] files = curFolder.listFiles(PdsFolderResolver.DB_SUBFOLDERS_NEW_STYLE_FILTER);
 
         for (File file : files) {
-            final PdsConsistentIdProcessor.FolderCandidate uid = parseSubFolderName(file, log);
+            final PdsFolderResolver.FolderCandidate uid = parseSubFolderName(file, log);
 
             if (uid != null)
                 indexes.add(uid.nodeIndex());
@@ -679,7 +679,7 @@ public class IgniteUidAsConsistentIdMigrationTest extends GridCommonAbstractTest
         assertDirectoryExist(DataStorageConfiguration.DFLT_BINARY_METADATA_PATH, subDirName);
         assertDirectoryExist(DataStorageConfiguration.DFLT_WAL_PATH, subDirName);
         assertDirectoryExist(DataStorageConfiguration.DFLT_WAL_ARCHIVE_PATH, subDirName);
-        assertDirectoryExist(PdsConsistentIdProcessor.DB_DEFAULT_FOLDER, subDirName);
+        assertDirectoryExist(PdsFolderResolver.DB_DEFAULT_FOLDER, subDirName);
     }
 
     /**
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
index 5c6c7c0..60b8020 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
@@ -96,7 +96,7 @@ import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.CREATE;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.DELETE;
 import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
-import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.genNewStyleSubfolderName;
+import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver.genNewStyleSubfolderName;
 
 /**
  * Test suite for WAL segments reader and event generator.
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
index 4698c00..d3b93ce 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
@@ -91,7 +91,6 @@ import org.apache.ignite.internal.processors.cache.persistence.checkpoint.Checkp
 import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointEntryType;
 import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointMarkersStorage;
 import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
-import org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor;
 import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.CompactablePageIO;
@@ -136,6 +135,7 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGNITE_INSTAN
 import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
 import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DATA_FILENAME;
 import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver.genNewStyleSubfolderName;
 
 /**
  *
@@ -784,7 +784,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
      */
     private File cacheDir(final String cacheName, final String consId) throws IgniteCheckedException {
         final String subfolderName
-            = PdsConsistentIdProcessor.genNewStyleSubfolderName(0, UUID.fromString(consId));
+            = genNewStyleSubfolderName(0, UUID.fromString(consId));
 
         final File dbDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false);