IGNITE-15146 Handle lifecycle properly for the standalone context (#9273)

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 50837b9..f9ea6a5 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
@@ -29,6 +29,7 @@
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import java.nio.channels.FileChannel;
+import java.nio.file.DirectoryStream;
 import java.nio.file.FileVisitResult;
 import java.nio.file.Files;
 import java.nio.file.Path;
@@ -549,6 +550,20 @@
     }
 
     /**
+     * @param snpName Snapshot name.
+     * @return Local snapshot directory for snapshot with given name.
+     * @throws IgniteCheckedException If directory doesn't exists.
+     */
+    private File resolveSnapshotDir(String snpName) throws IgniteCheckedException {
+        File snpDir = snapshotLocalDir(snpName);
+
+        if (!snpDir.exists())
+            throw new IgniteCheckedException("Snapshot directory doesn't exists: " + snpDir.getAbsolutePath());
+
+        return snpDir;
+    }
+
+    /**
      * @return Node snapshot working directory.
      */
     public File snapshotTmpDir() {
@@ -1087,11 +1102,20 @@
         A.notNullOrEmpty(snpName, "Snapshot name cannot be null or empty.");
         A.ensure(U.alphanumericUnderscore(snpName), "Snapshot name must satisfy the following name pattern: a-zA-Z0-9_");
 
-        File[] smfs = snapshotLocalDir(snpName).listFiles((dir, name) ->
-            name.toLowerCase().endsWith(SNAPSHOT_METAFILE_EXT));
+        List<File> smfs = new ArrayList<>();
 
-        if (smfs == null)
-            throw new IgniteException("Snapshot directory doesn't exists or an I/O error occurred during directory read.");
+        try (DirectoryStream<Path> ds = Files.newDirectoryStream(snapshotLocalDir(snpName).toPath())) {
+            for (Path d : ds) {
+                if (Files.isRegularFile(d) && d.getFileName().toString().toLowerCase().endsWith(SNAPSHOT_METAFILE_EXT))
+                    smfs.add(d.toFile());
+            }
+        }
+        catch (IOException e) {
+            throw new IgniteException(e);
+        }
+
+        if (smfs.isEmpty())
+            throw new IgniteException("Snapshot metadata files not found: " + snpName);
 
         Map<String, SnapshotMetadata> metasMap = new HashMap<>();
         SnapshotMetadata prev = null;
@@ -1341,20 +1365,55 @@
     /**
      * @param snpName Snapshot name.
      * @param folderName The node folder name, usually it's the same as the U.maskForFileName(consistentId).
+     * @return Standalone kernal context related to the snapshot.
+     * @throws IgniteCheckedException If fails.
+     */
+    public StandaloneGridKernalContext createStandaloneKernalContext(String snpName, String folderName) throws IgniteCheckedException {
+        File snpDir = resolveSnapshotDir(snpName);
+
+        return new StandaloneGridKernalContext(log,
+            resolveBinaryWorkDir(snpDir.getAbsolutePath(), folderName),
+            resolveMappingFileStoreWorkDir(snpDir.getAbsolutePath()));
+    }
+
+    /**
+     * @param grpName Cache group name.
+     * @param partId Partition id.
+     * @param pageStore File page store to iterate over.
+     * @return Iterator over partition.
+     * @throws IgniteCheckedException If and error occurs.
+     */
+    public GridCloseableIterator<CacheDataRow> partitionRowIterator(GridKernalContext ctx,
+        String grpName,
+        int partId,
+        FilePageStore pageStore
+    ) throws IgniteCheckedException {
+        CacheObjectContext coctx = new CacheObjectContext(ctx, grpName, null, false,
+            false, false, false, false);
+
+        GridCacheSharedContext<?, ?> sctx = new GridCacheSharedContext<>(ctx, null, null, null,
+            null, null, null, null, null, null,
+            null, null, null, null, null,
+            null, null, null, null, null, null);
+
+        return new DataPageIterator(sctx, coctx, pageStore, partId);
+    }
+
+    /**
+     * @param snpName Snapshot name.
+     * @param folderName The node folder name, usually it's the same as the U.maskForFileName(consistentId).
      * @param grpName Cache group name.
      * @param partId Partition id.
      * @return Iterator over partition.
      * @throws IgniteCheckedException If and error occurs.
      */
-    public GridCloseableIterator<CacheDataRow> partitionRowIterator(String snpName,
+    public GridCloseableIterator<CacheDataRow> partitionRowIterator(GridKernalContext ctx,
+        String snpName,
         String folderName,
         String grpName,
         int partId
     ) throws IgniteCheckedException {
-        File snpDir = snapshotLocalDir(snpName);
-
-        if (!snpDir.exists())
-            throw new IgniteCheckedException("Snapshot directory doesn't exists: " + snpDir.getAbsolutePath());
+        File snpDir = resolveSnapshotDir(snpName);
 
         File nodePath = new File(snpDir, databaseRelativePath(folderName));
 
@@ -1386,19 +1445,24 @@
                 val -> {
                 });
 
-        GridKernalContext kctx = new StandaloneGridKernalContext(log,
-            resolveBinaryWorkDir(snpDir.getAbsolutePath(), folderName),
-            resolveMappingFileStoreWorkDir(snpDir.getAbsolutePath()));
+        GridCloseableIterator<CacheDataRow> partIter = partitionRowIterator(ctx, grpName, partId, pageStore);
 
-        CacheObjectContext coctx = new CacheObjectContext(kctx, grpName, null, false,
-            false, false, false, false);
+        return new GridCloseableIteratorAdapter<CacheDataRow>() {
+            /** {@inheritDoc} */
+            @Override protected CacheDataRow onNext() throws IgniteCheckedException {
+                return partIter.nextX();
+            }
 
-        GridCacheSharedContext<?, ?> sctx = new GridCacheSharedContext<>(kctx, null, null, null,
-            null, null, null, null, null, null,
-            null, null, null, null, null,
-            null, null, null, null, null, null);
+            /** {@inheritDoc} */
+            @Override protected boolean onHasNext() throws IgniteCheckedException {
+                return partIter.hasNextX();
+            }
 
-        return new DataPageIterator(sctx, coctx, pageStore, partId);
+            /** {@inheritDoc} */
+            @Override protected void onClose() {
+                U.closeQuiet(pageStore);
+            }
+        };
     }
 
     /**
@@ -1921,8 +1985,8 @@
 
                 copy(ioFactory, part, snpPart, len);
 
-                if (log.isInfoEnabled()) {
-                    log.info("Partition has been snapshot [snapshotDir=" + dbDir.getAbsolutePath() +
+                if (log.isDebugEnabled()) {
+                    log.debug("Partition has been snapshot [snapshotDir=" + dbDir.getAbsolutePath() +
                         ", cacheDirName=" + cacheDirName + ", part=" + part.getName() +
                         ", length=" + part.length() + ", snapshot=" + snpPart.getName() + ']');
                 }
@@ -1936,8 +2000,8 @@
         @Override public void sendDelta0(File delta, String cacheDirName, GroupPartitionId pair) {
             File snpPart = getPartitionFile(dbDir, cacheDirName, pair.getPartitionId());
 
-            if (log.isInfoEnabled()) {
-                log.info("Start partition snapshot recovery with the given delta page file [part=" + snpPart +
+            if (log.isDebugEnabled()) {
+                log.debug("Start partition snapshot recovery with the given delta page file [part=" + snpPart +
                     ", delta=" + delta + ']');
             }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTask.java
index 3710183..0e99e8c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTask.java
@@ -40,6 +40,8 @@
 import org.apache.ignite.compute.ComputeJobResult;
 import org.apache.ignite.compute.ComputeJobResultPolicy;
 import org.apache.ignite.compute.ComputeTaskAdapter;
+import org.apache.ignite.internal.GridComponent;
+import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStore;
@@ -239,17 +241,22 @@
                 .order(ByteOrder.nativeOrder()));
 
             try {
-                U.doInParallel(
-                    ignite.context().getSystemExecutorService(),
-                    partFiles,
-                    part -> {
-                        String grpName = cacheGroupName(part.getParentFile());
-                        int grpId = CU.cacheId(grpName);
-                        int partId = partId(part.getName());
+                GridKernalContext snpCtx = snpMgr.createStandaloneKernalContext(snpName, meta.folderName());
 
-                        FilePageStoreManager storeMgr = (FilePageStoreManager)ignite.context().cache().context().pageStore();
+                for (GridComponent comp : snpCtx)
+                    comp.start();
 
-                        try {
+                try {
+                    U.doInParallel(
+                        snpMgr.snapshotExecutorService(),
+                        partFiles,
+                        part -> {
+                            String grpName = cacheGroupName(part.getParentFile());
+                            int grpId = CU.cacheId(grpName);
+                            int partId = partId(part.getName());
+
+                            FilePageStoreManager storeMgr = (FilePageStoreManager)ignite.context().cache().context().pageStore();
+
                             try (FilePageStore pageStore = (FilePageStore)storeMgr.getPageStoreFactory(grpId, false)
                                 .createPageStore(getTypeByPartId(partId),
                                     part::toPath,
@@ -302,20 +309,24 @@
                                     GridDhtPartitionState.OWNING,
                                     false,
                                     size,
-                                    snpMgr.partitionRowIterator(snpName, meta.folderName(), grpName, partId));
+                                    snpMgr.partitionRowIterator(snpCtx, grpName, partId, pageStore));
 
                                 assert hash != null : "OWNING must have hash: " + key;
 
                                 res.put(key, hash);
                             }
-                        }
-                        catch (IOException e) {
-                            throw new IgniteCheckedException(e);
-                        }
+                            catch (IOException e) {
+                                throw new IgniteCheckedException(e);
+                            }
 
-                        return null;
-                    }
-                );
+                            return null;
+                        }
+                    );
+                }
+                finally {
+                    for (GridComponent comp : snpCtx)
+                        comp.stop(true);
+                }
             }
             catch (IgniteCheckedException e) {
                 throw new IgniteException(e);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
index 68fc4324..9bbab22 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
@@ -34,6 +34,7 @@
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.internal.GridComponent;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.pagemem.wal.WALIterator;
 import org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType;
@@ -172,10 +173,20 @@
     ) throws IgniteCheckedException, IllegalArgumentException {
         iteratorParametersBuilder.validate();
 
+        GridCacheSharedContext<?, ?> sctx;
+
+        if (iteratorParametersBuilder.sharedCtx == null) {
+            sctx = prepareSharedCtx(iteratorParametersBuilder);
+
+            for (GridComponent comp : sctx.kernalContext())
+                comp.start();
+        }
+        else
+            sctx = iteratorParametersBuilder.sharedCtx;
+
         return new StandaloneWalRecordsIterator(
             iteratorParametersBuilder.log == null ? log : iteratorParametersBuilder.log,
-            iteratorParametersBuilder.sharedCtx == null ? prepareSharedCtx(iteratorParametersBuilder) :
-                iteratorParametersBuilder.sharedCtx,
+            sctx,
             iteratorParametersBuilder.ioFactory,
             resolveWalFiles(iteratorParametersBuilder),
             iteratorParametersBuilder.filter,
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
index 2951f1e..5c32442 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
@@ -19,7 +19,9 @@
 
 import java.io.File;
 import java.lang.reflect.Field;
+import java.util.Collections;
 import java.util.Iterator;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
@@ -109,12 +111,18 @@
 import org.jetbrains.annotations.Nullable;
 
 /**
- * Dummy grid kernal context
+ * Dummy context for offline utilities. All grid components registered in the standalone context
+ * must be properly stopped since the lifecycle of them are controlled by kernal.
+ *
+ * @see org.apache.ignite.internal.GridComponent#stop(boolean)
  */
 public class StandaloneGridKernalContext implements GridKernalContext {
     /** Config for fake Ignite instance. */
     private final IgniteConfiguration cfg;
 
+    /** List of registered components. */
+    private final List<GridComponent> comps = new LinkedList<>();
+
     /** Logger. */
     private IgniteLogger log;
 
@@ -170,6 +178,8 @@
 
         this.cacheObjProcessor = binaryProcessor(this, binaryMetadataFileStoreDir);
 
+        comps.add(cacheObjProcessor);
+
         if (marshallerMappingFileStoreDir != null) {
             marshallerCtx.setMarshallerMappingFileStoreDir(marshallerMappingFileStoreDir);
             marshallerCtx.onMarshallerProcessorStarted(this, null);
@@ -185,15 +195,14 @@
      * {@code null} means no specific folder is configured. <br> In this case folder for metadata is composed from work
      * directory and consistentId
      * @return Cache object processor able to restore data records content into binary objects
-     * @throws IgniteCheckedException Throws in case of initialization errors.
      */
     private IgniteCacheObjectProcessor binaryProcessor(
         final GridKernalContext ctx,
-        final File binaryMetadataFileStoreDir) throws IgniteCheckedException {
+        final File binaryMetadataFileStoreDir) {
 
         final CacheObjectBinaryProcessorImpl processor = new CacheObjectBinaryProcessorImpl(ctx);
         processor.setBinaryMetadataFileStoreDir(binaryMetadataFileStoreDir);
-        processor.start();
+
         return processor;
     }
 
@@ -226,7 +235,7 @@
 
     /** {@inheritDoc} */
     @Override public List<GridComponent> components() {
-        return null;
+        return Collections.unmodifiableList(comps);
     }
 
     /** {@inheritDoc} */
@@ -709,7 +718,7 @@
 
     /** {@inheritDoc} */
     @NotNull @Override public Iterator<GridComponent> iterator() {
-        return null;
+        return comps.iterator();
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
index e7c18ad..74c7a69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
@@ -24,6 +24,7 @@
 import java.util.stream.Collectors;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridComponent;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.pagemem.wal.record.DataEntry;
 import org.apache.ignite.internal.pagemem.wal.record.DataRecord;
@@ -518,7 +519,8 @@
 
         curWalSegmIdx = Integer.MAX_VALUE;
 
-        sharedCtx.kernalContext().cacheObjects().stop(true);
+        for (GridComponent comp : sharedCtx.kernalContext())
+            comp.stop(true);
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java
index e38bed7..c6a3055 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java
@@ -48,7 +48,6 @@
 import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.cluster.ClusterState;
@@ -99,9 +98,6 @@
     /** Default snapshot name. */
     protected static final String SNAPSHOT_NAME = "testSnapshot";
 
-    /** Default number of partitions for cache. */
-    protected static final int CACHE_PARTS_COUNT = 8;
-
     /** Number of cache keys to pre-create at node start. */
     protected static final int CACHE_KEYS_RANGE = 1024;
 
@@ -191,8 +187,7 @@
     protected <K, V> CacheConfiguration<K, V> txCacheConfig(CacheConfiguration<K, V> ccfg) {
         return ccfg.setCacheMode(CacheMode.PARTITIONED)
             .setBackups(2)
-            .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL)
-            .setAffinity(new RendezvousAffinityFunction(false, CACHE_PARTS_COUNT));
+            .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
     }
 
     /**
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java
index ae08924..c57ce1a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java
@@ -529,6 +529,31 @@
         assertContains(log, b.toString(), "Failed to read page (CRC validation failed)");
     }
 
+    /** @throws Exception If fails. */
+    @Test
+    public void testClusterSnapshotCheckMultipleTimes() throws Exception {
+        IgniteEx ignite = startGridsWithCache(3, dfltCacheCfg, CACHE_KEYS_RANGE);
+
+        startClientGrid();
+
+        ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get();
+
+        int iterations = 10;
+
+        // Warmup.
+        for (int i = 0; i < iterations; i++)
+            snp(ignite).checkSnapshot(SNAPSHOT_NAME).get();
+
+        int activeThreadsCntBefore = Thread.activeCount();
+
+        for (int i = 0; i < iterations; i++)
+            snp(ignite).checkSnapshot(SNAPSHOT_NAME).get();
+
+        int createdThreads = Thread.activeCount() - activeThreadsCntBefore;
+
+        assertTrue("Threads created: " + createdThreads, createdThreads < iterations);
+    }
+
     /**
      * @param cls Class of running task.
      * @param results Results of compute.
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java
index 139b9a3..c4d6a7e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java
@@ -42,6 +42,7 @@
 import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.cache.CacheExistsException;
 import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -547,7 +548,8 @@
     /** @throws Exception If failed. */
     @Test
     public void testNodeFailDuringFilesCopy() throws Exception {
-        dfltCacheCfg.setCacheMode(CacheMode.REPLICATED);
+        dfltCacheCfg.setCacheMode(CacheMode.REPLICATED)
+            .setAffinity(new RendezvousAffinityFunction());
 
         startGridsWithSnapshot(3, CACHE_KEYS_RANGE);
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotSelfTest.java
index 03c7af6..c0946b4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotSelfTest.java
@@ -43,7 +43,6 @@
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -107,8 +106,7 @@
     /** Cache configuration for test. */
     private static final CacheConfiguration<Integer, Integer> atomicCcfg = new CacheConfiguration<Integer, Integer>("atomicCacheName")
         .setAtomicityMode(CacheAtomicityMode.ATOMIC)
-        .setBackups(2)
-        .setAffinity(new RendezvousAffinityFunction(false, CACHE_PARTS_COUNT));
+        .setBackups(2);
 
     /** {@code true} if node should be started in separate jvm. */
     protected volatile boolean jvm;
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManagerSelfTest.java
index 2260ada..35cda7e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManagerSelfTest.java
@@ -431,7 +431,8 @@
 
         Map<Integer, Value> iterated = new HashMap<>();
 
-        try (GridCloseableIterator<CacheDataRow> iter = snp(ignite).partitionRowIterator(SNAPSHOT_NAME,
+        try (GridCloseableIterator<CacheDataRow> iter = snp(ignite).partitionRowIterator(ignite.context(),
+            SNAPSHOT_NAME,
             ignite.context().pdsFolderResolver().resolveFolders().folderName(),
             ccfg.getName(),
             0)
@@ -474,7 +475,8 @@
 
         int rows = 0;
 
-        try (GridCloseableIterator<CacheDataRow> iter = snp(ignite).partitionRowIterator(SNAPSHOT_NAME,
+        try (GridCloseableIterator<CacheDataRow> iter = snp(ignite).partitionRowIterator(ignite.context(),
+            SNAPSHOT_NAME,
             ignite.context().pdsFolderResolver().resolveFolders().folderName(),
             dfltCacheCfg.getName(),
             0)
@@ -516,7 +518,8 @@
 
         int rows = 0;
 
-        try (GridCloseableIterator<CacheDataRow> iter = snp(ignite).partitionRowIterator(SNAPSHOT_NAME,
+        try (GridCloseableIterator<CacheDataRow> iter = snp(ignite).partitionRowIterator(ignite.context(),
+            SNAPSHOT_NAME,
             ignite.context().pdsFolderResolver().resolveFolders().folderName(),
             dfltCacheCfg.getName(),
             0)