From 95e2c778fedfeed2f02b37b20cb32a47a9e4c1b1 Mon Sep 17 00:00:00 2001 From: Nikolay Date: Thu, 6 Jul 2023 22:57:23 +0300 Subject: [PATCH] IGNITE-19915 Remove obsolete IgniteCacheSnapshotManager (#10824) --- .../GridCachePartitionExchangeManager.java | 15 -- .../processors/cache/GridCacheProcessor.java | 9 - .../cache/GridCacheSharedContext.java | 19 -- .../GridDistributedTxRemoteAdapter.java | 4 +- .../GridDhtPartitionsExchangeFuture.java | 38 ---- .../GridCacheDatabaseSharedManager.java | 35 +--- .../persistence/GridCacheOffheapManager.java | 102 +---------- .../checkpoint/CheckpointContextImpl.java | 13 +- .../checkpoint/CheckpointListener.java | 10 -- .../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 - .../file/FilePageStoreManager.java | 39 +--- .../persistence/pagemem/PageMemoryImpl.java | 20 +-- .../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 | 8 +- .../BPlusTreeReuseListPageMemoryImplTest.java | 8 +- ...itePageMemReplaceDelayedWriteUnitTest.java | 2 +- .../IndexStoragePageMemoryImplTest.java | 8 +- .../pagemem/PageMemoryImplNoLoadTest.java | 8 +- .../pagemem/PageMemoryImplTest.java | 12 +- .../hashmap/GridCacheTestContext.java | 2 - 32 files changed, 31 insertions(+), 654 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 b14064dafc1aa..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); @@ -755,13 +743,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/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/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java index 7cb74ed16d1fe..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 @@ -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; @@ -128,7 +127,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; @@ -146,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; @@ -327,9 +324,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 +525,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 +562,6 @@ private DataRegionConfiguration createDefragmentationMappingRegionConfig(long re this::cacheGroupContexts, this::getPageMemoryForCacheGroup, resolveThrottlingPolicy(), - snapshotMgr, dataStorageMetricsImpl(), kernalCtx.longJvmPauseDetector(), kernalCtx.failure(), @@ -816,7 +807,6 @@ private void prepareCacheDefragmentation(List cacheNames) throws IgniteC () -> regions, this::getPageMemoryForCacheGroup, resolveThrottlingPolicy(), - snapshotMgr, dataStorageMetricsImpl(), kernalCtx.longJvmPauseDetector(), kernalCtx.failure(), @@ -919,8 +909,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); @@ -1215,22 +1203,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( @@ -1245,15 +1217,12 @@ 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); getCheckpointer().currentProgress().updateEvictedPages(1); }, - changeTracker, + trackable, this, memMetrics, resolveThrottlingPolicy(), @@ -1568,8 +1537,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/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/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/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, 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..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 @@ -84,7 +84,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,11 +234,8 @@ 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; + /** Obsolete feature. Flag keeped for compatibility with existing PDS. */ + private final boolean trackable; /** Pages write throttle. */ private PagesWriteThrottlePolicy writeThrottle; @@ -275,7 +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 changeTracker Callback invoked to track changes in pages. + * @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. @@ -288,7 +284,7 @@ public PageMemoryImpl( PageReadWriteManager pmPageMgr, int pageSize, PageStoreWriter flushDirtyPage, - @Nullable GridInClosure3X changeTracker, + boolean trackable, CheckpointLockStateChecker stateChecker, DataRegionMetricsImpl dataRegionMetrics, @Nullable ThrottlingPolicy throttlingPlc, @@ -308,7 +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.changeTracker = changeTracker; + this.trackable = trackable; this.stateChecker = stateChecker; this.throttlingPlc = throttlingPlc != null ? throttlingPlc : ThrottlingPolicy.CHECKPOINT_BUFFER_ONLY; this.cpProgressProvider = cpProgressProvider; @@ -557,7 +553,7 @@ else if (throttlingPlc == ThrottlingPolicy.CHECKPOINT_BUFFER_ONLY) seg.writeLock().lock(); - boolean isTrackingPage = changeTracker != null && + boolean isTrackingPage = trackable && PageIdUtils.pageIndex(trackingIO.trackingPageFor(pageId, realPageSize(grpId))) == PageIdUtils.pageIndex(pageId); if (isTrackingPage && PageIdUtils.flag(pageId) == PageIdAllocator.FLAG_AUX) @@ -1719,10 +1715,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..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 @@ -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,7 @@ 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, () -> 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..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 @@ -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,7 @@ 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, () -> 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..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, null, () -> 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 14d1380acd559..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 @@ -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,7 @@ 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, () -> 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..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 @@ -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,7 @@ 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, () -> 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..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 @@ -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,7 @@ private PageMemoryImpl createPageMemory( sharedCtx.pageStore(), PAGE_SIZE, replaceWriter, - new GridInClosure3X() { - @Override public void applyx(Long page, FullPageId fullId, PageMemoryEx pageMem) { - } - }, + true, () -> true, new DataRegionMetricsImpl(igniteCfg.getDataStorageConfiguration().getDefaultDataRegionConfiguration(), kernalCtx), throttlingPlc, @@ -671,10 +666,7 @@ private PageMemoryImpl createPageMemory( sharedCtx.pageStore(), PAGE_SIZE, replaceWriter, - new GridInClosure3X() { - @Override public void applyx(Long page, FullPageId fullId, PageMemoryEx pageMem) { - } - }, + true, () -> 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(),