From 035677cf22d767d677086dd64ba98bf247721342 Mon Sep 17 00:00:00 2001 From: nizhikov Date: Wed, 5 Jul 2023 15:01:20 +0300 Subject: [PATCH 1/7] IGNITE-19915 WIP --- .../cache/GridCacheSharedContext.java | 7 ---- .../GridCacheDatabaseSharedManager.java | 15 -------- .../checkpoint/CheckpointManager.java | 7 +--- .../checkpoint/CheckpointPagesWriter.java | 13 ------- .../CheckpointPagesWriterFactory.java | 8 ----- .../checkpoint/CheckpointProgressImpl.java | 35 ------------------- .../checkpoint/CheckpointWorkflow.java | 25 +------------ .../persistence/checkpoint/Checkpointer.java | 34 ------------------ .../LightweightCheckpointManager.java | 6 ---- 9 files changed, 2 insertions(+), 148 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java index b14064dafc1aa..7685e6e038423 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java @@ -755,13 +755,6 @@ public IgniteCacheDatabaseSharedManager database() { return dbMgr; } - /** - * @return Snapshot manager. - */ - public IgniteCacheSnapshotManager snapshot() { - return snpMgr; - } - /** * @return Page store manager. {@code Null} if persistence is not enabled. */ 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 7cb74ed16d1fe..5b7a4710d5963 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 @@ -128,7 +128,6 @@ import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImpl; import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageReadWriteManager; import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager; import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO; import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; @@ -327,9 +326,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan /** This is the earliest WAL pointer that was reserved during preloading. */ private final AtomicReference reservedForPreloading = new AtomicReference<>(); - /** Snapshot manager. */ - private IgniteCacheSnapshotManager snapshotMgr; - /** * MetaStorage instance. Value {@code null} means storage not initialized yet. * Guarded by {@link GridCacheDatabaseSharedManager#checkpointReadLock()} @@ -531,8 +527,6 @@ private DataRegionConfiguration createDefragmentationMappingRegionConfig(long re @Override protected void start0() throws IgniteCheckedException { super.start0(); - snapshotMgr = cctx.snapshot(); - IgnitePageStoreManager store = cctx.pageStore(); assert store instanceof FilePageStoreManager : "Invalid page store manager was created: " + store; @@ -570,7 +564,6 @@ private DataRegionConfiguration createDefragmentationMappingRegionConfig(long re this::cacheGroupContexts, this::getPageMemoryForCacheGroup, resolveThrottlingPolicy(), - snapshotMgr, dataStorageMetricsImpl(), kernalCtx.longJvmPauseDetector(), kernalCtx.failure(), @@ -816,7 +809,6 @@ private void prepareCacheDefragmentation(List cacheNames) throws IgniteC () -> regions, this::getPageMemoryForCacheGroup, resolveThrottlingPolicy(), - snapshotMgr, dataStorageMetricsImpl(), kernalCtx.longJvmPauseDetector(), kernalCtx.failure(), @@ -919,8 +911,6 @@ private void readMetastore() throws IgniteCheckedException { log.debug("Activate database manager [id=" + cctx.localNodeId() + " topVer=" + cctx.discovery().topologyVersionEx() + " ]"); - snapshotMgr = cctx.snapshot(); - checkpointManager.init(); super.onActivate(ctx); @@ -1245,9 +1235,6 @@ private long[] calculateFragmentSizes(String regionName, int concLvl, long cache (fullId, pageBuf, tag) -> { memMetrics.onPageWritten(); - // We can write only page from disk into snapshot. - snapshotMgr.beforePageWrite(fullId); - // Write page to disk. pmPageMgr.write(fullId.groupId(), fullId.pageId(), pageBuf, tag, true); @@ -1568,8 +1555,6 @@ private String cacheInfo(GridCacheContext cacheCtx) { if (!gctx.persistenceEnabled()) continue; - snapshotMgr.onCacheGroupStop(gctx, destroy); - PageMemoryEx pageMem = (PageMemoryEx)dataRegion.pageMemory(); Collection grpIds = destroyed.computeIfAbsent(pageMem, k -> new HashSet<>()); 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 8a45435845b28..9445307b1f533 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 @@ -40,7 +40,6 @@ import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; 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.snapshot.IgniteCacheSnapshotManager; import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; import org.apache.ignite.internal.processors.failure.FailureProcessor; import org.apache.ignite.internal.util.StripedExecutor; @@ -101,7 +100,6 @@ public class CheckpointManager { * @param cacheGroupContexts Cache group contexts. * @param pageMemoryGroupResolver Page memory resolver. * @param throttlingPolicy Throttling policy. - * @param snapshotMgr Snapshot manager. * @param persStoreMetrics Persistence metrics. * @param longJvmPauseDetector Long JVM pause detector. * @param failureProcessor Failure processor. @@ -123,7 +121,6 @@ public CheckpointManager( Supplier> cacheGroupContexts, IgniteThrowableFunction pageMemoryGroupResolver, PageMemoryImpl.ThrottlingPolicy throttlingPolicy, - IgniteCacheSnapshotManager snapshotMgr, DataStorageMetricsImpl persStoreMetrics, LongJVMPauseDetector longJvmPauseDetector, FailureProcessor failureProcessor, @@ -155,7 +152,6 @@ public CheckpointManager( checkpointWorkflow = new CheckpointWorkflow( logger, wal, - snapshotMgr, checkpointMarkersStorage, lock, persistenceCfg.getCheckpointWriteOrder(), @@ -177,7 +173,7 @@ public CheckpointManager( }; checkpointPagesWriterFactory = new CheckpointPagesWriterFactory( - logger, snapshotMgr, + logger, (pageMemEx, fullPage, buf, tag) -> pageStoreManager.write(fullPage.groupId(), fullPage.pageId(), buf, tag, true), persStoreMetrics, throttlingPolicy, threadBuf, @@ -191,7 +187,6 @@ public CheckpointManager( logger, longJvmPauseDetector, failureProcessor, - snapshotMgr, persStoreMetrics, cacheProcessor, checkpointWorkflow, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointPagesWriter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointPagesWriter.java index 5bb7364ece9c0..9349ee1b38f69 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointPagesWriter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointPagesWriter.java @@ -33,7 +33,6 @@ import org.apache.ignite.internal.processors.cache.persistence.pagemem.CheckpointMetricsTracker; 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.snapshot.IgniteCacheSnapshotManager; import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; import org.apache.ignite.internal.util.GridConcurrentMultiPairQueue; import org.apache.ignite.internal.util.future.CountDownFuture; @@ -66,9 +65,6 @@ public class CheckpointPagesWriter implements Runnable { /** Some action which will be executed every time before page will be written. */ private final Runnable beforePageWrite; - /** Snapshot manager. */ - private final IgniteCacheSnapshotManager snapshotMgr; - /** Data storage metrics. */ private final DataStorageMetricsImpl persStoreMetrics; @@ -98,7 +94,6 @@ public class CheckpointPagesWriter implements Runnable { * @param updStores Updating storage. * @param doneFut Done future. * @param beforePageWrite Action to be performed before every page write. - * @param snapshotManager Snapshot manager. * @param log Logger. * @param dsMetrics Data storage metrics. * @param buf Thread local byte buffer. @@ -114,7 +109,6 @@ public class CheckpointPagesWriter implements Runnable { ConcurrentLinkedHashMap updStores, CountDownFuture doneFut, Runnable beforePageWrite, - IgniteCacheSnapshotManager snapshotManager, IgniteLogger log, DataStorageMetricsImpl dsMetrics, ThreadLocal buf, @@ -129,7 +123,6 @@ public class CheckpointPagesWriter implements Runnable { this.updStores = updStores; this.doneFut = doneFut; this.beforePageWrite = beforePageWrite; - this.snapshotMgr = snapshotManager; this.log = log; this.persStoreMetrics = dsMetrics; this.threadBuf = buf; @@ -142,8 +135,6 @@ public class CheckpointPagesWriter implements Runnable { /** {@inheritDoc} */ @Override public void run() { - snapshotMgr.beforeCheckpointPageWritten(); - GridConcurrentMultiPairQueue writePageIds = this.writePageIds; try { @@ -198,8 +189,6 @@ private GridConcurrentMultiPairQueue writePages( PageMemoryEx pageMem = res.getKey(); - snapshotMgr.beforePageWrite(fullId); - tmpWriteBuf.rewind(); PageStoreWriter pageStoreWriter = @@ -214,8 +203,6 @@ private GridConcurrentMultiPairQueue writePages( if (cpPageId.equals(FullPageId.NULL_PAGE)) break; - snapshotMgr.beforePageWrite(cpPageId); - tmpWriteBuf.rewind(); pageMem.checkpointWritePage(cpPageId, tmpWriteBuf, pageStoreWriter, tracker); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointPagesWriterFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointPagesWriterFactory.java index 8c882e1473eeb..4713b83e0e3b2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointPagesWriterFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointPagesWriterFactory.java @@ -34,7 +34,6 @@ import org.apache.ignite.internal.processors.cache.persistence.pagemem.CheckpointMetricsTracker; 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.snapshot.IgniteCacheSnapshotManager; import org.apache.ignite.internal.util.GridConcurrentMultiPairQueue; import org.apache.ignite.internal.util.future.CountDownFuture; import org.apache.ignite.internal.util.lang.IgniteThrowableFunction; @@ -50,9 +49,6 @@ public class CheckpointPagesWriterFactory { /** Logger. */ private final IgniteLogger log; - /** Snapshot manager. */ - private final IgniteCacheSnapshotManager snapshotMgr; - /** Data storage metrics. */ private final DataStorageMetricsImpl persStoreMetrics; @@ -70,7 +66,6 @@ public class CheckpointPagesWriterFactory { /** * @param logger Logger. - * @param snapshotMgr Snapshot manager. * @param checkpointPageWriter Checkpoint page writer. * @param persStoreMetrics Persistence metrics. * @param throttlingPolicy Throttling policy. @@ -79,14 +74,12 @@ public class CheckpointPagesWriterFactory { */ CheckpointPagesWriterFactory( Function, IgniteLogger> logger, - IgniteCacheSnapshotManager snapshotMgr, CheckpointPagesWriter.CheckpointPageWriter checkpointPageWriter, DataStorageMetricsImpl persStoreMetrics, PageMemoryImpl.ThrottlingPolicy throttlingPolicy, ThreadLocal threadBuf, IgniteThrowableFunction pageMemoryGroupResolver ) { - this.snapshotMgr = snapshotMgr; this.log = logger.apply(getClass()); this.persStoreMetrics = persStoreMetrics; this.threadBuf = threadBuf; @@ -120,7 +113,6 @@ CheckpointPagesWriter build( updStores, doneWriteFut, beforePageWrite, - snapshotMgr, log, persStoreMetrics, threadBuf, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointProgressImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointProgressImpl.java index 1f0e7cebd471d..f45c48bcb029b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointProgressImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointProgressImpl.java @@ -24,7 +24,6 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.cache.persistence.CheckpointState; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotOperation; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.U; @@ -50,12 +49,6 @@ public class CheckpointProgressImpl implements CheckpointProgress { /** Cause of fail, which has happened during the checkpoint or null if checkpoint was successful. */ private volatile Throwable failCause; - /** Flag indicates that snapshot operation will be performed after checkpoint. */ - private volatile boolean nextSnapshot; - - /** Snapshot operation that should be performed if {@link #nextSnapshot} set to true. */ - private volatile SnapshotOperation snapshotOperation; - /** Partitions destroy queue. */ private final PartitionDestroyQueue destroyQueue = new PartitionDestroyQueue(); @@ -162,13 +155,6 @@ private void doFinishFuturesWhichLessOrEqualTo(@NotNull CheckpointState lastStat return destroyQueue; } - /** - * @return Flag indicates that snapshot operation will be performed after checkpoint. - */ - public boolean nextSnapshot() { - return nextSnapshot; - } - /** * @return Scheduled time of checkpoint. */ @@ -195,27 +181,6 @@ public void reason(String reason) { this.reason = reason; } - /** - * @return Snapshot operation that should be performed if set to true. - */ - public SnapshotOperation snapshotOperation() { - return snapshotOperation; - } - - /** - * @param snapshotOperation New snapshot operation that should be performed if set to true. - */ - public void snapshotOperation(SnapshotOperation snapshotOperation) { - this.snapshotOperation = snapshotOperation; - } - - /** - * @param nextSnapshot New flag indicates that snapshot operation will be performed after checkpoint. - */ - public void nextSnapshot(boolean nextSnapshot) { - this.nextSnapshot = nextSnapshot; - } - /** {@inheritDoc} */ @Override public AtomicInteger writtenPagesCounter() { return writtenPagesCntr; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointWorkflow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointWorkflow.java index 4fda960a3ae92..bf6518933ad42 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointWorkflow.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointWorkflow.java @@ -60,7 +60,6 @@ import org.apache.ignite.internal.processors.cache.persistence.pagemem.CheckpointMetricsTracker; import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx; import org.apache.ignite.internal.processors.cache.persistence.partstate.PartitionAllocationMap; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager; import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.GridConcurrentMultiPairQueue; @@ -72,7 +71,6 @@ import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.util.worker.WorkProgressDispatcher; -import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.thread.IgniteThreadPoolExecutor; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentLinkedHashMap; @@ -129,9 +127,6 @@ public class CheckpointWorkflow { /** Write ahead log. */ private final IgniteWriteAheadLogManager wal; - /** Snapshot manager. */ - private final IgniteCacheSnapshotManager snapshotMgr; - /** Checkpoint lock. */ private final CheckpointReadWriteLock checkpointReadWriteLock; @@ -165,7 +160,6 @@ public class CheckpointWorkflow { /** * @param logger Logger. * @param wal WAL manager. - * @param snapshotManager Snapshot manager. * @param checkpointMarkersStorage Checkpoint mark storage. * @param checkpointReadWriteLock Checkpoint read write lock. * @param checkpointWriteOrder Checkpoint write order. @@ -177,7 +171,6 @@ public class CheckpointWorkflow { CheckpointWorkflow( Function, IgniteLogger> logger, IgniteWriteAheadLogManager wal, - IgniteCacheSnapshotManager snapshotManager, CheckpointMarkersStorage checkpointMarkersStorage, CheckpointReadWriteLock checkpointReadWriteLock, CheckpointWriteOrder checkpointWriteOrder, @@ -187,7 +180,6 @@ public class CheckpointWorkflow { String igniteInstanceName ) { this.wal = wal; - this.snapshotMgr = snapshotManager; this.checkpointReadWriteLock = checkpointReadWriteLock; this.dataRegions = dataRegions; this.cacheGroupsContexts = cacheGroupContexts; @@ -240,8 +232,6 @@ public Checkpoint markCheckpointBegin( memoryRecoveryRecordPtr = null; - IgniteFuture snapFut = null; - CheckpointPagesInfoHolder cpPagesHolder; int dirtyPagesCount; @@ -283,9 +273,6 @@ curr, new PartitionAllocationMap(), checkpointCollectPagesInfoPool, workProgress tracker.onListenersExecuteEnd(); - if (curr.nextSnapshot()) - snapFut = snapshotMgr.onMarkCheckPointBegin(curr.snapshotOperation(), ctx0.partitionStatMap()); - fillCacheGroupState(cpRec); //There are allowable to replace pages only after checkpoint entry was stored to disk. @@ -297,7 +284,7 @@ curr, new PartitionAllocationMap(), checkpointCollectPagesInfoPool, workProgress hasPartitionsToDestroy = !curr.getDestroyQueue().pendingReqs().isEmpty(); - if (dirtyPagesCount > 0 || curr.nextSnapshot() || hasPartitionsToDestroy) { + if (dirtyPagesCount > 0 || hasPartitionsToDestroy) { // No page updates for this checkpoint are allowed from now on. if (wal != null) cpPtr = wal.log(cpRec); @@ -319,16 +306,6 @@ curr, new PartitionAllocationMap(), checkpointCollectPagesInfoPool, workProgress for (CheckpointListener lsnr : dbLsnrs) lsnr.onCheckpointBegin(ctx0); - if (snapFut != null) { - try { - snapFut.get(); - } - catch (IgniteException e) { - U.error(log, "Failed to wait for snapshot operation initialization: " + - curr.snapshotOperation(), e); - } - } - if (dirtyPagesCount > 0 || hasPartitionsToDestroy) { tracker.onWalCpRecordFsyncStart(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/Checkpointer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/Checkpointer.java index a870f19481059..e09d91451a027 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/Checkpointer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/Checkpointer.java @@ -48,8 +48,6 @@ import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager; import org.apache.ignite.internal.processors.cache.persistence.pagemem.CheckpointMetricsTracker; import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotOperation; import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; import org.apache.ignite.internal.processors.failure.FailureProcessor; import org.apache.ignite.internal.processors.performancestatistics.PerformanceStatisticsProcessor; @@ -76,7 +74,6 @@ import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION; import static org.apache.ignite.internal.LongJVMPauseDetector.DEFAULT_JVM_PAUSE_DETECTOR_THRESHOLD; import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED; -import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.LOCK_RELEASED; import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.IGNITE_PDS_CHECKPOINT_TEST_SKIP_SYNC; import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.IGNITE_PDS_SKIP_CHECKPOINT_ON_NODE_STOP; import static org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointReadWriteLock.CHECKPOINT_RUNNER_THREAD_PREFIX; @@ -136,9 +133,6 @@ public class Checkpointer extends GridWorker { /** Failure processor. */ private final FailureProcessor failureProcessor; - /** Snapshot manager. */ - private final IgniteCacheSnapshotManager snapshotMgr; - /** Metrics. */ private final DataStorageMetricsImpl persStoreMetrics; @@ -192,7 +186,6 @@ public class Checkpointer extends GridWorker { * @param logger Logger. * @param detector Long JVM pause detector. * @param failureProcessor Failure processor. - * @param snapshotManager Snapshot manager. * @param dsMetrics Data storage metrics. * @param cacheProcessor Cache processor. * @param checkpoint Implementation of checkpoint. @@ -208,7 +201,6 @@ public class Checkpointer extends GridWorker { Function, IgniteLogger> logger, LongJVMPauseDetector detector, FailureProcessor failureProcessor, - IgniteCacheSnapshotManager snapshotManager, DataStorageMetricsImpl dsMetrics, GridCacheProcessor cacheProcessor, CheckpointWorkflow checkpoint, @@ -221,7 +213,6 @@ public class Checkpointer extends GridWorker { this.pauseDetector = detector; this.checkpointFreq = checkpointFrequency; this.failureProcessor = failureProcessor; - this.snapshotMgr = snapshotManager; this.checkpointWorkflow = checkpoint; this.checkpointPagesWriterFactory = factory; this.persStoreMetrics = dsMetrics; @@ -382,29 +373,6 @@ public CheckpointProgress scheduleCheckpoint( return sched; } - /** - * @param snapshotOperation Snapshot operation. - */ - public IgniteInternalFuture wakeupForSnapshotCreation(SnapshotOperation snapshotOperation) { - GridFutureAdapter ret; - - synchronized (this) { - scheduledCp.nextCpNanos(System.nanoTime()); - - scheduledCp.reason("snapshot"); - - scheduledCp.nextSnapshot(true); - - scheduledCp.snapshotOperation(snapshotOperation); - - ret = scheduledCp.futureFor(LOCK_RELEASED); - - notifyAll(); - } - - return ret; - } - /** * */ @@ -476,8 +444,6 @@ private void doCheckpoint() { tracker.onFsyncStart(); } - snapshotMgr.afterCheckpointPageWritten(); - int destroyedPartitionsCnt = destroyEvictedPartitions(); // Must mark successful checkpoint only if there are no exceptions or interrupts. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/LightweightCheckpointManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/LightweightCheckpointManager.java index b3fa9e795c2cf..8a82c6e626f42 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/LightweightCheckpointManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/LightweightCheckpointManager.java @@ -36,7 +36,6 @@ import org.apache.ignite.internal.processors.cache.persistence.DataStorageMetricsImpl; 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.snapshot.IgniteCacheSnapshotManager; import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; import org.apache.ignite.internal.processors.failure.FailureProcessor; import org.apache.ignite.internal.util.StripedExecutor; @@ -85,7 +84,6 @@ public class LightweightCheckpointManager { * @param dataRegions Data regions. * @param pageMemoryGroupResolver Page memory resolver. * @param throttlingPolicy Throttling policy. - * @param snapshotMgr Snapshot manager. * @param persStoreMetrics Persistence metrics. * @param longJvmPauseDetector Long JVM pause detector. * @param failureProcessor Failure processor. @@ -100,7 +98,6 @@ public LightweightCheckpointManager( Supplier> dataRegions, IgniteThrowableFunction pageMemoryGroupResolver, PageMemoryImpl.ThrottlingPolicy throttlingPolicy, - IgniteCacheSnapshotManager snapshotMgr, DataStorageMetricsImpl persStoreMetrics, LongJVMPauseDetector longJvmPauseDetector, FailureProcessor failureProcessor, @@ -111,7 +108,6 @@ public LightweightCheckpointManager( checkpointWorkflow = new CheckpointWorkflow( logger, null, - snapshotMgr, null, lock, persistenceCfg.getCheckpointWriteOrder(), @@ -134,7 +130,6 @@ public LightweightCheckpointManager( checkpointPagesWriterFactory = new CheckpointPagesWriterFactory( logger, - snapshotMgr, (pageMemEx, fullPage, buf, tag) -> pageMemEx.pageManager().write(fullPage.groupId(), fullPage.pageId(), buf, tag, true), persStoreMetrics, @@ -150,7 +145,6 @@ public LightweightCheckpointManager( logger, longJvmPauseDetector, failureProcessor, - snapshotMgr, persStoreMetrics, cacheProcessor, checkpointWorkflow, From b5072e904d57f9e3f51254927787ce80f934393d Mon Sep 17 00:00:00 2001 From: nizhikov Date: Wed, 5 Jul 2023 16:48:52 +0300 Subject: [PATCH 2/7] IGNITE-19915 Remove obsolete IgniteCacheSnapshotManager --- .../GridCachePartitionExchangeManager.java | 15 -- .../processors/cache/GridCacheProcessor.java | 9 - .../cache/GridCacheSharedContext.java | 12 -- .../GridDistributedTxRemoteAdapter.java | 4 +- .../GridDhtPartitionsExchangeFuture.java | 38 ---- .../cache/mvcc/MvccProcessorImpl.java | 3 +- .../GridCacheDatabaseSharedManager.java | 30 +--- .../persistence/GridCacheOffheapManager.java | 102 +---------- .../IgniteCacheDatabaseSharedManager.java | 23 +-- .../checkpoint/CheckpointContextImpl.java | 13 +- .../checkpoint/CheckpointListener.java | 10 -- .../file/FilePageStoreManager.java | 39 +--- .../persistence/pagemem/PageMemoryImpl.java | 52 +----- .../snapshot/IgniteCacheSnapshotManager.java | 167 ------------------ .../snapshot/IgniteSnapshotManager.java | 2 +- .../wal/reader/IgniteWalIteratorFactory.java | 2 +- .../transactions/IgniteTxLocalAdapter.java | 21 +-- ...IgniteSequentialNodeCrashRecoveryTest.java | 10 -- .../checkpoint/LightweightCheckpointTest.java | 1 - .../IgniteWalIteratorSwitchSegmentTest.java | 2 - .../pagemem/BPlusTreePageMemoryImplTest.java | 7 - .../BPlusTreeReuseListPageMemoryImplTest.java | 7 - ...itePageMemReplaceDelayedWriteUnitTest.java | 2 +- .../IndexStoragePageMemoryImplTest.java | 7 - .../pagemem/PageMemoryImplNoLoadTest.java | 7 - .../pagemem/PageMemoryImplTest.java | 10 -- .../hashmap/GridCacheTestContext.java | 2 - 27 files changed, 30 insertions(+), 567 deletions(-) delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteCacheSnapshotManager.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index 3957151db29be..6ef94e2a61370 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java @@ -1343,16 +1343,6 @@ public List exchangeAwareComponents() { * @param grps Cache groups for partitions refresh. */ public void refreshPartitions(@NotNull Collection grps) { - // TODO https://issues.apache.org/jira/browse/IGNITE-6857 - if (cctx.snapshot().snapshotOperationInProgress()) { - if (log.isDebugEnabled()) - log.debug("Schedule resend parititions due to snapshot in progress"); - - scheduleResendPartitions(); - - return; - } - if (grps.isEmpty()) { if (log.isDebugEnabled()) log.debug("Skip partitions refresh, there are no cache groups for partition refresh."); @@ -3460,11 +3450,6 @@ else if (task instanceof ForceRebalanceExchangeTask) { forcedRebFut = ((ForceRebalanceExchangeTask)task).forcedRebalanceFuture(); for (CacheGroupContext grp : assignsSet.descendingSet()) { - boolean disableRebalance = cctx.snapshot().partitionsAreFrozen(grp); - - if (disableRebalance) - continue; - RebalanceFuture cur = grp.preloader().prepare(exchId, exchFut, cnt, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 85b4bcef3d7f2..61696306fe782 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -125,7 +125,6 @@ import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetastorageLifecycleListener; import org.apache.ignite.internal.processors.cache.persistence.metastorage.ReadOnlyMetastorage; import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager; import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotDiscoveryMessage; import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList; @@ -1084,8 +1083,6 @@ private void stopCache(GridCacheAdapter cache, boolean cancel, boolean cal ctx.kernalContext().continuous().onCacheStop(ctx); - ctx.kernalContext().cache().context().snapshot().onCacheStop(ctx, callDestroy); - ctx.kernalContext().coordinators().onCacheStop(ctx); ctx.group().stopCache(ctx, clearCache); @@ -3071,11 +3068,6 @@ private GridCacheSharedContext createSharedContext( if (snapshotMgr == null) snapshotMgr = new IgniteSnapshotManager(ctx); - IgniteCacheSnapshotManager snpMgr = ctx.plugins().createComponent(IgniteCacheSnapshotManager.class); - - if (snpMgr == null) - snpMgr = new IgniteCacheSnapshotManager(); - CacheObjectTransformerManager transMgr = ctx.plugins().createComponent(CacheObjectTransformerManager.class); GridCacheIoManager ioMgr = new GridCacheIoManager(); @@ -3101,7 +3093,6 @@ private GridCacheSharedContext createSharedContext( walStateMgr, dbMgr, snapshotMgr, - snpMgr, depMgr, exchMgr, topMgr, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java index 7685e6e038423..e6f9f70e12455 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java @@ -54,7 +54,6 @@ import org.apache.ignite.internal.processors.cache.mvcc.MvccProcessor; import org.apache.ignite.internal.processors.cache.persistence.DataRegion; import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager; import org.apache.ignite.internal.processors.cache.store.CacheStoreManager; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; @@ -124,9 +123,6 @@ public class GridCacheSharedContext { /** Database manager. */ private IgniteCacheDatabaseSharedManager dbMgr; - /** Snapshot manager. */ - private IgniteCacheSnapshotManager snpMgr; - /** Page store manager. {@code Null} if persistence is not enabled. */ @Nullable private IgnitePageStoreManager pageStoreMgr; @@ -209,7 +205,6 @@ public class GridCacheSharedContext { * @param walStateMgr WAL state manager. * @param depMgr Deployment manager. * @param dbMgr Database manager. - * @param snpMgr Snapshot manager. * @param exchMgr Exchange manager. * @param affMgr Affinity manager. * @param ioMgr IO manager. @@ -230,7 +225,6 @@ public GridCacheSharedContext( WalStateManager walStateMgr, IgniteCacheDatabaseSharedManager dbMgr, IgniteSnapshotManager snapshotMgr, - IgniteCacheSnapshotManager snpMgr, GridCacheDeploymentManager depMgr, GridCachePartitionExchangeManager exchMgr, CacheAffinitySharedManager affMgr, @@ -258,7 +252,6 @@ public GridCacheSharedContext( walStateMgr, dbMgr, snapshotMgr, - snpMgr, depMgr, exchMgr, affMgr, @@ -301,8 +294,6 @@ public GridCacheSharedContext( stateAwareMgrs.add(dbMgr); - stateAwareMgrs.add(snpMgr); - stateAwareMgrs.add(snapshotMgr); for (PluginProvider prv : kernalCtx.plugins().allProviders()) @@ -441,7 +432,6 @@ void onReconnected(boolean active) throws IgniteCheckedException { walStateMgr, dbMgr, snapshotMgr, - snpMgr, new GridCacheDeploymentManager(), new GridCachePartitionExchangeManager(), affMgr, @@ -493,7 +483,6 @@ private void setManagers( WalStateManager walStateMgr, IgniteCacheDatabaseSharedManager dbMgr, IgniteSnapshotManager snapshotMgr, - IgniteCacheSnapshotManager snpMgr, GridCacheDeploymentManager depMgr, GridCachePartitionExchangeManager exchMgr, CacheAffinitySharedManager affMgr, @@ -518,7 +507,6 @@ private void setManagers( this.walStateMgr = add(mgrs, walStateMgr); this.dbMgr = add(mgrs, dbMgr); this.snapshotMgr = add(mgrs, snapshotMgr); - this.snpMgr = add(mgrs, snpMgr); this.jtaMgr = add(mgrs, jtaMgr); this.depMgr = add(mgrs, depMgr); this.exchMgr = add(mgrs, exchMgr); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java index ce1e953bedd5f..236f38b7c96ec 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java @@ -497,7 +497,7 @@ private void commitIfLocked() throws IgniteCheckedException { try { assert !txState.mvccEnabled() || mvccSnapshot != null : "Mvcc is not initialized: " + this; - Collection entries = near() || cctx.snapshot().needTxReadLogging() ? allEntries() : writeEntries(); + Collection entries = near() ? allEntries() : writeEntries(); // Data entry to write to WAL. List dataEntries = null; @@ -602,7 +602,7 @@ else if (conflictCtx.isMerge()) { GridCacheVersion dhtVer = cached.isNear() ? writeVersion() : null; if (!near() && cacheCtx.group().logDataRecords() && - op != NOOP && op != RELOAD && (op != READ || cctx.snapshot().needTxReadLogging())) { + op != NOOP && op != RELOAD && op != READ) { if (dataEntries == null) dataEntries = new ArrayList<>(entries.size()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java index e16d73b4cca2c..85957f22af9f1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java @@ -1080,17 +1080,6 @@ else if (msg instanceof WalStateAbstractMessage) assert false; } - if (cctx.localNode().isClient()) { - cctx.exchange().exchangerBlockingSectionBegin(); - - try { - tryToPerformLocalSnapshotOperation(); - } - finally { - cctx.exchange().exchangerBlockingSectionEnd(); - } - } - for (PartitionsExchangeAware comp : cctx.exchange().exchangeAwareComponents()) comp.onInitAfterTopologyLock(this); @@ -1807,30 +1796,6 @@ else if (!skipWaitOnLocalJoin) { // Skip partition release if node has locally j } } - /** - * Try to start local snapshot operation if it is needed by discovery event - */ - private void tryToPerformLocalSnapshotOperation() { - try { - long start = System.nanoTime(); - - IgniteInternalFuture fut = cctx.snapshot().tryStartLocalSnapshotOperation(firstDiscoEvt, exchId.topologyVersion()); - - if (fut != null) { - fut.get(); - - long end = System.nanoTime(); - - if (log.isInfoEnabled()) - log.info("Snapshot initialization completed [topVer=" + exchangeId().topologyVersion() + - ", time=" + U.nanosToMillis(end - start) + "ms]"); - } - } - catch (IgniteException | IgniteCheckedException e) { - U.error(log, "Error while starting snapshot operation", e); - } - } - /** * Change WAL mode if needed. */ @@ -2515,9 +2480,6 @@ private String exchangeTimingsLogMessage(String header, List timings) { grpValidRes = m; } - if (!cctx.localNode().isClient()) - tryToPerformLocalSnapshotOperation(); - if (err == null) cctx.coordinators().onExchangeDone(events().discoveryCache()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java index cfc98852b76f7..a04be7c5fc95f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java @@ -373,8 +373,7 @@ public MvccProcessorImpl(GridKernalContext ctx) { mgr.addDataRegion( dscfg, - createTxLogRegion(dscfg), - CU.isPersistenceEnabled(ctx.config())); + createTxLogRegion(dscfg)); } /** {@inheritDoc} */ 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 5b7a4710d5963..b7451bf72e7f2 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 @@ -66,7 +66,6 @@ import org.apache.ignite.internal.mem.DirectMemoryProvider; import org.apache.ignite.internal.mem.DirectMemoryRegion; import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp; -import org.apache.ignite.internal.pagemem.FullPageId; import org.apache.ignite.internal.pagemem.PageIdAllocator; import org.apache.ignite.internal.pagemem.PageMemory; import org.apache.ignite.internal.pagemem.PageUtils; @@ -145,7 +144,6 @@ import org.apache.ignite.internal.util.StripedExecutor; import org.apache.ignite.internal.util.TimeBag; import org.apache.ignite.internal.util.future.GridFutureAdapter; -import org.apache.ignite.internal.util.lang.GridInClosure3X; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.X; @@ -471,7 +469,7 @@ public IgniteInternalFuture enableCheckpoints(boolean enable) { @Override protected void initDataRegions0(DataStorageConfiguration memCfg) throws IgniteCheckedException { super.initDataRegions0(memCfg); - addDataRegion(memCfg, createMetastoreDataRegionConfig(memCfg), false); + addDataRegion(memCfg, createMetastoreDataRegionConfig(memCfg)); List regionMetrics = dataRegionMap.values().stream() .map(DataRegion::metrics) @@ -655,7 +653,6 @@ private DataStorageConfiguration configureDataRegionForDefragmentation( addDataRegion( memCfg, createDefragmentationDataRegionConfig(totalDefrRegionSize - mappingRegionSize), - true, new DefragmentationPageReadWriteManager(cctx.kernalContext(), "defrgPartitionsStore") ) ); @@ -664,7 +661,6 @@ private DataStorageConfiguration configureDataRegionForDefragmentation( addDataRegion( memCfg, createDefragmentationMappingRegionConfig(mappingRegionSize), - true, new DefragmentationPageReadWriteManager(cctx.kernalContext(), "defrgLinkMappingStore") ) ); @@ -836,8 +832,8 @@ public CachePartitionDefragmentationManager defragmentationManager() { /** {@inheritDoc} */ @Override public DataRegion addDataRegion(DataStorageConfiguration dataStorageCfg, DataRegionConfiguration dataRegionCfg, - boolean trackable, PageReadWriteManager pmPageMgr) throws IgniteCheckedException { - DataRegion region = super.addDataRegion(dataStorageCfg, dataRegionCfg, trackable, pmPageMgr); + PageReadWriteManager pmPageMgr) throws IgniteCheckedException { + DataRegion region = super.addDataRegion(dataStorageCfg, dataRegionCfg, pmPageMgr); checkpointedDataRegions.add(region); @@ -1184,11 +1180,10 @@ private long[] calculateFragmentSizes(String regionName, int concLvl, long cache DataStorageConfiguration memCfg, DataRegionConfiguration plcCfg, DataRegionMetricsImpl memMetrics, - final boolean trackable, PageReadWriteManager pmPageMgr ) { if (!plcCfg.isPersistenceEnabled()) - return super.createPageMemory(memProvider, memCfg, plcCfg, memMetrics, trackable, pmPageMgr); + return super.createPageMemory(memProvider, memCfg, plcCfg, memMetrics, pmPageMgr); memMetrics.persistenceEnabled(true); @@ -1205,22 +1200,6 @@ private long[] calculateFragmentSizes(String regionName, int concLvl, long cache chpBufSize = cacheSize; } - GridInClosure3X changeTracker; - - if (trackable) - changeTracker = new GridInClosure3X() { - @Override public void applyx( - Long page, - FullPageId fullId, - PageMemoryEx pageMem - ) throws IgniteCheckedException { - if (trackable) - snapshotMgr.onChangeTrackerPage(page, fullId, pageMem); - } - }; - else - changeTracker = null; - PageMemoryImpl pageMem = new PageMemoryImpl( wrapMetricsPersistentMemoryProvider(memProvider, memMetrics), calculateFragmentSizes( @@ -1240,7 +1219,6 @@ private long[] calculateFragmentSizes(String regionName, int concLvl, long cache getCheckpointer().currentProgress().updateEvictedPages(1); }, - changeTracker, this, memMetrics, resolveThrottlingPolicy(), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java index 7622305cce446..43d69a62dbe2f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java @@ -127,7 +127,6 @@ import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR; import static org.apache.ignite.internal.processors.cache.GridCacheTtlManager.DFLT_UNWIND_THROTTLING_TIMEOUT; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.EVICTED; -import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.MOVING; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.OWNING; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.RENTING; import static org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler.isWalDeltaRecordNeeded; @@ -261,7 +260,7 @@ public IndexStorage getIndexStorage() { @Override public void onMarkCheckpointBegin(Context ctx) throws IgniteCheckedException { assert grp.dataRegion().pageMemory() instanceof PageMemoryEx; - syncMetadata(ctx); + syncMetadata(ctx.executor()); } /** {@inheritDoc} */ @@ -269,35 +268,7 @@ public IndexStorage getIndexStorage() { assert F.size(cacheDataStores().iterator(), CacheDataStore::destroyed) == 0; // Optimization: reducing the holding time of checkpoint write lock. - syncMetadata(ctx, ctx.executor(), false); - } - - /** - * Syncs and saves meta-information of all data structures to page memory. - * - * @throws IgniteCheckedException If failed. - */ - private void syncMetadata(Context ctx) throws IgniteCheckedException { - Executor execSvc = ctx.executor(); - - boolean needSnapshot = ctx.nextSnapshot() && ctx.needToSnapshot(grp.cacheOrGroupName()); - - if (needSnapshot) { - if (execSvc == null) - addPartitions(ctx); - else { - execSvc.execute(() -> { - try { - addPartitions(ctx); - } - catch (IgniteCheckedException e) { - throw new IgniteException(e); - } - }); - } - } - - syncMetadata(ctx, ctx.executor(), needSnapshot); + syncMetadata(ctx.executor()); } /** @@ -306,12 +277,12 @@ private void syncMetadata(Context ctx) throws IgniteCheckedException { * @param execSvc Executor service to run save process * @throws IgniteCheckedException If failed. */ - private void syncMetadata(Context ctx, Executor execSvc, boolean needSnapshot) throws IgniteCheckedException { + private void syncMetadata(Executor execSvc) throws IgniteCheckedException { if (execSvc == null) { reuseList.saveMetadata(grp.statisticsHolderData()); for (CacheDataStore store : cacheDataStores()) - saveStoreMetadata(store, ctx, false, needSnapshot); + saveStoreMetadata(store, false); } else { execSvc.execute(() -> { @@ -326,7 +297,7 @@ private void syncMetadata(Context ctx, Executor execSvc, boolean needSnapshot) t for (CacheDataStore store : cacheDataStores()) execSvc.execute(() -> { try { - saveStoreMetadata(store, ctx, false, needSnapshot); + saveStoreMetadata(store, false); } catch (IgniteCheckedException e) { throw new IgniteException(e); @@ -344,9 +315,7 @@ private void syncMetadata(Context ctx, Executor execSvc, boolean needSnapshot) t */ private void saveStoreMetadata( CacheDataStore store, - Context ctx, - boolean beforeDestroy, - boolean needSnapshot + boolean beforeDestroy ) throws IgniteCheckedException { RowStore rowStore0 = store.rowStore(); @@ -495,42 +464,7 @@ else if (updCntrsBytes != null && link != 0) { else cntrsPageId = 0L; - int pageCnt; - - if (needSnapshot) { - pageCnt = this.ctx.pageStore().pages(grpId, store.partId()); - - io.setCandidatePageCount(partMetaPageAddr, size == 0 ? 0 : pageCnt); - - if (state == OWNING) { - assert part != null; - - if (!addPartition( - part, - ctx.partitionStatMap(), - partMetaPageAddr, - io, - grpId, - store.partId(), - this.ctx.pageStore().pages(grpId, store.partId()), - store.fullSize() - )) - U.warn(log, "Partition was concurrently evicted grpId=" + grpId + - ", partitionId=" + part.id()); - } - else if (state == MOVING || state == RENTING) { - if (ctx.partitionStatMap().forceSkipIndexPartition(grpId)) { - if (log.isInfoEnabled()) - log.info("Will not include SQL indexes to snapshot because there is " + - "a partition not in " + OWNING + " state [grp=" + grp.cacheOrGroupName() + - ", partId=" + store.partId() + ", state=" + state + ']'); - } - } - - changed = true; - } - else - pageCnt = io.getCandidatePageCount(partMetaPageAddr); + int pageCnt = io.getCandidatePageCount(partMetaPageAddr); if (changed && isWalDeltaRecordNeeded(pageMem, grpId, partMetaId, partMetaPage, wal, null)) wal.log(new MetaPageUpdatePartitionDataRecordV3( @@ -561,11 +495,7 @@ else if (state == MOVING || state == RENTING) { pageMem.releasePage(grpId, partMetaId, partMetaPage); } } - else if (needSnapshot) - tryAddEmptyPartitionToSnapshot(store, ctx); } - else if (needSnapshot) - tryAddEmptyPartitionToSnapshot(store, ctx); } /** {@inheritDoc} */ @@ -719,22 +649,6 @@ private void updateState(GridDhtLocalPartition part, int stateId) { } } - /** - * Check that we need to snapshot this partition and add it to map. - * - * @param store Store. - * @param ctx Snapshot context. - */ - private void tryAddEmptyPartitionToSnapshot(CacheDataStore store, Context ctx) { - GridDhtLocalPartition locPart = getPartition(store); - - if (locPart != null && locPart.state() == OWNING) { - ctx.partitionStatMap().put( - new GroupPartitionId(grp.groupId(), store.partId()), - new PagesAllocationRange(0, 0)); - } - } - /** * @param store Store. * @@ -963,7 +877,7 @@ private static boolean addPartition( ctx.database().checkpointReadLock(); try { - saveStoreMetadata(store, null, true, false); + saveStoreMetadata(store, true); } finally { ctx.database().checkpointReadUnlock(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java index 2458224b3514a..ae541fdf3cbf8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java @@ -322,13 +322,12 @@ protected void initDataRegions0(DataStorageConfiguration memCfg) throws IgniteCh if (dataRegionCfgs != null) { for (DataRegionConfiguration dataRegionCfg : dataRegionCfgs) - addDataRegion(memCfg, dataRegionCfg, dataRegionCfg.isPersistenceEnabled()); + addDataRegion(memCfg, dataRegionCfg); } addDataRegion( memCfg, - memCfg.getDefaultDataRegionConfiguration(), - memCfg.getDefaultDataRegionConfiguration().isPersistenceEnabled() + memCfg.getDefaultDataRegionConfiguration() ); addDataRegion( @@ -337,8 +336,7 @@ protected void initDataRegions0(DataStorageConfiguration memCfg) throws IgniteCh memCfg.getSystemDataRegionConfiguration().getInitialSize(), memCfg.getSystemDataRegionConfiguration().getMaxSize(), persistenceEnabled - ), - persistenceEnabled + ) ); addDataRegion( @@ -346,8 +344,7 @@ protected void initDataRegions0(DataStorageConfiguration memCfg) throws IgniteCh createVolatileDataRegion( memCfg.getSystemDataRegionConfiguration().getInitialSize(), memCfg.getSystemDataRegionConfiguration().getMaxSize() - ), - false + ) ); for (DatabaseLifecycleListener lsnr : getDatabaseListeners(cctx.kernalContext())) @@ -369,10 +366,9 @@ protected List getDatabaseListeners(GridKernalContext */ public DataRegion addDataRegion( DataStorageConfiguration dataStorageCfg, - DataRegionConfiguration dataRegionCfg, - boolean trackable + DataRegionConfiguration dataRegionCfg ) throws IgniteCheckedException { - return addDataRegion(dataStorageCfg, dataRegionCfg, trackable, cctx.pageStore()); + return addDataRegion(dataStorageCfg, dataRegionCfg, cctx.pageStore()); } /** @@ -384,7 +380,6 @@ public DataRegion addDataRegion( protected DataRegion addDataRegion( DataStorageConfiguration dataStorageCfg, DataRegionConfiguration dataRegionCfg, - boolean trackable, PageReadWriteManager pmPageMgr ) throws IgniteCheckedException { String dataRegionName = dataRegionCfg.getName(); @@ -399,7 +394,7 @@ protected DataRegion addDataRegion( cctx.kernalContext(), dataRegionMetricsProvider(dataRegionCfg)); - DataRegion region = initMemory(dataStorageCfg, dataRegionCfg, memMetrics, trackable, pmPageMgr); + DataRegion region = initMemory(dataStorageCfg, dataRegionCfg, memMetrics, pmPageMgr); dataRegionMap.put(dataRegionName, region); @@ -1262,14 +1257,13 @@ private DataRegion initMemory( DataStorageConfiguration memCfg, DataRegionConfiguration plcCfg, DataRegionMetricsImpl memMetrics, - boolean trackable, PageReadWriteManager pmPageMgr ) throws IgniteCheckedException { if (plcCfg.getMemoryAllocator() == null) plcCfg.setMemoryAllocator(memCfg.getMemoryAllocator()); PageMemory pageMem = createPageMemory(createOrReuseMemoryProvider(plcCfg), memCfg, plcCfg, memMetrics, - trackable, pmPageMgr); + pmPageMgr); return new DataRegion(pageMem, plcCfg, memMetrics, createPageEvictionTracker(plcCfg, pageMem)); } @@ -1376,7 +1370,6 @@ protected PageMemory createPageMemory( DataStorageConfiguration memCfg, DataRegionConfiguration memPlcCfg, DataRegionMetricsImpl memMetrics, - boolean trackable, PageReadWriteManager pmPageMgr ) { memMetrics.persistenceEnabled(false); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointContextImpl.java index 44bb25dfa0dfb..8c940141eca35 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointContextImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointContextImpl.java @@ -24,7 +24,6 @@ import org.apache.ignite.internal.processors.cache.persistence.partstate.PartitionAllocationMap; import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; -import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.util.worker.WorkProgressDispatcher; import org.apache.ignite.thread.IgniteThreadPoolExecutor; @@ -78,11 +77,6 @@ public class CheckpointContextImpl implements CheckpointListener.Context { return curr; } - /** {@inheritDoc} */ - @Override public boolean nextSnapshot() { - return curr.nextSnapshot(); - } - /** {@inheritDoc} */ @Override public void walFlush(boolean flush) { forceWalFlush = flush; @@ -90,7 +84,7 @@ public class CheckpointContextImpl implements CheckpointListener.Context { /** {@inheritDoc} */ @Override public boolean walFlush() { - return forceWalFlush || nextSnapshot(); + return forceWalFlush; } /** {@inheritDoc} */ @@ -103,11 +97,6 @@ public class CheckpointContextImpl implements CheckpointListener.Context { return map; } - /** {@inheritDoc} */ - @Override public boolean needToSnapshot(String cacheOrGrpName) { - return curr.snapshotOperation().cacheGroupIds().contains(CU.cacheId(cacheOrGrpName)); - } - /** {@inheritDoc} */ @Override public Executor executor() { return asyncRunner == null ? null : cmd -> { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointListener.java index 24c1c27ecf6c4..c18cd7b1a1c06 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointListener.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointListener.java @@ -36,11 +36,6 @@ public interface Context { */ public CheckpointProgress progress(); - /** - * @return {@code True} if a snapshot have to be created after. - */ - public boolean nextSnapshot(); - /** * @param flush If {@code True} then will flush WAL after a Checkpoint begin. */ @@ -61,11 +56,6 @@ public interface Context { */ public PartitionAllocationMap partitionStatMap(); - /** - * @param cacheOrGrpName Cache or group name. - */ - public boolean needToSnapshot(String cacheOrGrpName); - /** * @return Context executor. */ 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 b1e75502b6584..af0a4975d1e78 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 @@ -25,7 +25,6 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.PathMatcher; -import java.nio.file.StandardCopyOption; import java.util.AbstractList; import java.util.ArrayList; import java.util.Arrays; @@ -71,7 +70,6 @@ import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMetrics; import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageReadWriteManager; import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageReadWriteManagerImpl; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager; import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.util.GridStripedReadWriteLock; import org.apache.ignite.internal.util.typedef.X; @@ -803,46 +801,11 @@ public static boolean checkAndInitCacheWorkDir(File cacheWorkDir, IgniteLogger l throw new IgniteCheckedException("Failed to initialize cache working directory " + "(a file with the same name already exists): " + cacheWorkDir.getAbsolutePath()); - File lockF = new File(cacheWorkDir, IgniteCacheSnapshotManager.SNAPSHOT_RESTORE_STARTED_LOCK_FILENAME); - Path cacheWorkDirPath = cacheWorkDir.toPath(); Path tmp = cacheWorkDirPath.getParent().resolve(cacheWorkDir.getName() + TMP_SUFFIX); - if (Files.exists(tmp) && Files.isDirectory(tmp) && - Files.exists(tmp.resolve(IgniteCacheSnapshotManager.TEMP_FILES_COMPLETENESS_MARKER))) { - - U.warn(log, "Ignite node crashed during the snapshot restore process " + - "(there is a snapshot restore lock file left for cache). But old version of cache was saved. " + - "Trying to restore it. Cache - [" + cacheWorkDir.getAbsolutePath() + ']'); - - U.delete(cacheWorkDir); - - try { - Files.move(tmp, cacheWorkDirPath, StandardCopyOption.ATOMIC_MOVE); - - cacheWorkDirPath.resolve(IgniteCacheSnapshotManager.TEMP_FILES_COMPLETENESS_MARKER).toFile().delete(); - } - catch (IOException e) { - throw new IgniteCheckedException(e); - } - } - else if (lockF.exists()) { - U.warn(log, "Ignite node crashed during the snapshot restore process " + - "(there is a snapshot restore lock file left for cache). Will remove both the lock file and " + - "incomplete cache directory [cacheDir=" + cacheWorkDir.getAbsolutePath() + ']'); - - boolean deleted = U.delete(cacheWorkDir); - - if (!deleted) - throw new IgniteCheckedException("Failed to remove obsolete cache working directory " + - "(remove the directory manually and make sure the work folder has correct permissions): " + - cacheWorkDir.getAbsolutePath()); - - cacheWorkDir.mkdirs(); - } - else - dirExisted = true; + dirExisted = true; if (!cacheWorkDir.exists()) throw new IgniteCheckedException("Failed to initialize cache working directory " + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java index 5acbe2dab88d4..931e3de6ac8a2 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java @@ -62,7 +62,6 @@ import org.apache.ignite.internal.pagemem.wal.record.CheckpointRecord; import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; -import org.apache.ignite.internal.pagemem.wal.record.delta.InitNewPageRecord; import org.apache.ignite.internal.pagemem.wal.record.delta.PageDeltaRecord; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.persistence.CheckpointLockStateChecker; @@ -84,7 +83,6 @@ import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.OffheapReadWriteLock; import org.apache.ignite.internal.util.future.CountDownFuture; -import org.apache.ignite.internal.util.lang.GridInClosure3X; import org.apache.ignite.internal.util.lang.GridPlainRunnable; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; @@ -235,12 +233,6 @@ public class PageMemoryImpl implements PageMemoryEx { */ @Nullable private final DelayedPageReplacementTracker delayedPageReplacementTracker; - /** - * Callback invoked to track changes in pages. - * {@code Null} if page tracking functionality is disabled - * */ - @Nullable private final GridInClosure3X changeTracker; - /** Pages write throttle. */ private PagesWriteThrottlePolicy writeThrottle; @@ -275,7 +267,6 @@ public class PageMemoryImpl implements PageMemoryEx { * @param pmPageMgr Page store manager. * @param pageSize Page size. * @param flushDirtyPage write callback invoked when a dirty page is removed for replacement. - * @param changeTracker Callback invoked to track changes in pages. * @param stateChecker Checkpoint lock state provider. Used to ensure lock is held by thread, which modify pages. * @param dataRegionMetrics Memory metrics to track dirty pages count and page replace rate. * @param throttlingPlc Write throttle enabled and its type. Null equal to none. @@ -288,7 +279,6 @@ public PageMemoryImpl( PageReadWriteManager pmPageMgr, int pageSize, PageStoreWriter flushDirtyPage, - @Nullable GridInClosure3X changeTracker, CheckpointLockStateChecker stateChecker, DataRegionMetricsImpl dataRegionMetrics, @Nullable ThrottlingPolicy throttlingPlc, @@ -308,7 +298,6 @@ public PageMemoryImpl( getBoolean(IGNITE_DELAYED_REPLACED_PAGE_WRITE, DFLT_DELAYED_REPLACED_PAGE_WRITE) ? new DelayedPageReplacementTracker(pageSize, flushDirtyPage, log, sizes.length - 1) : null; - this.changeTracker = changeTracker; this.stateChecker = stateChecker; this.throttlingPlc = throttlingPlc != null ? throttlingPlc : ThrottlingPolicy.CHECKPOINT_BUFFER_ONLY; this.cpProgressProvider = cpProgressProvider; @@ -557,12 +546,6 @@ else if (throttlingPlc == ThrottlingPolicy.CHECKPOINT_BUFFER_ONLY) seg.writeLock().lock(); - boolean isTrackingPage = changeTracker != null && - PageIdUtils.pageIndex(trackingIO.trackingPageFor(pageId, realPageSize(grpId))) == PageIdUtils.pageIndex(pageId); - - if (isTrackingPage && PageIdUtils.flag(pageId) == PageIdAllocator.FLAG_AUX) - pageId = PageIdUtils.pageId(PageIdUtils.partId(pageId), PageIdAllocator.FLAG_DATA, PageIdUtils.pageIndex(pageId)); - FullPageId fullId = new FullPageId(pageId, grpId); try { @@ -611,34 +594,6 @@ else if (throttlingPlc == ThrottlingPolicy.CHECKPOINT_BUFFER_ONLY) setDirty(fullId, absPtr, true, true); - if (isTrackingPage) { - long pageAddr = absPtr + PAGE_OVERHEAD; - - // We are inside segment write lock, so no other thread can pin this tracking page yet. - // We can modify page buffer directly. - if (PageIO.getType(pageAddr) == 0) { - PageMetrics metrics = dataRegionMetrics.cacheGrpPageMetrics(grpId); - - trackingIO.initNewPage(pageAddr, pageId, realPageSize(grpId), metrics); - - if (!ctx.wal().disabled(fullId.groupId(), fullId.pageId())) { - if (!ctx.wal().isAlwaysWriteFullPages()) - ctx.wal().log( - new InitNewPageRecord( - grpId, - pageId, - trackingIO.getType(), - trackingIO.getVersion(), pageId - ) - ); - else { - ctx.wal().log(new PageSnapshot(fullId, absPtr + PAGE_OVERHEAD, pageSize(), - realPageSize(fullId.groupId()))); - } - } - } - } - seg.pageReplacementPolicy.onMiss(relPtr); seg.loadedPages.put(grpId, PageIdUtils.effectivePageId(pageId), relPtr, seg.partGeneration(grpId, partId)); @@ -669,8 +624,7 @@ else if (throttlingPlc == ThrottlingPolicy.CHECKPOINT_BUFFER_ONLY) if (delayedPageReplacementTracker != null) delayedPageReplacementTracker.delayedPageWrite().finishReplacement(); - //we have allocated 'tracking' page, we need to allocate regular one - return isTrackingPage ? allocatePage(grpId, partId, flags) : pageId; + return pageId; } /** @@ -1719,10 +1673,6 @@ private void writeUnlockPage( boolean wasDirty = isDirty(page); try { - //if page is for restore, we shouldn't mark it as changed - if (!restore && markDirty && !wasDirty && changeTracker != null) - changeTracker.apply(page, fullId, this); - boolean pageWalRec = markDirty && walPlc != FALSE && (walPlc == TRUE || !wasDirty); assert PageIO.getCrc(page + PAGE_OVERHEAD) == 0; //TODO GG-11480 diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteCacheSnapshotManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteCacheSnapshotManager.java deleted file mode 100644 index 5c751fda96d47..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteCacheSnapshotManager.java +++ /dev/null @@ -1,167 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.processors.cache.persistence.snapshot; - -import java.util.UUID; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.events.DiscoveryEvent; -import org.apache.ignite.internal.GridKernalContext; -import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.pagemem.FullPageId; -import org.apache.ignite.internal.pagemem.PageMemory; -import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.internal.processors.cache.CacheGroupContext; -import org.apache.ignite.internal.processors.cache.GridCacheContext; -import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; -import org.apache.ignite.internal.processors.cache.persistence.partstate.PartitionAllocationMap; -import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport; -import org.apache.ignite.lang.IgniteFuture; -import org.jetbrains.annotations.Nullable; - -/** - * Snapshot manager stub. - * - * @deprecated Use {@link IgniteSnapshotManager}. - */ -@Deprecated -public class IgniteCacheSnapshotManager - extends GridCacheSharedManagerAdapter implements IgniteChangeGlobalStateSupport { - /** Snapshot started lock filename. */ - public static final String SNAPSHOT_RESTORE_STARTED_LOCK_FILENAME = "snapshot-started.loc"; - - /** Temp files completeness marker. */ - public static final String TEMP_FILES_COMPLETENESS_MARKER = "finished.tmp"; - - /** - * Try to start local snapshot operation if it's required by discovery event. - * - * @param discoveryEvt Discovery event. - * @param topVer topology version on the moment when this method was called - * - * @throws IgniteCheckedException if failed - */ - @Nullable public IgniteInternalFuture tryStartLocalSnapshotOperation( - @Nullable DiscoveryEvent discoveryEvt, AffinityTopologyVersion topVer - ) throws IgniteCheckedException { - return null; - } - - /** - * @param initiatorNodeId Initiator node id. - * @param snapshotOperation Snapshot operation. - */ - @Nullable public IgniteInternalFuture startLocalSnapshotOperation( - UUID initiatorNodeId, - T snapshotOperation, - AffinityTopologyVersion topVer - ) throws IgniteCheckedException { - return null; - } - - /** - * @param snapshotOperation current snapshot operation. - * @param map (cacheId, partId) -> (lastAllocatedIndex, count) - * - * @return {@code true} if next operation must be snapshot, {@code false} if checkpoint must be executed. - */ - public IgniteFuture onMarkCheckPointBegin( - T snapshotOperation, - PartitionAllocationMap map - ) throws IgniteCheckedException { - return null; - } - - /** - * - */ - public boolean partitionsAreFrozen(CacheGroupContext grp) { - return false; - } - - /** - * - */ - public boolean snapshotOperationInProgress() { - return false; - } - - /** - * - */ - public void beforeCheckpointPageWritten() { - // No-op. - } - - /** - * - */ - public void afterCheckpointPageWritten() { - // No-op. - } - - /** - * @param fullId Full id. - */ - public void beforePageWrite(FullPageId fullId) { - // No-op. - } - - /** - * @param cctx Cctx. - * @param destroy Destroy flag. - */ - public void onCacheStop(GridCacheContext cctx, boolean destroy) { - // No-op. - } - - /** - * @param gctx Cctx. - * @param destroy Destroy flag. - */ - public void onCacheGroupStop(CacheGroupContext gctx, boolean destroy) { - // No-op. - } - - /** - * - */ - public void onChangeTrackerPage( - Long page, - FullPageId fullId, - PageMemory pageMem - ) throws IgniteCheckedException { - // No-op. - } - - /** {@inheritDoc} */ - @Override public void onActivate(GridKernalContext kctx) throws IgniteCheckedException { - // No-op. - } - - /** {@inheritDoc} */ - @Override public void onDeActivate(GridKernalContext kctx) { - // No-op. - } - - /** - * @return {@code True} if TX READ records must be logged in WAL. - */ - public boolean needTxReadLogging() { - return false; - } -} 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 1a49899e9bc9a..780470bf2e273 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 @@ -2544,7 +2544,7 @@ public GridCloseableIterator partitionRowIterator(GridKernalContex false, false, false, false); GridCacheSharedContext sctx = new GridCacheSharedContext<>(ctx, null, null, null, - null, null, null, null, null, null, + null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java index ef998e3aca2a7..15a43b5a659bb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java @@ -403,7 +403,7 @@ private FileDescriptor readFileDescriptor(File file, FileIOFactory ioFactory) { return new GridCacheSharedContext<>( kernalCtx, null, null, null, - null, null, null, dbMgr, null, null, + null, null, null, dbMgr, null, null, null, null, null, null, null, null, null, null, null, null, null ); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index cda2e0a9cb1a5..1f3a3f02b8845 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -34,11 +34,8 @@ import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.NodeStoppingException; -import org.apache.ignite.internal.pagemem.wal.record.DataEntry; -import org.apache.ignite.internal.pagemem.wal.record.DataRecord; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; -import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.CacheInvokeEntry; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.EntryProcessorResourceInjectorProxy; @@ -543,7 +540,7 @@ protected GridCacheEntryEx entryEx(GridCacheContext cacheCtx, IgniteTxKey key, A checkValid(); - Collection commitEntries = (near() || cctx.snapshot().needTxReadLogging()) ? allEntries() : writeEntries(); + Collection commitEntries = near() ? allEntries() : writeEntries(); boolean empty = F.isEmpty(commitEntries) && !queryEnlisted(); @@ -816,22 +813,6 @@ else if (op == RELOAD) { updateNearEntrySafely(cacheCtx, txEntry.key(), GridCacheEntryEx::innerReload); } else if (op == READ) { - CacheGroupContext grp = cacheCtx.group(); - - if (grp.logDataRecords() && cctx.snapshot().needTxReadLogging()) { - ptr = grp.wal().log(new DataRecord(new DataEntry( - cacheCtx.cacheId(), - txEntry.key(), - val, - op, - nearXidVersion(), - writeVersion(), - 0, - txEntry.key().partition(), - txEntry.updateCounter(), - DataEntry.flags(CU.txOnPrimary(this))))); - } - ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry); if (expiry != null) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteSequentialNodeCrashRecoveryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteSequentialNodeCrashRecoveryTest.java index 746474a64fbe9..3e3b4fb5106b2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteSequentialNodeCrashRecoveryTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteSequentialNodeCrashRecoveryTest.java @@ -302,11 +302,6 @@ private static class DummyCheckpointContext implements CheckpointListener.Contex return null; } - /** {@inheritDoc} */ - @Override public boolean nextSnapshot() { - return false; - } - /** {@inheritDoc} */ @Override public IgniteInternalFuture finishedStateFut() { return null; @@ -317,11 +312,6 @@ private static class DummyCheckpointContext implements CheckpointListener.Contex return null; } - /** {@inheritDoc} */ - @Override public boolean needToSnapshot(String cacheOrGrpName) { - return false; - } - /** {@inheritDoc} */ @Override public void walFlush(boolean flush) { // No-op. 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 9044e1b208b17..a5072cfa4f3b6 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 @@ -151,7 +151,6 @@ public void testLightCheckpointAbleToStoreOnlyGivenDataRegion() throws Exception () -> Arrays.asList(regionForCheckpoint), grpId -> getPageMemoryForCacheGroup(grpId, db, context), PageMemoryImpl.ThrottlingPolicy.CHECKPOINT_BUFFER_ONLY, - context.cache().context().snapshot(), db.dataStorageMetricsImpl(), context.longJvmPauseDetector(), context.failure(), diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java index f03a64e81874e..724a500a0760d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java @@ -184,7 +184,6 @@ private void checkInvariantSwitchSegmentSize(int serVer) throws Exception { null, null, null, - null, null) ).createSerializer(serVer); @@ -473,7 +472,6 @@ private T2 initiate( null, null, null, - null, new GridCacheIoManager(), null, null, diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java index 93a5ea9bd098f..ebe0d196238e3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java @@ -26,7 +26,6 @@ import org.apache.ignite.internal.managers.systemview.JmxSystemViewExporterSpi; import org.apache.ignite.internal.mem.DirectMemoryProvider; import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider; -import org.apache.ignite.internal.pagemem.FullPageId; import org.apache.ignite.internal.pagemem.PageMemory; import org.apache.ignite.internal.processors.cache.CacheDiagnosticManager; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; @@ -39,7 +38,6 @@ import org.apache.ignite.internal.processors.performancestatistics.PerformanceStatisticsProcessor; import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor; import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor; -import org.apache.ignite.internal.util.typedef.CIX3; import org.apache.ignite.lang.IgniteOutClosure; import org.apache.ignite.spi.encryption.noop.NoopEncryptionSpi; import org.apache.ignite.spi.metric.noop.NoopMetricExporterSpi; @@ -98,7 +96,6 @@ public class BPlusTreePageMemoryImplTest extends BPlusTreeSelfTest { null, null, null, - null, new CacheDiagnosticManager(), null ); @@ -117,10 +114,6 @@ public class BPlusTreePageMemoryImplTest extends BPlusTreeSelfTest { (fullPageId, byteBuf, tag) -> { assert false : "No page replacement should happen during the test"; }, - new CIX3() { - @Override public void applyx(Long aLong, FullPageId fullPageId, PageMemoryEx ex) { - } - }, () -> true, new DataRegionMetricsImpl(new DataRegionConfiguration(), cctx), PageMemoryImpl.ThrottlingPolicy.DISABLED, diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java index c7d5a92696bbf..8bd5ce3ef073e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java @@ -26,7 +26,6 @@ import org.apache.ignite.internal.managers.systemview.JmxSystemViewExporterSpi; import org.apache.ignite.internal.mem.DirectMemoryProvider; import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider; -import org.apache.ignite.internal.pagemem.FullPageId; import org.apache.ignite.internal.pagemem.PageMemory; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetricsImpl; @@ -38,7 +37,6 @@ import org.apache.ignite.internal.processors.performancestatistics.PerformanceStatisticsProcessor; import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor; import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor; -import org.apache.ignite.internal.util.lang.GridInClosure3X; import org.apache.ignite.lang.IgniteOutClosure; import org.apache.ignite.spi.encryption.noop.NoopEncryptionSpi; import org.apache.ignite.spi.metric.noop.NoopMetricExporterSpi; @@ -98,7 +96,6 @@ public class BPlusTreeReuseListPageMemoryImplTest extends BPlusTreeReuseSelfTest null, null, null, - null, null ); @@ -116,10 +113,6 @@ public class BPlusTreeReuseListPageMemoryImplTest extends BPlusTreeReuseSelfTest (fullPageId, byteBuf, tag) -> { assert false : "No page replacement (rotation with disk) should happen during the test"; }, - new GridInClosure3X() { - @Override public void applyx(Long page, FullPageId fullPageId, PageMemoryEx pageMem) { - } - }, () -> true, new DataRegionMetricsImpl(new DataRegionConfiguration(), cctx), PageMemoryImpl.ThrottlingPolicy.DISABLED, diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java index 1d80c893b76ed..68d30400217c5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java @@ -255,7 +255,7 @@ private PageMemoryImpl createPageMemory(IgniteConfiguration cfg, PageStoreWriter IgniteOutClosure clo = () -> Mockito.mock(CheckpointProgressImpl.class); PageMemoryImpl memory = new PageMemoryImpl(provider, sizes, sctx, sctx.pageStore(), pageSize, - pageWriter, null, () -> true, memMetrics, PageMemoryImpl.ThrottlingPolicy.DISABLED, + pageWriter, () -> true, memMetrics, PageMemoryImpl.ThrottlingPolicy.DISABLED, clo); memory.start(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java index 14d1380acd559..0d9e28d60aee6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java @@ -27,7 +27,6 @@ import org.apache.ignite.internal.managers.systemview.JmxSystemViewExporterSpi; import org.apache.ignite.internal.mem.DirectMemoryProvider; import org.apache.ignite.internal.mem.file.MappedFileMemoryProvider; -import org.apache.ignite.internal.pagemem.FullPageId; import org.apache.ignite.internal.pagemem.PageMemory; import org.apache.ignite.internal.processors.cache.CacheDiagnosticManager; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; @@ -40,7 +39,6 @@ import org.apache.ignite.internal.processors.performancestatistics.PerformanceStatisticsProcessor; import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor; import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor; -import org.apache.ignite.internal.util.lang.GridInClosure3X; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteOutClosure; import org.apache.ignite.spi.encryption.noop.NoopEncryptionSpi; @@ -113,7 +111,6 @@ public class IndexStoragePageMemoryImplTest extends IndexStorageSelfTest { null, null, null, - null, new CacheDiagnosticManager(), null ); @@ -128,10 +125,6 @@ public class IndexStoragePageMemoryImplTest extends IndexStorageSelfTest { (fullPageId, byteBuf, tag) -> { assert false : "No page replacement (rotation with disk) should happen during the test"; }, - new GridInClosure3X() { - @Override public void applyx(Long page, FullPageId fullId, PageMemoryEx pageMem) { - } - }, () -> true, new DataRegionMetricsImpl(new DataRegionConfiguration(), cctx), PageMemoryImpl.ThrottlingPolicy.DISABLED, diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java index 1f2cc68f36636..3364181caaf88 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java @@ -27,7 +27,6 @@ import org.apache.ignite.internal.managers.systemview.JmxSystemViewExporterSpi; import org.apache.ignite.internal.mem.DirectMemoryProvider; import org.apache.ignite.internal.mem.file.MappedFileMemoryProvider; -import org.apache.ignite.internal.pagemem.FullPageId; import org.apache.ignite.internal.pagemem.PageMemory; import org.apache.ignite.internal.pagemem.impl.PageMemoryNoLoadSelfTest; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; @@ -39,7 +38,6 @@ import org.apache.ignite.internal.processors.performancestatistics.PerformanceStatisticsProcessor; import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor; import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor; -import org.apache.ignite.internal.util.lang.GridInClosure3X; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteOutClosure; import org.apache.ignite.spi.encryption.noop.NoopEncryptionSpi; @@ -103,7 +101,6 @@ public class PageMemoryImplNoLoadTest extends PageMemoryNoLoadSelfTest { null, null, null, - null, null ); @@ -118,10 +115,6 @@ public class PageMemoryImplNoLoadTest extends PageMemoryNoLoadSelfTest { (fullPageId, byteBuf, tag) -> { assert false : "No page replacement (rotation with disk) should happen during the test"; }, - new GridInClosure3X() { - @Override public void applyx(Long page, FullPageId fullId, PageMemoryEx pageMem) { - } - }, () -> true, new DataRegionMetricsImpl(new DataRegionConfiguration(), cctx), PageMemoryImpl.ThrottlingPolicy.DISABLED, diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java index 94b79039cb079..84e256232b3ba 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java @@ -61,7 +61,6 @@ import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor; import org.apache.ignite.internal.util.GridMultiCollectionWrapper; import org.apache.ignite.internal.util.future.GridFinishedFuture; -import org.apache.ignite.internal.util.lang.GridInClosure3X; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgniteOutClosure; @@ -633,7 +632,6 @@ private PageMemoryImpl createPageMemory( null, null, null, - null, null ); @@ -655,10 +653,6 @@ private PageMemoryImpl createPageMemory( sharedCtx.pageStore(), PAGE_SIZE, replaceWriter, - new GridInClosure3X() { - @Override public void applyx(Long page, FullPageId fullId, PageMemoryEx pageMem) { - } - }, () -> true, new DataRegionMetricsImpl(igniteCfg.getDataStorageConfiguration().getDefaultDataRegionConfiguration(), kernalCtx), throttlingPlc, @@ -671,10 +665,6 @@ private PageMemoryImpl createPageMemory( sharedCtx.pageStore(), PAGE_SIZE, replaceWriter, - new GridInClosure3X() { - @Override public void applyx(Long page, FullPageId fullId, PageMemoryEx pageMem) { - } - }, () -> true, new DataRegionMetricsImpl(igniteCfg.getDataStorageConfiguration().getDefaultDataRegionConfiguration(), kernalCtx), throttlingPlc, diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java index 4bb9cebb92e45..6d21728e87c04 100644 --- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java +++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java @@ -42,7 +42,6 @@ import org.apache.ignite.internal.processors.cache.dr.GridOsCacheDrManager; import org.apache.ignite.internal.processors.cache.jta.CacheNoopJtaManager; import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager; import org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager; import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager; import org.apache.ignite.internal.processors.cache.store.CacheOsStoreManager; @@ -75,7 +74,6 @@ public GridCacheTestContext(GridTestKernalContext ctx) throws Exception { new WalStateManager(null), new IgniteCacheDatabaseSharedManager(ctx), null, - new IgniteCacheSnapshotManager(), new GridCacheDeploymentManager(), new GridCachePartitionExchangeManager(), new CacheAffinitySharedManager(), From 4840979a83142a26836e966838051d7a14892768 Mon Sep 17 00:00:00 2001 From: nizhikov Date: Wed, 5 Jul 2023 18:09:07 +0300 Subject: [PATCH 3/7] IGNITE-19915 Rename. --- .../snapshot/SnapshotCompressionBasicTest.java | 2 +- .../ignite/util/GridCommandHandlerTest.java | 6 +++--- .../apache/ignite/internal/IgniteKernal.java | 2 +- .../management/snapshot/SnapshotCheckTask.java | 2 +- .../snapshot/SnapshotCreateTask.java | 2 +- .../snapshot/SnapshotRestoreTask.java | 4 ++-- .../snapshot/SnapshotStatusTask.java | 2 +- .../encryption/GridEncryptionManager.java | 4 ++-- .../encryption/GroupKeyChangeProcess.java | 4 ++-- .../processors/cache/ClusterCachesInfo.java | 8 ++++---- .../processors/cache/GridCacheProcessor.java | 10 +++++----- .../cache/GridCacheSharedContext.java | 18 +++++++++--------- .../distributed/near/GridNearTxLocal.java | 2 +- .../GridCacheDatabaseSharedManager.java | 4 ++-- .../snapshot/IgniteSnapshotManager.java | 8 ++++---- .../IncrementalSnapshotFutureTask.java | 2 +- .../snapshot/IncrementalSnapshotProcessor.java | 6 +++--- .../IncrementalSnapshotVerificationTask.java | 4 ++-- .../snapshot/SnapshotFutureTask.java | 2 +- .../snapshot/SnapshotHandlerRestoreTask.java | 4 ++-- .../snapshot/SnapshotMXBeanImpl.java | 2 +- .../SnapshotMetadataVerificationTask.java | 4 ++-- .../SnapshotPartitionsVerifyHandler.java | 2 +- .../snapshot/SnapshotPartitionsVerifyTask.java | 4 ++-- .../SnapshotResponseRemoteFutureTask.java | 4 ++-- .../snapshot/SnapshotRestoreProcess.java | 18 +++++++++--------- .../snapshot/SnapshotRestoreStatusTask.java | 2 +- .../cache/transactions/IgniteTxHandler.java | 4 ++-- .../datastreamer/DataStreamerImpl.java | 2 +- .../snapshot/AbstractSnapshotSelfTest.java | 6 +++--- .../IgniteClusterSnapshotHandlerTest.java | 4 ++-- .../IgniteClusterSnapshotRestoreSelfTest.java | 4 ++-- .../IgniteClusterSnapshotSelfTest.java | 6 +++--- .../IgniteSnapshotConsistencyTest.java | 2 +- .../snapshot/IgniteSnapshotMXBeanTest.java | 2 +- .../IgniteSnapshotRestoreFromRemoteTest.java | 2 +- ...iteClusterSnapshotCheckWithIndexesTest.java | 6 +++--- .../IgniteClusterSnapshotMetricsTest.java | 2 +- 38 files changed, 86 insertions(+), 86 deletions(-) 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 4ccdf4ca3c3d6..7c926f19d48b2 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 @@ -331,7 +331,7 @@ protected void createTestSnapshot() throws Exception { for (String snpName : Arrays.asList(SNAPSHOT_WITH_HOLES, SNAPSHOT_WITHOUT_HOLES)) { snp(ignite).createSnapshot(snpName, null, false, onlyPrimary).get(TIMEOUT); - IdleVerifyResultV2 res = ignite.context().cache().context().snapshotMgr().checkSnapshot(snpName, null) + IdleVerifyResultV2 res = ignite.context().cache().context().snapshot().checkSnapshot(snpName, null) .get().idleVerifyResult(); StringBuilder b = new StringBuilder(); 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 abb7a50c4fb58..87d1731c4ad01 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 @@ -3654,7 +3654,7 @@ public void testSnapshotRestoreCancelAndStatus() throws Exception { awaitPartitionMapExchange(); CountDownLatch ioStartLatch = new CountDownLatch(1); - IgniteSnapshotManager snpMgr = ig.context().cache().context().snapshotMgr(); + IgniteSnapshotManager snpMgr = ig.context().cache().context().snapshot(); // Replace the IO factory in the snapshot manager so we have enough time to test the status command. snpMgr.ioFactory(new SlowDownFileIoFactory(snpMgr.ioFactory(), getTestTimeout() / locPartsCnt, ioStartLatch)); @@ -3695,7 +3695,7 @@ public void testSnapshotRestoreCancelAndStatus() throws Exception { // Make sure the context disappeared at node 1. boolean ctxDisposed = - waitForCondition(() -> !grid(1).context().cache().context().snapshotMgr().isRestoring(), getTestTimeout()); + waitForCondition(() -> !grid(1).context().cache().context().snapshot().isRestoring(), getTestTimeout()); assertTrue(ctxDisposed); @@ -3815,7 +3815,7 @@ private void checkSnapshotStatus(boolean isCreating, boolean isRestoring, boolea assertTrue(waitForCondition(() -> srvs.stream().allMatch( ignite -> { - IgniteSnapshotManager mgr = ((IgniteEx)ignite).context().cache().context().snapshotMgr(); + IgniteSnapshotManager mgr = ((IgniteEx)ignite).context().cache().context().snapshot(); return isCreating == mgr.isSnapshotCreating() && isRestoring == mgr.isRestoring(); }), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index e61a1a8772f7e..8990df38d444a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -2868,7 +2868,7 @@ private Collection baselineNodes() { /** {@inheritDoc} */ @Override public IgniteSnapshot snapshot() { - return ctx.cache().context().snapshotMgr(); + return ctx.cache().context().snapshot(); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCheckTask.java b/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCheckTask.java index c467585d8629b..3e777156d5a76 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCheckTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCheckTask.java @@ -54,7 +54,7 @@ protected SnapshotCheckJob(SnapshotCheckCommandArg arg, boolean debug) { /** {@inheritDoc} */ @Override protected SnapshotPartitionsVerifyTaskResult run(SnapshotCheckCommandArg arg) throws IgniteException { - IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshotMgr(); + IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshot(); return new IgniteFutureImpl<>(snpMgr.checkSnapshot(arg.snapshotName(), arg.src(), arg.increment())).get(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCreateTask.java b/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCreateTask.java index e236ad6fc7cf2..c0eca7a8932f2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCreateTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCreateTask.java @@ -52,7 +52,7 @@ protected SnapshotCreateJob(SnapshotCreateCommandArg arg, boolean debug) { /** {@inheritDoc} */ @Override protected String run(SnapshotCreateCommandArg arg) throws IgniteException { - IgniteFutureImpl fut = ignite.context().cache().context().snapshotMgr().createSnapshot( + IgniteFutureImpl fut = ignite.context().cache().context().snapshot().createSnapshot( arg.snapshotName(), arg.dest(), arg.incremental(), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotRestoreTask.java b/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotRestoreTask.java index d238c7bc69d70..c28b7cf24370d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotRestoreTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotRestoreTask.java @@ -58,7 +58,7 @@ protected SnapshotStartRestoreJob(SnapshotRestoreCommandArg arg, boolean debug) /** {@inheritDoc} */ @Override protected String run(SnapshotRestoreCommandArg arg) throws IgniteException { - IgniteFutureImpl fut = ignite.context().cache().context().snapshotMgr().restoreSnapshot( + IgniteFutureImpl fut = ignite.context().cache().context().snapshot().restoreSnapshot( arg.snapshotName(), arg.src(), arg.groups() == null ? null : Arrays.asList(arg.groups()), @@ -125,7 +125,7 @@ protected SnapshotRestoreStatusJob(SnapshotRestoreCommandArg arg, boolean debug) /** {@inheritDoc} */ @Override protected String run(SnapshotRestoreCommandArg arg) throws IgniteException { - boolean state = ignite.context().cache().context().snapshotMgr().restoreStatus(arg.snapshotName()).get(); + boolean state = ignite.context().cache().context().snapshot().restoreStatus(arg.snapshotName()).get(); return "Snapshot cache group restore operation is " + (state ? "" : "NOT ") + "running [snapshot=" + arg.snapshotName() + ']'; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotStatusTask.java b/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotStatusTask.java index 88fea0f7eab3f..7451f81ff85f5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotStatusTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotStatusTask.java @@ -111,7 +111,7 @@ protected SnapshotStatusJob(@Nullable NoArg arg, boolean debug) { if (!CU.isPersistenceEnabled(ignite.context().config())) return null; - IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshotMgr(); + IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshot(); SnapshotOperationRequest req = snpMgr.currentCreateRequest(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java index f1f5f11c5537c..e26ebd55ff762 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java @@ -1528,8 +1528,8 @@ private IgniteInternalFuture prepareMasterKeyChange(MasterKeyChange "The previous change was not completed.")); } - if (ctx.cache().context().snapshotMgr().isSnapshotCreating() - || ctx.cache().context().snapshotMgr().isRestoring()) { + if (ctx.cache().context().snapshot().isSnapshotCreating() + || ctx.cache().context().snapshot().isRestoring()) { return new GridFinishedFuture<>(new IgniteException("Master key change was rejected. Snapshot operation " + "is in progress.")); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GroupKeyChangeProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GroupKeyChangeProcess.java index a8b63bce21593..7167257c6a29a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GroupKeyChangeProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GroupKeyChangeProcess.java @@ -197,8 +197,8 @@ private IgniteInternalFuture prepare(ChangeCacheEncryptionRequest r "The previous change was not completed.")); } - if (ctx.cache().context().snapshotMgr().isSnapshotCreating() - || ctx.cache().context().snapshotMgr().isRestoring()) { + if (ctx.cache().context().snapshot().isSnapshotCreating() + || ctx.cache().context().snapshot().isRestoring()) { return new GridFinishedFuture<>(new IgniteException("Cache group key change was rejected. " + "Snapshot operation is in progress.")); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java index 3109bc5c21c35..248b9c77319e9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java @@ -622,7 +622,7 @@ public boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, AffinityTop assert restartIds.size() <= 1 : batch.requests(); - Collection nodes = ctx.cache().context().snapshotMgr().cacheStartRequiredAliveNodes(F.first(restartIds)); + Collection nodes = ctx.cache().context().snapshot().cacheStartRequiredAliveNodes(F.first(restartIds)); for (UUID nodeId : nodes) { ClusterNode node = ctx.discovery().node(nodeId); @@ -872,7 +872,7 @@ private boolean processStopCacheRequest( AffinityTopologyVersion topVer, boolean checkForAlreadyDeleted ) { - if (ctx.cache().context().snapshotMgr().isSnapshotCreating()) { + if (ctx.cache().context().snapshot().isSnapshotCreating()) { IgniteCheckedException err = new IgniteCheckedException(SNP_IN_PROGRESS_ERR_MSG); U.warn(log, err); @@ -1119,9 +1119,9 @@ else if (encMgr.masterKeyDigest() != null && } if (err == null && req.restartId() == null) { - IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); + IgniteSnapshotManager snpMgr = ctx.cache().context().snapshot(); - if (snapshotMgr.isRestoring(ccfg)) { + if (snpMgr.isRestoring(ccfg)) { err = new IgniteCheckedException("Cache start failed. A cache or group with the same name is " + "currently being restored from a snapshot [cache=" + cacheName + (ccfg.getGroupName() == null ? "" : ", group=" + ccfg.getGroupName()) + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 61696306fe782..7ab80391ae9df 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -3063,10 +3063,10 @@ private GridCacheSharedContext createSharedContext( WalStateManager walStateMgr = new WalStateManager(ctx); - IgniteSnapshotManager snapshotMgr = ctx.plugins().createComponent(IgniteSnapshotManager.class); + IgniteSnapshotManager snpMgr = ctx.plugins().createComponent(IgniteSnapshotManager.class); - if (snapshotMgr == null) - snapshotMgr = new IgniteSnapshotManager(ctx); + if (snpMgr == null) + snpMgr = new IgniteSnapshotManager(ctx); CacheObjectTransformerManager transMgr = ctx.plugins().createComponent(CacheObjectTransformerManager.class); @@ -3092,7 +3092,7 @@ private GridCacheSharedContext createSharedContext( walMgr, walStateMgr, dbMgr, - snapshotMgr, + snpMgr, depMgr, exchMgr, topMgr, @@ -4331,7 +4331,7 @@ private IgniteNodeValidationResult validateRestartingCaches(ClusterNode node) { * @return Node validation result if there was an issue with the joining node, {@code null} otherwise. */ private IgniteNodeValidationResult validateRestoringCaches(ClusterNode node) { - if (ctx.cache().context().snapshotMgr().isRestoring()) { + if (ctx.cache().context().snapshot().isRestoring()) { String msg = "Joining node during caches restore is not allowed [joiningNodeId=" + node.id() + ']'; return new IgniteNodeValidationResult(node.id(), msg); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java index e6f9f70e12455..1cfad22daef56 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java @@ -127,7 +127,7 @@ public class GridCacheSharedContext { @Nullable private IgnitePageStoreManager pageStoreMgr; /** Snapshot manager for persistence caches. See {@link IgniteSnapshot}. */ - private IgniteSnapshotManager snapshotMgr; + private IgniteSnapshotManager snpMgr; /** Affinity manager. */ private CacheAffinitySharedManager affMgr; @@ -224,7 +224,7 @@ public GridCacheSharedContext( @Nullable IgniteWriteAheadLogManager walMgr, WalStateManager walStateMgr, IgniteCacheDatabaseSharedManager dbMgr, - IgniteSnapshotManager snapshotMgr, + IgniteSnapshotManager snpMgr, GridCacheDeploymentManager depMgr, GridCachePartitionExchangeManager exchMgr, CacheAffinitySharedManager affMgr, @@ -251,7 +251,7 @@ public GridCacheSharedContext( walMgr, walStateMgr, dbMgr, - snapshotMgr, + snpMgr, depMgr, exchMgr, affMgr, @@ -294,7 +294,7 @@ public GridCacheSharedContext( stateAwareMgrs.add(dbMgr); - stateAwareMgrs.add(snapshotMgr); + stateAwareMgrs.add(snpMgr); for (PluginProvider prv : kernalCtx.plugins().allProviders()) if (prv instanceof IgniteChangeGlobalStateSupport) @@ -431,7 +431,7 @@ void onReconnected(boolean active) throws IgniteCheckedException { cdcWalMgr, walStateMgr, dbMgr, - snapshotMgr, + snpMgr, new GridCacheDeploymentManager(), new GridCachePartitionExchangeManager(), affMgr, @@ -482,7 +482,7 @@ private void setManagers( IgniteWriteAheadLogManager cdcWalMgr, WalStateManager walStateMgr, IgniteCacheDatabaseSharedManager dbMgr, - IgniteSnapshotManager snapshotMgr, + IgniteSnapshotManager snpMgr, GridCacheDeploymentManager depMgr, GridCachePartitionExchangeManager exchMgr, CacheAffinitySharedManager affMgr, @@ -506,7 +506,7 @@ private void setManagers( this.cdcWalMgr = walMgr == null ? add(mgrs, cdcWalMgr) : cdcWalMgr; this.walStateMgr = add(mgrs, walStateMgr); this.dbMgr = add(mgrs, dbMgr); - this.snapshotMgr = add(mgrs, snapshotMgr); + this.snpMgr = add(mgrs, snpMgr); this.jtaMgr = add(mgrs, jtaMgr); this.depMgr = add(mgrs, depMgr); this.exchMgr = add(mgrs, exchMgr); @@ -753,8 +753,8 @@ public IgniteCacheDatabaseSharedManager database() { /** * @return Page storage snapshot manager. */ - public IgniteSnapshotManager snapshotMgr() { - return snapshotMgr; + public IgniteSnapshotManager snapshot() { + return snpMgr; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java index c5555d97ac930..5211f626cd353 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java @@ -3981,7 +3981,7 @@ public IgniteInternalFuture commitNearTxLocalAsync() { commitOrRollbackStartTime.compareAndSet(0, System.nanoTime()); if (!onePhaseCommit) - incrementalSnapshotId(cctx.snapshotMgr().incrementalSnapshotId()); + incrementalSnapshotId(cctx.snapshot().incrementalSnapshotId()); try { // Make sure that here are no exceptions. 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 b7451bf72e7f2..b2aad6ea4ec1f 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 @@ -1513,7 +1513,7 @@ private String cacheInfo(GridCacheContext cacheCtx) { .map(t -> t.get1().groupId()) .collect(toList()); - cctx.snapshotMgr().onCacheGroupsStopped(stoppedGrpIds); + cctx.snapshot().onCacheGroupsStopped(stoppedGrpIds); initiallyLocWalDisabledGrps.removeAll(stoppedGrpIds); initiallyGlobalWalDisabledGrps.removeAll(stoppedGrpIds); @@ -3117,7 +3117,7 @@ public boolean isCheckpointInapplicableForWalRebalance(Long cpTs, int grpId) thr if (lastCpTs != 0) metaStorage.write(checkpointInapplicableCpAndGroupIdToKey(lastCpTs, grpId), true); - cctx.snapshotMgr().disableIncrementalSnapshotsCreation(metaStorage, grpId); + cctx.snapshot().disableIncrementalSnapshotsCreation(metaStorage, grpId); } catch (IgniteCheckedException e) { log.error("Failed to mark last checkpoint as inapplicable for WAL rebalance for group: " + grpId, e); 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 780470bf2e273..5296053a964a0 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 @@ -4545,7 +4545,7 @@ public CreateSnapshotCallable(String snpName, boolean incremental, boolean onlyP if (incremental) ignite.snapshot().createIncrementalSnapshot(snpName).get(); else - ignite.context().cache().context().snapshotMgr().createSnapshot(snpName, null, false, onlyPrimary).get(); + ignite.context().cache().context().snapshot().createSnapshot(snpName, null, false, onlyPrimary).get(); return null; } @@ -4579,12 +4579,12 @@ public CancelSnapshotCallable(UUID reqId, String snpName) { /** {@inheritDoc} */ @Override public Boolean call() throws Exception { if (reqId != null) - return ignite.context().cache().context().snapshotMgr().cancelLocalSnapshotOperations(reqId); + return ignite.context().cache().context().snapshot().cancelLocalSnapshotOperations(reqId); else { - if (ignite.context().cache().context().snapshotMgr().cancelLocalSnapshotTask(snpName)) + if (ignite.context().cache().context().snapshot().cancelLocalSnapshotTask(snpName)) return true; - return ignite.context().cache().context().snapshotMgr().cancelLocalRestoreTask(snpName).get(); + return ignite.context().cache().context().snapshot().cancelLocalRestoreTask(snpName).get(); } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotFutureTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotFutureTask.java index e9be26974f7ad..53c4cdf41d9bd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotFutureTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotFutureTask.java @@ -123,7 +123,7 @@ public IncrementalSnapshotFutureTask( /** {@inheritDoc} */ @Override public boolean start() { try { - File incSnpDir = cctx.snapshotMgr().incrementalSnapshotLocalDir(snpName, snpPath, incIdx); + File incSnpDir = cctx.snapshot().incrementalSnapshotLocalDir(snpName, snpPath, incIdx); if (!incSnpDir.mkdirs() && !incSnpDir.exists()) { onDone(new IgniteException("Can't create snapshot directory [dir=" + incSnpDir.getAbsolutePath() + ']')); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotProcessor.java index 998d5130a7600..f4f994809bc12 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotProcessor.java @@ -96,7 +96,7 @@ void process( Consumer dataEntryHnd, @Nullable Consumer txHnd ) throws IgniteCheckedException, IOException { - IncrementalSnapshotMetadata meta = cctx.snapshotMgr() + IncrementalSnapshotMetadata meta = cctx.snapshot() .readIncrementalSnapshotMetadata(snpName, snpPath, incIdx); File[] segments = walSegments(meta.folderName()); @@ -158,7 +158,7 @@ void process( } UUID prevIncSnpId = incIdx > 1 - ? cctx.snapshotMgr().readIncrementalSnapshotMetadata(snpName, snpPath, incIdx - 1).requestId() + ? cctx.snapshot().readIncrementalSnapshotMetadata(snpName, snpPath, incIdx - 1).requestId() : null; IgnitePredicate txVerFilter = prevIncSnpId != null @@ -225,7 +225,7 @@ private File[] walSegments(String folderName) throws IgniteCheckedException { File[] segments = null; for (int i = 1; i <= incIdx; i++) { - File incSnpDir = cctx.snapshotMgr().incrementalSnapshotLocalDir(snpName, snpPath, i); + File incSnpDir = cctx.snapshot().incrementalSnapshotLocalDir(snpName, snpPath, i); if (!incSnpDir.exists()) throw new IgniteCheckedException("Incremental snapshot doesn't exists [dir=" + incSnpDir + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerificationTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerificationTask.java index c64cadb0f1e52..29a8c97865472 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerificationTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerificationTask.java @@ -401,7 +401,7 @@ else if (txRec.state() == TransactionState.ROLLED_BACK) { /** Checks that current baseline topology matches baseline topology of the snapshot. */ private void checkBaseline(BaselineTopology blt) throws IgniteCheckedException, IOException { - IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshotMgr(); + IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshot(); File snpDir = snpMgr.snapshotLocalDir(snpName, snpPath); SnapshotMetadata meta = snpMgr.readSnapshotMetadata(snpDir, ignite.localNode().consistentId().toString()); @@ -414,7 +414,7 @@ private void checkBaseline(BaselineTopology blt) throws IgniteCheckedException, /** @return Collection of snapshotted transactional caches, key is a cache ID. */ private Map readTxCachesData() throws IgniteCheckedException, IOException { - File snpDir = ignite.context().cache().context().snapshotMgr().snapshotLocalDir(snpName, snpPath); + File snpDir = ignite.context().cache().context().snapshot().snapshotLocalDir(snpName, snpPath); String folderName = ignite.context().pdsFolderResolver().resolveFolders().folderName(); 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 7fc31dbdfb7c4..b774fc716f588 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 @@ -159,7 +159,7 @@ class SnapshotFutureTask extends AbstractSnapshotFutureTask deltaWriterFactory = - cctx.snapshotMgr().sequentialWrite() ? IndexedPageStoreSerialWriter::new : PageStoreSerialWriter::new; + cctx.snapshot().sequentialWrite() ? IndexedPageStoreSerialWriter::new : PageStoreSerialWriter::new; /** * @param cctx Shared context. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerRestoreTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerRestoreTask.java index 5c34511ce2755..063ed7082037e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerRestoreTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerRestoreTask.java @@ -87,7 +87,7 @@ public class SnapshotHandlerRestoreTask extends AbstractSnapshotVerificationTask String snapshotName = F.first(F.first(metas.values())).snapshotName(); try { - ignite.context().cache().context().snapshotMgr().handlers().completeAll( + ignite.context().cache().context().snapshot().handlers().completeAll( SnapshotHandlerType.RESTORE, snapshotName, clusterResults, execNodes, wrns -> {}); } catch (Exception e) { @@ -151,7 +151,7 @@ public SnapshotHandlerRestoreJob( /** {@inheritDoc} */ @Override public Map> execute() { try { - IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshotMgr(); + IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshot(); File snpDir = snpMgr.snapshotLocalDir(snpName, snpPath); SnapshotMetadata meta = snpMgr.readSnapshotMetadata(snpDir, consistentId); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMXBeanImpl.java index 59032be79e51c..0a04afb1251c3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMXBeanImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMXBeanImpl.java @@ -46,7 +46,7 @@ public class SnapshotMXBeanImpl implements SnapshotMXBean { * @param ctx Kernal context. */ public SnapshotMXBeanImpl(GridKernalContext ctx) { - mgr = ctx.cache().context().snapshotMgr(); + mgr = ctx.cache().context().snapshot(); metricMgr = ctx.metric(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataVerificationTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataVerificationTask.java index 1695ad692efc9..0a425bd2f0cee 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataVerificationTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataVerificationTask.java @@ -89,7 +89,7 @@ public MetadataVerificationJob(SnapshotMetadataVerificationTaskArg arg) { /** {@inheritDoc} */ @Override public List execute() throws IgniteException { - IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshotMgr(); + IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshot(); List snpMeta = snpMgr.readSnapshotMetadatas(arg.snapshotName(), arg.snapshotPath()); @@ -112,7 +112,7 @@ public MetadataVerificationJob(SnapshotMetadataVerificationTaskArg arg) { /** Checks that all incremental snapshots are present, contain correct metafile and WAL segments. */ public void checkIncrementalSnapshots(SnapshotMetadata fullMeta, SnapshotMetadataVerificationTaskArg arg) { try { - IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshotMgr(); + IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshot(); // Incremental snapshot must contain ClusterSnapshotRecord. long startSeg = fullMeta.snapshotRecordPointer().index(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java index e0e654d49ae81..a3fb5356ab2bd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java @@ -158,7 +158,7 @@ public SnapshotPartitionsVerifyHandler(GridCacheSharedContext cctx) { ThreadLocal buff = ThreadLocal.withInitial(() -> ByteBuffer.allocateDirect(meta.pageSize()) .order(ByteOrder.nativeOrder())); - IgniteSnapshotManager snpMgr = cctx.snapshotMgr(); + IgniteSnapshotManager snpMgr = cctx.snapshot(); GridKernalContext snpCtx = snpMgr.createStandaloneKernalContext(cctx.kernalContext().compress(), opCtx.snapshotDirectory(), meta.folderName()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTask.java index 2fc31021490c4..c2967ca4ce536 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTask.java @@ -127,8 +127,8 @@ public VerifySnapshotPartitionsJob( } try { - File snpDir = cctx.snapshotMgr().snapshotLocalDir(snpName, snpPath); - SnapshotMetadata meta = cctx.snapshotMgr().readSnapshotMetadata(snpDir, consId); + File snpDir = cctx.snapshot().snapshotLocalDir(snpName, snpPath); + SnapshotMetadata meta = cctx.snapshot().readSnapshotMetadata(snpDir, consId); return new SnapshotPartitionsVerifyHandler(cctx) .invoke(new SnapshotHandlerContext(meta, rqGrps, ignite.localNode(), snpDir, false, check)); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotResponseRemoteFutureTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotResponseRemoteFutureTask.java index 90e562f426559..2ed90af572513 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotResponseRemoteFutureTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotResponseRemoteFutureTask.java @@ -75,7 +75,7 @@ public SnapshotResponseRemoteFutureTask( return false; try { - List metas = cctx.snapshotMgr().readSnapshotMetadatas(snpName, snpPath); + List metas = cctx.snapshot().readSnapshotMetadatas(snpName, snpPath); Function findMeta = pair -> { for (SnapshotMetadata meta : metas) { @@ -108,7 +108,7 @@ public SnapshotResponseRemoteFutureTask( snpSndr.init(partsToSend.size()); - File snpDir = cctx.snapshotMgr().snapshotLocalDir(snpName, snpPath); + File snpDir = cctx.snapshot().snapshotLocalDir(snpName, snpPath); CompletableFuture.runAsync(() -> partsToSend.forEach((gp, meta) -> { if (err.get() != null) 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 f16543a4fa435..028d53f7acf85 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 @@ -268,7 +268,7 @@ public IgniteFutureImpl start( int incIdx, boolean check ) { - IgniteSnapshotManager snpMgr = ctx.cache().context().snapshotMgr(); + IgniteSnapshotManager snpMgr = ctx.cache().context().snapshot(); ClusterSnapshotFuture fut0; try { @@ -660,7 +660,7 @@ private IgniteInternalFuture prepare(SnapshotO lastOpCtx = opCtx0; DiscoveryDataClusterState state = ctx.state().clusterState(); - IgniteSnapshotManager snpMgr = ctx.cache().context().snapshotMgr(); + IgniteSnapshotManager snpMgr = ctx.cache().context().snapshot(); if (state.state() != ClusterState.ACTIVE || state.transition()) throw new IgniteCheckedException(OP_REJECT_MSG + "The cluster should be active."); @@ -777,7 +777,7 @@ private void enrichContext( // Metastorage can be restored only manually by directly copying files. boolean skipCompressCheck = false; for (SnapshotMetadata meta : metas) { - for (File snpCacheDir : cctx.snapshotMgr().snapshotCacheDirectories(req.snapshotName(), req.snapshotPath(), meta.folderName(), + for (File snpCacheDir : cctx.snapshot().snapshotCacheDirectories(req.snapshotName(), req.snapshotPath(), meta.folderName(), name -> !METASTORAGE_CACHE_NAME.equals(name))) { String grpName = FilePageStoreManager.cacheGroupName(snpCacheDir); @@ -970,7 +970,7 @@ private IgniteInternalFuture preload(UUID reqId) { AbstractSnapshotVerificationTask.checkMissedMetadata(allMetas); - IgniteSnapshotManager snpMgr = ctx.cache().context().snapshotMgr(); + IgniteSnapshotManager snpMgr = ctx.cache().context().snapshot(); synchronized (this) { opCtx0.stopFut = new IgniteFutureImpl<>(retFut.chain(f -> null)); @@ -992,7 +992,7 @@ private IgniteInternalFuture preload(UUID reqId) { SnapshotMetadata meta = F.first(opCtx0.metasPerNode.get(opCtx0.opNodeId)); File dir = opCtx0.incIdx > 0 ? - ctx.cache().context().snapshotMgr() + ctx.cache().context().snapshot() .incrementalSnapshotLocalDir(opCtx0.snpName, opCtx0.snpPath, opCtx0.incIdx) : snpDir; @@ -1130,7 +1130,7 @@ private IgniteInternalFuture preload(UUID reqId) { ", grpParts=" + partitionsMapToString(m.getValue(), cacheGrpNames) + "]"); } - ctx.cache().context().snapshotMgr() + ctx.cache().context().snapshot() .requestRemoteSnapshotFiles(m.getKey(), opCtx0.reqId, opCtx0.snpName, @@ -1585,7 +1585,7 @@ private void finishIncrementalSnapshotRestore(UUID reqId, Map res .map(cacheId -> CU.cacheOrGroupName(ctx.cache().cacheDescriptor(cacheId).cacheConfiguration())) .collect(Collectors.toSet()); - ctx.cache().context().snapshotMgr() + ctx.cache().context().snapshot() .warnAtomicCachesInIncrementalSnapshot(opCtx0.snpName, opCtx0.incIdx, cacheGrps); } @@ -1675,7 +1675,7 @@ private IgniteInternalFuture rollback(UUID reqId) { } try { - ctx.cache().context().snapshotMgr().snapshotExecutorService().execute(() -> { + ctx.cache().context().snapshot().snapshotExecutorService().execute(() -> { if (log.isInfoEnabled()) { log.info("Removing restored cache directories [reqId=" + reqId + ", snapshot=" + opCtx0.snpName + ", dirs=" + opCtx0.dirs + ']'); @@ -1760,7 +1760,7 @@ private void copyLocalAsync( Path partFile = Paths.get(targetDir.getAbsolutePath(), FilePageStoreManager.getPartitionFileName(partFut.partId)); int grpId = groupIdFromTmpDir(targetDir); - IgniteSnapshotManager snapMgr = ctx.cache().context().snapshotMgr(); + IgniteSnapshotManager snapMgr = ctx.cache().context().snapshot(); CompletableFuture copyPartFut = CompletableFuture.supplyAsync(() -> { if (opCtx.stopChecker.getAsBoolean()) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStatusTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStatusTask.java index 14a395779c9d4..82a450b89daca 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStatusTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStatusTask.java @@ -57,7 +57,7 @@ class SnapshotRestoreStatusTask extends ComputeTaskAdapter { private transient IgniteEx ignite; @Override public Boolean execute() throws IgniteException { - return ignite.context().cache().context().snapshotMgr().isRestoring(snpName); + return ignite.context().cache().context().snapshot().isRestoring(snpName); } }, node); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java index 4f8ef80316e7b..ed310a629baae 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java @@ -288,7 +288,7 @@ public IgniteTxHandler(GridCacheSharedContext ctx) { private void processIncrementalSnapshotAwareMessage(UUID nodeId, IncrementalSnapshotAwareMessage msg) { // Skip if node has joined after incremental snapshot started. if (ctx.localNode().order() <= msg.snapshotTopologyVersion()) { - ctx.snapshotMgr().handleIncrementalSnapshotId(msg.id(), msg.snapshotTopologyVersion()); + ctx.snapshot().handleIncrementalSnapshotId(msg.id(), msg.snapshotTopologyVersion()); setIncrementalSnapshotIdIfRequired(msg); } @@ -1285,7 +1285,7 @@ else if (nearTx != null) if (dhtTx != null) { dhtTx.onePhaseCommit(true); dhtTx.needReturnValue(req.needReturnValue()); - dhtTx.incrementalSnapshotId(ctx.snapshotMgr().incrementalSnapshotId()); + dhtTx.incrementalSnapshotId(ctx.snapshot().incrementalSnapshotId()); finish(dhtTx, req); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java index b98adfef6ea40..862f2ae1e18c5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java @@ -2386,7 +2386,7 @@ else if (ttl == CU.TTL_NOT_CHANGED) */ private static void snapshotWarning(GridCacheContext cctx) { if (cctx.group().persistenceEnabled()) - cctx.kernalContext().cache().context().snapshotMgr().streamerWarning(); + cctx.kernalContext().cache().context().snapshot().streamerWarning(); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java index 802497a2d72b2..10bac7ada444c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java @@ -315,7 +315,7 @@ protected void ensureCacheAbsent(CacheConfiguration ccfg) throws IgniteChe assertNull("nodeId=" + kctx.localNodeId() + ", cache=" + cacheName, desc); boolean success = GridTestUtils.waitForCondition( - () -> !kctx.cache().context().snapshotMgr().isRestoring(), + () -> !kctx.cache().context().snapshot().isRestoring(), TIMEOUT); assertTrue("The process has not finished on the node " + kctx.localNodeId(), success); @@ -715,7 +715,7 @@ protected void checkSnapshot(String snpName, String snpPath) throws IgniteChecke * @return Snapshot manager related to given ignite instance. */ public static IgniteSnapshotManager snp(IgniteEx ignite) { - return ignite.context().cache().context().snapshotMgr(); + return ignite.context().cache().context().snapshot(); } /** @@ -823,7 +823,7 @@ protected static IgniteInternalFuture startLocalSnapshotTask( boolean withMetaStorage, SnapshotSender snpSndr ) throws IgniteCheckedException { - AbstractSnapshotFutureTask task = cctx.snapshotMgr().registerSnapshotTask(snpName, cctx.localNodeId(), null, + AbstractSnapshotFutureTask task = cctx.snapshot().registerSnapshotTask(snpName, cctx.localNodeId(), null, parts, withMetaStorage, snpSndr); if (!(task instanceof SnapshotFutureTask)) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotHandlerTest.java index 1c3ae2ef5bc18..78deb26d28bab 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotHandlerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotHandlerTest.java @@ -148,7 +148,7 @@ public void testClusterSnapshotHandlers() throws Exception { */ private void changeMetadataRequestIdOnDisk(UUID newReqId) throws Exception { for (Ignite grid : G.allGrids()) { - IgniteSnapshotManager snpMgr = ((IgniteEx)grid).context().cache().context().snapshotMgr(); + IgniteSnapshotManager snpMgr = ((IgniteEx)grid).context().cache().context().snapshot(); String constId = grid.cluster().localNode().consistentId().toString(); File snpDir = snpMgr.snapshotLocalDir(SNAPSHOT_NAME); @@ -392,7 +392,7 @@ public void testHandlerSnapshotLocation() throws Exception { try { IgniteEx ignite = startGridsWithCache(1, CACHE_KEYS_RANGE, valueBuilder(), dfltCacheCfg); - IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshotMgr(); + IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshot(); createAndCheckSnapshot(ignite, snpName, snpDir.getAbsolutePath(), TIMEOUT); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java index 3fd9d6dde406f..897eaef99f47e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java @@ -155,7 +155,7 @@ public void testClusterSnapshotRestoreFromCustomDir() throws Exception { ignite.destroyCache(DEFAULT_CACHE_NAME); awaitPartitionMapExchange(); - ignite.context().cache().context().snapshotMgr().restoreSnapshot(SNAPSHOT_NAME, snpDir.getAbsolutePath(), null) + ignite.context().cache().context().snapshot().restoreSnapshot(SNAPSHOT_NAME, snpDir.getAbsolutePath(), null) .get(); IgniteCache cache = ignite.cache(DEFAULT_CACHE_NAME); @@ -674,7 +674,7 @@ public void testNodeFailDuringFilesCopy() throws Exception { String failingFilePath = Paths.get(CACHE_DIR_PREFIX + DEFAULT_CACHE_NAME, PART_FILE_PREFIX + (dfltCacheCfg.getAffinity().partitions() / 2) + FILE_SUFFIX).toString(); - grid(2).context().cache().context().snapshotMgr().ioFactory( + grid(2).context().cache().context().snapshot().ioFactory( new CustomFileIOFactory(new RandomAccessFileIOFactory(), file -> { if (file.getPath().endsWith(failingFilePath)) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotSelfTest.java index 9aa317c32123f..8861ee9a33beb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotSelfTest.java @@ -691,7 +691,7 @@ public void testRecoveryClusterSnapshotJvmHalted() throws Exception { assertTrue( "Snapshot directory must be empty", - grid2.context().cache().context().snapshotMgr().localSnapshotNames(null).isEmpty() + grid2.context().cache().context().snapshot().localSnapshotNames(null).isEmpty() ); createAndCheckSnapshot(ignite, SNAPSHOT_NAME); @@ -797,7 +797,7 @@ public void testClusterSnapshotWithExplicitPathError() throws Exception { GridKernalContext kctx = grid(idx).context(); assertThrowsAnyCause(log, - () -> kctx.cache().context().snapshotMgr() + () -> kctx.cache().context().snapshot() .createSnapshot(SNAPSHOT_NAME, invalidPath, false, onlyPrimary) .get(TIMEOUT), IgniteCheckedException.class, @@ -811,7 +811,7 @@ public void testClusterSnapshotWithExplicitPathError() throws Exception { // Check on coordinator. check.accept(0); - waitForCondition(() -> !grid(1).context().cache().context().snapshotMgr().isSnapshotCreating(), TIMEOUT); + waitForCondition(() -> !grid(1).context().cache().context().snapshot().isSnapshotCreating(), TIMEOUT); // Check on non-coordinator. check.accept(1); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotConsistencyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotConsistencyTest.java index 370b92cdd07fd..85fae13794110 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotConsistencyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotConsistencyTest.java @@ -210,7 +210,7 @@ public void testConcurrentPutWithStaleTopologyVersion() throws Exception { snpFut.get(getTestTimeout()); putFut.get(getTestTimeout()); - IdleVerifyResultV2 snpVerifyRes = crd.context().cache().context().snapshotMgr() + IdleVerifyResultV2 snpVerifyRes = crd.context().cache().context().snapshot() .checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); assertFalse(snpVerifyRes.hasConflicts()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotMXBeanTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotMXBeanTest.java index 91b88a45461b0..8e2a6ce30649a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotMXBeanTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotMXBeanTest.java @@ -308,7 +308,7 @@ public void testStatus() throws Exception { private void checkSnapshotStatus(boolean isCreating, boolean isRestoring, boolean isIncremental, String expName) throws Exception { assertTrue(waitForCondition(() -> G.allGrids().stream().allMatch( ignite -> { - IgniteSnapshotManager mgr = ((IgniteEx)ignite).context().cache().context().snapshotMgr(); + IgniteSnapshotManager mgr = ((IgniteEx)ignite).context().cache().context().snapshot(); return isCreating == mgr.isSnapshotCreating() && isRestoring == mgr.isRestoring(); }), diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotRestoreFromRemoteTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotRestoreFromRemoteTest.java index 40bd60290e6aa..92dde5e93d5b3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotRestoreFromRemoteTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotRestoreFromRemoteTest.java @@ -243,7 +243,7 @@ public void testRestoreFromAnEmptyNode() throws Exception { awaitPartitionMapExchange(); // Ensure that the snapshot check command succeeds. - IdleVerifyResultV2 res = emptyNode.context().cache().context().snapshotMgr() + IdleVerifyResultV2 res = emptyNode.context().cache().context().snapshot() .checkSnapshot(SNAPSHOT_NAME, null).get(TIMEOUT).idleVerifyResult(); StringBuilder buf = new StringBuilder(); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckWithIndexesTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckWithIndexesTest.java index 39c6926a230ee..284e000512fa0 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckWithIndexesTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckWithIndexesTest.java @@ -44,7 +44,7 @@ public void testClusterSnapshotCheckEmptyCache() throws Exception { snp(ignite).createSnapshot(SNAPSHOT_NAME, null, false, onlyPrimary).get(TIMEOUT); - IdleVerifyResultV2 res = ignite.context().cache().context().snapshotMgr() + IdleVerifyResultV2 res = ignite.context().cache().context().snapshot() .checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); StringBuilder b = new StringBuilder(); @@ -62,7 +62,7 @@ public void testClusterSnapshotCheckWithIndexes() throws Exception { createAndCheckSnapshot(ignite, SNAPSHOT_NAME, null, TIMEOUT); - IdleVerifyResultV2 res = ignite.context().cache().context().snapshotMgr() + IdleVerifyResultV2 res = ignite.context().cache().context().snapshot() .checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); StringBuilder b = new StringBuilder(); @@ -89,7 +89,7 @@ public void testClusterSnapshotCheckWithNodeFilter() throws Exception { createAndCheckSnapshot(grid(0), SNAPSHOT_NAME, null, TIMEOUT); - IdleVerifyResultV2 res = grid(0).context().cache().context().snapshotMgr() + IdleVerifyResultV2 res = grid(0).context().cache().context().snapshot() .checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); StringBuilder b = new StringBuilder(); 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 4041b5a3dc7f7..6f7c6be48d596 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 @@ -194,7 +194,7 @@ public void testRestoreSnapshotError() throws Exception { AtomicBoolean failFlag = new AtomicBoolean(); - ignite.context().cache().context().snapshotMgr().ioFactory((file, modes) -> { + ignite.context().cache().context().snapshot().ioFactory((file, modes) -> { FileIO delegate = ioFactory.create(file, modes); if (file.getPath().endsWith(failingFilePath)) { From c8231bab03e1b224d43ad1a53c3eb3c7169cd3e6 Mon Sep 17 00:00:00 2001 From: nizhikov Date: Wed, 5 Jul 2023 18:23:50 +0300 Subject: [PATCH 4/7] IGNITE-19915 Rename. --- .../persistence/pagemem/PageMemoryImpl.java | 38 ++++++++++++++++++- 1 file changed, 37 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java index 931e3de6ac8a2..14086cc122673 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java @@ -62,6 +62,7 @@ import org.apache.ignite.internal.pagemem.wal.record.CheckpointRecord; import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.pagemem.wal.record.delta.InitNewPageRecord; import org.apache.ignite.internal.pagemem.wal.record.delta.PageDeltaRecord; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.persistence.CheckpointLockStateChecker; @@ -546,6 +547,12 @@ else if (throttlingPlc == ThrottlingPolicy.CHECKPOINT_BUFFER_ONLY) seg.writeLock().lock(); + boolean isTrackingPage = + PageIdUtils.pageIndex(trackingIO.trackingPageFor(pageId, realPageSize(grpId))) == PageIdUtils.pageIndex(pageId); + + if (isTrackingPage && PageIdUtils.flag(pageId) == PageIdAllocator.FLAG_AUX) + pageId = PageIdUtils.pageId(PageIdUtils.partId(pageId), PageIdAllocator.FLAG_DATA, PageIdUtils.pageIndex(pageId)); + FullPageId fullId = new FullPageId(pageId, grpId); try { @@ -594,6 +601,34 @@ else if (throttlingPlc == ThrottlingPolicy.CHECKPOINT_BUFFER_ONLY) setDirty(fullId, absPtr, true, true); + if (isTrackingPage) { + long pageAddr = absPtr + PAGE_OVERHEAD; + + // We are inside segment write lock, so no other thread can pin this tracking page yet. + // We can modify page buffer directly. + if (PageIO.getType(pageAddr) == 0) { + PageMetrics metrics = dataRegionMetrics.cacheGrpPageMetrics(grpId); + + trackingIO.initNewPage(pageAddr, pageId, realPageSize(grpId), metrics); + + if (!ctx.wal().disabled(fullId.groupId(), fullId.pageId())) { + if (!ctx.wal().isAlwaysWriteFullPages()) + ctx.wal().log( + new InitNewPageRecord( + grpId, + pageId, + trackingIO.getType(), + trackingIO.getVersion(), pageId + ) + ); + else { + ctx.wal().log(new PageSnapshot(fullId, absPtr + PAGE_OVERHEAD, pageSize(), + realPageSize(fullId.groupId()))); + } + } + } + } + seg.pageReplacementPolicy.onMiss(relPtr); seg.loadedPages.put(grpId, PageIdUtils.effectivePageId(pageId), relPtr, seg.partGeneration(grpId, partId)); @@ -624,7 +659,8 @@ else if (throttlingPlc == ThrottlingPolicy.CHECKPOINT_BUFFER_ONLY) if (delayedPageReplacementTracker != null) delayedPageReplacementTracker.delayedPageWrite().finishReplacement(); - return pageId; + //we have allocated 'tracking' page, we need to allocate regular one + return isTrackingPage ? allocatePage(grpId, partId, flags) : pageId; } /** From 7f33c743981f2fd95ecfea21f466988fc7af3f5a Mon Sep 17 00:00:00 2001 From: nizhikov Date: Wed, 5 Jul 2023 22:32:52 +0300 Subject: [PATCH 5/7] IGNITE-19915 Keep trackable flag for compatibility with existing PDS. --- .../cache/mvcc/MvccProcessorImpl.java | 3 ++- .../GridCacheDatabaseSharedManager.java | 12 ++++++---- .../IgniteCacheDatabaseSharedManager.java | 23 ++++++++++++------- .../persistence/pagemem/PageMemoryImpl.java | 8 ++++++- .../pagemem/BPlusTreePageMemoryImplTest.java | 1 + .../BPlusTreeReuseListPageMemoryImplTest.java | 1 + ...itePageMemReplaceDelayedWriteUnitTest.java | 2 +- .../IndexStoragePageMemoryImplTest.java | 1 + .../pagemem/PageMemoryImplNoLoadTest.java | 1 + .../pagemem/PageMemoryImplTest.java | 2 ++ 10 files changed, 39 insertions(+), 15 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java index a04be7c5fc95f..cfc98852b76f7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java @@ -373,7 +373,8 @@ public MvccProcessorImpl(GridKernalContext ctx) { mgr.addDataRegion( dscfg, - createTxLogRegion(dscfg)); + createTxLogRegion(dscfg), + CU.isPersistenceEnabled(ctx.config())); } /** {@inheritDoc} */ 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 b2aad6ea4ec1f..c0751c712e621 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 @@ -469,7 +469,7 @@ public IgniteInternalFuture enableCheckpoints(boolean enable) { @Override protected void initDataRegions0(DataStorageConfiguration memCfg) throws IgniteCheckedException { super.initDataRegions0(memCfg); - addDataRegion(memCfg, createMetastoreDataRegionConfig(memCfg)); + addDataRegion(memCfg, createMetastoreDataRegionConfig(memCfg), false); List regionMetrics = dataRegionMap.values().stream() .map(DataRegion::metrics) @@ -653,6 +653,7 @@ private DataStorageConfiguration configureDataRegionForDefragmentation( addDataRegion( memCfg, createDefragmentationDataRegionConfig(totalDefrRegionSize - mappingRegionSize), + true, new DefragmentationPageReadWriteManager(cctx.kernalContext(), "defrgPartitionsStore") ) ); @@ -661,6 +662,7 @@ private DataStorageConfiguration configureDataRegionForDefragmentation( addDataRegion( memCfg, createDefragmentationMappingRegionConfig(mappingRegionSize), + true, new DefragmentationPageReadWriteManager(cctx.kernalContext(), "defrgLinkMappingStore") ) ); @@ -832,8 +834,8 @@ public CachePartitionDefragmentationManager defragmentationManager() { /** {@inheritDoc} */ @Override public DataRegion addDataRegion(DataStorageConfiguration dataStorageCfg, DataRegionConfiguration dataRegionCfg, - PageReadWriteManager pmPageMgr) throws IgniteCheckedException { - DataRegion region = super.addDataRegion(dataStorageCfg, dataRegionCfg, pmPageMgr); + boolean trackable, PageReadWriteManager pmPageMgr) throws IgniteCheckedException { + DataRegion region = super.addDataRegion(dataStorageCfg, dataRegionCfg, trackable, pmPageMgr); checkpointedDataRegions.add(region); @@ -1180,10 +1182,11 @@ private long[] calculateFragmentSizes(String regionName, int concLvl, long cache DataStorageConfiguration memCfg, DataRegionConfiguration plcCfg, DataRegionMetricsImpl memMetrics, + final boolean trackable, PageReadWriteManager pmPageMgr ) { if (!plcCfg.isPersistenceEnabled()) - return super.createPageMemory(memProvider, memCfg, plcCfg, memMetrics, pmPageMgr); + return super.createPageMemory(memProvider, memCfg, plcCfg, memMetrics, trackable, pmPageMgr); memMetrics.persistenceEnabled(true); @@ -1219,6 +1222,7 @@ private long[] calculateFragmentSizes(String regionName, int concLvl, long cache getCheckpointer().currentProgress().updateEvictedPages(1); }, + trackable, this, memMetrics, resolveThrottlingPolicy(), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java index ae541fdf3cbf8..2458224b3514a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java @@ -322,12 +322,13 @@ protected void initDataRegions0(DataStorageConfiguration memCfg) throws IgniteCh if (dataRegionCfgs != null) { for (DataRegionConfiguration dataRegionCfg : dataRegionCfgs) - addDataRegion(memCfg, dataRegionCfg); + addDataRegion(memCfg, dataRegionCfg, dataRegionCfg.isPersistenceEnabled()); } addDataRegion( memCfg, - memCfg.getDefaultDataRegionConfiguration() + memCfg.getDefaultDataRegionConfiguration(), + memCfg.getDefaultDataRegionConfiguration().isPersistenceEnabled() ); addDataRegion( @@ -336,7 +337,8 @@ protected void initDataRegions0(DataStorageConfiguration memCfg) throws IgniteCh memCfg.getSystemDataRegionConfiguration().getInitialSize(), memCfg.getSystemDataRegionConfiguration().getMaxSize(), persistenceEnabled - ) + ), + persistenceEnabled ); addDataRegion( @@ -344,7 +346,8 @@ protected void initDataRegions0(DataStorageConfiguration memCfg) throws IgniteCh createVolatileDataRegion( memCfg.getSystemDataRegionConfiguration().getInitialSize(), memCfg.getSystemDataRegionConfiguration().getMaxSize() - ) + ), + false ); for (DatabaseLifecycleListener lsnr : getDatabaseListeners(cctx.kernalContext())) @@ -366,9 +369,10 @@ protected List getDatabaseListeners(GridKernalContext */ public DataRegion addDataRegion( DataStorageConfiguration dataStorageCfg, - DataRegionConfiguration dataRegionCfg + DataRegionConfiguration dataRegionCfg, + boolean trackable ) throws IgniteCheckedException { - return addDataRegion(dataStorageCfg, dataRegionCfg, cctx.pageStore()); + return addDataRegion(dataStorageCfg, dataRegionCfg, trackable, cctx.pageStore()); } /** @@ -380,6 +384,7 @@ public DataRegion addDataRegion( protected DataRegion addDataRegion( DataStorageConfiguration dataStorageCfg, DataRegionConfiguration dataRegionCfg, + boolean trackable, PageReadWriteManager pmPageMgr ) throws IgniteCheckedException { String dataRegionName = dataRegionCfg.getName(); @@ -394,7 +399,7 @@ protected DataRegion addDataRegion( cctx.kernalContext(), dataRegionMetricsProvider(dataRegionCfg)); - DataRegion region = initMemory(dataStorageCfg, dataRegionCfg, memMetrics, pmPageMgr); + DataRegion region = initMemory(dataStorageCfg, dataRegionCfg, memMetrics, trackable, pmPageMgr); dataRegionMap.put(dataRegionName, region); @@ -1257,13 +1262,14 @@ private DataRegion initMemory( DataStorageConfiguration memCfg, DataRegionConfiguration plcCfg, DataRegionMetricsImpl memMetrics, + boolean trackable, PageReadWriteManager pmPageMgr ) throws IgniteCheckedException { if (plcCfg.getMemoryAllocator() == null) plcCfg.setMemoryAllocator(memCfg.getMemoryAllocator()); PageMemory pageMem = createPageMemory(createOrReuseMemoryProvider(plcCfg), memCfg, plcCfg, memMetrics, - pmPageMgr); + trackable, pmPageMgr); return new DataRegion(pageMem, plcCfg, memMetrics, createPageEvictionTracker(plcCfg, pageMem)); } @@ -1370,6 +1376,7 @@ protected PageMemory createPageMemory( DataStorageConfiguration memCfg, DataRegionConfiguration memPlcCfg, DataRegionMetricsImpl memMetrics, + boolean trackable, PageReadWriteManager pmPageMgr ) { memMetrics.persistenceEnabled(false); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java index 14086cc122673..eb440f10e5bbe 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java @@ -234,6 +234,9 @@ public class PageMemoryImpl implements PageMemoryEx { */ @Nullable private final DelayedPageReplacementTracker delayedPageReplacementTracker; + /** Obsolete feature. Flag keeped for compatibility with existing PDS. */ + private final boolean trackable; + /** Pages write throttle. */ private PagesWriteThrottlePolicy writeThrottle; @@ -268,6 +271,7 @@ public class PageMemoryImpl implements PageMemoryEx { * @param pmPageMgr Page store manager. * @param pageSize Page size. * @param flushDirtyPage write callback invoked when a dirty page is removed for replacement. + * @param trackable Obsolete flag. Keeped for compatibility with existing PDS. * @param stateChecker Checkpoint lock state provider. Used to ensure lock is held by thread, which modify pages. * @param dataRegionMetrics Memory metrics to track dirty pages count and page replace rate. * @param throttlingPlc Write throttle enabled and its type. Null equal to none. @@ -280,6 +284,7 @@ public PageMemoryImpl( PageReadWriteManager pmPageMgr, int pageSize, PageStoreWriter flushDirtyPage, + boolean trackable, CheckpointLockStateChecker stateChecker, DataRegionMetricsImpl dataRegionMetrics, @Nullable ThrottlingPolicy throttlingPlc, @@ -299,6 +304,7 @@ public PageMemoryImpl( getBoolean(IGNITE_DELAYED_REPLACED_PAGE_WRITE, DFLT_DELAYED_REPLACED_PAGE_WRITE) ? new DelayedPageReplacementTracker(pageSize, flushDirtyPage, log, sizes.length - 1) : null; + this.trackable = trackable; this.stateChecker = stateChecker; this.throttlingPlc = throttlingPlc != null ? throttlingPlc : ThrottlingPolicy.CHECKPOINT_BUFFER_ONLY; this.cpProgressProvider = cpProgressProvider; @@ -547,7 +553,7 @@ else if (throttlingPlc == ThrottlingPolicy.CHECKPOINT_BUFFER_ONLY) seg.writeLock().lock(); - boolean isTrackingPage = + boolean isTrackingPage = trackable && PageIdUtils.pageIndex(trackingIO.trackingPageFor(pageId, realPageSize(grpId))) == PageIdUtils.pageIndex(pageId); if (isTrackingPage && PageIdUtils.flag(pageId) == PageIdAllocator.FLAG_AUX) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java index ebe0d196238e3..6a30f43d97b70 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java @@ -114,6 +114,7 @@ public class BPlusTreePageMemoryImplTest extends BPlusTreeSelfTest { (fullPageId, byteBuf, tag) -> { assert false : "No page replacement should happen during the test"; }, + true, () -> true, new DataRegionMetricsImpl(new DataRegionConfiguration(), cctx), PageMemoryImpl.ThrottlingPolicy.DISABLED, diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java index 8bd5ce3ef073e..32b2b5936b00c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java @@ -113,6 +113,7 @@ public class BPlusTreeReuseListPageMemoryImplTest extends BPlusTreeReuseSelfTest (fullPageId, byteBuf, tag) -> { assert false : "No page replacement (rotation with disk) should happen during the test"; }, + true, () -> true, new DataRegionMetricsImpl(new DataRegionConfiguration(), cctx), PageMemoryImpl.ThrottlingPolicy.DISABLED, diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java index 68d30400217c5..c1e7140dd63f1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java @@ -255,7 +255,7 @@ private PageMemoryImpl createPageMemory(IgniteConfiguration cfg, PageStoreWriter IgniteOutClosure clo = () -> Mockito.mock(CheckpointProgressImpl.class); PageMemoryImpl memory = new PageMemoryImpl(provider, sizes, sctx, sctx.pageStore(), pageSize, - pageWriter, () -> true, memMetrics, PageMemoryImpl.ThrottlingPolicy.DISABLED, + pageWriter, false, () -> true, memMetrics, PageMemoryImpl.ThrottlingPolicy.DISABLED, clo); memory.start(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java index 0d9e28d60aee6..70bc4431b6077 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java @@ -125,6 +125,7 @@ public class IndexStoragePageMemoryImplTest extends IndexStorageSelfTest { (fullPageId, byteBuf, tag) -> { assert false : "No page replacement (rotation with disk) should happen during the test"; }, + true, () -> true, new DataRegionMetricsImpl(new DataRegionConfiguration(), cctx), PageMemoryImpl.ThrottlingPolicy.DISABLED, diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java index 3364181caaf88..3aaf5233dd20c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java @@ -115,6 +115,7 @@ public class PageMemoryImplNoLoadTest extends PageMemoryNoLoadSelfTest { (fullPageId, byteBuf, tag) -> { assert false : "No page replacement (rotation with disk) should happen during the test"; }, + true, () -> true, new DataRegionMetricsImpl(new DataRegionConfiguration(), cctx), PageMemoryImpl.ThrottlingPolicy.DISABLED, diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java index 84e256232b3ba..400d7f5070fca 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java @@ -653,6 +653,7 @@ private PageMemoryImpl createPageMemory( sharedCtx.pageStore(), PAGE_SIZE, replaceWriter, + true, () -> true, new DataRegionMetricsImpl(igniteCfg.getDataStorageConfiguration().getDefaultDataRegionConfiguration(), kernalCtx), throttlingPlc, @@ -665,6 +666,7 @@ private PageMemoryImpl createPageMemory( sharedCtx.pageStore(), PAGE_SIZE, replaceWriter, + true, () -> true, new DataRegionMetricsImpl(igniteCfg.getDataStorageConfiguration().getDefaultDataRegionConfiguration(), kernalCtx), throttlingPlc, From 86b809f258f5e49a7120a733a87992d1979088ba Mon Sep 17 00:00:00 2001 From: nizhikov Date: Wed, 5 Jul 2023 22:34:21 +0300 Subject: [PATCH 6/7] IGNITE-19915 Keep trackable flag for compatibility with existing PDS. --- .../snapshot/SnapshotCompressionBasicTest.java | 2 +- .../ignite/util/GridCommandHandlerTest.java | 6 +++--- .../apache/ignite/internal/IgniteKernal.java | 2 +- .../management/snapshot/SnapshotCheckTask.java | 2 +- .../snapshot/SnapshotCreateTask.java | 2 +- .../snapshot/SnapshotRestoreTask.java | 4 ++-- .../snapshot/SnapshotStatusTask.java | 2 +- .../encryption/GridEncryptionManager.java | 4 ++-- .../encryption/GroupKeyChangeProcess.java | 4 ++-- .../processors/cache/ClusterCachesInfo.java | 6 +++--- .../processors/cache/GridCacheProcessor.java | 2 +- .../cache/GridCacheSharedContext.java | 2 +- .../distributed/near/GridNearTxLocal.java | 2 +- .../GridCacheDatabaseSharedManager.java | 4 ++-- .../snapshot/IgniteSnapshotManager.java | 8 ++++---- .../IncrementalSnapshotFutureTask.java | 2 +- .../snapshot/IncrementalSnapshotProcessor.java | 6 +++--- .../IncrementalSnapshotVerificationTask.java | 4 ++-- .../snapshot/SnapshotFutureTask.java | 2 +- .../snapshot/SnapshotHandlerRestoreTask.java | 4 ++-- .../snapshot/SnapshotMXBeanImpl.java | 2 +- .../SnapshotMetadataVerificationTask.java | 4 ++-- .../SnapshotPartitionsVerifyHandler.java | 2 +- .../snapshot/SnapshotPartitionsVerifyTask.java | 4 ++-- .../SnapshotResponseRemoteFutureTask.java | 4 ++-- .../snapshot/SnapshotRestoreProcess.java | 18 +++++++++--------- .../snapshot/SnapshotRestoreStatusTask.java | 2 +- .../cache/transactions/IgniteTxHandler.java | 4 ++-- .../datastreamer/DataStreamerImpl.java | 2 +- .../snapshot/AbstractSnapshotSelfTest.java | 6 +++--- .../IgniteClusterSnapshotHandlerTest.java | 4 ++-- .../IgniteClusterSnapshotRestoreSelfTest.java | 4 ++-- .../IgniteClusterSnapshotSelfTest.java | 6 +++--- .../IgniteSnapshotConsistencyTest.java | 2 +- .../snapshot/IgniteSnapshotMXBeanTest.java | 2 +- .../IgniteSnapshotRestoreFromRemoteTest.java | 2 +- ...iteClusterSnapshotCheckWithIndexesTest.java | 6 +++--- .../IgniteClusterSnapshotMetricsTest.java | 2 +- 38 files changed, 73 insertions(+), 73 deletions(-) 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 7c926f19d48b2..4ccdf4ca3c3d6 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 @@ -331,7 +331,7 @@ protected void createTestSnapshot() throws Exception { for (String snpName : Arrays.asList(SNAPSHOT_WITH_HOLES, SNAPSHOT_WITHOUT_HOLES)) { snp(ignite).createSnapshot(snpName, null, false, onlyPrimary).get(TIMEOUT); - IdleVerifyResultV2 res = ignite.context().cache().context().snapshot().checkSnapshot(snpName, null) + IdleVerifyResultV2 res = ignite.context().cache().context().snapshotMgr().checkSnapshot(snpName, null) .get().idleVerifyResult(); StringBuilder b = new StringBuilder(); 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 87d1731c4ad01..abb7a50c4fb58 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 @@ -3654,7 +3654,7 @@ public void testSnapshotRestoreCancelAndStatus() throws Exception { awaitPartitionMapExchange(); CountDownLatch ioStartLatch = new CountDownLatch(1); - IgniteSnapshotManager snpMgr = ig.context().cache().context().snapshot(); + IgniteSnapshotManager snpMgr = ig.context().cache().context().snapshotMgr(); // Replace the IO factory in the snapshot manager so we have enough time to test the status command. snpMgr.ioFactory(new SlowDownFileIoFactory(snpMgr.ioFactory(), getTestTimeout() / locPartsCnt, ioStartLatch)); @@ -3695,7 +3695,7 @@ public void testSnapshotRestoreCancelAndStatus() throws Exception { // Make sure the context disappeared at node 1. boolean ctxDisposed = - waitForCondition(() -> !grid(1).context().cache().context().snapshot().isRestoring(), getTestTimeout()); + waitForCondition(() -> !grid(1).context().cache().context().snapshotMgr().isRestoring(), getTestTimeout()); assertTrue(ctxDisposed); @@ -3815,7 +3815,7 @@ private void checkSnapshotStatus(boolean isCreating, boolean isRestoring, boolea assertTrue(waitForCondition(() -> srvs.stream().allMatch( ignite -> { - IgniteSnapshotManager mgr = ((IgniteEx)ignite).context().cache().context().snapshot(); + IgniteSnapshotManager mgr = ((IgniteEx)ignite).context().cache().context().snapshotMgr(); return isCreating == mgr.isSnapshotCreating() && isRestoring == mgr.isRestoring(); }), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 8990df38d444a..e61a1a8772f7e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -2868,7 +2868,7 @@ private Collection baselineNodes() { /** {@inheritDoc} */ @Override public IgniteSnapshot snapshot() { - return ctx.cache().context().snapshot(); + return ctx.cache().context().snapshotMgr(); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCheckTask.java b/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCheckTask.java index 3e777156d5a76..c467585d8629b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCheckTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCheckTask.java @@ -54,7 +54,7 @@ protected SnapshotCheckJob(SnapshotCheckCommandArg arg, boolean debug) { /** {@inheritDoc} */ @Override protected SnapshotPartitionsVerifyTaskResult run(SnapshotCheckCommandArg arg) throws IgniteException { - IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshot(); + IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshotMgr(); return new IgniteFutureImpl<>(snpMgr.checkSnapshot(arg.snapshotName(), arg.src(), arg.increment())).get(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCreateTask.java b/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCreateTask.java index c0eca7a8932f2..e236ad6fc7cf2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCreateTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotCreateTask.java @@ -52,7 +52,7 @@ protected SnapshotCreateJob(SnapshotCreateCommandArg arg, boolean debug) { /** {@inheritDoc} */ @Override protected String run(SnapshotCreateCommandArg arg) throws IgniteException { - IgniteFutureImpl fut = ignite.context().cache().context().snapshot().createSnapshot( + IgniteFutureImpl fut = ignite.context().cache().context().snapshotMgr().createSnapshot( arg.snapshotName(), arg.dest(), arg.incremental(), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotRestoreTask.java b/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotRestoreTask.java index c28b7cf24370d..d238c7bc69d70 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotRestoreTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotRestoreTask.java @@ -58,7 +58,7 @@ protected SnapshotStartRestoreJob(SnapshotRestoreCommandArg arg, boolean debug) /** {@inheritDoc} */ @Override protected String run(SnapshotRestoreCommandArg arg) throws IgniteException { - IgniteFutureImpl fut = ignite.context().cache().context().snapshot().restoreSnapshot( + IgniteFutureImpl fut = ignite.context().cache().context().snapshotMgr().restoreSnapshot( arg.snapshotName(), arg.src(), arg.groups() == null ? null : Arrays.asList(arg.groups()), @@ -125,7 +125,7 @@ protected SnapshotRestoreStatusJob(SnapshotRestoreCommandArg arg, boolean debug) /** {@inheritDoc} */ @Override protected String run(SnapshotRestoreCommandArg arg) throws IgniteException { - boolean state = ignite.context().cache().context().snapshot().restoreStatus(arg.snapshotName()).get(); + boolean state = ignite.context().cache().context().snapshotMgr().restoreStatus(arg.snapshotName()).get(); return "Snapshot cache group restore operation is " + (state ? "" : "NOT ") + "running [snapshot=" + arg.snapshotName() + ']'; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotStatusTask.java b/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotStatusTask.java index 7451f81ff85f5..88fea0f7eab3f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotStatusTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/management/snapshot/SnapshotStatusTask.java @@ -111,7 +111,7 @@ protected SnapshotStatusJob(@Nullable NoArg arg, boolean debug) { if (!CU.isPersistenceEnabled(ignite.context().config())) return null; - IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshot(); + IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshotMgr(); SnapshotOperationRequest req = snpMgr.currentCreateRequest(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java index e26ebd55ff762..f1f5f11c5537c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java @@ -1528,8 +1528,8 @@ private IgniteInternalFuture prepareMasterKeyChange(MasterKeyChange "The previous change was not completed.")); } - if (ctx.cache().context().snapshot().isSnapshotCreating() - || ctx.cache().context().snapshot().isRestoring()) { + if (ctx.cache().context().snapshotMgr().isSnapshotCreating() + || ctx.cache().context().snapshotMgr().isRestoring()) { return new GridFinishedFuture<>(new IgniteException("Master key change was rejected. Snapshot operation " + "is in progress.")); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GroupKeyChangeProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GroupKeyChangeProcess.java index 7167257c6a29a..a8b63bce21593 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GroupKeyChangeProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GroupKeyChangeProcess.java @@ -197,8 +197,8 @@ private IgniteInternalFuture prepare(ChangeCacheEncryptionRequest r "The previous change was not completed.")); } - if (ctx.cache().context().snapshot().isSnapshotCreating() - || ctx.cache().context().snapshot().isRestoring()) { + if (ctx.cache().context().snapshotMgr().isSnapshotCreating() + || ctx.cache().context().snapshotMgr().isRestoring()) { return new GridFinishedFuture<>(new IgniteException("Cache group key change was rejected. " + "Snapshot operation is in progress.")); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java index 248b9c77319e9..715a8711b0828 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java @@ -622,7 +622,7 @@ public boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, AffinityTop assert restartIds.size() <= 1 : batch.requests(); - Collection nodes = ctx.cache().context().snapshot().cacheStartRequiredAliveNodes(F.first(restartIds)); + Collection nodes = ctx.cache().context().snapshotMgr().cacheStartRequiredAliveNodes(F.first(restartIds)); for (UUID nodeId : nodes) { ClusterNode node = ctx.discovery().node(nodeId); @@ -872,7 +872,7 @@ private boolean processStopCacheRequest( AffinityTopologyVersion topVer, boolean checkForAlreadyDeleted ) { - if (ctx.cache().context().snapshot().isSnapshotCreating()) { + if (ctx.cache().context().snapshotMgr().isSnapshotCreating()) { IgniteCheckedException err = new IgniteCheckedException(SNP_IN_PROGRESS_ERR_MSG); U.warn(log, err); @@ -1119,7 +1119,7 @@ else if (encMgr.masterKeyDigest() != null && } if (err == null && req.restartId() == null) { - IgniteSnapshotManager snpMgr = ctx.cache().context().snapshot(); + IgniteSnapshotManager snpMgr = ctx.cache().context().snapshotMgr(); if (snpMgr.isRestoring(ccfg)) { err = new IgniteCheckedException("Cache start failed. A cache or group with the same name is " + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 7ab80391ae9df..191e98365a144 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -4331,7 +4331,7 @@ private IgniteNodeValidationResult validateRestartingCaches(ClusterNode node) { * @return Node validation result if there was an issue with the joining node, {@code null} otherwise. */ private IgniteNodeValidationResult validateRestoringCaches(ClusterNode node) { - if (ctx.cache().context().snapshot().isRestoring()) { + if (ctx.cache().context().snapshotMgr().isRestoring()) { String msg = "Joining node during caches restore is not allowed [joiningNodeId=" + node.id() + ']'; return new IgniteNodeValidationResult(node.id(), msg); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java index 1cfad22daef56..f5b40804d5441 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java @@ -753,7 +753,7 @@ public IgniteCacheDatabaseSharedManager database() { /** * @return Page storage snapshot manager. */ - public IgniteSnapshotManager snapshot() { + public IgniteSnapshotManager snapshotMgr() { return snpMgr; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java index 5211f626cd353..c5555d97ac930 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java @@ -3981,7 +3981,7 @@ public IgniteInternalFuture commitNearTxLocalAsync() { commitOrRollbackStartTime.compareAndSet(0, System.nanoTime()); if (!onePhaseCommit) - incrementalSnapshotId(cctx.snapshot().incrementalSnapshotId()); + incrementalSnapshotId(cctx.snapshotMgr().incrementalSnapshotId()); try { // Make sure that here are no exceptions. 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 c0751c712e621..dfb7ea2afd5ea 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 @@ -1517,7 +1517,7 @@ private String cacheInfo(GridCacheContext cacheCtx) { .map(t -> t.get1().groupId()) .collect(toList()); - cctx.snapshot().onCacheGroupsStopped(stoppedGrpIds); + cctx.snapshotMgr().onCacheGroupsStopped(stoppedGrpIds); initiallyLocWalDisabledGrps.removeAll(stoppedGrpIds); initiallyGlobalWalDisabledGrps.removeAll(stoppedGrpIds); @@ -3121,7 +3121,7 @@ public boolean isCheckpointInapplicableForWalRebalance(Long cpTs, int grpId) thr if (lastCpTs != 0) metaStorage.write(checkpointInapplicableCpAndGroupIdToKey(lastCpTs, grpId), true); - cctx.snapshot().disableIncrementalSnapshotsCreation(metaStorage, grpId); + cctx.snapshotMgr().disableIncrementalSnapshotsCreation(metaStorage, grpId); } catch (IgniteCheckedException e) { log.error("Failed to mark last checkpoint as inapplicable for WAL rebalance for group: " + grpId, e); 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 5296053a964a0..780470bf2e273 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 @@ -4545,7 +4545,7 @@ public CreateSnapshotCallable(String snpName, boolean incremental, boolean onlyP if (incremental) ignite.snapshot().createIncrementalSnapshot(snpName).get(); else - ignite.context().cache().context().snapshot().createSnapshot(snpName, null, false, onlyPrimary).get(); + ignite.context().cache().context().snapshotMgr().createSnapshot(snpName, null, false, onlyPrimary).get(); return null; } @@ -4579,12 +4579,12 @@ public CancelSnapshotCallable(UUID reqId, String snpName) { /** {@inheritDoc} */ @Override public Boolean call() throws Exception { if (reqId != null) - return ignite.context().cache().context().snapshot().cancelLocalSnapshotOperations(reqId); + return ignite.context().cache().context().snapshotMgr().cancelLocalSnapshotOperations(reqId); else { - if (ignite.context().cache().context().snapshot().cancelLocalSnapshotTask(snpName)) + if (ignite.context().cache().context().snapshotMgr().cancelLocalSnapshotTask(snpName)) return true; - return ignite.context().cache().context().snapshot().cancelLocalRestoreTask(snpName).get(); + return ignite.context().cache().context().snapshotMgr().cancelLocalRestoreTask(snpName).get(); } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotFutureTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotFutureTask.java index 53c4cdf41d9bd..e9be26974f7ad 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotFutureTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotFutureTask.java @@ -123,7 +123,7 @@ public IncrementalSnapshotFutureTask( /** {@inheritDoc} */ @Override public boolean start() { try { - File incSnpDir = cctx.snapshot().incrementalSnapshotLocalDir(snpName, snpPath, incIdx); + File incSnpDir = cctx.snapshotMgr().incrementalSnapshotLocalDir(snpName, snpPath, incIdx); if (!incSnpDir.mkdirs() && !incSnpDir.exists()) { onDone(new IgniteException("Can't create snapshot directory [dir=" + incSnpDir.getAbsolutePath() + ']')); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotProcessor.java index f4f994809bc12..998d5130a7600 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotProcessor.java @@ -96,7 +96,7 @@ void process( Consumer dataEntryHnd, @Nullable Consumer txHnd ) throws IgniteCheckedException, IOException { - IncrementalSnapshotMetadata meta = cctx.snapshot() + IncrementalSnapshotMetadata meta = cctx.snapshotMgr() .readIncrementalSnapshotMetadata(snpName, snpPath, incIdx); File[] segments = walSegments(meta.folderName()); @@ -158,7 +158,7 @@ void process( } UUID prevIncSnpId = incIdx > 1 - ? cctx.snapshot().readIncrementalSnapshotMetadata(snpName, snpPath, incIdx - 1).requestId() + ? cctx.snapshotMgr().readIncrementalSnapshotMetadata(snpName, snpPath, incIdx - 1).requestId() : null; IgnitePredicate txVerFilter = prevIncSnpId != null @@ -225,7 +225,7 @@ private File[] walSegments(String folderName) throws IgniteCheckedException { File[] segments = null; for (int i = 1; i <= incIdx; i++) { - File incSnpDir = cctx.snapshot().incrementalSnapshotLocalDir(snpName, snpPath, i); + File incSnpDir = cctx.snapshotMgr().incrementalSnapshotLocalDir(snpName, snpPath, i); if (!incSnpDir.exists()) throw new IgniteCheckedException("Incremental snapshot doesn't exists [dir=" + incSnpDir + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerificationTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerificationTask.java index 29a8c97865472..c64cadb0f1e52 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerificationTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerificationTask.java @@ -401,7 +401,7 @@ else if (txRec.state() == TransactionState.ROLLED_BACK) { /** Checks that current baseline topology matches baseline topology of the snapshot. */ private void checkBaseline(BaselineTopology blt) throws IgniteCheckedException, IOException { - IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshot(); + IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshotMgr(); File snpDir = snpMgr.snapshotLocalDir(snpName, snpPath); SnapshotMetadata meta = snpMgr.readSnapshotMetadata(snpDir, ignite.localNode().consistentId().toString()); @@ -414,7 +414,7 @@ private void checkBaseline(BaselineTopology blt) throws IgniteCheckedException, /** @return Collection of snapshotted transactional caches, key is a cache ID. */ private Map readTxCachesData() throws IgniteCheckedException, IOException { - File snpDir = ignite.context().cache().context().snapshot().snapshotLocalDir(snpName, snpPath); + File snpDir = ignite.context().cache().context().snapshotMgr().snapshotLocalDir(snpName, snpPath); String folderName = ignite.context().pdsFolderResolver().resolveFolders().folderName(); 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 b774fc716f588..7fc31dbdfb7c4 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 @@ -159,7 +159,7 @@ class SnapshotFutureTask extends AbstractSnapshotFutureTask deltaWriterFactory = - cctx.snapshot().sequentialWrite() ? IndexedPageStoreSerialWriter::new : PageStoreSerialWriter::new; + cctx.snapshotMgr().sequentialWrite() ? IndexedPageStoreSerialWriter::new : PageStoreSerialWriter::new; /** * @param cctx Shared context. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerRestoreTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerRestoreTask.java index 063ed7082037e..5c34511ce2755 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerRestoreTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerRestoreTask.java @@ -87,7 +87,7 @@ public class SnapshotHandlerRestoreTask extends AbstractSnapshotVerificationTask String snapshotName = F.first(F.first(metas.values())).snapshotName(); try { - ignite.context().cache().context().snapshot().handlers().completeAll( + ignite.context().cache().context().snapshotMgr().handlers().completeAll( SnapshotHandlerType.RESTORE, snapshotName, clusterResults, execNodes, wrns -> {}); } catch (Exception e) { @@ -151,7 +151,7 @@ public SnapshotHandlerRestoreJob( /** {@inheritDoc} */ @Override public Map> execute() { try { - IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshot(); + IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshotMgr(); File snpDir = snpMgr.snapshotLocalDir(snpName, snpPath); SnapshotMetadata meta = snpMgr.readSnapshotMetadata(snpDir, consistentId); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMXBeanImpl.java index 0a04afb1251c3..59032be79e51c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMXBeanImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMXBeanImpl.java @@ -46,7 +46,7 @@ public class SnapshotMXBeanImpl implements SnapshotMXBean { * @param ctx Kernal context. */ public SnapshotMXBeanImpl(GridKernalContext ctx) { - mgr = ctx.cache().context().snapshot(); + mgr = ctx.cache().context().snapshotMgr(); metricMgr = ctx.metric(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataVerificationTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataVerificationTask.java index 0a425bd2f0cee..1695ad692efc9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataVerificationTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataVerificationTask.java @@ -89,7 +89,7 @@ public MetadataVerificationJob(SnapshotMetadataVerificationTaskArg arg) { /** {@inheritDoc} */ @Override public List execute() throws IgniteException { - IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshot(); + IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshotMgr(); List snpMeta = snpMgr.readSnapshotMetadatas(arg.snapshotName(), arg.snapshotPath()); @@ -112,7 +112,7 @@ public MetadataVerificationJob(SnapshotMetadataVerificationTaskArg arg) { /** Checks that all incremental snapshots are present, contain correct metafile and WAL segments. */ public void checkIncrementalSnapshots(SnapshotMetadata fullMeta, SnapshotMetadataVerificationTaskArg arg) { try { - IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshot(); + IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshotMgr(); // Incremental snapshot must contain ClusterSnapshotRecord. long startSeg = fullMeta.snapshotRecordPointer().index(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java index a3fb5356ab2bd..e0e654d49ae81 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java @@ -158,7 +158,7 @@ public SnapshotPartitionsVerifyHandler(GridCacheSharedContext cctx) { ThreadLocal buff = ThreadLocal.withInitial(() -> ByteBuffer.allocateDirect(meta.pageSize()) .order(ByteOrder.nativeOrder())); - IgniteSnapshotManager snpMgr = cctx.snapshot(); + IgniteSnapshotManager snpMgr = cctx.snapshotMgr(); GridKernalContext snpCtx = snpMgr.createStandaloneKernalContext(cctx.kernalContext().compress(), opCtx.snapshotDirectory(), meta.folderName()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTask.java index c2967ca4ce536..2fc31021490c4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyTask.java @@ -127,8 +127,8 @@ public VerifySnapshotPartitionsJob( } try { - File snpDir = cctx.snapshot().snapshotLocalDir(snpName, snpPath); - SnapshotMetadata meta = cctx.snapshot().readSnapshotMetadata(snpDir, consId); + File snpDir = cctx.snapshotMgr().snapshotLocalDir(snpName, snpPath); + SnapshotMetadata meta = cctx.snapshotMgr().readSnapshotMetadata(snpDir, consId); return new SnapshotPartitionsVerifyHandler(cctx) .invoke(new SnapshotHandlerContext(meta, rqGrps, ignite.localNode(), snpDir, false, check)); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotResponseRemoteFutureTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotResponseRemoteFutureTask.java index 2ed90af572513..90e562f426559 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotResponseRemoteFutureTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotResponseRemoteFutureTask.java @@ -75,7 +75,7 @@ public SnapshotResponseRemoteFutureTask( return false; try { - List metas = cctx.snapshot().readSnapshotMetadatas(snpName, snpPath); + List metas = cctx.snapshotMgr().readSnapshotMetadatas(snpName, snpPath); Function findMeta = pair -> { for (SnapshotMetadata meta : metas) { @@ -108,7 +108,7 @@ public SnapshotResponseRemoteFutureTask( snpSndr.init(partsToSend.size()); - File snpDir = cctx.snapshot().snapshotLocalDir(snpName, snpPath); + File snpDir = cctx.snapshotMgr().snapshotLocalDir(snpName, snpPath); CompletableFuture.runAsync(() -> partsToSend.forEach((gp, meta) -> { if (err.get() != null) 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 028d53f7acf85..f16543a4fa435 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 @@ -268,7 +268,7 @@ public IgniteFutureImpl start( int incIdx, boolean check ) { - IgniteSnapshotManager snpMgr = ctx.cache().context().snapshot(); + IgniteSnapshotManager snpMgr = ctx.cache().context().snapshotMgr(); ClusterSnapshotFuture fut0; try { @@ -660,7 +660,7 @@ private IgniteInternalFuture prepare(SnapshotO lastOpCtx = opCtx0; DiscoveryDataClusterState state = ctx.state().clusterState(); - IgniteSnapshotManager snpMgr = ctx.cache().context().snapshot(); + IgniteSnapshotManager snpMgr = ctx.cache().context().snapshotMgr(); if (state.state() != ClusterState.ACTIVE || state.transition()) throw new IgniteCheckedException(OP_REJECT_MSG + "The cluster should be active."); @@ -777,7 +777,7 @@ private void enrichContext( // Metastorage can be restored only manually by directly copying files. boolean skipCompressCheck = false; for (SnapshotMetadata meta : metas) { - for (File snpCacheDir : cctx.snapshot().snapshotCacheDirectories(req.snapshotName(), req.snapshotPath(), meta.folderName(), + for (File snpCacheDir : cctx.snapshotMgr().snapshotCacheDirectories(req.snapshotName(), req.snapshotPath(), meta.folderName(), name -> !METASTORAGE_CACHE_NAME.equals(name))) { String grpName = FilePageStoreManager.cacheGroupName(snpCacheDir); @@ -970,7 +970,7 @@ private IgniteInternalFuture preload(UUID reqId) { AbstractSnapshotVerificationTask.checkMissedMetadata(allMetas); - IgniteSnapshotManager snpMgr = ctx.cache().context().snapshot(); + IgniteSnapshotManager snpMgr = ctx.cache().context().snapshotMgr(); synchronized (this) { opCtx0.stopFut = new IgniteFutureImpl<>(retFut.chain(f -> null)); @@ -992,7 +992,7 @@ private IgniteInternalFuture preload(UUID reqId) { SnapshotMetadata meta = F.first(opCtx0.metasPerNode.get(opCtx0.opNodeId)); File dir = opCtx0.incIdx > 0 ? - ctx.cache().context().snapshot() + ctx.cache().context().snapshotMgr() .incrementalSnapshotLocalDir(opCtx0.snpName, opCtx0.snpPath, opCtx0.incIdx) : snpDir; @@ -1130,7 +1130,7 @@ private IgniteInternalFuture preload(UUID reqId) { ", grpParts=" + partitionsMapToString(m.getValue(), cacheGrpNames) + "]"); } - ctx.cache().context().snapshot() + ctx.cache().context().snapshotMgr() .requestRemoteSnapshotFiles(m.getKey(), opCtx0.reqId, opCtx0.snpName, @@ -1585,7 +1585,7 @@ private void finishIncrementalSnapshotRestore(UUID reqId, Map res .map(cacheId -> CU.cacheOrGroupName(ctx.cache().cacheDescriptor(cacheId).cacheConfiguration())) .collect(Collectors.toSet()); - ctx.cache().context().snapshot() + ctx.cache().context().snapshotMgr() .warnAtomicCachesInIncrementalSnapshot(opCtx0.snpName, opCtx0.incIdx, cacheGrps); } @@ -1675,7 +1675,7 @@ private IgniteInternalFuture rollback(UUID reqId) { } try { - ctx.cache().context().snapshot().snapshotExecutorService().execute(() -> { + ctx.cache().context().snapshotMgr().snapshotExecutorService().execute(() -> { if (log.isInfoEnabled()) { log.info("Removing restored cache directories [reqId=" + reqId + ", snapshot=" + opCtx0.snpName + ", dirs=" + opCtx0.dirs + ']'); @@ -1760,7 +1760,7 @@ private void copyLocalAsync( Path partFile = Paths.get(targetDir.getAbsolutePath(), FilePageStoreManager.getPartitionFileName(partFut.partId)); int grpId = groupIdFromTmpDir(targetDir); - IgniteSnapshotManager snapMgr = ctx.cache().context().snapshot(); + IgniteSnapshotManager snapMgr = ctx.cache().context().snapshotMgr(); CompletableFuture copyPartFut = CompletableFuture.supplyAsync(() -> { if (opCtx.stopChecker.getAsBoolean()) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStatusTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStatusTask.java index 82a450b89daca..14a395779c9d4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStatusTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStatusTask.java @@ -57,7 +57,7 @@ class SnapshotRestoreStatusTask extends ComputeTaskAdapter { private transient IgniteEx ignite; @Override public Boolean execute() throws IgniteException { - return ignite.context().cache().context().snapshot().isRestoring(snpName); + return ignite.context().cache().context().snapshotMgr().isRestoring(snpName); } }, node); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java index ed310a629baae..4f8ef80316e7b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java @@ -288,7 +288,7 @@ public IgniteTxHandler(GridCacheSharedContext ctx) { private void processIncrementalSnapshotAwareMessage(UUID nodeId, IncrementalSnapshotAwareMessage msg) { // Skip if node has joined after incremental snapshot started. if (ctx.localNode().order() <= msg.snapshotTopologyVersion()) { - ctx.snapshot().handleIncrementalSnapshotId(msg.id(), msg.snapshotTopologyVersion()); + ctx.snapshotMgr().handleIncrementalSnapshotId(msg.id(), msg.snapshotTopologyVersion()); setIncrementalSnapshotIdIfRequired(msg); } @@ -1285,7 +1285,7 @@ else if (nearTx != null) if (dhtTx != null) { dhtTx.onePhaseCommit(true); dhtTx.needReturnValue(req.needReturnValue()); - dhtTx.incrementalSnapshotId(ctx.snapshot().incrementalSnapshotId()); + dhtTx.incrementalSnapshotId(ctx.snapshotMgr().incrementalSnapshotId()); finish(dhtTx, req); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java index 862f2ae1e18c5..b98adfef6ea40 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java @@ -2386,7 +2386,7 @@ else if (ttl == CU.TTL_NOT_CHANGED) */ private static void snapshotWarning(GridCacheContext cctx) { if (cctx.group().persistenceEnabled()) - cctx.kernalContext().cache().context().snapshot().streamerWarning(); + cctx.kernalContext().cache().context().snapshotMgr().streamerWarning(); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java index 10bac7ada444c..802497a2d72b2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java @@ -315,7 +315,7 @@ protected void ensureCacheAbsent(CacheConfiguration ccfg) throws IgniteChe assertNull("nodeId=" + kctx.localNodeId() + ", cache=" + cacheName, desc); boolean success = GridTestUtils.waitForCondition( - () -> !kctx.cache().context().snapshot().isRestoring(), + () -> !kctx.cache().context().snapshotMgr().isRestoring(), TIMEOUT); assertTrue("The process has not finished on the node " + kctx.localNodeId(), success); @@ -715,7 +715,7 @@ protected void checkSnapshot(String snpName, String snpPath) throws IgniteChecke * @return Snapshot manager related to given ignite instance. */ public static IgniteSnapshotManager snp(IgniteEx ignite) { - return ignite.context().cache().context().snapshot(); + return ignite.context().cache().context().snapshotMgr(); } /** @@ -823,7 +823,7 @@ protected static IgniteInternalFuture startLocalSnapshotTask( boolean withMetaStorage, SnapshotSender snpSndr ) throws IgniteCheckedException { - AbstractSnapshotFutureTask task = cctx.snapshot().registerSnapshotTask(snpName, cctx.localNodeId(), null, + AbstractSnapshotFutureTask task = cctx.snapshotMgr().registerSnapshotTask(snpName, cctx.localNodeId(), null, parts, withMetaStorage, snpSndr); if (!(task instanceof SnapshotFutureTask)) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotHandlerTest.java index 78deb26d28bab..1c3ae2ef5bc18 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotHandlerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotHandlerTest.java @@ -148,7 +148,7 @@ public void testClusterSnapshotHandlers() throws Exception { */ private void changeMetadataRequestIdOnDisk(UUID newReqId) throws Exception { for (Ignite grid : G.allGrids()) { - IgniteSnapshotManager snpMgr = ((IgniteEx)grid).context().cache().context().snapshot(); + IgniteSnapshotManager snpMgr = ((IgniteEx)grid).context().cache().context().snapshotMgr(); String constId = grid.cluster().localNode().consistentId().toString(); File snpDir = snpMgr.snapshotLocalDir(SNAPSHOT_NAME); @@ -392,7 +392,7 @@ public void testHandlerSnapshotLocation() throws Exception { try { IgniteEx ignite = startGridsWithCache(1, CACHE_KEYS_RANGE, valueBuilder(), dfltCacheCfg); - IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshot(); + IgniteSnapshotManager snpMgr = ignite.context().cache().context().snapshotMgr(); createAndCheckSnapshot(ignite, snpName, snpDir.getAbsolutePath(), TIMEOUT); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java index 897eaef99f47e..3fd9d6dde406f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java @@ -155,7 +155,7 @@ public void testClusterSnapshotRestoreFromCustomDir() throws Exception { ignite.destroyCache(DEFAULT_CACHE_NAME); awaitPartitionMapExchange(); - ignite.context().cache().context().snapshot().restoreSnapshot(SNAPSHOT_NAME, snpDir.getAbsolutePath(), null) + ignite.context().cache().context().snapshotMgr().restoreSnapshot(SNAPSHOT_NAME, snpDir.getAbsolutePath(), null) .get(); IgniteCache cache = ignite.cache(DEFAULT_CACHE_NAME); @@ -674,7 +674,7 @@ public void testNodeFailDuringFilesCopy() throws Exception { String failingFilePath = Paths.get(CACHE_DIR_PREFIX + DEFAULT_CACHE_NAME, PART_FILE_PREFIX + (dfltCacheCfg.getAffinity().partitions() / 2) + FILE_SUFFIX).toString(); - grid(2).context().cache().context().snapshot().ioFactory( + grid(2).context().cache().context().snapshotMgr().ioFactory( new CustomFileIOFactory(new RandomAccessFileIOFactory(), file -> { if (file.getPath().endsWith(failingFilePath)) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotSelfTest.java index 8861ee9a33beb..9aa317c32123f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotSelfTest.java @@ -691,7 +691,7 @@ public void testRecoveryClusterSnapshotJvmHalted() throws Exception { assertTrue( "Snapshot directory must be empty", - grid2.context().cache().context().snapshot().localSnapshotNames(null).isEmpty() + grid2.context().cache().context().snapshotMgr().localSnapshotNames(null).isEmpty() ); createAndCheckSnapshot(ignite, SNAPSHOT_NAME); @@ -797,7 +797,7 @@ public void testClusterSnapshotWithExplicitPathError() throws Exception { GridKernalContext kctx = grid(idx).context(); assertThrowsAnyCause(log, - () -> kctx.cache().context().snapshot() + () -> kctx.cache().context().snapshotMgr() .createSnapshot(SNAPSHOT_NAME, invalidPath, false, onlyPrimary) .get(TIMEOUT), IgniteCheckedException.class, @@ -811,7 +811,7 @@ public void testClusterSnapshotWithExplicitPathError() throws Exception { // Check on coordinator. check.accept(0); - waitForCondition(() -> !grid(1).context().cache().context().snapshot().isSnapshotCreating(), TIMEOUT); + waitForCondition(() -> !grid(1).context().cache().context().snapshotMgr().isSnapshotCreating(), TIMEOUT); // Check on non-coordinator. check.accept(1); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotConsistencyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotConsistencyTest.java index 85fae13794110..370b92cdd07fd 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotConsistencyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotConsistencyTest.java @@ -210,7 +210,7 @@ public void testConcurrentPutWithStaleTopologyVersion() throws Exception { snpFut.get(getTestTimeout()); putFut.get(getTestTimeout()); - IdleVerifyResultV2 snpVerifyRes = crd.context().cache().context().snapshot() + IdleVerifyResultV2 snpVerifyRes = crd.context().cache().context().snapshotMgr() .checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); assertFalse(snpVerifyRes.hasConflicts()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotMXBeanTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotMXBeanTest.java index 8e2a6ce30649a..91b88a45461b0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotMXBeanTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotMXBeanTest.java @@ -308,7 +308,7 @@ public void testStatus() throws Exception { private void checkSnapshotStatus(boolean isCreating, boolean isRestoring, boolean isIncremental, String expName) throws Exception { assertTrue(waitForCondition(() -> G.allGrids().stream().allMatch( ignite -> { - IgniteSnapshotManager mgr = ((IgniteEx)ignite).context().cache().context().snapshot(); + IgniteSnapshotManager mgr = ((IgniteEx)ignite).context().cache().context().snapshotMgr(); return isCreating == mgr.isSnapshotCreating() && isRestoring == mgr.isRestoring(); }), diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotRestoreFromRemoteTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotRestoreFromRemoteTest.java index 92dde5e93d5b3..40bd60290e6aa 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotRestoreFromRemoteTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotRestoreFromRemoteTest.java @@ -243,7 +243,7 @@ public void testRestoreFromAnEmptyNode() throws Exception { awaitPartitionMapExchange(); // Ensure that the snapshot check command succeeds. - IdleVerifyResultV2 res = emptyNode.context().cache().context().snapshot() + IdleVerifyResultV2 res = emptyNode.context().cache().context().snapshotMgr() .checkSnapshot(SNAPSHOT_NAME, null).get(TIMEOUT).idleVerifyResult(); StringBuilder buf = new StringBuilder(); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckWithIndexesTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckWithIndexesTest.java index 284e000512fa0..39c6926a230ee 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckWithIndexesTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckWithIndexesTest.java @@ -44,7 +44,7 @@ public void testClusterSnapshotCheckEmptyCache() throws Exception { snp(ignite).createSnapshot(SNAPSHOT_NAME, null, false, onlyPrimary).get(TIMEOUT); - IdleVerifyResultV2 res = ignite.context().cache().context().snapshot() + IdleVerifyResultV2 res = ignite.context().cache().context().snapshotMgr() .checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); StringBuilder b = new StringBuilder(); @@ -62,7 +62,7 @@ public void testClusterSnapshotCheckWithIndexes() throws Exception { createAndCheckSnapshot(ignite, SNAPSHOT_NAME, null, TIMEOUT); - IdleVerifyResultV2 res = ignite.context().cache().context().snapshot() + IdleVerifyResultV2 res = ignite.context().cache().context().snapshotMgr() .checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); StringBuilder b = new StringBuilder(); @@ -89,7 +89,7 @@ public void testClusterSnapshotCheckWithNodeFilter() throws Exception { createAndCheckSnapshot(grid(0), SNAPSHOT_NAME, null, TIMEOUT); - IdleVerifyResultV2 res = grid(0).context().cache().context().snapshot() + IdleVerifyResultV2 res = grid(0).context().cache().context().snapshotMgr() .checkSnapshot(SNAPSHOT_NAME, null).get().idleVerifyResult(); StringBuilder b = new StringBuilder(); 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 6f7c6be48d596..4041b5a3dc7f7 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 @@ -194,7 +194,7 @@ public void testRestoreSnapshotError() throws Exception { AtomicBoolean failFlag = new AtomicBoolean(); - ignite.context().cache().context().snapshot().ioFactory((file, modes) -> { + ignite.context().cache().context().snapshotMgr().ioFactory((file, modes) -> { FileIO delegate = ioFactory.create(file, modes); if (file.getPath().endsWith(failingFilePath)) { From ae5bee00cc537ef9b201453530b2e5bfe7fb71a6 Mon Sep 17 00:00:00 2001 From: nizhikov Date: Wed, 5 Jul 2023 23:15:19 +0300 Subject: [PATCH 7/7] IGNITE-19915 Revert unnecessary changes. --- .../processors/cache/ClusterCachesInfo.java | 4 ++-- .../processors/cache/GridCacheProcessor.java | 8 ++++---- .../processors/cache/GridCacheSharedContext.java | 16 ++++++++-------- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java index 715a8711b0828..3109bc5c21c35 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java @@ -1119,9 +1119,9 @@ else if (encMgr.masterKeyDigest() != null && } if (err == null && req.restartId() == null) { - IgniteSnapshotManager snpMgr = ctx.cache().context().snapshotMgr(); + IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); - if (snpMgr.isRestoring(ccfg)) { + if (snapshotMgr.isRestoring(ccfg)) { err = new IgniteCheckedException("Cache start failed. A cache or group with the same name is " + "currently being restored from a snapshot [cache=" + cacheName + (ccfg.getGroupName() == null ? "" : ", group=" + ccfg.getGroupName()) + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 191e98365a144..61696306fe782 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -3063,10 +3063,10 @@ private GridCacheSharedContext createSharedContext( WalStateManager walStateMgr = new WalStateManager(ctx); - IgniteSnapshotManager snpMgr = ctx.plugins().createComponent(IgniteSnapshotManager.class); + IgniteSnapshotManager snapshotMgr = ctx.plugins().createComponent(IgniteSnapshotManager.class); - if (snpMgr == null) - snpMgr = new IgniteSnapshotManager(ctx); + if (snapshotMgr == null) + snapshotMgr = new IgniteSnapshotManager(ctx); CacheObjectTransformerManager transMgr = ctx.plugins().createComponent(CacheObjectTransformerManager.class); @@ -3092,7 +3092,7 @@ private GridCacheSharedContext createSharedContext( walMgr, walStateMgr, dbMgr, - snpMgr, + snapshotMgr, depMgr, exchMgr, topMgr, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java index f5b40804d5441..e6f9f70e12455 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java @@ -127,7 +127,7 @@ public class GridCacheSharedContext { @Nullable private IgnitePageStoreManager pageStoreMgr; /** Snapshot manager for persistence caches. See {@link IgniteSnapshot}. */ - private IgniteSnapshotManager snpMgr; + private IgniteSnapshotManager snapshotMgr; /** Affinity manager. */ private CacheAffinitySharedManager affMgr; @@ -224,7 +224,7 @@ public GridCacheSharedContext( @Nullable IgniteWriteAheadLogManager walMgr, WalStateManager walStateMgr, IgniteCacheDatabaseSharedManager dbMgr, - IgniteSnapshotManager snpMgr, + IgniteSnapshotManager snapshotMgr, GridCacheDeploymentManager depMgr, GridCachePartitionExchangeManager exchMgr, CacheAffinitySharedManager affMgr, @@ -251,7 +251,7 @@ public GridCacheSharedContext( walMgr, walStateMgr, dbMgr, - snpMgr, + snapshotMgr, depMgr, exchMgr, affMgr, @@ -294,7 +294,7 @@ public GridCacheSharedContext( stateAwareMgrs.add(dbMgr); - stateAwareMgrs.add(snpMgr); + stateAwareMgrs.add(snapshotMgr); for (PluginProvider prv : kernalCtx.plugins().allProviders()) if (prv instanceof IgniteChangeGlobalStateSupport) @@ -431,7 +431,7 @@ void onReconnected(boolean active) throws IgniteCheckedException { cdcWalMgr, walStateMgr, dbMgr, - snpMgr, + snapshotMgr, new GridCacheDeploymentManager(), new GridCachePartitionExchangeManager(), affMgr, @@ -482,7 +482,7 @@ private void setManagers( IgniteWriteAheadLogManager cdcWalMgr, WalStateManager walStateMgr, IgniteCacheDatabaseSharedManager dbMgr, - IgniteSnapshotManager snpMgr, + IgniteSnapshotManager snapshotMgr, GridCacheDeploymentManager depMgr, GridCachePartitionExchangeManager exchMgr, CacheAffinitySharedManager affMgr, @@ -506,7 +506,7 @@ private void setManagers( this.cdcWalMgr = walMgr == null ? add(mgrs, cdcWalMgr) : cdcWalMgr; this.walStateMgr = add(mgrs, walStateMgr); this.dbMgr = add(mgrs, dbMgr); - this.snpMgr = add(mgrs, snpMgr); + this.snapshotMgr = add(mgrs, snapshotMgr); this.jtaMgr = add(mgrs, jtaMgr); this.depMgr = add(mgrs, depMgr); this.exchMgr = add(mgrs, exchMgr); @@ -754,7 +754,7 @@ public IgniteCacheDatabaseSharedManager database() { * @return Page storage snapshot manager. */ public IgniteSnapshotManager snapshotMgr() { - return snpMgr; + return snapshotMgr; } /**