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 2022/05/11 15:24:39 UTC

[ignite] branch master updated: IGNITE-16938 StoredCacheData logic moved to GridLocalConfigManager (#10018)

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

nizhikov 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 cdf8200fa02 IGNITE-16938 StoredCacheData logic moved to GridLocalConfigManager (#10018)
cdf8200fa02 is described below

commit cdf8200fa02da8e88b6ca5a9fd9e89bb28f95c62
Author: Nikolay <ni...@apache.org>
AuthorDate: Wed May 11 18:24:31 2022 +0300

    IGNITE-16938 StoredCacheData logic moved to GridLocalConfigManager (#10018)
---
 .../pagemem/store/IgnitePageStoreManager.java      |  27 +-
 .../internal/processors/cache/CachesRegistry.java  |   2 +-
 .../processors/cache/GridCacheProcessor.java       |  21 +-
 .../processors/cache/GridLocalConfigManager.java   | 347 ++++++++++++++++++++-
 .../GridCacheDatabaseSharedManager.java            |   8 +-
 .../persistence/file/FilePageStoreManager.java     | 285 +----------------
 .../snapshot/IgniteSnapshotManager.java            |   9 +-
 .../persistence/snapshot/SnapshotFutureTask.java   |   6 +-
 .../snapshot/SnapshotPartitionsVerifyHandler.java  |   2 +-
 .../snapshot/SnapshotRestoreProcess.java           |   4 +-
 .../cluster/GridClusterStateProcessor.java         |   2 +-
 ...CacheConfigurationFileConsistencyCheckTest.java |   8 +-
 .../persistence/pagemem/NoOpPageStoreManager.java  |  21 +-
 13 files changed, 378 insertions(+), 364 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
index 0b3b8a746b2..30f058a46ed 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.pagemem.store;
 
 import java.nio.ByteBuffer;
-import java.util.Map;
 import java.util.function.Predicate;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -26,7 +25,6 @@ import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManager;
-import org.apache.ignite.internal.processors.cache.StoredCacheData;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMetrics;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageReadWriteManager;
 import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport;
@@ -61,10 +59,10 @@ public interface IgnitePageStoreManager extends GridCacheSharedManager, IgniteCh
      * Callback called when a cache is starting.
      *
      * @param grpDesc Cache group descriptor.
-     * @param cacheData Cache data of the cache being started.
+     * @param ccfg Cache configuration.
      * @throws IgniteCheckedException If failed to handle cache start callback.
      */
-    public void initializeForCache(CacheGroupDescriptor grpDesc, StoredCacheData cacheData)
+    public void initializeForCache(CacheGroupDescriptor grpDesc, CacheConfiguration<?, ?> ccfg)
         throws IgniteCheckedException;
 
     /**
@@ -176,27 +174,6 @@ public interface IgnitePageStoreManager extends GridCacheSharedManager, IgniteCh
      */
     public int pages(int grpId, int partId) throws IgniteCheckedException;
 
-    /**
-     * @return Saved cache configurations.
-     * @throws IgniteCheckedException If failed.
-     */
-    public Map<String, StoredCacheData> readCacheConfigurations() throws IgniteCheckedException;
-
-    /**
-     * @param cacheData Cache configuration.
-     * @param overwrite Whether stored configuration should be overwritten if it exists.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void storeCacheData(StoredCacheData cacheData, boolean overwrite) throws IgniteCheckedException;
-
-    /**
-     * Remove cache configuration data file.
-     *
-     * @param cacheData Cache configuration.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void removeCacheData(StoredCacheData cacheData) throws IgniteCheckedException;
-
     /**
      * @param grpId Cache group ID.
      * @return {@code True} if index store for given cache group existed before node started.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachesRegistry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachesRegistry.java
index f4b4fce8311..31aa0ac4693 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachesRegistry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachesRegistry.java
@@ -299,7 +299,7 @@ public class CachesRegistry {
             @Override public void run() {
                 try {
                     for (StoredCacheData data : cacheConfigsToPersist)
-                        cctx.cache().saveCacheConfiguration(data, false);
+                        cctx.cache().configManager().saveCacheConfiguration(data, false);
                 }
                 catch (IgniteCheckedException e) {
                     U.error(log, "Error while saving cache configurations on disk", e);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index c5efbf8bec7..2930a06594c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -1117,7 +1117,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             if (destroy && (pageStore = sharedCtx.pageStore()) != null) {
                 try {
-                    pageStore.removeCacheData(new StoredCacheData(ctx.config()));
+                    locCfgMgr.removeCacheData(new StoredCacheData(ctx.config()));
                 }
                 catch (IgniteCheckedException e) {
                     U.error(log, "Failed to delete cache configuration data while destroying cache" +
@@ -2200,7 +2200,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (sharedCtx.pageStore() != null && affNode)
             initializationProtector.protect(
                 desc.groupDescriptor().groupId(),
-                () -> sharedCtx.pageStore().initializeForCache(desc.groupDescriptor(), desc.toStoredData(splitter))
+                () -> sharedCtx.pageStore().initializeForCache(desc.groupDescriptor(), desc.toStoredData(splitter).config())
             );
     }
 
@@ -4084,18 +4084,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         locCfgMgr.saveCacheConfiguration(desc.toStoredData(splitter), true);
     }
 
-    /**
-     * Save cache configuration to persistent store if necessary.
-     *
-     * @param storedCacheData Stored cache data.
-     * @param overwrite Overwrite existing.
-     */
-    public void saveCacheConfiguration(StoredCacheData storedCacheData, boolean overwrite) throws IgniteCheckedException {
-        assert storedCacheData != null;
-
-        locCfgMgr.saveCacheConfiguration(storedCacheData, overwrite);
-    }
-
     /**
      * Remove all persistent files for all registered caches.
      */
@@ -4911,6 +4899,11 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         return (GridCacheSharedContext<K, V>)sharedCtx;
     }
 
+    /** @return Local config manager. */
+    public GridLocalConfigManager configManager() {
+        return locCfgMgr;
+    }
+
     /**
      * @return Transactions interface implementation.
      */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridLocalConfigManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridLocalConfigManager.java
index c59b3fbafda..ed69095434e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridLocalConfigManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridLocalConfigManager.java
@@ -17,8 +17,22 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Deque;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -26,21 +40,35 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.BiConsumer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.FailureContext;
+import org.apache.ignite.failure.FailureType;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.marshaller.MarshallerUtils;
 
+import static java.nio.file.Files.newDirectoryStream;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isPersistentCache;
+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.CACHE_DIR_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_GRP_DIR_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
 
 /**
  * Responsible for restoring local cache configurations (both from static configuration and persistence).
@@ -51,6 +79,9 @@ public class GridLocalConfigManager {
     private final boolean startClientCaches =
         IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_START_CACHES_ON_JOIN, false);
 
+    /** Listeners of configuration changes e.g. overwrite or remove actions. */
+    private final List<BiConsumer<String, File>> lsnrs = new CopyOnWriteArrayList<>();
+
     /** Caches stop sequence. */
     private final Deque<String> stopSeq = new LinkedList<>();
 
@@ -63,9 +94,18 @@ public class GridLocalConfigManager {
     /** Cache processor. */
     private final GridCacheProcessor cacheProcessor;
 
+    /** Absolute directory for file page store. Includes consistent id based folder. */
+    private final File storeWorkDir;
+
+    /** Marshaller. */
+    private final Marshaller marshaller;
+
     /** Context. */
     private final GridKernalContext ctx;
 
+    /** Lock which guards configuration changes. */
+    private final ReentrantReadWriteLock chgLock = new ReentrantReadWriteLock();
+
     /**
      * @param cacheProcessor Cache processor.
      * @param kernalCtx Kernal context.
@@ -73,27 +113,221 @@ public class GridLocalConfigManager {
     public GridLocalConfigManager(
         GridCacheProcessor cacheProcessor,
         GridKernalContext kernalCtx
-    ) {
+    ) throws IgniteCheckedException {
         this.cacheProcessor = cacheProcessor;
         ctx = kernalCtx;
         log = ctx.log(getClass());
+        marshaller = MarshallerUtils.jdkMarshaller(ctx.igniteInstanceName());
+
+        PdsFolderSettings<?> folderSettings = ctx.pdsFolderResolver().resolveFolders();
+
+        if (!ctx.clientNode() && folderSettings.persistentStoreRootPath() != null) {
+            storeWorkDir = folderSettings.persistentStoreNodePath();
+
+            U.ensureDirectory(storeWorkDir, "page store work directory", log);
+        }
+        else
+            storeWorkDir = null;
+    }
+
+    /**
+     * @param ccfgs List of cache configurations to process.
+     * @param ccfgCons Consumer which accepts found configurations files.
+     */
+    public void readConfigurationFiles(
+        List<CacheConfiguration<?, ?>> ccfgs,
+        BiConsumer<CacheConfiguration<?, ?>, File> ccfgCons
+    ) {
+        chgLock.writeLock().lock();
+
+        try {
+            for (CacheConfiguration<?, ?> ccfg : ccfgs) {
+                File cacheDir = cacheWorkDir(ccfg);
+
+                if (!cacheDir.exists())
+                    continue;
+
+                File[] ccfgFiles = cacheDir.listFiles((dir, name) -> name.endsWith(CACHE_DATA_FILENAME));
+
+                if (ccfgFiles == null)
+                    continue;
+
+                for (File ccfgFile : ccfgFiles)
+                    ccfgCons.accept(ccfg, ccfgFile);
+            }
+        }
+        finally {
+            chgLock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * @return Saved cache configurations.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Map<String, StoredCacheData> readCacheConfigurations() throws IgniteCheckedException {
+        if (ctx.clientNode())
+            return Collections.emptyMap();
+
+        File[] files = storeWorkDir.listFiles();
+
+        if (files == null)
+            return Collections.emptyMap();
+
+        Map<String, StoredCacheData> ccfgs = new HashMap<>();
+
+        Arrays.sort(files);
+
+        for (File file : files) {
+            if (file.isDirectory())
+                readCacheConfigurations(file, ccfgs);
+        }
+
+        return ccfgs;
+    }
+
+    /**
+     * @param dir Cache (group) directory.
+     * @param ccfgs Cache configurations.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void readCacheConfigurations(File dir, Map<String, StoredCacheData> ccfgs) throws IgniteCheckedException {
+        if (dir.getName().startsWith(CACHE_DIR_PREFIX)) {
+            File conf = new File(dir, CACHE_DATA_FILENAME);
+
+            if (conf.exists() && conf.length() > 0) {
+                StoredCacheData cacheData = readCacheData(conf);
+
+                String cacheName = cacheData.config().getName();
+
+                if (!ccfgs.containsKey(cacheName))
+                    ccfgs.put(cacheName, cacheData);
+                else {
+                    U.warn(log, "Cache with name=" + cacheName + " is already registered, skipping config file "
+                        + dir.getName());
+                }
+            }
+        }
+        else if (dir.getName().startsWith(CACHE_GRP_DIR_PREFIX))
+            readCacheGroupCaches(dir, ccfgs);
+    }
+
+    /**
+     * @param conf File with stored cache data.
+     * @return Cache data.
+     * @throws IgniteCheckedException If failed.
+     */
+    public StoredCacheData readCacheData(File conf) throws IgniteCheckedException {
+        try (InputStream stream = new BufferedInputStream(new FileInputStream(conf))) {
+            return marshaller.unmarshal(stream, U.resolveClassLoader(ctx.config()));
+        }
+        catch (IgniteCheckedException | IOException e) {
+            throw new IgniteCheckedException("An error occurred during cache configuration loading from file [file=" +
+                conf.getAbsolutePath() + "]", e);
+        }
+    }
+
+    /**
+     * @param conf File to store cache data.
+     * @param cacheData Cache data file.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void writeCacheData(StoredCacheData cacheData, File conf) throws IgniteCheckedException {
+        // Pre-existing file will be truncated upon stream open.
+        try (OutputStream stream = new BufferedOutputStream(new FileOutputStream(conf))) {
+            marshaller.marshal(cacheData, stream);
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("An error occurred during cache configuration writing to file [file=" +
+                conf.getAbsolutePath() + "]", e);
+        }
     }
 
     /**
      * Save cache configuration to persistent store if necessary.
      *
-     * @param storedCacheData Stored cache data.
+     * @param cacheData Stored cache data.
      * @param overwrite Overwrite existing.
      */
-    public void saveCacheConfiguration(StoredCacheData storedCacheData, boolean overwrite) throws IgniteCheckedException {
-        assert storedCacheData != null;
+    public void saveCacheConfiguration(
+        StoredCacheData cacheData,
+        boolean overwrite
+    ) throws IgniteCheckedException {
+        assert cacheData != null;
 
         GridCacheSharedContext<Object, Object> sharedContext = cacheProcessor.context();
 
-        if (sharedContext.pageStore() != null
+        boolean shouldStore = sharedContext.pageStore() != null
             && !sharedContext.kernalContext().clientNode()
-            && isPersistentCache(storedCacheData.config(), sharedContext.gridConfig().getDataStorageConfiguration()))
-            sharedContext.pageStore().storeCacheData(storedCacheData, overwrite);
+            && isPersistentCache(cacheData.config(), sharedContext.gridConfig().getDataStorageConfiguration());
+
+        if (!shouldStore)
+            return;
+
+        CacheConfiguration<?, ?> ccfg = cacheData.config();
+        File cacheWorkDir = cacheWorkDir(ccfg);
+
+        cacheProcessor.context().pageStore().checkAndInitCacheWorkDir(ccfg);
+
+        assert cacheWorkDir.exists() : "Work directory does not exist: " + cacheWorkDir;
+
+        File file = cacheConfigurationFile(ccfg);
+        Path filePath = file.toPath();
+
+        chgLock.readLock().lock();
+
+        try {
+            if (overwrite || !Files.exists(filePath) || Files.size(filePath) == 0) {
+                File tmp = new File(file.getParent(), file.getName() + TMP_SUFFIX);
+
+                if (tmp.exists() && !tmp.delete()) {
+                    log.warning("Failed to delete temporary cache config file" +
+                        "(make sure Ignite process has enough rights):" + file.getName());
+                }
+
+                writeCacheData(cacheData, tmp);
+
+                if (Files.exists(filePath) && Files.size(filePath) > 0) {
+                    for (BiConsumer<String, File> lsnr : lsnrs)
+                        lsnr.accept(ccfg.getName(), file);
+                }
+
+                Files.move(tmp.toPath(), file.toPath(), StandardCopyOption.REPLACE_EXISTING);
+            }
+        }
+        catch (IOException ex) {
+            ctx.failure().process(new FailureContext(FailureType.CRITICAL_ERROR, ex));
+
+            throw new IgniteCheckedException("Failed to persist cache configuration: " + ccfg.getName(), ex);
+        }
+        finally {
+            chgLock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Remove cache configuration from persistent store.
+     *
+     * @param cacheData Stored cache data.
+     */
+    public void removeCacheData(StoredCacheData cacheData) throws IgniteCheckedException {
+        chgLock.readLock().lock();
+
+        try {
+            CacheConfiguration<?, ?> ccfg = cacheData.config();
+            File file = cacheConfigurationFile(ccfg);
+
+            if (file.exists()) {
+                for (BiConsumer<String, File> lsnr : lsnrs)
+                    lsnr.accept(ccfg.getName(), file);
+
+                if (!file.delete())
+                    throw new IgniteCheckedException("Failed to delete cache configuration: " + ccfg.getName());
+            }
+        }
+        finally {
+            chgLock.readLock().unlock();
+        }
     }
 
     /**
@@ -135,6 +369,103 @@ public class GridLocalConfigManager {
         return discoData;
     }
 
+    /**
+     * @param lsnr Instance of listener to add.
+     */
+    public void addConfigurationChangeListener(BiConsumer<String, File> lsnr) {
+        assert chgLock.isWriteLockedByCurrentThread();
+
+        lsnrs.add(lsnr);
+    }
+
+    /**
+     * @param lsnr Instance of listener to remove.
+     */
+    public void removeConfigurationChangeListener(BiConsumer<String, File> lsnr) {
+        lsnrs.remove(lsnr);
+    }
+
+    /**
+     * Delete caches' configuration data files of cache group.
+     *
+     * @param ctx Cache group context.
+     * @throws IgniteCheckedException If fails.
+     */
+    public void removeCacheGroupConfigurationData(CacheGroupContext ctx) throws IgniteCheckedException {
+        File cacheGrpDir = cacheWorkDir(ctx.sharedGroup(), ctx.cacheOrGroupName());
+
+        if (cacheGrpDir != null && cacheGrpDir.exists()) {
+            DirectoryStream.Filter<Path> cacheCfgFileFilter = new DirectoryStream.Filter<Path>() {
+                @Override public boolean accept(Path path) {
+                    return Files.isRegularFile(path) && path.getFileName().toString().endsWith(CACHE_DATA_FILENAME);
+                }
+            };
+
+            try (DirectoryStream<Path> dirStream = newDirectoryStream(cacheGrpDir.toPath(), cacheCfgFileFilter)) {
+                for (Path path: dirStream)
+                    Files.deleteIfExists(path);
+            }
+            catch (IOException e) {
+                throw new IgniteCheckedException("Failed to delete cache configurations of group: " + ctx.toString(), e);
+            }
+        }
+    }
+
+    /**
+     * @param grpDir Group directory.
+     * @param ccfgs Cache configurations.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void readCacheGroupCaches(File grpDir, Map<String, StoredCacheData> ccfgs) throws IgniteCheckedException {
+        File[] files = grpDir.listFiles();
+
+        if (files == null)
+            return;
+
+        for (File file : files) {
+            if (!file.isDirectory() && file.getName().endsWith(CACHE_DATA_FILENAME) && file.length() > 0) {
+                StoredCacheData cacheData = readCacheData(file);
+
+                String cacheName = cacheData.config().getName();
+
+                if (!ccfgs.containsKey(cacheName))
+                    ccfgs.put(cacheName, cacheData);
+                else {
+                    U.warn(log, "Cache with name=" + cacheName + " is already registered, skipping config file "
+                        + file.getName() + " in group directory " + grpDir.getName());
+                }
+            }
+        }
+    }
+
+    /**
+     * @param ccfg Cache configuration.
+     * @return Cache configuration file with respect to {@link CacheConfiguration#getGroupName} value.
+     */
+    private File cacheConfigurationFile(CacheConfiguration<?, ?> ccfg) {
+        File cacheWorkDir = cacheWorkDir(ccfg);
+
+        return ccfg.getGroupName() == null ? new File(cacheWorkDir, CACHE_DATA_FILENAME) :
+            new File(cacheWorkDir, ccfg.getName() + CACHE_DATA_FILENAME);
+    }
+
+    /**
+     * @param ccfg Cache configuration.
+     * @return Store dir for given cache.
+     */
+    private File cacheWorkDir(CacheConfiguration<?, ?> ccfg) {
+        return FilePageStoreManager.cacheWorkDir(storeWorkDir, FilePageStoreManager.cacheDirName(ccfg));
+    }
+
+    /**
+     * @param isSharedGroup {@code True} if cache is sharing the same `underlying` cache.
+     * @param cacheOrGroupName Cache name.
+     * @return Store directory for given cache.
+     */
+    public File cacheWorkDir(boolean isSharedGroup, String cacheOrGroupName) {
+        return FilePageStoreManager.cacheWorkDir(storeWorkDir, FilePageStoreManager.cacheDirName(isSharedGroup, cacheOrGroupName));
+    }
+
     /**
      * @return {@code True} if need locally start all existing caches on client node start.
      */
@@ -168,7 +499,7 @@ public class GridLocalConfigManager {
         }
 
         if (CU.isPersistenceEnabled(config) && pageStoreManager != null) {
-            Map<String, StoredCacheData> storedCaches = pageStoreManager.readCacheConfigurations();
+            Map<String, StoredCacheData> storedCaches = readCacheConfigurations();
 
             if (!F.isEmpty(storedCaches)) {
                 List<String> skippedConfigs = new ArrayList<>();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 5d031e78874..28d58ab5dde 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -1573,6 +1573,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
         for (IgniteBiTuple<CacheGroupContext, Boolean> tup : stoppedGrps) {
             CacheGroupContext gctx = tup.get1();
+
             boolean destroy = tup.get2();
 
             int grpId = gctx.groupId();
@@ -1624,7 +1625,12 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 CacheGroupContext grp = tup.get1();
 
                 try {
-                    cctx.pageStore().shutdownForCacheGroup(grp, tup.get2());
+                    boolean destroy = tup.get2();
+
+                    cctx.pageStore().shutdownForCacheGroup(grp, destroy);
+
+                    if (destroy)
+                        cctx.cache().configManager().removeCacheGroupConfigurationData(grp);
                 }
                 catch (IgniteCheckedException e) {
                     U.error(log, "Failed to gracefully clean page store resources for destroyed cache " +
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
index 85cfaa84162..aa332e2ba7c 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
@@ -17,14 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.persistence.file;
 
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
 import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
 import java.nio.ByteBuffer;
 import java.nio.file.DirectoryStream;
 import java.nio.file.FileSystems;
@@ -37,16 +31,13 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.function.BiConsumer;
 import java.util.function.BiFunction;
 import java.util.function.Function;
 import java.util.function.Predicate;
@@ -70,7 +61,6 @@ import org.apache.ignite.internal.pagemem.store.PageStoreCollection;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
-import org.apache.ignite.internal.processors.cache.StoredCacheData;
 import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.persistence.StorageException;
@@ -89,8 +79,6 @@ import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.lang.IgniteOutClosure;
 import org.apache.ignite.maintenance.MaintenanceRegistry;
 import org.apache.ignite.maintenance.MaintenanceTask;
-import org.apache.ignite.marshaller.Marshaller;
-import org.apache.ignite.marshaller.MarshallerUtils;
 import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
@@ -156,15 +144,6 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
     /** Unique name for corrupted data files maintenance task. */
     public static final String CORRUPTED_DATA_FILES_MNTC_TASK_NAME = "corrupted-cache-data-files-task";
 
-    /** Listeners of configuration changes e.g. overwrite or remove actions. */
-    private final List<BiConsumer<String, File>> lsnrs = new CopyOnWriteArrayList<>();
-
-    /** Lock which guards configuration changes. */
-    private final ReentrantReadWriteLock chgLock = new ReentrantReadWriteLock();
-
-    /** Marshaller. */
-    private final Marshaller marshaller;
-
     /** Page manager. */
     private final PageReadWriteManager pmPageMgr;
 
@@ -224,8 +203,6 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
 
         pageStoreV1FileIoFactory = pageStoreFileIoFactory = dsCfg.getFileIOFactory();
 
-        marshaller = MarshallerUtils.jdkMarshaller(ctx.igniteInstanceName());
-
         pmPageMgr = new PageReadWriteManagerImpl(ctx, this, FilePageStoreManager.class.getSimpleName());
     }
 
@@ -476,17 +453,17 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
     }
 
     /** {@inheritDoc} */
-    @Override public void initializeForCache(CacheGroupDescriptor grpDesc, StoredCacheData cacheData) throws IgniteCheckedException {
+    @Override public void initializeForCache(CacheGroupDescriptor grpDesc, CacheConfiguration<?, ?> ccfg) throws IgniteCheckedException {
         assert storeWorkDir != null;
 
         int grpId = grpDesc.groupId();
 
         if (!idxCacheStores.containsKey(grpId)) {
-            CacheStoreHolder holder = initForCache(grpDesc, cacheData.config());
+            CacheStoreHolder holder = initForCache(grpDesc, ccfg);
 
             CacheStoreHolder old = idxCacheStores.put(grpId, holder);
 
-            assert old == null : "Non-null old store holder for cache: " + cacheData.config().getName();
+            assert old == null : "Non-null old store holder for cache: " + ccfg.getName();
         }
     }
 
@@ -513,65 +490,6 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
         }
     }
 
-    /** {@inheritDoc} */
-    @Override public void storeCacheData(StoredCacheData cacheData, boolean overwrite) throws IgniteCheckedException {
-        CacheConfiguration<?, ?> ccfg = cacheData.config();
-        File cacheWorkDir = cacheWorkDir(ccfg);
-
-        checkAndInitCacheWorkDir(cacheWorkDir);
-
-        assert cacheWorkDir.exists() : "Work directory does not exist: " + cacheWorkDir;
-
-        File file = cacheConfigurationFile(ccfg);
-        Path filePath = file.toPath();
-
-        chgLock.readLock().lock();
-
-        try {
-            if (overwrite || !Files.exists(filePath) || Files.size(filePath) == 0) {
-                File tmp = new File(file.getParent(), file.getName() + TMP_SUFFIX);
-
-                if (tmp.exists() && !tmp.delete()) {
-                    log.warning("Failed to delete temporary cache config file" +
-                            "(make sure Ignite process has enough rights):" + file.getName());
-                }
-
-                writeCacheData(cacheData, tmp);
-
-                if (Files.exists(filePath) && Files.size(filePath) > 0) {
-                    for (BiConsumer<String, File> lsnr : lsnrs)
-                        lsnr.accept(ccfg.getName(), file);
-                }
-
-                Files.move(tmp.toPath(), file.toPath(), StandardCopyOption.REPLACE_EXISTING);
-            }
-        }
-        catch (IOException ex) {
-            cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, ex));
-
-            throw new IgniteCheckedException("Failed to persist cache configuration: " + ccfg.getName(), ex);
-        }
-        finally {
-            chgLock.readLock().unlock();
-        }
-    }
-
-    /**
-     * @param lsnr Instance of listener to add.
-     */
-    public void addConfigurationChangeListener(BiConsumer<String, File> lsnr) {
-        assert chgLock.isWriteLockedByCurrentThread();
-
-        lsnrs.add(lsnr);
-    }
-
-    /**
-     * @param lsnr Instance of listener to remove.
-     */
-    public void removeConfigurationChangeListener(BiConsumer<String, File> lsnr) {
-        lsnrs.remove(lsnr);
-    }
-
     /** {@inheritDoc} */
     @Override public void shutdownForCacheGroup(CacheGroupContext grp, boolean destroy) throws IgniteCheckedException {
         grpsWithoutIdx.remove(grp.groupId());
@@ -581,9 +499,6 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
         if (old != null) {
             IgniteCheckedException ex = shutdown(old, /*clean files if destroy*/destroy, null);
 
-            if (destroy)
-                removeCacheGroupConfigurationData(grp);
-
             if (ex != null)
                 throw ex;
         }
@@ -948,84 +863,6 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
         return store.pages();
     }
 
-    /**
-     * @param ccfgs List of cache configurations to process.
-     * @param ccfgCons Consumer which accepts found configurations files.
-     */
-    public void readConfigurationFiles(List<CacheConfiguration<?, ?>> ccfgs,
-        BiConsumer<CacheConfiguration<?, ?>, File> ccfgCons) {
-        chgLock.writeLock().lock();
-
-        try {
-            for (CacheConfiguration<?, ?> ccfg : ccfgs) {
-                File cacheDir = cacheWorkDir(ccfg);
-
-                if (!cacheDir.exists())
-                    continue;
-
-                File[] ccfgFiles = cacheDir.listFiles((dir, name) ->
-                    name.endsWith(CACHE_DATA_FILENAME));
-
-                if (ccfgFiles == null)
-                    continue;
-
-                for (File ccfgFile : ccfgFiles)
-                    ccfgCons.accept(ccfg, ccfgFile);
-            }
-        }
-        finally {
-            chgLock.writeLock().unlock();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<String, StoredCacheData> readCacheConfigurations() throws IgniteCheckedException {
-        if (cctx.kernalContext().clientNode())
-            return Collections.emptyMap();
-
-        File[] files = storeWorkDir.listFiles();
-
-        if (files == null)
-            return Collections.emptyMap();
-
-        Map<String, StoredCacheData> ccfgs = new HashMap<>();
-
-        Arrays.sort(files);
-
-        for (File file : files) {
-            if (file.isDirectory())
-                readCacheConfigurations(file, ccfgs);
-        }
-
-        return ccfgs;
-    }
-
-    /**
-     * @param dir Cache (group) directory.
-     * @param ccfgs Cache configurations.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void readCacheConfigurations(File dir, Map<String, StoredCacheData> ccfgs) throws IgniteCheckedException {
-        if (dir.getName().startsWith(CACHE_DIR_PREFIX)) {
-            File conf = new File(dir, CACHE_DATA_FILENAME);
-
-            if (conf.exists() && conf.length() > 0) {
-                StoredCacheData cacheData = readCacheData(conf);
-
-                String cacheName = cacheData.config().getName();
-
-                if (!ccfgs.containsKey(cacheName))
-                    ccfgs.put(cacheName, cacheData);
-                else {
-                    U.warn(log, "Cache with name=" + cacheName + " is already registered, skipping config file "
-                        + dir.getName());
-                }
-            }
-        }
-        else if (dir.getName().startsWith(CACHE_GRP_DIR_PREFIX))
-            readCacheGroupCaches(dir, ccfgs);
-    }
-
     /**
      * @param dir Directory to check.
      * @param names Cache group names to filter.
@@ -1111,64 +948,6 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
             throw new IgniteException("Directory doesn't match the cache or cache group prefix: " + dir);
     }
 
-    /**
-     * @param grpDir Group directory.
-     * @param ccfgs Cache configurations.
-     * @throws IgniteCheckedException If failed.
-     */
-    private void readCacheGroupCaches(File grpDir, Map<String, StoredCacheData> ccfgs) throws IgniteCheckedException {
-        File[] files = grpDir.listFiles();
-
-        if (files == null)
-            return;
-
-        for (File file : files) {
-            if (!file.isDirectory() && file.getName().endsWith(CACHE_DATA_FILENAME) && file.length() > 0) {
-                StoredCacheData cacheData = readCacheData(file);
-
-                String cacheName = cacheData.config().getName();
-
-                if (!ccfgs.containsKey(cacheName))
-                    ccfgs.put(cacheName, cacheData);
-                else {
-                    U.warn(log, "Cache with name=" + cacheName + " is already registered, skipping config file "
-                            + file.getName() + " in group directory " + grpDir.getName());
-                }
-            }
-        }
-    }
-
-    /**
-     * @param conf File with stored cache data.
-     * @return Cache data.
-     * @throws IgniteCheckedException If failed.
-     */
-    public StoredCacheData readCacheData(File conf) throws IgniteCheckedException {
-        try (InputStream stream = new BufferedInputStream(new FileInputStream(conf))) {
-            return marshaller.unmarshal(stream, U.resolveClassLoader(igniteCfg));
-        }
-        catch (IgniteCheckedException | IOException e) {
-            throw new IgniteCheckedException("An error occurred during cache configuration loading from file [file=" +
-                conf.getAbsolutePath() + "]", e);
-        }
-    }
-
-    /**
-     * @param conf File to store cache data.
-     * @param cacheData Cache data file.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void writeCacheData(StoredCacheData cacheData, File conf) throws IgniteCheckedException {
-        // Pre-existing file will be truncated upon stream open.
-        try (OutputStream stream = new BufferedOutputStream(new FileOutputStream(conf))) {
-            marshaller.marshal(cacheData, stream);
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException("An error occurred during cache configuration writing to file [file=" +
-                conf.getAbsolutePath() + "]", e);
-        }
-    }
-
     /** {@inheritDoc} */
     @Override public boolean hasIndexStore(int grpId) {
         return !grpsWithoutIdx.contains(grpId);
@@ -1288,64 +1067,6 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
         return aggr;
     }
 
-    /**
-     * Delete caches' configuration data files of cache group.
-     *
-     * @param ctx Cache group context.
-     * @throws IgniteCheckedException If fails.
-     */
-    private void removeCacheGroupConfigurationData(CacheGroupContext ctx) throws IgniteCheckedException {
-        File cacheGrpDir = cacheWorkDir(ctx.sharedGroup(), ctx.cacheOrGroupName());
-
-        if (cacheGrpDir != null && cacheGrpDir.exists()) {
-            DirectoryStream.Filter<Path> cacheCfgFileFilter = new DirectoryStream.Filter<Path>() {
-                @Override public boolean accept(Path path) {
-                    return Files.isRegularFile(path) && path.getFileName().toString().endsWith(CACHE_DATA_FILENAME);
-                }
-            };
-
-            try (DirectoryStream<Path> dirStream = newDirectoryStream(cacheGrpDir.toPath(), cacheCfgFileFilter)) {
-                for (Path path: dirStream)
-                    Files.deleteIfExists(path);
-            }
-            catch (IOException e) {
-                throw new IgniteCheckedException("Failed to delete cache configurations of group: " + ctx.toString(), e);
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeCacheData(StoredCacheData cacheData) throws IgniteCheckedException {
-        chgLock.readLock().lock();
-
-        try {
-            CacheConfiguration<?, ?> ccfg = cacheData.config();
-            File file = cacheConfigurationFile(ccfg);
-
-            if (file.exists()) {
-                for (BiConsumer<String, File> lsnr : lsnrs)
-                    lsnr.accept(ccfg.getName(), file);
-
-                if (!file.delete())
-                    throw new IgniteCheckedException("Failed to delete cache configuration: " + ccfg.getName());
-            }
-        }
-        finally {
-            chgLock.readLock().unlock();
-        }
-    }
-
-    /**
-     * @param ccfg Cache configuration.
-     * @return Cache configuration file with respect to {@link CacheConfiguration#getGroupName} value.
-     */
-    private File cacheConfigurationFile(CacheConfiguration<?, ?> ccfg) {
-        File cacheWorkDir = cacheWorkDir(ccfg);
-
-        return ccfg.getGroupName() == null ? new File(cacheWorkDir, CACHE_DATA_FILENAME) :
-            new File(cacheWorkDir, ccfg.getName() + CACHE_DATA_FILENAME);
-    }
-
     /**
      * @param store Store to shutdown.
      * @param cleanFile {@code True} if files should be cleaned.
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 8e68788a034..ed560bb58c3 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
@@ -112,6 +112,7 @@ import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.CacheType;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
+import org.apache.ignite.internal.processors.cache.GridLocalConfigManager;
 import org.apache.ignite.internal.processors.cache.StoredCacheData;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware;
@@ -365,6 +366,9 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter
     /** File store manager to create page store for restore. */
     private volatile FilePageStoreManager storeMgr;
 
+    /** File store manager to create page store for restore. */
+    private volatile GridLocalConfigManager locCfgMgr;
+
     /** System discovery message listener. */
     private DiscoveryEventListener discoLsnr;
 
@@ -445,6 +449,7 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter
         assert cctx.pageStore() instanceof FilePageStoreManager;
 
         storeMgr = (FilePageStoreManager)cctx.pageStore();
+        locCfgMgr = cctx.cache().configManager();
 
         pdsSettings = cctx.kernalContext().pdsFolderResolver().resolveFolders();
 
@@ -3079,7 +3084,7 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter
 
                 copy(ioFactory, ccfg, targetCacheCfg, ccfg.length());
 
-                StoredCacheData cacheData = storeMgr.readCacheData(targetCacheCfg);
+                StoredCacheData cacheData = locCfgMgr.readCacheData(targetCacheCfg);
 
                 if (cacheData.config().isEncryptionEnabled()) {
                     EncryptionSpi encSpi = cctx.kernalContext().config().getEncryptionSpi();
@@ -3088,7 +3093,7 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter
 
                     cacheData.groupKeyEncrypted(new GroupKeyEncrypted(gKey.id(), encSpi.encryptKey(gKey.key())));
 
-                    storeMgr.writeCacheData(cacheData, targetCacheCfg);
+                    locCfgMgr.writeCacheData(cacheData, targetCacheCfg);
                 }
             }
             catch (IgniteCheckedException e) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFutureTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFutureTask.java
index 97f7d72f3b5..81d5f52644d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFutureTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFutureTask.java
@@ -417,7 +417,7 @@ class SnapshotFutureTask extends AbstractSnapshotFutureTask<Set<GroupPartitionId
                     MetaStorage.METASTORAGE_DIR_NAME);
             }
 
-            pageStore.readConfigurationFiles(ccfgs,
+            cctx.cache().configManager().readConfigurationFiles(ccfgs,
                 (ccfg, ccfgFile) -> ccfgSndrs.add(new CacheConfigurationSender(ccfg.getName(),
                     FilePageStoreManager.cacheDirName(ccfg), ccfgFile)));
         }
@@ -681,7 +681,7 @@ class SnapshotFutureTask extends AbstractSnapshotFutureTask<Set<GroupPartitionId
             this.cacheDirName = cacheDirName;
             this.ccfgFile = ccfgFile;
 
-            pageStore.addConfigurationChangeListener(this);
+            cctx.cache().configManager().addConfigurationChangeListener(this);
         }
 
         /**
@@ -743,7 +743,7 @@ class SnapshotFutureTask extends AbstractSnapshotFutureTask<Set<GroupPartitionId
         /** Close writer and remove listener. */
         private void close0() {
             sent = true;
-            pageStore.removeConfigurationChangeListener(this);
+            cctx.cache().configManager().removeConfigurationChangeListener(this);
 
             if (fromTemp)
                 U.delete(ccfgFile);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java
index 76e1f456c23..f6c2b78f3a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java
@@ -306,7 +306,7 @@ public class SnapshotPartitionsVerifyHandler implements SnapshotHandler<Map<Part
                 try (DirectoryStream<Path> ds = Files.newDirectoryStream(grpDirs.get(grpId).toPath(),
                     p -> Files.isRegularFile(p) && p.toString().endsWith(CACHE_DATA_FILENAME))) {
                     for (Path p : ds) {
-                        StoredCacheData cacheData = ((FilePageStoreManager)ctx.cache().context().pageStore()).readCacheData(p.toFile());
+                        StoredCacheData cacheData = ctx.cache().configManager().readCacheData(p.toFile());
 
                         GroupKeyEncrypted grpKeyEncrypted = cacheData.groupKeyEncrypted();
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreProcess.java
index b707edfebfa..01489646426 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreProcess.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreProcess.java
@@ -61,6 +61,7 @@ import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.managers.discovery.DiscoCache;
 import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.GridLocalConfigManager;
 import org.apache.ignite.internal.processors.cache.StoredCacheData;
 import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
 import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.ClusterSnapshotFuture;
@@ -660,6 +661,7 @@ public class SnapshotRestoreProcess {
 
         Map<String, StoredCacheData> cfgsByName = new HashMap<>();
         FilePageStoreManager pageStore = (FilePageStoreManager)cctx.pageStore();
+        GridLocalConfigManager locCfgMgr = cctx.cache().configManager();
 
         // Collect the cache configurations and prepare a temporary directory for copying files.
         // Metastorage can be restored only manually by directly copying files.
@@ -698,7 +700,7 @@ public class SnapshotRestoreProcess {
                         "[group=" + grpName + ", dir=" + tmpCacheDir + ']');
                 }
 
-                pageStore.readCacheConfigurations(snpCacheDir, cfgsByName);
+                locCfgMgr.readCacheConfigurations(snpCacheDir, cfgsByName);
             }
         }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
index f96c2eeac8e..1d5f55b17e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
@@ -1174,7 +1174,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
 
         if (activate(curState.state(), state) && !inMemoryMode) {
             try {
-                Map<String, StoredCacheData> cfgs = ctx.cache().context().pageStore().readCacheConfigurations();
+                Map<String, StoredCacheData> cfgs = ctx.cache().configManager().readCacheConfigurations();
 
                 if (!F.isEmpty(cfgs)) {
                     storedCfgs = new ArrayList<>(cfgs.values());
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheConfigurationFileConsistencyCheckTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheConfigurationFileConsistencyCheckTest.java
index 54d930c0b29..05d858088ac 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheConfigurationFileConsistencyCheckTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheConfigurationFileConsistencyCheckTest.java
@@ -35,6 +35,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.GridLocalConfigManager;
 import org.apache.ignite.internal.processors.cache.StoredCacheData;
 import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
 import org.apache.ignite.marshaller.Marshaller;
@@ -191,15 +192,12 @@ public class IgnitePdsCacheConfigurationFileConsistencyCheckTest extends GridCom
         for (int i = 0; i < NODES; i++) {
             IgniteEx ig = grid(i);
 
-            GridCacheSharedContext sharedCtx = ig.context().cache().context();
-
-            FilePageStoreManager pageStore = (FilePageStoreManager)sharedCtx.pageStore();
-
             StoredCacheData corrData = cacheDescr.toStoredData(ig.context().cache().splitter());
 
             corrData.config().setGroupName(ODD_GROUP_NAME);
 
-            pageStore.storeCacheData(corrData, true);
+            GridTestUtils.<GridLocalConfigManager>getFieldValue(ig.context().cache(), "locCfgMgr")
+                .saveCacheConfiguration(corrData, true);
         }
     }
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
index ffd8c923876..33d6f11c41b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
@@ -18,8 +18,6 @@
 package org.apache.ignite.internal.processors.cache.persistence.pagemem;
 
 import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -34,7 +32,6 @@ import org.apache.ignite.internal.pagemem.store.PageStore;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
-import org.apache.ignite.internal.processors.cache.StoredCacheData;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.lang.IgniteFuture;
 
@@ -62,8 +59,7 @@ public class NoOpPageStoreManager implements IgnitePageStoreManager {
     }
 
     /** {@inheritDoc} */
-    @Override public void initializeForCache(CacheGroupDescriptor grpDesc,
-        StoredCacheData cacheData) throws IgniteCheckedException {
+    @Override public void initializeForCache(CacheGroupDescriptor grpDesc, CacheConfiguration<?, ?> ccfg) throws IgniteCheckedException {
         // No-op.
     }
 
@@ -187,21 +183,6 @@ public class NoOpPageStoreManager implements IgnitePageStoreManager {
         // No-op.
     }
 
-    /** {@inheritDoc} */
-    @Override public Map<String, StoredCacheData> readCacheConfigurations() throws IgniteCheckedException {
-        return Collections.emptyMap();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void storeCacheData(StoredCacheData cacheData, boolean overwrite) throws IgniteCheckedException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeCacheData(StoredCacheData cacheData) throws IgniteCheckedException {
-        // No-op.
-    }
-
     /** {@inheritDoc} */
     @Override public boolean hasIndexStore(int grpId) {
         return false;