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 2166e3aa7ec IGNITE-24421 Move checkpoint dir to NodeFileTree (#11854)
2166e3aa7ec is described below
commit 2166e3aa7ec2652fd77a27039561600ab7612469
Author: Nikolay <[email protected]>
AuthorDate: Thu Feb 6 16:41:48 2025 +0300
IGNITE-24421 Move checkpoint dir to NodeFileTree (#11854)
---
.../jol/FileStoreHeapUtilizationJolBenchmark.java | 4 +-
...IgnitePersistenceCompatibilityAbstractTest.java | 3 +-
...gratingToWalV2SerializerWithCompactionTest.java | 18 ++----
.../snapshot/SnapshotCompressionBasicTest.java | 11 ++--
.../CacheObjectCompressionConsumptionTest.java | 9 ++-
.../indexreader/IgniteIndexReaderTest.java | 3 +-
.../apache/ignite/util/GridCommandHandlerTest.java | 10 ++--
.../management/persistence/PersistenceTask.java | 2 +-
.../GridCacheDatabaseSharedManager.java | 23 +++-----
.../persistence/checkpoint/CheckpointManager.java | 18 ++----
.../checkpoint/CheckpointMarkersStorage.java | 64 +++++++++++-----------
.../persistence/file/FilePageStoreManager.java | 37 +++++--------
.../cache/persistence/filename/NodeFileTree.java | 37 +++++++++++--
.../snapshot/IgniteSnapshotManager.java | 6 +-
.../persistence/snapshot/SnapshotFutureTask.java | 12 ++--
.../snapshot/SnapshotRestoreProcess.java | 6 +-
.../checkpoint/sharedfs/SharedFsCheckpointSpi.java | 5 +-
.../java/org/apache/ignite/cdc/CdcManagerTest.java | 16 +-----
.../wal/record/WALRecordSerializationTest.java | 10 +---
.../cache/WalModeChangeAdvancedSelfTest.java | 20 ++-----
.../IgnitePdsCheckpointMapSnapshotTest.java | 4 +-
.../persistence/IgnitePdsCorruptedStoreTest.java | 13 ++---
.../IgnitePdsSporadicDataRecordsOnBackupTest.java | 22 +++-----
...ocalWalModeChangeDuringRebalancingSelfTest.java | 8 +--
.../PagesPossibleCorruptionDiagnosticTest.java | 7 +--
.../WALPreloadingWithCompactionTest.java | 18 ++----
.../db/IgnitePdsDataRegionMetricsTest.java | 2 +-
.../CheckpointMarkerReadingErrorOnStartTest.java | 10 +---
.../db/checkpoint/LightweightCheckpointTest.java | 5 +-
.../persistence/db/wal/WalCompactionTest.java | 56 ++++++-------------
.../db/wal/WalDeletionArchiveAbstractTest.java | 2 +-
.../db/wal/reader/IgniteWalReaderTest.java | 62 +++++----------------
.../MaintenanceModeNodeSecurityTest.java | 9 +--
.../apache/ignite/testframework/GridTestUtils.java | 6 +-
.../junits/common/GridCommonAbstractTest.java | 14 +++--
.../utils/IgniteWalConverterSensitiveDataTest.java | 7 +--
.../file/LinuxNativeIoPluginProvider.java | 2 +-
.../persistence/db/wal/IgniteWalRecoveryTest.java | 3 +-
.../snapshot/IgniteClusterSnapshotMetricsTest.java | 6 +-
.../processors/query/h2/GridIndexRebuildTest.java | 19 ++-----
40 files changed, 233 insertions(+), 356 deletions(-)
diff --git
a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jol/FileStoreHeapUtilizationJolBenchmark.java
b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jol/FileStoreHeapUtilizationJolBenchmark.java
index 8cab20dea13..b20d5af22ee 100644
---
a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jol/FileStoreHeapUtilizationJolBenchmark.java
+++
b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jol/FileStoreHeapUtilizationJolBenchmark.java
@@ -16,6 +16,7 @@
*/
package org.apache.ignite.internal.benchmarks.jol;
+import java.io.File;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
@@ -35,6 +36,7 @@ import
org.apache.ignite.internal.processors.cache.persistence.filename.SharedFi
import org.apache.ignite.internal.util.typedef.F;
import org.apache.ignite.internal.util.typedef.G;
import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointSpi;
import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
import org.openjdk.jol.info.GraphLayout;
@@ -69,7 +71,7 @@ public class FileStoreHeapUtilizationJolBenchmark {
SharedFileTree sft = new SharedFileTree(U.defaultWorkDirectory());
- U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), "cp",
false));
+ U.delete(new File(sft.root(), SharedFsCheckpointSpi.DFLT_ROOT));
U.delete(sft.marshaller().getParentFile());
}
diff --git
a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePersistenceCompatibilityAbstractTest.java
b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePersistenceCompatibilityAbstractTest.java
index 32d4f66a196..c0cfcb87a41 100644
---
a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePersistenceCompatibilityAbstractTest.java
+++
b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePersistenceCompatibilityAbstractTest.java
@@ -23,6 +23,7 @@ import java.util.List;
import
org.apache.ignite.compatibility.testframework.junits.IgniteCompatibilityAbstractTest;
import
org.apache.ignite.compatibility.testframework.util.CompatibilityTestsUtils;
import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointSpi;
import static
org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
import static
org.apache.ignite.internal.processors.cache.persistence.filename.SharedFileTree.BINARY_METADATA_DIR;
@@ -34,7 +35,7 @@ import static
org.apache.ignite.internal.processors.cache.persistence.filename.S
public abstract class IgnitePersistenceCompatibilityAbstractTest extends
IgniteCompatibilityAbstractTest {
/** Persistence directories. */
private static final List<String> PERSISTENCE_DIRS
- = Arrays.asList(DFLT_STORE_DIR, BINARY_METADATA_DIR, "cp",
MARSHALLER_DIR);
+ = Arrays.asList(DFLT_STORE_DIR, BINARY_METADATA_DIR,
SharedFsCheckpointSpi.DFLT_ROOT, MARSHALLER_DIR);
/** {@inheritDoc} */
@Override protected void beforeTest() throws Exception {
diff --git
a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/MigratingToWalV2SerializerWithCompactionTest.java
b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/MigratingToWalV2SerializerWithCompactionTest.java
index 27e80d3801f..822b0b2563a 100644
---
a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/MigratingToWalV2SerializerWithCompactionTest.java
+++
b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/MigratingToWalV2SerializerWithCompactionTest.java
@@ -32,7 +32,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.configuration.WALMode;
import org.apache.ignite.internal.IgniteEx;
import
org.apache.ignite.internal.processors.cache.GridCacheAbstractFullApiSelfTest;
-import org.apache.ignite.internal.util.typedef.internal.U;
+import
org.apache.ignite.internal.processors.cache.persistence.filename.NodeFileTree;
import org.apache.ignite.lang.IgniteInClosure;
import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
import org.junit.Test;
@@ -122,14 +122,9 @@ public class MigratingToWalV2SerializerWithCompactionTest
extends IgnitePersiste
int expCompressedWalSegments = PAYLOAD_SIZE * ENTRIES * 4 /
WAL_SEGMENT_SIZE - 1;
- String nodeFolderName =
ignite.context().pdsFolderResolver().resolveFolders().folderName();
+ NodeFileTree ft = ignite.context().pdsFolderResolver().fileTree();
- File dbDir = U.resolveWorkDirectory(U.defaultWorkDirectory(),
"db", false);
- File walDir = new File(dbDir, "wal");
- File archiveDir = new File(walDir, "archive");
- File nodeArchiveDir = new File(archiveDir, nodeFolderName);
-
- File[] compressedSegments = nodeArchiveDir.listFiles(new
FilenameFilter() {
+ File[] compressedSegments = ft.walArchive().listFiles(new
FilenameFilter() {
@Override public boolean accept(File dir, String name) {
return name.endsWith(".wal.zip");
}
@@ -142,15 +137,12 @@ public class MigratingToWalV2SerializerWithCompactionTest
extends IgnitePersiste
stopAllGrids();
- File nodeLfsDir = new File(dbDir, nodeFolderName);
- File cpMarkersDir = new File(nodeLfsDir, "cp");
-
- File[] cpMarkers = cpMarkersDir.listFiles();
+ File[] cpMarkers = ft.checkpoint().listFiles();
assertNotNull(cpMarkers);
assertTrue(cpMarkers.length > 0);
- File cacheDir = new File(nodeLfsDir, "cache-" + TEST_CACHE_NAME);
+ File cacheDir = new File(ft.nodeStorage(), "cache-" +
TEST_CACHE_NAME);
File[] partFiles = cacheDir.listFiles(new FilenameFilter() {
@Override public boolean accept(File dir, String name) {
return name.startsWith("part");
diff --git
a/modules/compress/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCompressionBasicTest.java
b/modules/compress/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCompressionBasicTest.java
index 863f98f9222..a2e08f928eb 100644
---
a/modules/compress/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCompressionBasicTest.java
+++
b/modules/compress/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCompressionBasicTest.java
@@ -51,6 +51,7 @@ import org.apache.ignite.internal.IgniteEx;
import org.apache.ignite.internal.management.cache.IdleVerifyResult;
import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
import
org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIO;
+import
org.apache.ignite.internal.processors.cache.persistence.filename.NodeFileTree;
import org.apache.ignite.internal.processors.compress.CompressionProcessor;
import org.apache.ignite.internal.util.typedef.F;
import org.apache.ignite.internal.util.typedef.G;
@@ -288,10 +289,12 @@ public class SnapshotCompressionBasicTest extends
AbstractSnapshotSelfTest {
continue;
}
- U.delete(U.resolveWorkDirectory(dir.toString(), "cp", false));
- U.delete(U.resolveWorkDirectory(dir.toString(),
DFLT_STORE_DIR, false));
- U.delete(nodeFileTree(dir.toString()).marshaller());
- U.delete(nodeFileTree(dir.toString()).binaryMetaRoot());
+ NodeFileTree ft = nodeFileTree(dir.toString());
+
+ U.delete(ft.checkpoint());
+ U.delete(ft.nodeStorage().getParentFile());
+ U.delete(ft.marshaller());
+ U.delete(ft.binaryMetaRoot());
}
}
catch (IOException e) {
diff --git
a/modules/compress/src/test/java/org/apache/ignite/internal/processors/cache/transform/CacheObjectCompressionConsumptionTest.java
b/modules/compress/src/test/java/org/apache/ignite/internal/processors/cache/transform/CacheObjectCompressionConsumptionTest.java
index fefabd38316..ff05e7a205f 100644
---
a/modules/compress/src/test/java/org/apache/ignite/internal/processors/cache/transform/CacheObjectCompressionConsumptionTest.java
+++
b/modules/compress/src/test/java/org/apache/ignite/internal/processors/cache/transform/CacheObjectCompressionConsumptionTest.java
@@ -17,6 +17,7 @@
package org.apache.ignite.internal.processors.cache.transform;
+import java.io.File;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
@@ -40,7 +41,6 @@ import org.apache.ignite.internal.client.thin.TcpClientCache;
import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
import org.apache.ignite.internal.util.typedef.G;
import org.apache.ignite.internal.util.typedef.T3;
-import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.metric.MetricRegistry;
import org.apache.ignite.spi.communication.CommunicationSpi;
import org.apache.ignite.spi.metric.LongMetric;
@@ -49,7 +49,6 @@ import
org.apache.ignite.testframework.junits.WithSystemProperty;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
-import static
org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
import static
org.apache.ignite.internal.processors.metric.GridMetricManager.CLIENT_CONNECTOR_METRICS;
import static
org.apache.ignite.internal.util.nio.GridNioServer.RECEIVED_BYTES_METRIC_NAME;
import static
org.apache.ignite.internal.util.nio.GridNioServer.SENT_BYTES_METRIC_NAME;
@@ -361,10 +360,10 @@ public class CacheObjectCompressionConsumptionTest
extends AbstractCacheObjectCo
mem += metrics.getTotalAllocatedSize();
- String nodeFolder =
((IgniteEx)node).context().pdsFolderResolver().resolveFolders().folderName();
+ File nodeFolder =
((IgniteEx)node).context().pdsFolderResolver().fileTree().nodeStorage();
- pers += FileUtils.sizeOfDirectory(
- U.resolveWorkDirectory(U.defaultWorkDirectory(),
DFLT_STORE_DIR + "/" + nodeFolder, false));
+ if (nodeFolder != null)
+ pers += FileUtils.sizeOfDirectory(nodeFolder);
if (mode != ConsumptionTestMode.PERSISTENT)
assertEquals(0, pers);
diff --git
a/modules/control-utility/src/test/java/org/apache/ignite/internal/commandline/indexreader/IgniteIndexReaderTest.java
b/modules/control-utility/src/test/java/org/apache/ignite/internal/commandline/indexreader/IgniteIndexReaderTest.java
index 81b15f0bd10..8fa739842d4 100644
---
a/modules/control-utility/src/test/java/org/apache/ignite/internal/commandline/indexreader/IgniteIndexReaderTest.java
+++
b/modules/control-utility/src/test/java/org/apache/ignite/internal/commandline/indexreader/IgniteIndexReaderTest.java
@@ -54,7 +54,6 @@ import org.apache.ignite.internal.logger.IgniteLoggerEx;
import org.apache.ignite.internal.pagemem.PageIdAllocator;
import
org.apache.ignite.internal.processors.cache.persistence.IndexStorageImpl;
import
org.apache.ignite.internal.processors.cache.persistence.file.FilePageStore;
-import
org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
import org.apache.ignite.internal.processors.query.QueryUtils;
import org.apache.ignite.internal.util.GridStringBuilder;
@@ -228,7 +227,7 @@ public class IgniteIndexReaderTest extends
GridCommandHandlerAbstractTest {
try (IgniteEx node = startGrid(0)) {
populateData(node, null);
- workDir =
((FilePageStoreManager)node.context().cache().context().pageStore()).workDir();
+ workDir =
node.context().pdsFolderResolver().fileTree().nodeStorage();
}
}
diff --git
a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
index 8371fa7b0cf..9999a84df3f 100644
---
a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
+++
b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
@@ -103,6 +103,7 @@ import
org.apache.ignite.internal.processors.cache.persistence.db.IgniteCacheGro
import
org.apache.ignite.internal.processors.cache.persistence.diagnostic.pagelocktracker.dumpprocessors.ToFileDumpProcessor;
import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
import
org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import
org.apache.ignite.internal.processors.cache.persistence.filename.NodeFileTree;
import
org.apache.ignite.internal.processors.cache.persistence.snapshot.DataStreamerUpdatesHandler;
import
org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager;
import
org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotPartitionsVerifyTaskResult;
@@ -328,10 +329,7 @@ public class GridCommandHandlerTest extends
GridCommandHandlerClusterPerMethodAb
IgniteEx ig0 = startGrid(0);
IgniteEx ig1 = startGrid(1);
- String ig1Folder =
ig1.context().pdsFolderResolver().resolveFolders().folderName();
- File dbDir =
U.resolveWorkDirectory(ig1.configuration().getWorkDirectory(), "db", false);
-
- File ig1LfsDir = new File(dbDir, ig1Folder);
+ NodeFileTree ft1 = ig1.context().pdsFolderResolver().fileTree();
ig0.cluster().baselineAutoAdjustEnabled(false);
ig0.cluster().state(ACTIVE);
@@ -362,7 +360,7 @@ public class GridCommandHandlerTest extends
GridCommandHandlerClusterPerMethodAb
stopGrid(1);
- File[] cpMarkers = new File(ig1LfsDir, "cp").listFiles();
+ File[] cpMarkers = ft1.checkpoint().listFiles();
for (File cpMark : cpMarkers) {
if (cpMark.getName().contains("-END"))
@@ -371,7 +369,7 @@ public class GridCommandHandlerTest extends
GridCommandHandlerClusterPerMethodAb
assertThrows(log, () -> startGrid(1), Exception.class, null);
- return ig1LfsDir;
+ return ft1.nodeStorage();
}
/**
diff --git
a/modules/core/src/main/java/org/apache/ignite/internal/management/persistence/PersistenceTask.java
b/modules/core/src/main/java/org/apache/ignite/internal/management/persistence/PersistenceTask.java
index 4394e4c0af3..95ec61ac20c 100644
---
a/modules/core/src/main/java/org/apache/ignite/internal/management/persistence/PersistenceTask.java
+++
b/modules/core/src/main/java/org/apache/ignite/internal/management/persistence/PersistenceTask.java
@@ -110,7 +110,7 @@ public class PersistenceTask extends
VisorOneNodeTask<PersistenceTaskArg, Persis
MaintenanceRegistry mntcReg =
ignite.context().maintenanceRegistry();
MaintenanceTask task =
mntcReg.activeMaintenanceTask(CORRUPTED_DATA_FILES_MNTC_TASK_NAME);
- File workDir =
((FilePageStoreManager)ignite.context().cache().context().pageStore()).workDir();
+ File workDir =
ignite.context().pdsFolderResolver().fileTree().nodeStorage();
if (arg instanceof PersistenceBackupAllTaskArg)
return backupAll(workDir);
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 eb004cd4f4b..8b2c02a72de 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
@@ -566,7 +566,8 @@ public class GridCacheDatabaseSharedManager extends
IgniteCacheDatabaseSharedMan
kernalCtx.cache(),
() ->
cpFreqDeviation.getOrDefault(DEFAULT_CHECKPOINT_DEVIATION),
kernalCtx.pools().getSystemExecutorService(),
- kernalCtx.marshallerContext().jdkMarshaller()
+ kernalCtx.marshallerContext().jdkMarshaller(),
+ kernalCtx.pdsFolderResolver().fileTree()
);
final NodeFileLockHolder preLocked = kernalCtx.pdsFolderResolver()
@@ -745,8 +746,9 @@ public class GridCacheDatabaseSharedManager extends
IgniteCacheDatabaseSharedMan
if (cctx.kernalContext().clientNode())
return;
- fileLockHolder = preLocked == null ?
- new NodeFileLockHolder(storeMgr.workDir().getPath(),
cctx.kernalContext(), log) : preLocked;
+ fileLockHolder = preLocked == null
+ ? new
NodeFileLockHolder(cctx.kernalContext().pdsFolderResolver().fileTree().nodeStorage().getPath(),
cctx.kernalContext(), log)
+ : preLocked;
if (!fileLockHolder.isLocked()) {
if (log.isDebugEnabled())
@@ -1314,7 +1316,9 @@ public class GridCacheDatabaseSharedManager extends
IgniteCacheDatabaseSharedMan
"Invalid page store manager was created: " +
cctx.pageStore();
Path anyIdxPartFile = IgniteUtils.searchFileRecursively(
-
((FilePageStoreManager)cctx.pageStore()).workDir().toPath(),
FilePageStoreManager.INDEX_FILE_NAME);
+
cctx.kernalContext().pdsFolderResolver().fileTree().nodeStorage().toPath(),
+ FilePageStoreManager.INDEX_FILE_NAME
+ );
if (anyIdxPartFile != null) {
memCfg.setPageSize(resolvePageSizeFromPartitionFile(anyIdxPartFile));
@@ -1839,13 +1843,6 @@ public class GridCacheDatabaseSharedManager extends
IgniteCacheDatabaseSharedMan
return lastCheckpointEntry == null ? null :
lastCheckpointEntry.checkpointMark();
}
- /**
- * @return Checkpoint directory.
- */
- public File checkpointDirectory() {
- return checkpointManager.checkpointDirectory();
- }
-
/**
* @param lsnr Listener.
* @param dataRegion Data region for which listener is corresponded to.
@@ -1889,10 +1886,8 @@ public class GridCacheDatabaseSharedManager extends
IgniteCacheDatabaseSharedMan
if (mntcTask != null) {
log.warning("Maintenance task found, stop restoring memory");
- File workDir = ((FilePageStoreManager)cctx.pageStore()).workDir();
-
mntcRegistry.registerWorkflowCallback(CORRUPTED_DATA_FILES_MNTC_TASK_NAME,
- new CorruptedPdsMaintenanceCallback(workDir,
+ new
CorruptedPdsMaintenanceCallback(cctx.kernalContext().pdsFolderResolver().fileTree().nodeStorage(),
Arrays.asList(mntcTask.parameters().split(Pattern.quote(File.separator))))
);
diff --git
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointManager.java
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointManager.java
index 5cba085cd6a..854d1a16a61 100644
---
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointManager.java
+++
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointManager.java
@@ -17,7 +17,6 @@
package org.apache.ignite.internal.processors.cache.persistence.checkpoint;
-import java.io.File;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.util.Collection;
@@ -38,6 +37,7 @@ import
org.apache.ignite.internal.processors.cache.persistence.DataRegion;
import
org.apache.ignite.internal.processors.cache.persistence.DataStorageMetricsImpl;
import
org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
import
org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import
org.apache.ignite.internal.processors.cache.persistence.filename.NodeFileTree;
import
org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
import
org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImpl;
import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer;
@@ -108,7 +108,7 @@ public class CheckpointManager {
* @param cpFreqDeviation Distributed checkpoint frequency deviation.
* @param checkpointMapSnapshotExecutor Checkpoint map snapshot executor.
* @param marsh JDK marshaller.
- * @throws IgniteCheckedException if fail.
+ * @param ft Node file tree.
*/
public CheckpointManager(
Function<Class<?>, IgniteLogger> logger,
@@ -129,8 +129,9 @@ public class CheckpointManager {
GridCacheProcessor cacheProcessor,
Supplier<Integer> cpFreqDeviation,
Executor checkpointMapSnapshotExecutor,
- JdkMarshaller marsh
- ) throws IgniteCheckedException {
+ JdkMarshaller marsh,
+ NodeFileTree ft
+ ) {
CheckpointHistory cpHistory = new CheckpointHistory(
persistenceCfg,
logger,
@@ -147,7 +148,7 @@ public class CheckpointManager {
logger,
cpHistory,
ioFactory,
- pageStoreManager.workDir().getAbsolutePath(),
+ ft,
lock,
checkpointMapSnapshotExecutor,
marsh
@@ -259,13 +260,6 @@ public class CheckpointManager {
checkpointWorkflow.memoryRecoveryRecordPtr(memoryRecoveryRecordPtr);
}
- /**
- * @return Checkpoint directory.
- */
- public File checkpointDirectory() {
- return checkpointMarkersStorage.cpDir;
- }
-
/**
* @return Checkpoint storage.
*/
diff --git
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointMarkersStorage.java
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointMarkersStorage.java
index 55cfd4abc79..8926dd2eda5 100644
---
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointMarkersStorage.java
+++
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointMarkersStorage.java
@@ -48,9 +48,9 @@ import
org.apache.ignite.internal.processors.cache.persistence.StorageException;
import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
import
org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
import
org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import
org.apache.ignite.internal.processors.cache.persistence.filename.NodeFileTree;
import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer;
import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.marshaller.jdk.JdkMarshaller;
import org.jetbrains.annotations.Nullable;
@@ -87,7 +87,10 @@ public class CheckpointMarkersStorage {
/** Logger. */
protected IgniteLogger log;
- /** Checkpoint history. */
+ /**
+ * Checkpoint history.
+ * Non-final for tests.
+ */
private CheckpointHistory cpHistory;
/** File I/O factory for writing checkpoint markers. */
@@ -96,8 +99,8 @@ public class CheckpointMarkersStorage {
/** Checkpoint read-write lock. */
private final CheckpointReadWriteLock lock;
- /** Checkpoint metadata directory ("cp"), contains files with checkpoint
start and end */
- public final File cpDir;
+ /** Node file tree. */
+ private final NodeFileTree ft;
/** Temporary write buffer. */
private final ByteBuffer tmpWriteBuf;
@@ -116,31 +119,28 @@ public class CheckpointMarkersStorage {
* @param logger Ignite logger.
* @param history Checkpoint history.
* @param factory IO factory.
- * @param absoluteWorkDir Directory path to checkpoint markers folder.
+ * @param ft Node file tree.
* @param lock Checkpoint read-write lock.
* @param checkpointMapSnapshotExecutor Checkpoint map snapshot executor.
* @param marsh JDK marshaler.
- * @throws IgniteCheckedException if fail.
*/
CheckpointMarkersStorage(
String igniteInstanceName,
Function<Class<?>, IgniteLogger> logger,
CheckpointHistory history,
FileIOFactory factory,
- String absoluteWorkDir,
+ NodeFileTree ft,
CheckpointReadWriteLock lock,
Executor checkpointMapSnapshotExecutor,
JdkMarshaller marsh
- ) throws IgniteCheckedException {
+ ) {
this.log = logger.apply(getClass());
cpHistory = history;
ioFactory = factory;
this.lock = lock;
+ this.ft = ft;
- cpDir = Paths.get(absoluteWorkDir, "cp").toFile();
-
- if (!U.mkdirs(cpDir))
- throw new IgniteCheckedException("Could not create directory for
checkpoint metadata: " + cpDir);
+ ft.mkdirCheckpoint();
//File index + offset + length.
tmpWriteBuf = ByteBuffer.allocateDirect(Long.BYTES + Integer.BYTES +
Integer.BYTES);
@@ -156,13 +156,13 @@ public class CheckpointMarkersStorage {
*/
public void cleanupTempCheckpointDirectory() throws IgniteCheckedException
{
try {
- try (DirectoryStream<Path> files =
Files.newDirectoryStream(cpDir.toPath(), TMP_FILE_MATCHER::matches)) {
+ try (DirectoryStream<Path> files =
Files.newDirectoryStream(ft.checkpoint().toPath(), TMP_FILE_MATCHER::matches)) {
for (Path path : files)
Files.delete(path);
}
}
catch (IOException e) {
- throw new IgniteCheckedException("Failed to cleanup checkpoint
directory from temporary files: " + cpDir, e);
+ throw new IgniteCheckedException("Failed to cleanup checkpoint
directory from temporary files: " + ft.checkpoint(), e);
}
}
@@ -174,13 +174,13 @@ public class CheckpointMarkersStorage {
cpHistory.clear();
try {
- try (DirectoryStream<Path> files =
Files.newDirectoryStream(cpDir.toPath())) {
+ try (DirectoryStream<Path> files =
Files.newDirectoryStream(ft.checkpoint().toPath())) {
for (Path path : files)
Files.delete(path);
}
}
catch (IOException e) {
- throw new IgniteCheckedException("Failed to cleanup checkpoint
directory: " + cpDir, e);
+ throw new IgniteCheckedException("Failed to cleanup checkpoint
directory: " + ft.checkpoint(), e);
}
}
@@ -188,8 +188,8 @@ public class CheckpointMarkersStorage {
* Filling internal structures with data from disk.
*/
public void initialize() throws IgniteCheckedException {
- File snapshotFile = new File(cpDir, EARLIEST_CP_SNAPSHOT_FILE);
- File snapshotTmpFile = new File(cpDir, EARLIEST_CP_SNAPSHOT_TMP_FILE);
+ File snapshotFile = new File(ft.checkpoint(),
EARLIEST_CP_SNAPSHOT_FILE);
+ File snapshotTmpFile = new File(ft.checkpoint(),
EARLIEST_CP_SNAPSHOT_TMP_FILE);
if (snapshotTmpFile.exists()) {
if (!IgniteUtils.delete(snapshotTmpFile)) {
@@ -274,15 +274,13 @@ public class CheckpointMarkersStorage {
WALPointer startPtr = CheckpointStatus.NULL_PTR;
WALPointer endPtr = CheckpointStatus.NULL_PTR;
- File dir = cpDir;
-
- if (!dir.exists()) {
+ if (!ft.checkpoint().exists()) {
log.warning("Read checkpoint status: checkpoint directory is not
found.");
return new CheckpointStatus(0, startId, startPtr, endId, endPtr);
}
- File[] files = dir.listFiles();
+ File[] files = ft.checkpoint().listFiles();
for (File file : files) {
Matcher matcher = CP_FILE_NAME_PATTERN.matcher(file.getName());
@@ -326,11 +324,11 @@ public class CheckpointMarkersStorage {
* @return List of checkpoints.
*/
private List<CheckpointEntry> retrieveHistory() throws
IgniteCheckedException {
- if (!cpDir.exists())
+ if (!ft.checkpoint().exists())
return Collections.emptyList();
try (DirectoryStream<Path> cpFiles = Files.newDirectoryStream(
- cpDir.toPath(),
+ ft.checkpoint().toPath(),
path ->
CP_FILE_NAME_PATTERN.matcher(path.toFile().getName()).matches())
) {
List<CheckpointEntry> checkpoints = new ArrayList<>();
@@ -435,8 +433,8 @@ public class CheckpointMarkersStorage {
* @throws IgniteCheckedException If failed to delete.
*/
private void removeCheckpointFiles(CheckpointEntry cpEntry) throws
IgniteCheckedException {
- Path startFile = new File(cpDir.getAbsolutePath(),
checkpointFileName(cpEntry, CheckpointEntryType.START)).toPath();
- Path endFile = new File(cpDir.getAbsolutePath(),
checkpointFileName(cpEntry, CheckpointEntryType.END)).toPath();
+ Path startFile = new File(ft.checkpoint().getAbsolutePath(),
checkpointFileName(cpEntry, CheckpointEntryType.START)).toPath();
+ Path endFile = new File(ft.checkpoint().getAbsolutePath(),
checkpointFileName(cpEntry, CheckpointEntryType.END)).toPath();
try {
if (Files.exists(startFile))
@@ -467,7 +465,7 @@ public class CheckpointMarkersStorage {
String tmpFileName = fileName + FilePageStoreManager.TMP_SUFFIX;
try {
- try (FileIO io =
ioFactory.create(Paths.get(cpDir.getAbsolutePath(), skipSync ? fileName :
tmpFileName).toFile(),
+ try (FileIO io =
ioFactory.create(Paths.get(ft.checkpoint().getAbsolutePath(), skipSync ?
fileName : tmpFileName).toFile(),
StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE)) {
io.writeFully(entryBuf);
@@ -478,8 +476,12 @@ public class CheckpointMarkersStorage {
io.force(true);
}
- if (!skipSync)
- Files.move(Paths.get(cpDir.getAbsolutePath(), tmpFileName),
Paths.get(cpDir.getAbsolutePath(), fileName));
+ if (!skipSync) {
+ Files.move(
+ Paths.get(ft.checkpoint().getAbsolutePath(), tmpFileName),
+ Paths.get(ft.checkpoint().getAbsolutePath(), fileName)
+ );
+ }
}
catch (IOException e) {
throw new StorageException("Failed to write checkpoint entry
[ptr=" + cp.checkpointMark()
@@ -628,11 +630,11 @@ public class CheckpointMarkersStorage {
lock.readUnlock();
}
- File targetFile = new File(cpDir,
EARLIEST_CP_SNAPSHOT_FILE);
+ File targetFile = new File(ft.checkpoint(),
EARLIEST_CP_SNAPSHOT_FILE);
// For fail-safety we should first write the snapshot to a
temporary file
// and then atomically rename it
- File tmpFile = new File(cpDir,
EARLIEST_CP_SNAPSHOT_TMP_FILE);
+ File tmpFile = new File(ft.checkpoint(),
EARLIEST_CP_SNAPSHOT_TMP_FILE);
if (tmpFile.exists() && !IgniteUtils.delete(tmpFile)) {
log.error("Failed to delete temporary checkpoint
snapshot file: " + tmpFile.getAbsolutePath());
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 c86c2ecb99a..dd0c0cbe59a 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
@@ -64,7 +64,7 @@ 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;
import
org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils;
-import
org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings;
+import
org.apache.ignite.internal.processors.cache.persistence.filename.NodeFileTree;
import
org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage;
import
org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMetrics;
import
org.apache.ignite.internal.processors.cache.persistence.pagemem.PageReadWriteManager;
@@ -173,8 +173,8 @@ public class FilePageStoreManager extends
GridCacheSharedManagerAdapter implemen
/** */
private final DataStorageConfiguration dsCfg;
- /** Absolute directory for file page store. Includes consistent id based
folder. */
- private File storeWorkDir;
+ /** Node file tree. */
+ private NodeFileTree ft;
/** */
private final Set<Integer> grpsWithoutIdx = Collections.newSetFromMap(new
ConcurrentHashMap<Integer, Boolean>());
@@ -212,24 +212,22 @@ public class FilePageStoreManager extends
GridCacheSharedManagerAdapter implemen
if (ctx.clientNode())
return;
- final PdsFolderSettings folderSettings =
ctx.pdsFolderResolver().resolveFolders();
+ ft = ctx.pdsFolderResolver().fileTree();
- storeWorkDir = folderSettings.persistentStoreNodePath();
-
- U.ensureDirectory(storeWorkDir, "page store work directory", log);
+ U.ensureDirectory(ft.nodeStorage(), "page store work directory", log);
String tmpDir = System.getProperty("java.io.tmpdir");
- if (tmpDir != null &&
storeWorkDir.getAbsolutePath().startsWith(tmpDir)) {
+ if (tmpDir != null &&
ft.nodeStorage().getAbsolutePath().startsWith(tmpDir)) {
log.warning("Persistence store directory is in the temp directory
and may be cleaned." +
"To avoid this set \"IGNITE_HOME\" environment variable
properly or " +
"change location of persistence directories in data storage
configuration " +
"(see DataStorageConfiguration#walPath,
DataStorageConfiguration#walArchivePath, " +
"DataStorageConfiguration#storagePath properties). " +
- "Current persistence store directory is: [" +
storeWorkDir.getAbsolutePath() + "]");
+ "Current persistence store directory is: [" +
ft.nodeStorage().getAbsolutePath() + "]");
}
- File[] files = storeWorkDir.listFiles();
+ File[] files = ft.nodeStorage().listFiles();
for (File file : files) {
if (file.isDirectory()) {
@@ -273,7 +271,7 @@ public class FilePageStoreManager extends
GridCacheSharedManagerAdapter implemen
@Override public void cleanupPersistentSpace() throws
IgniteCheckedException {
try {
try (DirectoryStream<Path> files = newDirectoryStream(
- storeWorkDir.toPath(), entry -> {
+ ft.nodeStorage().toPath(), entry -> {
String name = entry.toFile().getName();
return !name.equals(MetaStorage.METASTORAGE_DIR_NAME) &&
@@ -437,7 +435,7 @@ public class FilePageStoreManager extends
GridCacheSharedManagerAdapter implemen
/** {@inheritDoc} */
@Override public void initializeForCache(CacheGroupDescriptor grpDesc,
CacheConfiguration<?, ?> ccfg) throws IgniteCheckedException {
- assert storeWorkDir != null;
+ assert ft != null;
int grpId = grpDesc.groupId();
@@ -452,7 +450,7 @@ public class FilePageStoreManager extends
GridCacheSharedManagerAdapter implemen
/** {@inheritDoc} */
@Override public void initializeForMetastorage() throws
IgniteCheckedException {
- assert storeWorkDir != null;
+ assert ft.nodeStorage() != null;
int grpId = MetaStorage.METASTORAGE_CACHE_ID;
@@ -461,7 +459,7 @@ public class FilePageStoreManager extends
GridCacheSharedManagerAdapter implemen
PageMetrics pageMetrics =
dataRegion.metrics().cacheGrpPageMetrics(grpId);
CacheStoreHolder holder = initDir(
- new File(storeWorkDir, MetaStorage.METASTORAGE_DIR_NAME),
+ new File(ft.nodeStorage(), MetaStorage.METASTORAGE_DIR_NAME),
grpId,
MetaStorage.METASTORAGE_CACHE_NAME,
MetaStorage.METASTORAGE_PARTITIONS.size(),
@@ -954,19 +952,12 @@ public class FilePageStoreManager extends
GridCacheSharedManagerAdapter implemen
return pageCnt;
}
- /**
- * @return Store work dir. Includes consistent-id based folder
- */
- public File workDir() {
- return storeWorkDir;
- }
-
/**
* @param ccfg Cache configuration.
* @return Store dir for given cache.
*/
public File cacheWorkDir(CacheConfiguration<?, ?> ccfg) {
- return cacheWorkDir(storeWorkDir, cacheDirName(ccfg));
+ return cacheWorkDir(ft.nodeStorage(), cacheDirName(ccfg));
}
/**
@@ -975,7 +966,7 @@ public class FilePageStoreManager extends
GridCacheSharedManagerAdapter implemen
* @return Store directory for given cache.
*/
public File cacheWorkDir(boolean isSharedGroup, String cacheOrGroupName) {
- return cacheWorkDir(storeWorkDir, cacheDirName(isSharedGroup,
cacheOrGroupName));
+ return cacheWorkDir(ft.nodeStorage(), cacheDirName(isSharedGroup,
cacheOrGroupName));
}
/**
diff --git
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/NodeFileTree.java
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/NodeFileTree.java
index c7bff12f133..702adcc4e49 100644
---
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/NodeFileTree.java
+++
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/NodeFileTree.java
@@ -25,6 +25,7 @@ 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.S;
import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointSpi;
import org.jetbrains.annotations.Nullable;
import static
org.apache.ignite.configuration.DataStorageConfiguration.DFLT_WAL_ARCHIVE_PATH;
@@ -56,7 +57,7 @@ import static
org.apache.ignite.internal.processors.cache.persistence.filename.P
* <pre>
* ❯ tree
* .
← root (work directory, shared between all local nodes).
- * ├── cp
+ * ├── cp
← default sharedfs root. See {@link SharedFsCheckpointSpi}.
* │ └── sharedfs
* │ └── BinaryMarshaller
* ├── db
← db (shared between all local nodes).
@@ -86,7 +87,7 @@ import static
org.apache.ignite.internal.processors.cache.persistence.filename.P
* │ │ │ ├── part-1.bin
* ...
* │ │ │ └── part-9.bin
- * │ │ ├── cp
+ * │ │ ├── cp
← checkpoint (node 0).
* │ │ │ ├── 1737804007693-96128bb0-5361-495a-b593-53dc4339a56d-END.bin
* │ │ │ └── 1737804007693-96128bb0-5361-495a-b593-53dc4339a56d-START.bin
* │ │ ├── lock
@@ -102,7 +103,7 @@ import static
org.apache.ignite.internal.processors.cache.persistence.filename.P
* ...
* │ │ ├── cache-tx-cache
* ...
- * │ │ ├── cp
+ * │ │ ├── cp
← checkpoint (node 1).
* ...
* │ │ ├── lock
* │ │ ├── maintenance_tasks.mntc
@@ -149,6 +150,9 @@ public class NodeFileTree extends SharedFileTree {
/** Default snapshot directory for loading remote snapshots. */
public static final String SNAPSHOT_TMP_DIR = "snp";
+ /** Checkpoint directory name. */
+ public static final String CHECKPOINT_DIR = "cp";
+
/** Folder name for consistent id. */
private final String folderName;
@@ -158,6 +162,9 @@ public class NodeFileTree extends SharedFileTree {
/** Path to the storage directory. */
private final @Nullable File nodeStorage;
+ /** Path to the checkpoint directory. */
+ private final @Nullable File checkpoint;
+
/** Path to the directory containing active WAL segments. */
private final @Nullable File wal;
@@ -216,6 +223,7 @@ public class NodeFileTree extends SharedFileTree {
walCdc = rootRelative(DFLT_WAL_CDC_PATH);
nodeStorage = rootRelative(DB_DEFAULT_FOLDER);
snpTmpRoot = new File(nodeStorage, SNAPSHOT_TMP_DIR);
+ checkpoint = new File(nodeStorage, CHECKPOINT_DIR);
}
/**
@@ -247,6 +255,7 @@ public class NodeFileTree extends SharedFileTree {
? rootRelative(DB_DEFAULT_FOLDER)
: resolveDirectory(dsCfg.getStoragePath());
snpTmpRoot = new File(nodeStorage, SNAPSHOT_TMP_DIR);
+ checkpoint = new File(nodeStorage, CHECKPOINT_DIR);
wal = resolveDirectory(dsCfg.getWalPath());
walArchive = resolveDirectory(dsCfg.getWalArchivePath());
walCdc = resolveDirectory(dsCfg.getCdcWalPath());
@@ -254,6 +263,7 @@ public class NodeFileTree extends SharedFileTree {
else {
nodeStorage = null;
snpTmpRoot = null;
+ checkpoint = null;
wal = null;
walArchive = null;
walCdc = null;
@@ -265,6 +275,11 @@ public class NodeFileTree extends SharedFileTree {
return nodeStorage;
}
+ /** @return Folder name. */
+ public String folderName() {
+ return folderName;
+ }
+
/** @return Path to binary metadata directory. */
public File binaryMeta() {
return binaryMeta;
@@ -290,6 +305,11 @@ public class NodeFileTree extends SharedFileTree {
return snpTmpRoot;
}
+ /** @return Path to the checkpoint directory. */
+ public File checkpoint() {
+ return checkpoint;
+ }
+
/**
* Creates {@link #binaryMeta()} directory.
* @return Created directory.
@@ -308,6 +328,15 @@ public class NodeFileTree extends SharedFileTree {
return mkdir(snpTmpRoot, "temp directory for snapshot creation");
}
+ /**
+ * Creates {@link #checkpoint()} directory.
+ * @return Created directory.
+ * @see #checkpoint()
+ */
+ public File mkdirCheckpoint() {
+ return mkdir(checkpoint, "checkpoint metadata directory");
+ }
+
/** @return {@code True} if WAL archive enabled. */
public boolean walArchiveEnabled() {
return walArchive != null && wal != null && !walArchive.equals(wal);
@@ -319,7 +348,7 @@ public class NodeFileTree extends SharedFileTree {
* @param cfg Configured directory path.
* @return Initialized directory.
*/
- private File resolveDirectory(String cfg) {
+ public File resolveDirectory(String cfg) {
File sharedDir = new File(cfg);
return sharedDir.isAbsolute()
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 6660474e8ad..cc1259cdbf0 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
@@ -2661,7 +2661,7 @@ public class IgniteSnapshotManager extends
GridCacheSharedManagerAdapter
srcNodeId,
reqId,
snpName,
- ft.snapshotTempRoot(),
+ ft,
ioFactory,
snpSndr,
parts,
@@ -3893,9 +3893,9 @@ public class IgniteSnapshotManager extends
GridCacheSharedManagerAdapter
try {
task.partsLeft.compareAndSet(-1, partsCnt);
- File cacheDir =
FilePageStoreManager.cacheWorkDir(storeMgr.workDir(), cacheDirName);
+ File cacheDir =
FilePageStoreManager.cacheWorkDir(ft.nodeStorage(), cacheDirName);
- File tmpCacheDir =
U.resolveWorkDirectory(storeMgr.workDir().getAbsolutePath(),
+ File tmpCacheDir =
U.resolveWorkDirectory(ft.nodeStorage().getAbsolutePath(),
formatTmpDirName(cacheDir).getName(), false);
return Paths.get(tmpCacheDir.getAbsolutePath(),
getPartitionFileName(partId)).toString();
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 ddae6250e03..8a672fb4239 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
@@ -56,6 +56,7 @@ import
org.apache.ignite.internal.processors.cache.persistence.checkpoint.Checkp
import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
import
org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
import
org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import
org.apache.ignite.internal.processors.cache.persistence.filename.NodeFileTree;
import
org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage;
import
org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
@@ -97,6 +98,9 @@ class SnapshotFutureTask extends
AbstractCreateSnapshotFutureTask implements Che
/** Snapshot working directory on file system. */
private final File tmpSnpWorkDir;
+ /** Node file tree. */
+ private final NodeFileTree ft;
+
/** IO factory which will be used for creating snapshot delta-writers. */
private final FileIOFactory ioFactory;
@@ -154,7 +158,6 @@ class SnapshotFutureTask extends
AbstractCreateSnapshotFutureTask implements Che
* @param srcNodeId Node id which cause snapshot task creation.
* @param reqId Snapshot operation request ID.
* @param snpName Unique identifier of snapshot process.
- * @param tmpWorkDir Working directory for intermediate snapshot results.
* @param ioFactory Factory to working with snapshot files.
* @param snpSndr Factory which produces snapshot receiver instance.
* @param parts Map of cache groups and its partitions to include into
snapshot, if set of partitions
@@ -165,7 +168,7 @@ class SnapshotFutureTask extends
AbstractCreateSnapshotFutureTask implements Che
UUID srcNodeId,
UUID reqId,
String snpName,
- File tmpWorkDir,
+ NodeFileTree ft,
FileIOFactory ioFactory,
SnapshotSender snpSndr,
Map<Integer, Set<Integer>> parts,
@@ -180,7 +183,8 @@ class SnapshotFutureTask extends
AbstractCreateSnapshotFutureTask implements Che
assert cctx.pageStore() instanceof FilePageStoreManager : "Snapshot
task can work only with physical files.";
assert !parts.containsKey(MetaStorage.METASTORAGE_CACHE_ID) : "The
withMetaStorage must be used instead.";
- this.tmpSnpWorkDir = new File(tmpWorkDir, snpName);
+ this.tmpSnpWorkDir = new File(ft.snapshotTempRoot(), snpName);
+ this.ft = ft;
this.ioFactory = ioFactory;
this.withMetaStorage = withMetaStorage;
this.pageStore = (FilePageStoreManager)cctx.pageStore();
@@ -408,7 +412,7 @@ class SnapshotFutureTask extends
AbstractCreateSnapshotFutureTask implements Che
return runAsync(() -> {
snpSndr.sendPart(
- getPartitionFile(pageStore.workDir(), cacheDirName,
partId),
+ getPartitionFile(ft.nodeStorage(), cacheDirName, partId),
cacheDirName,
pair,
partLen);
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 0132058d31e..7674610ea11 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
@@ -204,11 +204,9 @@ public class SnapshotRestoreProcess {
* @throws IgniteCheckedException If it was not possible to delete some
temporary directory.
*/
protected void cleanup() throws IgniteCheckedException {
- FilePageStoreManager pageStore =
(FilePageStoreManager)ctx.cache().context().pageStore();
+ File nodeStorage = ctx.pdsFolderResolver().fileTree().nodeStorage();
- File dbDir = pageStore.workDir();
-
- for (File dir : dbDir.listFiles(dir -> dir.isDirectory() &&
dir.getName().startsWith(TMP_CACHE_DIR_PREFIX))) {
+ for (File dir : nodeStorage.listFiles(dir -> dir.isDirectory() &&
dir.getName().startsWith(TMP_CACHE_DIR_PREFIX))) {
if (!U.delete(dir)) {
throw new IgniteCheckedException("Unable to remove temporary
directory, " +
"try deleting it manually [dir=" + dir + ']');
diff --git
a/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java
b/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java
index d967b8da77a..50a58e7fcb2 100644
---
a/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java
+++
b/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java
@@ -121,6 +121,9 @@ import org.jetbrains.annotations.Nullable;
@IgniteSpiMultipleInstancesSupport(true)
@IgniteSpiConsistencyChecked(optional = false)
public class SharedFsCheckpointSpi extends IgniteSpiAdapter implements
CheckpointSpi {
+ /** Default root checkpoint directory. */
+ public static final String DFLT_ROOT = "cp";
+
/**
* Default checkpoint directory. Note that this path is relative to {@code
IGNITE_HOME/work} folder
* if {@code IGNITE_HOME} system or environment variable specified,
otherwise it is relative to
@@ -128,7 +131,7 @@ public class SharedFsCheckpointSpi extends IgniteSpiAdapter
implements Checkpoin
*
* @see
org.apache.ignite.configuration.IgniteConfiguration#getWorkDirectory()
*/
- public static final String DFLT_DIR_PATH = "cp/sharedfs";
+ public static final String DFLT_DIR_PATH = DFLT_ROOT + "/sharedfs";
/** */
private static final String CODES = "0123456789QWERTYUIOPASDFGHJKLZXCVBNM";
diff --git
a/modules/core/src/test/java/org/apache/ignite/cdc/CdcManagerTest.java
b/modules/core/src/test/java/org/apache/ignite/cdc/CdcManagerTest.java
index 645604b563f..cdace86065d 100644
--- a/modules/core/src/test/java/org/apache/ignite/cdc/CdcManagerTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cdc/CdcManagerTest.java
@@ -41,7 +41,6 @@ import
org.apache.ignite.internal.pagemem.wal.record.RolloverType;
import
org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
import
org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
import
org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
-import
org.apache.ignite.internal.processors.cache.persistence.filename.NodeFileTree;
import
org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
import
org.apache.ignite.internal.processors.cache.persistence.wal.reader.IgniteWalIteratorFactory;
import
org.apache.ignite.internal.processors.configuration.distributed.DistributedChangeableProperty;
@@ -159,11 +158,9 @@ public class CdcManagerTest extends GridCommonAbstractTest
{
assertEquals(0, walMgr.currentSegment());
- File walDir = walDir(ign);
-
stopGrid(0);
- Path seg =
Arrays.stream(walDir.listFiles()).sorted().findFirst().get().toPath();
+ Path seg =
Arrays.stream(ign.context().pdsFolderResolver().fileTree().wal().listFiles()).sorted().findFirst().get().toPath();
ByteBuffer walBuf = ByteBuffer.wrap(Files.readAllBytes(seg));
ByteBuffer cdcBuf = ByteBuffer.wrap(cdcMgr(ign).buf.array());
@@ -205,7 +202,7 @@ public class CdcManagerTest extends GridCommonAbstractTest {
stopGrid(0);
- File seg =
Arrays.stream(walDir(ign).listFiles()).sorted().findFirst().get();
+ File seg =
Arrays.stream(ign.context().pdsFolderResolver().fileTree().wal().listFiles()).sorted().findFirst().get();
int len0 = writtenLength(seg);
ByteBuffer buf0 = cdcMgr.buf;
@@ -322,11 +319,9 @@ public class CdcManagerTest extends GridCommonAbstractTest
{
for (int i = 0; i < 10_000; i++)
ign.cache(DEFAULT_CACHE_NAME).put(i, i);
- File walDir = walDir(ign);
-
stopGrid(0);
- List<File> segs = Arrays.stream(walDir.listFiles()).sorted()
+ List<File> segs =
Arrays.stream(ign.context().pdsFolderResolver().fileTree().wal().listFiles()).sorted()
.limit(2)
.collect(Collectors.toList());
@@ -364,11 +359,6 @@ public class CdcManagerTest extends GridCommonAbstractTest
{
}
}
- /** Get WAL directory. */
- private File walDir(IgniteEx ign) throws Exception {
- return new NodeFileTree(ign.configuration(),
ign.context().pdsFolderResolver().resolveFolders().folderName()).wal();
- }
-
/** @return Length of the all written records in the specified segment. */
private int writtenLength(File walSegment) throws Exception {
IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log);
diff --git
a/modules/core/src/test/java/org/apache/ignite/internal/pagemem/wal/record/WALRecordSerializationTest.java
b/modules/core/src/test/java/org/apache/ignite/internal/pagemem/wal/record/WALRecordSerializationTest.java
index b0027611473..b236135c6fc 100644
---
a/modules/core/src/test/java/org/apache/ignite/internal/pagemem/wal/record/WALRecordSerializationTest.java
+++
b/modules/core/src/test/java/org/apache/ignite/internal/pagemem/wal/record/WALRecordSerializationTest.java
@@ -18,7 +18,6 @@
package org.apache.ignite.internal.pagemem.wal.record;
import java.io.File;
-import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
@@ -32,7 +31,6 @@ import
org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
import org.apache.ignite.internal.pagemem.wal.WALIterator;
import
org.apache.ignite.internal.processors.cache.persistence.wal.FileDescriptor;
import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer;
-import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.testframework.GridTestUtils;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
import org.apache.ignite.testframework.wal.record.RecordUtils;
@@ -182,13 +180,7 @@ public class WALRecordSerializationTest extends
GridCommonAbstractTest {
ignite.context().cache().context().database().checkpointReadUnlock();
}
- String nodeFolderName =
ignite.context().pdsFolderResolver().resolveFolders().folderName();
- File nodeArchiveDir = Paths.get(
- U.resolveWorkDirectory(U.defaultWorkDirectory(), "db",
false).getAbsolutePath(),
- "wal",
- "archive",
- nodeFolderName
- ).toFile();
+ File nodeArchiveDir =
ignite.context().pdsFolderResolver().fileTree().walArchive();
File walSegment = new File(nodeArchiveDir,
FileDescriptor.fileName(lastPointer.index()));
File walZipSegment = new File(nodeArchiveDir,
FileDescriptor.fileName(lastPointer.index()) + ZIP_SUFFIX);
diff --git
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WalModeChangeAdvancedSelfTest.java
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WalModeChangeAdvancedSelfTest.java
index 5861a20aa3a..4e683042bfb 100644
---
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WalModeChangeAdvancedSelfTest.java
+++
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WalModeChangeAdvancedSelfTest.java
@@ -26,14 +26,13 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteException;
import org.apache.ignite.internal.IgniteClientReconnectAbstractTest;
import org.apache.ignite.internal.IgniteEx;
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
import
org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
-import org.apache.ignite.internal.util.typedef.internal.U;
+import
org.apache.ignite.internal.processors.cache.persistence.filename.NodeFileTree;
import org.apache.ignite.testframework.GridTestUtils;
import org.apache.ignite.testframework.GridTestUtils.SF;
import org.junit.Ignore;
@@ -159,11 +158,7 @@ public class WalModeChangeAdvancedSelfTest extends
WalModeChangeCommonAbstractSe
File cacheToClean = cacheDir(srv, CACHE_NAME);
- String ig0Folder =
srv.context().pdsFolderResolver().resolveFolders().folderName();
- File dbDir =
U.resolveWorkDirectory(srv.configuration().getWorkDirectory(), "db", false);
-
- File ig0LfsDir = new File(dbDir, ig0Folder);
- File ig0CpDir = new File(ig0LfsDir, "cp");
+ NodeFileTree ft0 = srv.context().pdsFolderResolver().fileTree();
srv.cluster().state(ACTIVE);
@@ -176,7 +171,7 @@ public class WalModeChangeAdvancedSelfTest extends
WalModeChangeCommonAbstractSe
stopAllGrids(true);
- File[] cpMarkers = ig0CpDir.listFiles();
+ File[] cpMarkers = ft0.checkpoint().listFiles();
for (File cpMark : cpMarkers) {
if (cpMark.getName().contains("-END"))
@@ -290,13 +285,8 @@ public class WalModeChangeAdvancedSelfTest extends
WalModeChangeCommonAbstractSe
}
/** */
- private File cacheDir(Ignite ig, String cacheName) throws
IgniteCheckedException {
- String igFolder =
((IgniteEx)ig).context().pdsFolderResolver().resolveFolders().folderName();
- File dbDir =
U.resolveWorkDirectory(ig.configuration().getWorkDirectory(), "db", false);
-
- File igPdsFolder = new File(dbDir, igFolder);
-
- return new File(igPdsFolder, "cache-" + cacheName);
+ private File cacheDir(Ignite ig, String cacheName) {
+ return new
File(((IgniteEx)ig).context().pdsFolderResolver().fileTree().nodeStorage(),
"cache-" + cacheName);
}
/** */
diff --git
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCheckpointMapSnapshotTest.java
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCheckpointMapSnapshotTest.java
index 1203c582510..18a9d76f07f 100644
---
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCheckpointMapSnapshotTest.java
+++
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCheckpointMapSnapshotTest.java
@@ -159,7 +159,7 @@ public class IgnitePdsCheckpointMapSnapshotTest extends
GridCommonAbstractTest {
public void testCorruptedCpMap() throws Exception {
IgniteEx grid = startGrid(0);
- File cpDir = dbMgr(grid).checkpointManager.checkpointDirectory();
+ File cpDir =
grid.context().pdsFolderResolver().fileTree().checkpoint();
File cpSnapshotMap = new File(cpDir,
CheckpointMarkersStorage.EARLIEST_CP_SNAPSHOT_FILE);
@@ -203,7 +203,7 @@ public class IgnitePdsCheckpointMapSnapshotTest extends
GridCommonAbstractTest {
if (removeSnapshot) {
// Remove checkpoint map snapshot
- File cpDir = dbMgr(grid).checkpointManager.checkpointDirectory();
+ File cpDir =
grid.context().pdsFolderResolver().fileTree().checkpoint();
File cpSnapshotMap = new File(cpDir,
CheckpointMarkersStorage.EARLIEST_CP_SNAPSHOT_FILE);
diff --git
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedStoreTest.java
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedStoreTest.java
index 265e62de5f8..fdc55559520 100644
---
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedStoreTest.java
+++
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedStoreTest.java
@@ -53,6 +53,7 @@ import
org.apache.ignite.internal.processors.cache.persistence.file.FileIODecora
import
org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
import
org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
import
org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import
org.apache.ignite.internal.processors.cache.persistence.filename.NodeFileTree;
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.PageIO;
@@ -64,7 +65,6 @@ import
org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
import org.junit.Test;
import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_SKIP_CRC;
-import static
org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
import static
org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage.METASTORAGE_CACHE_ID;
/**
@@ -169,14 +169,13 @@ public class IgnitePdsCorruptedStoreTest extends
GridCommonAbstractTest {
ignite.cluster().state(ClusterState.INACTIVE);
+ NodeFileTree ft = ignite.context().pdsFolderResolver().fileTree();
+
stopGrid(0);
System.setProperty(IGNITE_PDS_SKIP_CRC, "false");
- File dbDir = U.resolveWorkDirectory(U.defaultWorkDirectory(),
DFLT_STORE_DIR, false);
- File walDir = new File(dbDir, "wal");
-
- U.delete(walDir);
+ U.delete(ft.wal());
try {
startGrid(0);
@@ -317,9 +316,7 @@ public class IgnitePdsCorruptedStoreTest extends
GridCommonAbstractTest {
ignite0.cluster().state(ClusterState.INACTIVE);
- FilePageStoreManager storeMgr =
((FilePageStoreManager)ignite0.context().cache().context().pageStore());
-
- File workDir = storeMgr.workDir();
+ File workDir =
ignite0.context().pdsFolderResolver().fileTree().nodeStorage();
File metaStoreDir = new File(workDir,
MetaStorage.METASTORAGE_CACHE_NAME.toLowerCase());
File metaStoreFile = new File(metaStoreDir,
String.format(FilePageStoreManager.PART_FILE_TEMPLATE, 0));
diff --git
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSporadicDataRecordsOnBackupTest.java
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSporadicDataRecordsOnBackupTest.java
index 18b2dbd8b4b..a6dea1956dc 100644
---
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSporadicDataRecordsOnBackupTest.java
+++
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSporadicDataRecordsOnBackupTest.java
@@ -17,7 +17,6 @@
package org.apache.ignite.internal.processors.cache.persistence;
-import java.io.File;
import java.util.Collections;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicBoolean;
@@ -40,11 +39,11 @@ import
org.apache.ignite.internal.pagemem.wal.record.DataEntry;
import org.apache.ignite.internal.pagemem.wal.record.DataRecord;
import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
import org.apache.ignite.internal.processors.cache.GridCacheOperation;
+import
org.apache.ignite.internal.processors.cache.persistence.filename.NodeFileTree;
import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer;
import
org.apache.ignite.internal.processors.cache.persistence.wal.reader.IgniteWalIteratorFactory;
import
org.apache.ignite.internal.processors.cache.persistence.wal.reader.IgniteWalIteratorFactory.IteratorParametersBuilder;
import org.apache.ignite.internal.util.typedef.internal.CU;
-import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.lang.IgniteBiTuple;
import org.apache.ignite.testframework.GridTestUtils;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -119,8 +118,8 @@ public class IgnitePdsSporadicDataRecordsOnBackupTest
extends GridCommonAbstract
grid(0).cluster().state(ClusterState.ACTIVE);
- String nodeFolderName0 =
ig0.context().pdsFolderResolver().resolveFolders().folderName();
- String nodeFolderName1 =
ig1.context().pdsFolderResolver().resolveFolders().folderName();
+ NodeFileTree ft0 = ig0.context().pdsFolderResolver().fileTree();
+ NodeFileTree ft1 = ig1.context().pdsFolderResolver().fileTree();
IgniteCache<Integer, Long> cache = grid(0).cache(TX_CACHE_NAME);
@@ -139,7 +138,7 @@ public class IgnitePdsSporadicDataRecordsOnBackupTest
extends GridCommonAbstract
stopAllGrids();
- assertEquals(0, findSporadicDataRecords(nodeFolderName0) +
findSporadicDataRecords(nodeFolderName1));
+ assertEquals(0, findSporadicDataRecords(ft0) +
findSporadicDataRecords(ft1));
}
/**
@@ -148,19 +147,14 @@ public class IgnitePdsSporadicDataRecordsOnBackupTest
extends GridCommonAbstract
*
* @throws IgniteCheckedException If failed.
*/
- private long findSporadicDataRecords(String nodeFolderName) throws
IgniteCheckedException {
- File dbDir = new File(U.defaultWorkDirectory(), "db");
- File commonWalDir = new File(dbDir, "wal");
- File walDir = new File(commonWalDir, nodeFolderName);
- File walArchiveDir = new File(new File(commonWalDir, "archive"),
nodeFolderName);
-
- assertTrue(walDir.exists());
- assertTrue(walArchiveDir.exists());
+ private long findSporadicDataRecords(NodeFileTree ft) throws
IgniteCheckedException {
+ assertTrue(ft.wal().exists());
+ assertTrue(ft.walArchive().exists());
IteratorParametersBuilder params = new IteratorParametersBuilder();
params.bufferSize(1024 * 1024);
- params.filesOrDirs(walDir, walArchiveDir);
+ params.filesOrDirs(ft.wal(), ft.walArchive());
params.filter((type, pointer) -> type ==
WALRecord.RecordType.DATA_RECORD_V2);
int cacheId = CU.cacheId(TX_CACHE_NAME);
diff --git
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java
index 57a7e36402b..1f3343e4533 100644
---
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java
+++
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java
@@ -681,12 +681,6 @@ public class LocalWalModeChangeDuringRebalancingSelfTest
extends GridCommonAbstr
IgniteEx ig0 = startGrid(0);
IgniteEx ig1 = startGrid(1);
- String ig1Folder =
ig1.context().pdsFolderResolver().resolveFolders().folderName();
- File dbDir =
U.resolveWorkDirectory(ig1.configuration().getWorkDirectory(), "db", false);
-
- File ig1LfsDir = new File(dbDir, ig1Folder);
- File ig1CpDir = new File(ig1LfsDir, "cp");
-
ig0.cluster().baselineAutoAdjustEnabled(false);
ig0.cluster().state(ACTIVE);
@@ -714,7 +708,7 @@ public class LocalWalModeChangeDuringRebalancingSelfTest
extends GridCommonAbstr
ig0 = startGrid(0);
- File[] cpMarkers = ig1CpDir.listFiles();
+ File[] cpMarkers =
ig1.context().pdsFolderResolver().fileTree().checkpoint().listFiles();
for (File cpMark : cpMarkers) {
if (cpMark.getName().contains("-END"))
diff --git
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/PagesPossibleCorruptionDiagnosticTest.java
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/PagesPossibleCorruptionDiagnosticTest.java
index 3c46dc91b79..5099290f73c 100644
---
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/PagesPossibleCorruptionDiagnosticTest.java
+++
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/PagesPossibleCorruptionDiagnosticTest.java
@@ -37,7 +37,6 @@ import
org.apache.ignite.internal.processors.cache.PartitionUpdateCounter;
import
org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIOFactory;
import
org.apache.ignite.internal.processors.cache.persistence.file.FilePageStore;
import
org.apache.ignite.internal.processors.cache.persistence.file.FileVersionCheckingFactory;
-import
org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings;
import
org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
import
org.apache.ignite.internal.processors.cache.persistence.freelist.CorruptedFreeListException;
import
org.apache.ignite.internal.processors.cache.persistence.tree.reuse.LongListReuseBag;
@@ -118,11 +117,7 @@ public class PagesPossibleCorruptionDiagnosticTest extends
GridCommonAbstractTes
* @throws IgniteCheckedException If failed.
*/
private FilePageStore filePageStore(IgniteEx ignite, int partId) throws
IgniteCheckedException {
- final PdsFolderSettings folderSettings =
ignite.context().pdsFolderResolver().resolveFolders();
-
- File storeWorkDir = new File(folderSettings.persistentStoreRootPath(),
folderSettings.folderName());
-
- File cacheWorkDir = new File(storeWorkDir, CACHE_DIR_PREFIX +
DEFAULT_CACHE_NAME);
+ File cacheWorkDir = new
File(ignite.context().pdsFolderResolver().fileTree().nodeStorage(),
CACHE_DIR_PREFIX + DEFAULT_CACHE_NAME);
File partFile = new File(cacheWorkDir, format(PART_FILE_TEMPLATE,
partId));
diff --git
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/WALPreloadingWithCompactionTest.java
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/WALPreloadingWithCompactionTest.java
index 34598edbc20..9bcfaa6f6a6 100644
---
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/WALPreloadingWithCompactionTest.java
+++
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/WALPreloadingWithCompactionTest.java
@@ -19,7 +19,6 @@
package org.apache.ignite.internal.processors.cache.persistence;
import java.io.File;
-import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteSystemProperties;
import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
import org.apache.ignite.cluster.ClusterState;
@@ -29,8 +28,8 @@ import
org.apache.ignite.configuration.DataStorageConfiguration;
import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.configuration.WALMode;
import org.apache.ignite.internal.IgniteEx;
+import
org.apache.ignite.internal.processors.cache.persistence.filename.NodeFileTree;
import
org.apache.ignite.internal.processors.cache.persistence.wal.FileDescriptor;
-import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
import org.junit.Test;
@@ -117,17 +116,12 @@ public class WALPreloadingWithCompactionTest extends
GridCommonAbstractTest {
* Check that there's only compacted version of given segment.
* @param ignite Ignite instance.
* @param segment Segment index.
- * @throws IgniteCheckedException If failed.
*/
- private void checkThatOnlyZipSegmentExists(IgniteEx ignite, int segment)
throws IgniteCheckedException {
- String nodeFolderName =
ignite.context().pdsFolderResolver().resolveFolders().folderName();
-
- File dbDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), "db",
false);
- File walDir = new File(dbDir, "wal");
- File archiveDir = new File(walDir, "archive");
- File nodeArchiveDir = new File(archiveDir, nodeFolderName);
- File walZipSegment = new File(nodeArchiveDir,
FileDescriptor.fileName(segment) + ".zip");
- File walRawSegment = new File(nodeArchiveDir,
FileDescriptor.fileName(segment));
+ private void checkThatOnlyZipSegmentExists(IgniteEx ignite, int segment) {
+ NodeFileTree ft = ignite.context().pdsFolderResolver().fileTree();
+
+ File walZipSegment = new File(ft.walArchive(),
FileDescriptor.fileName(segment) + ".zip");
+ File walRawSegment = new File(ft.walArchive(),
FileDescriptor.fileName(segment));
assertTrue(walZipSegment.exists());
assertFalse(walRawSegment.exists());
diff --git
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java
index e1401e6eb1e..db5ca30441f 100644
---
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java
+++
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java
@@ -404,7 +404,7 @@ public class IgnitePdsDataRegionMetricsTest extends
GridCommonAbstractTest {
boolean metaStore = METASTORAGE_CACHE_NAME.equals(cacheName);
- File cacheWorkDir = metaStore ? new File(pageStoreMgr.workDir(),
METASTORAGE_DIR_NAME) :
+ File cacheWorkDir = metaStore ? new
File(node.context().pdsFolderResolver().fileTree().nodeStorage(),
METASTORAGE_DIR_NAME) :
pageStoreMgr.cacheWorkDir(node.cachex(cacheName).configuration());
long totalPersistenceSize = 0;
diff --git
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/CheckpointMarkerReadingErrorOnStartTest.java
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/CheckpointMarkerReadingErrorOnStartTest.java
index 55d80b31dfa..da5b44ce268 100644
---
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/CheckpointMarkerReadingErrorOnStartTest.java
+++
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/CheckpointMarkerReadingErrorOnStartTest.java
@@ -26,7 +26,7 @@ import
org.apache.ignite.configuration.DataRegionConfiguration;
import org.apache.ignite.configuration.DataStorageConfiguration;
import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.internal.IgniteEx;
-import
org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings;
+import
org.apache.ignite.internal.processors.cache.persistence.filename.NodeFileTree;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
import org.junit.Test;
@@ -76,15 +76,11 @@ public class CheckpointMarkerReadingErrorOnStartTest
extends GridCommonAbstractT
forceCheckpoint();
- final PdsFolderSettings folderSettings =
ignite.context().pdsFolderResolver().resolveFolders();
-
- File storeWorkDir = new File(folderSettings.persistentStoreRootPath(),
folderSettings.folderName());
-
- File cpMarkersDir = new File(storeWorkDir, "cp");
+ final NodeFileTree ft =
ignite.context().pdsFolderResolver().fileTree();
stopGrid(0);
- File[] cpMarkers = cpMarkersDir.listFiles();
+ File[] cpMarkers = ft.checkpoint().listFiles();
assertNotNull(cpMarkers);
diff --git
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/LightweightCheckpointTest.java
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/LightweightCheckpointTest.java
index 16f8555e6bd..b3bafc32f5e 100644
---
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/LightweightCheckpointTest.java
+++
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/LightweightCheckpointTest.java
@@ -18,7 +18,6 @@
package org.apache.ignite.internal.processors.cache.persistence.db.checkpoint;
import java.io.File;
-import java.nio.file.Paths;
import java.util.Arrays;
import org.apache.ignite.IgniteCache;
import org.apache.ignite.IgniteCheckedException;
@@ -40,7 +39,6 @@ import
org.apache.ignite.internal.processors.cache.persistence.checkpoint.Lightw
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.pagemem.PageMemoryImpl;
-import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
import org.junit.Test;
@@ -195,8 +193,7 @@ public class LightweightCheckpointTest extends
GridCommonAbstractTest {
waitForCondition(() ->
!db2.getCheckpointer().currentProgress().inProgress(), 10_000);
- String nodeFolderName =
ignite0.context().pdsFolderResolver().resolveFolders().folderName();
- File cpMarkersDir = Paths.get(U.defaultWorkDirectory(), "db",
nodeFolderName, "cp").toFile();
+ File cpMarkersDir =
ignite0.context().pdsFolderResolver().fileTree().checkpoint();
//then: Expected only two pairs checkpoint markers - both from the
start of node.
assertEquals(4, cpMarkersDir.listFiles().length);
diff --git
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java
index 4fec9ef585a..f833c7c5c82 100644
---
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java
+++
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java
@@ -20,7 +20,6 @@ import java.io.File;
import java.io.FilenameFilter;
import java.io.RandomAccessFile;
import java.nio.ByteBuffer;
-import java.nio.file.Paths;
import java.util.Arrays;
import java.util.Collections;
import java.util.Set;
@@ -44,6 +43,7 @@ import
org.apache.ignite.internal.pagemem.wal.record.PageSnapshot;
import org.apache.ignite.internal.pagemem.wal.record.RolloverType;
import org.apache.ignite.internal.processors.cache.persistence.DummyPageIO;
import
org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
+import
org.apache.ignite.internal.processors.cache.persistence.filename.NodeFileTree;
import
org.apache.ignite.internal.processors.cache.persistence.wal.FileDescriptor;
import org.apache.ignite.internal.util.GridUnsafe;
import org.apache.ignite.internal.util.typedef.internal.U;
@@ -190,13 +190,9 @@ public class WalCompactionTest extends
GridCommonAbstractTest {
ig.context().cache().context().database().wakeupForCheckpoint("Forced
checkpoint").get();
ig.context().cache().context().database().wakeupForCheckpoint("Forced
checkpoint").get();
- String nodeFolderName =
ig.context().pdsFolderResolver().resolveFolders().folderName();
+ NodeFileTree ft = ig.context().pdsFolderResolver().fileTree();
- File dbDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), "db",
false);
- File walDir = new File(dbDir, "wal");
- File archiveDir = new File(walDir, "archive");
- File nodeArchiveDir = new File(archiveDir, nodeFolderName);
- File walSegment = new File(nodeArchiveDir, FileDescriptor.fileName(0)
+ ZIP_SUFFIX);
+ File walSegment = new File(ft.walArchive(), FileDescriptor.fileName(0)
+ ZIP_SUFFIX);
// Allow compressor to compress WAL segments.
assertTrue(GridTestUtils.waitForCondition(walSegment::exists, 15_000));
@@ -205,15 +201,12 @@ public class WalCompactionTest extends
GridCommonAbstractTest {
stopAllGrids();
- File nodeLfsDir = new File(dbDir, nodeFolderName);
- File cpMarkersDir = new File(nodeLfsDir, "cp");
-
- File[] cpMarkers = cpMarkersDir.listFiles();
+ File[] cpMarkers = ft.checkpoint().listFiles();
assertNotNull(cpMarkers);
assertTrue(cpMarkers.length > 0);
- File cacheDir = new File(nodeLfsDir, "cache-" + CACHE_NAME);
+ File cacheDir = new File(ft.nodeStorage(), "cache-" + CACHE_NAME);
File[] lfsFiles = cacheDir.listFiles();
assertNotNull(lfsFiles);
@@ -340,15 +333,11 @@ public class WalCompactionTest extends
GridCommonAbstractTest {
assertTrue(System.currentTimeMillis() - start < 15_000);
- String nodeFolderName =
ig.context().pdsFolderResolver().resolveFolders().folderName();
+ NodeFileTree ft = ig.context().pdsFolderResolver().fileTree();
stopAllGrids();
- File dbDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), "db",
false);
- File walDir = new File(dbDir, "wal");
- File archiveDir = new File(walDir, "archive");
- File nodeArchiveDir = new File(archiveDir, nodeFolderName);
- File walSegment = new File(nodeArchiveDir, FileDescriptor.fileName(0));
+ File walSegment = new File(ft.walArchive(),
FileDescriptor.fileName(0));
assertTrue("" + walSegment.length(), walSegment.length() <
200_000_000);
}
@@ -377,18 +366,14 @@ public class WalCompactionTest extends
GridCommonAbstractTest {
ig.context().cache().context().database().wakeupForCheckpoint("Forced
checkpoint").get();
ig.context().cache().context().database().wakeupForCheckpoint("Forced
checkpoint").get();
- String nodeFolderName =
ig.context().pdsFolderResolver().resolveFolders().folderName();
+ NodeFileTree ft = ig.context().pdsFolderResolver().fileTree();
stopAllGrids();
int emptyIdx = 5;
- File dbDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), "db",
false);
- File walDir = new File(dbDir, "wal");
- File archiveDir = new File(walDir, "archive");
- File nodeArchiveDir = new File(archiveDir, nodeFolderName);
- File walSegment = new File(nodeArchiveDir,
FileDescriptor.fileName(emptyIdx));
- File zippedWalSegment = new File(nodeArchiveDir,
FileDescriptor.fileName(emptyIdx + 1) + ZIP_SUFFIX);
+ File walSegment = new File(ft.walArchive(),
FileDescriptor.fileName(emptyIdx));
+ File zippedWalSegment = new File(ft.walArchive(),
FileDescriptor.fileName(emptyIdx + 1) + ZIP_SUFFIX);
long start = U.currentTimeMillis();
do {
@@ -414,7 +399,7 @@ public class WalCompactionTest extends
GridCommonAbstractTest {
// Allow compressor to compress WAL segments.
assertTrue(GridTestUtils.waitForCondition(zippedWalSegment::exists,
15_000));
- File[] compressedSegments = nodeArchiveDir.listFiles(new
FilenameFilter() {
+ File[] compressedSegments = ft.walArchive().listFiles(new
FilenameFilter() {
@Override public boolean accept(File dir, String name) {
return name.endsWith(".wal.zip");
}
@@ -431,7 +416,7 @@ public class WalCompactionTest extends
GridCommonAbstractTest {
assertTrue(maxIdx > emptyIdx);
if (!walSegment.exists()) {
- File[] list = nodeArchiveDir.listFiles();
+ File[] list = ft.walArchive().listFiles();
Arrays.sort(list);
@@ -466,13 +451,9 @@ public class WalCompactionTest extends
GridCommonAbstractTest {
ig.context().cache().context().database().wakeupForCheckpoint("Forced
checkpoint").get();
ig.context().cache().context().database().wakeupForCheckpoint("Forced
checkpoint").get();
- String nodeFolderName =
ig.context().pdsFolderResolver().resolveFolders().folderName();
-
- File dbDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), "db",
false);
- File nodeLfsDir = new File(dbDir, nodeFolderName);
- File cpMarkersDir = new File(nodeLfsDir, "cp");
+ NodeFileTree ft = ig.context().pdsFolderResolver().fileTree();
- Set<String> cpMarkersToSave =
Arrays.stream(cpMarkersDir.listFiles()).map(File::getName).collect(toSet());
+ Set<String> cpMarkersToSave =
Arrays.stream(ft.checkpoint().listFiles()).map(File::getName).collect(toSet());
assertTrue(cpMarkersToSave.size() >= 2);
@@ -503,9 +484,8 @@ public class WalCompactionTest extends
GridCommonAbstractTest {
ig.context().cache().context().database().wakeupForCheckpoint("Forced
checkpoint").get();
ig.context().cache().context().database().wakeupForCheckpoint("Forced
checkpoint").get();
- File nodeArchiveDir = dbDir.toPath().resolve(Paths.get("wal",
"archive", nodeFolderName)).toFile();
- File unzippedWalSegment = new File(nodeArchiveDir,
FileDescriptor.fileName(0));
- File walSegment = new File(nodeArchiveDir, FileDescriptor.fileName(0)
+ ZIP_SUFFIX);
+ File unzippedWalSegment = new File(ft.walArchive(),
FileDescriptor.fileName(0));
+ File walSegment = new File(ft.walArchive(), FileDescriptor.fileName(0)
+ ZIP_SUFFIX);
// Allow compressor to compress WAL segments.
assertTrue(GridTestUtils.waitForCondition(() ->
!unzippedWalSegment.exists(), 15_000));
@@ -515,12 +495,12 @@ public class WalCompactionTest extends
GridCommonAbstractTest {
stopAllGrids();
- File[] cpMarkers = cpMarkersDir.listFiles((dir, name) ->
!cpMarkersToSave.contains(name));
+ File[] cpMarkers = ft.checkpoint().listFiles((dir, name) ->
!cpMarkersToSave.contains(name));
assertNotNull(cpMarkers);
assertTrue(cpMarkers.length > 0);
- File cacheDir = new File(nodeLfsDir, "cache-" + CACHE_NAME);
+ File cacheDir = new File(ft.nodeStorage(), "cache-" + CACHE_NAME);
File[] lfsFiles = cacheDir.listFiles();
assertNotNull(lfsFiles);
diff --git
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java
index 6fe799a25c3..340b84d5c0c 100644
---
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java
+++
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java
@@ -221,7 +221,7 @@ public abstract class WalDeletionArchiveAbstractTest
extends GridCommonAbstractT
assertTrue(hist.checkpoints().size() < checkpointCnt + startHistSize);
- File[] cpFiles = dbMgr.checkpointDirectory().listFiles();
+ File[] cpFiles =
((IgniteEx)ignite).context().pdsFolderResolver().fileTree().checkpoint().listFiles();
assertTrue(cpFiles.length <= (checkpointCnt * 2 + 1)); // starts &
ends + node_start
}
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 76f691ae2b0..1555a3b67f7 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
@@ -1129,13 +1129,11 @@ public class IgniteWalReaderTest extends
GridCommonAbstractTest {
ignite.cluster().state(ClusterState.INACTIVE);
- String subfolderName1 = genDbSubfolderName(ignite, 0);
- String subfolderName2 = genDbSubfolderName(ignite1, 1);
+ NodeFileTree ft1 = nodeFileTree(genDbSubfolderName(ignite, 0));
+ NodeFileTree ft2 = nodeFileTree(genDbSubfolderName(ignite1, 1));
stopAllGrids();
- String workDir = U.defaultWorkDirectory();
-
IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log);
Map<GridCacheOperation, Integer> operationsFound = new
EnumMap<>(GridCacheOperation.class);
@@ -1153,11 +1151,7 @@ public class IgniteWalReaderTest extends
GridCommonAbstractTest {
scanIterateAndCount(
factory,
- createIteratorParametersBuilder(workDir, subfolderName1)
- .filesOrDirs(
- workDir + "/db/wal/" + subfolderName1,
- workDir + "/db/wal/archive/" + subfolderName1
- ),
+ createIteratorParametersBuilder(ft1.root().getAbsolutePath(),
ft1.folderName()).filesOrDirs(ft1.wal(), ft1.walArchive()),
1,
1,
null, drHnd
@@ -1167,11 +1161,7 @@ public class IgniteWalReaderTest extends
GridCommonAbstractTest {
scanIterateAndCount(
factory,
- createIteratorParametersBuilder(workDir, subfolderName2)
- .filesOrDirs(
- workDir + "/db/wal/" + subfolderName2,
- workDir + "/db/wal/archive/" + subfolderName2
- ),
+ createIteratorParametersBuilder(ft2.root().getAbsolutePath(),
ft2.folderName()).filesOrDirs(ft2.wal(), ft2.walArchive()),
1,
1,
null,
@@ -1215,14 +1205,12 @@ public class IgniteWalReaderTest extends
GridCommonAbstractTest {
awaitPartitionMapExchange(false, true, null);
- String subfolderName1 = genDbSubfolderName(ignite, 0);
- String subfolderName2 = genDbSubfolderName(ignite1, 1);
- String subfolderName3 = genDbSubfolderName(ignite2, 2);
+ NodeFileTree ft1 = nodeFileTree(genDbSubfolderName(ignite, 0));
+ NodeFileTree ft2 = nodeFileTree(genDbSubfolderName(ignite1, 1));
+ NodeFileTree ft3 = nodeFileTree(genDbSubfolderName(ignite2, 2));
stopAllGrids();
- String workDir = U.defaultWorkDirectory();
-
IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log);
Map<GridCacheOperation, Integer> operationsFound = new
EnumMap<>(GridCacheOperation.class);
@@ -1240,11 +1228,7 @@ public class IgniteWalReaderTest extends
GridCommonAbstractTest {
scanIterateAndCount(
factory,
- createIteratorParametersBuilder(workDir, subfolderName1)
- .filesOrDirs(
- workDir + "/db/wal/" + subfolderName1,
- workDir + "/db/wal/archive/" + subfolderName1
- ),
+ createIteratorParametersBuilder(ft1.root().getAbsolutePath(),
ft1.folderName()).filesOrDirs(ft1.wal(), ft2.walArchive()),
1,
1,
null, drHnd
@@ -1254,11 +1238,7 @@ public class IgniteWalReaderTest extends
GridCommonAbstractTest {
scanIterateAndCount(
factory,
- createIteratorParametersBuilder(workDir, subfolderName2)
- .filesOrDirs(
- workDir + "/db/wal/" + subfolderName2,
- workDir + "/db/wal/archive/" + subfolderName2
- ),
+ createIteratorParametersBuilder(ft2.root().getAbsolutePath(),
ft2.folderName()).filesOrDirs(ft2.wal(), ft2.walArchive()),
1,
1,
null,
@@ -1269,11 +1249,7 @@ public class IgniteWalReaderTest extends
GridCommonAbstractTest {
scanIterateAndCount(
factory,
- createIteratorParametersBuilder(workDir, subfolderName3)
- .filesOrDirs(
- workDir + "/db/wal/" + subfolderName3,
- workDir + "/db/wal/archive/" + subfolderName3
- ),
+ createIteratorParametersBuilder(ft3.root().getAbsolutePath(),
ft3.folderName()).filesOrDirs(ft3.wal(), ft3.walArchive()),
1,
0,
null,
@@ -1304,13 +1280,11 @@ public class IgniteWalReaderTest extends
GridCommonAbstractTest {
ignite.cluster().state(ClusterState.INACTIVE);
- String subfolderName1 = genDbSubfolderName(ignite, 0);
- String subfolderName2 = genDbSubfolderName(ignite1, 1);
+ NodeFileTree ft1 = nodeFileTree(genDbSubfolderName(ignite, 0));
+ NodeFileTree ft2 = nodeFileTree(genDbSubfolderName(ignite1, 1));
stopAllGrids();
- String workDir = U.defaultWorkDirectory();
-
IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log);
StringBuilder sb = new StringBuilder();
@@ -1346,11 +1320,7 @@ public class IgniteWalReaderTest extends
GridCommonAbstractTest {
scanIterateAndCount(
factory,
- createIteratorParametersBuilder(workDir, subfolderName1)
- .filesOrDirs(
- workDir + "/db/wal/" + subfolderName1,
- workDir + "/db/wal/archive/" + subfolderName1
- ),
+ createIteratorParametersBuilder(ft1.root().getAbsolutePath(),
ft1.folderName()).filesOrDirs(ft1.wal(), ft1.walArchive()),
1,
1,
null, drHnd
@@ -1358,11 +1328,7 @@ public class IgniteWalReaderTest extends
GridCommonAbstractTest {
scanIterateAndCount(
factory,
- createIteratorParametersBuilder(workDir, subfolderName2)
- .filesOrDirs(
- workDir + "/db/wal/" + subfolderName2,
- workDir + "/db/wal/archive/" + subfolderName2
- ),
+ createIteratorParametersBuilder(ft2.root().getAbsolutePath(),
ft2.folderName()).filesOrDirs(ft2.wal(), ft2.walArchive()),
1,
1,
null,
diff --git
a/modules/core/src/test/java/org/apache/ignite/internal/processors/security/maintenance/MaintenanceModeNodeSecurityTest.java
b/modules/core/src/test/java/org/apache/ignite/internal/processors/security/maintenance/MaintenanceModeNodeSecurityTest.java
index 1b22fed4dce..cd712fc56b9 100644
---
a/modules/core/src/test/java/org/apache/ignite/internal/processors/security/maintenance/MaintenanceModeNodeSecurityTest.java
+++
b/modules/core/src/test/java/org/apache/ignite/internal/processors/security/maintenance/MaintenanceModeNodeSecurityTest.java
@@ -27,9 +27,9 @@ import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.internal.IgniteEx;
import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
import
org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import
org.apache.ignite.internal.processors.cache.persistence.filename.NodeFileTree;
import org.apache.ignite.internal.processors.security.AbstractSecurityTest;
import
org.apache.ignite.internal.processors.security.AbstractTestSecurityPluginProvider;
-import org.apache.ignite.internal.util.typedef.internal.U;
import org.junit.Test;
import static org.apache.ignite.cluster.ClusterState.ACTIVE;
@@ -63,10 +63,7 @@ public class MaintenanceModeNodeSecurityTest extends
AbstractSecurityTest {
.setAffinity(new RendezvousAffinityFunction(false, 32))
.setBackups(1));
- String srvPdsFolder =
srv.context().pdsFolderResolver().resolveFolders().folderName();
- File dbDir =
U.resolveWorkDirectory(srv.configuration().getWorkDirectory(), "db", false);
-
- File srvLfsDir = new File(dbDir, srvPdsFolder);
+ NodeFileTree ft = srv.context().pdsFolderResolver().fileTree();
for (int k = 0; k < 1000; k++)
cache.put(k, k);
@@ -84,7 +81,7 @@ public class MaintenanceModeNodeSecurityTest extends
AbstractSecurityTest {
stopGrid(1);
- File[] cpMarkers = new File(srvLfsDir, "cp").listFiles();
+ File[] cpMarkers = ft.checkpoint().listFiles();
for (File cpMark : cpMarkers) {
if (cpMark.getName().contains("-END"))
diff --git
a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
index af8658cfdda..70db19dad97 100644
---
a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
+++
b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
@@ -101,8 +101,6 @@ import
org.apache.ignite.internal.processors.cache.GridCacheContext;
import
org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
import
org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
import
org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter;
-import
org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
-import
org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
import org.apache.ignite.internal.processors.odbc.ClientListenerProcessor;
import org.apache.ignite.internal.processors.port.GridPortRecord;
import org.apache.ignite.internal.util.GridBusyLock;
@@ -2355,9 +2353,7 @@ public final class GridTestUtils {
* @param ignite Ignite node.
*/
public static void deleteLastCheckpointEndMarker(IgniteEx ignite) throws
IOException {
- IgniteCacheDatabaseSharedManager dbSharedMgr =
ignite.context().cache().context().database();
-
- Path cpDir =
((GridCacheDatabaseSharedManager)dbSharedMgr).checkpointDirectory().toPath();
+ Path cpDir =
ignite.context().pdsFolderResolver().fileTree().checkpoint().toPath();
try (Stream<Path> files = Files.list(cpDir)) {
Optional<Path> endMarker = files
diff --git
a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index bead48ae60f..11cab394765 100755
---
a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++
b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -111,6 +111,7 @@ import
org.apache.ignite.internal.processors.cache.distributed.dht.topology.Grid
import
org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter;
import
org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
import
org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
+import
org.apache.ignite.internal.processors.cache.persistence.filename.NodeFileTree;
import
org.apache.ignite.internal.processors.cache.persistence.filename.SharedFileTree;
import
org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage;
import
org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
@@ -140,6 +141,7 @@ import org.apache.ignite.marshaller.jdk.JdkMarshaller;
import org.apache.ignite.mxbean.MXBeanDescription;
import org.apache.ignite.resources.IgniteInstanceResource;
import org.apache.ignite.resources.LoggerResource;
+import org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointSpi;
import org.apache.ignite.testframework.GridTestNode;
import org.apache.ignite.testframework.GridTestUtils;
import org.apache.ignite.testframework.junits.GridAbstractTest;
@@ -1970,7 +1972,7 @@ public abstract class GridCommonAbstractTest extends
GridAbstractTest {
protected void cleanPersistenceDir(boolean saveSnp) throws Exception {
assertTrue("Grids are not stopped", F.isEmpty(G.allGrids()));
- U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), "cp",
false));
+ U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(),
SharedFsCheckpointSpi.DFLT_ROOT, false));
U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(),
DFLT_STORE_DIR, false));
SharedFileTree sft = sharedFileTree();
@@ -1985,12 +1987,14 @@ public abstract class GridCommonAbstractTest extends
GridAbstractTest {
/**
* @param consistentId Node consistentId.
*/
- protected void cleanPersistenceDir(String consistentId) throws Exception {
+ protected void cleanPersistenceDir(String consistentId) {
String dn2DirName = consistentId.replace(".", "_");
- U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(),
DFLT_STORE_DIR + "/" + dn2DirName, true));
- U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(),
DFLT_STORE_DIR + "/wal/" + dn2DirName, true));
- U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(),
DFLT_STORE_DIR + "/wal/archive/" + dn2DirName, true));
+ NodeFileTree ft = nodeFileTree(dn2DirName);
+
+ U.delete(ft.nodeStorage());
+ U.delete(ft.wal());
+ U.delete(ft.walArchive());
}
/**
diff --git
a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterSensitiveDataTest.java
b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterSensitiveDataTest.java
index 866b5087e9b..a5d5f9f55cf 100644
---
a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterSensitiveDataTest.java
+++
b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterSensitiveDataTest.java
@@ -18,7 +18,6 @@
package org.apache.ignite.development.utils;
import java.io.ByteArrayOutputStream;
-import java.io.File;
import java.io.PrintStream;
import java.io.Serializable;
import java.nio.charset.StandardCharsets;
@@ -130,11 +129,7 @@ public class IgniteWalConverterSensitiveDataTest extends
GridCommonAbstractTest
IgniteConfiguration cfg = crd.configuration();
- String wd = cfg.getWorkDirectory();
- String wp = cfg.getDataStorageConfiguration().getWalPath();
- String fn =
kernalCtx.pdsFolderResolver().resolveFolders().folderName();
-
- walDirPath = wd + File.separator + wp + File.separator + fn;
+ walDirPath =
kernalCtx.pdsFolderResolver().fileTree().wal().getAbsolutePath();
pageSize = cfg.getDataStorageConfiguration().getPageSize();
stopGrid(nodeId);
diff --git
a/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/LinuxNativeIoPluginProvider.java
b/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/LinuxNativeIoPluginProvider.java
index 9feb39f9f28..43aca728988 100644
---
a/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/LinuxNativeIoPluginProvider.java
+++
b/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/LinuxNativeIoPluginProvider.java
@@ -175,7 +175,7 @@ public class LinuxNativeIoPluginProvider implements
PluginProvider {
final AlignedBuffersDirectFileIOFactory factory = new
AlignedBuffersDirectFileIOFactory(
ignite.log(),
- pageStore.workDir(),
+ ignite.context().pdsFolderResolver().fileTree().nodeStorage(),
pageStore.pageSize(),
backupIoFactory);
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 9494be9f7a5..907ea597678 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
@@ -130,6 +130,7 @@ import
org.apache.ignite.transactions.TransactionConcurrency;
import org.apache.ignite.transactions.TransactionIsolation;
import org.junit.Assert;
import org.junit.Test;
+
import static
org.apache.ignite.IgniteSystemProperties.IGNITE_DISABLE_WAL_DURING_REBALANCING;
import static
org.apache.ignite.configuration.DataStorageConfiguration.DFLT_CHECKPOINT_FREQ;
import static
org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGNITE_INSTANCE_NAME;
@@ -567,7 +568,7 @@ public class IgniteWalRecoveryTest extends
GridCommonAbstractTest {
String cpEndFileName =
CheckpointMarkersStorage.checkpointFileName(cpEntry, CheckpointEntryType.END);
-
Files.delete(Paths.get(dbMgr.checkpointDirectory().getAbsolutePath(),
cpEndFileName));
+
Files.delete(Paths.get(ig2.context().pdsFolderResolver().fileTree().checkpoint().getAbsolutePath(),
cpEndFileName));
log.info("Checkpoint marker removed [cpEndFileName=" +
cpEndFileName + ']');
}
diff --git
a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotMetricsTest.java
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotMetricsTest.java
index 1c3ef2a3c2e..ffb9bc69af3 100644
---
a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotMetricsTest.java
+++
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotMetricsTest.java
@@ -47,7 +47,6 @@ import
org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
import
org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
import
org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
import
org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
-import
org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings;
import
org.apache.ignite.internal.processors.configuration.distributed.DistributedChangeableProperty;
import org.apache.ignite.internal.util.typedef.F;
import org.apache.ignite.internal.util.typedef.G;
@@ -287,10 +286,7 @@ public class IgniteClusterSnapshotMetricsTest extends
IgniteClusterSnapshotResto
assertEquals(-1, processedSize.value());
// Calculate transfer rate limit.
- PdsFolderSettings<?> folderSettings =
ignite.context().pdsFolderResolver().resolveFolders();
- File storeWorkDir = new File(folderSettings.persistentStoreRootPath(),
folderSettings.folderName());
-
- long rate = FileUtils.sizeOfDirectory(storeWorkDir) / 5;
+ long rate =
FileUtils.sizeOfDirectory(ignite.context().pdsFolderResolver().fileTree().nodeStorage())
/ 5;
// Limit snapshot transfer rate.
DistributedChangeableProperty<Serializable> rateProp =
diff --git
a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildTest.java
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildTest.java
index 8ecdbc3f7af..e0208decc50 100644
---
a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildTest.java
+++
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildTest.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.query.h2;
import java.io.File;
import java.nio.file.Files;
-import java.nio.file.Paths;
import java.util.Collections;
import java.util.Date;
import java.util.LinkedHashMap;
@@ -47,6 +46,7 @@ import
org.apache.ignite.internal.management.cache.CacheValidateIndexesCommandAr
import org.apache.ignite.internal.management.cache.ValidateIndexesJobResult;
import org.apache.ignite.internal.management.cache.ValidateIndexesTask;
import org.apache.ignite.internal.management.cache.ValidateIndexesTaskResult;
+import
org.apache.ignite.internal.processors.cache.persistence.filename.NodeFileTree;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.internal.visor.VisorTaskArgument;
import org.apache.ignite.testframework.ListeningTestLogger;
@@ -341,21 +341,14 @@ public class GridIndexRebuildTest extends
GridCommonAbstractTest {
/** */
private void cleanPersistenceFiles(String igName) throws Exception {
- String ig1DbPath = Paths.get(DFLT_STORE_DIR, igName).toString();
+ NodeFileTree ft = nodeFileTree(igName);
- File igDbDir = U.resolveWorkDirectory(U.defaultWorkDirectory(),
ig1DbPath, false);
+ U.delete(ft.nodeStorage());
- U.delete(igDbDir);
+ Files.createDirectory(ft.nodeStorage().toPath());
- Files.createDirectory(igDbDir.toPath());
-
- String ig1DbWalPath = Paths.get(DFLT_STORE_DIR, "wal",
igName).toString();
-
- U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(),
ig1DbWalPath, false));
-
- ig1DbWalPath = Paths.get(DFLT_STORE_DIR, "wal", "archive",
igName).toString();
-
- U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(),
ig1DbWalPath, false));
+ U.delete(ft.wal());
+ U.delete(ft.walArchive());
}
/** */