From bfe7cd8a9d83092140b642505b880a8c7b369e2e Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Thu, 1 Nov 2018 13:10:40 +0300 Subject: [PATCH] IGNITE-9420 Logical recovery (squash just for review) --- .../ignite/internal/GridKernalContext.java | 5 + .../internal/GridKernalContextImpl.java | 15 + .../apache/ignite/internal/IgniteKernal.java | 2 + .../eventstorage/GridEventStorageManager.java | 3 + .../pagemem/wal/record/DataRecord.java | 7 + .../wal/record/MemoryRecoveryRecord.java | 9 +- .../wal/record/MetastoreDataRecord.java | 8 +- .../MetaPageUpdatePartitionDataRecord.java | 3 +- .../cache/CacheAffinitySharedManager.java | 9 +- .../processors/cache/CacheGroupContext.java | 288 +++++- .../processors/cache/GridCacheContext.java | 44 +- .../cache/GridCacheEventManager.java | 4 + .../processors/cache/GridCacheProcessor.java | 323 ++++-- .../GridDhtPartitionsExchangeFuture.java | 36 +- .../GridDhtPartitionTopologyImpl.java | 2 - .../cache/mvcc/MvccProcessorImpl.java | 16 +- .../DatabaseLifecycleListener.java | 23 +- .../GridCacheDatabaseSharedManager.java | 926 +++++++++--------- .../IgniteCacheDatabaseSharedManager.java | 98 +- .../persistence/file/AsyncFileIOFactory.java | 9 - .../file/EncryptedFileIOFactory.java | 7 - .../cache/persistence/file/FileIOFactory.java | 10 +- .../cache/persistence/file/FilePageStore.java | 7 +- .../file/RandomAccessFileIOFactory.java | 9 - .../MetastorageLifecycleListener.java | 4 +- .../partstate/PartitionRecoverState.java | 52 + .../snapshot/IgniteCacheSnapshotManager.java | 4 - .../wal/FileWriteAheadLogManager.java | 58 +- .../FsyncModeFileWriteAheadLogManager.java | 50 +- .../reader/StandaloneGridKernalContext.java | 5 + .../serializer/RecordDataV1Serializer.java | 4 +- .../IgniteClusterActivateDeactivateTest.java | 2 +- ...PdsBinaryMetadataOnClusterRestartTest.java | 6 +- .../IgnitePdsCorruptedIndexTest.java | 10 - .../IgnitePdsCorruptedStoreTest.java | 11 +- .../IgnitePdsPartitionFilesDestroyTest.java | 10 - .../IgnitePdsTaskCancelingTest.java | 5 - ...alModeChangeDuringRebalancingSelfTest.java | 5 - .../db/CheckpointBufferDeadlockTest.java | 5 - .../db/IgniteLogicalRecoveryTest.java | 556 +++++++++++ .../db/IgnitePdsDataRegionMetricsTest.java | 3 +- .../IgnitePdsDiskErrorsRecoveringTest.java | 71 +- ...IgniteNodeStoppedDuringDisableWALTest.java | 11 +- .../db/wal/IgniteWalFlushFailoverTest.java | 5 - ...lushMultiNodeFailoverAbstractSelfTest.java | 5 - .../wal/IgniteWalFormatFileFailoverTest.java | 9 +- .../wal/IgniteWalHistoryReservationsTest.java | 16 +- .../db/wal/IgniteWalRebalanceTest.java | 5 - .../wal/WalRecoveryTxLogicalRecordsTest.java | 6 +- .../pagemem/PagesWriteThrottleSmokeTest.java | 5 - .../AbstractNodeJoinTemplate.java | 4 +- .../wal/AbstractWalDeltaConsistencyTest.java | 2 + .../wal/ExplicitWalDeltaConsistencyTest.java | 2 + .../wal/memtracker/PageMemoryTracker.java | 13 +- .../PageMemoryTrackerPluginProvider.java | 19 +- .../StandaloneWalRecordsIteratorTest.java | 5 - ...tegrityWithPrimaryIndexCorruptionTest.java | 3 - .../hashmap/GridCacheTestContext.java | 1 + .../junits/GridAbstractTest.java | 7 +- .../AlignedBuffersDirectFileIOFactory.java | 5 - .../IgnitePdsWithIndexingCoreTestSuite.java | 3 + 61 files changed, 1928 insertions(+), 922 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/partstate/PartitionRecoverState.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteLogicalRecoveryTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java index 970b8e7150ac5..a43312cc8ac6a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java @@ -698,4 +698,9 @@ public interface GridKernalContext extends Iterable { * @return Default uncaught exception handler used by thread pools. */ public Thread.UncaughtExceptionHandler uncaughtExceptionHandler(); + + /** + * @return {@code True} if node is in recovery mode (before join to topology). + */ + public boolean recoveryMode(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java index 6f6f2d408cf34..8a42664cc1a84 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java @@ -415,6 +415,9 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable /** Failure processor. */ private FailureProcessor failureProc; + /** Recovery mode flag. Flag is set to {@code false} when discovery manager started. */ + private boolean recoveryMode = true; + /** * No-arg constructor is required by externalization. */ @@ -1180,6 +1183,18 @@ void disconnected(boolean disconnected) { return hnd; } + /** {@inheritDoc} */ + @Override public boolean recoveryMode() { + return recoveryMode; + } + + /** + * @param recoveryMode Recovery mode. + */ + public void recoveryMode(boolean recoveryMode) { + this.recoveryMode = recoveryMode; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(GridKernalContextImpl.class, this); 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 250fbd78d3a54..b20023864ef07 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 @@ -1048,6 +1048,8 @@ public void start( ctx.cache().context().database().notifyMetaStorageSubscribersOnReadyForRead(); ctx.cache().context().database().startMemoryRestore(ctx); + + ctx.recoveryMode(false); } catch (Throwable e) { U.error( diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java index d4daab85c0ed7..92a2eefe0bcf3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java @@ -316,6 +316,9 @@ public void record(DiscoveryEvent evt, DiscoCache discoCache) { private void record0(Event evt, Object... params) { assert evt != null; + if (ctx.recoveryMode()) + return; + if (!enterBusy()) return; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataRecord.java index 7a4d6b8793a21..d5ab53a7226c2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataRecord.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataRecord.java @@ -76,6 +76,13 @@ public DataRecord(List writeEntries, long timestamp) { this.writeEntries = writeEntries; } + /** + * @param writeEntries Write entries. + */ + public void setWriteEntries(List writeEntries) { + this.writeEntries = writeEntries; + } + /** * @return Collection of write entries. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MemoryRecoveryRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MemoryRecoveryRecord.java index 92658cc4ce788..5a48b340d0ca4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MemoryRecoveryRecord.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MemoryRecoveryRecord.java @@ -20,16 +20,11 @@ import org.apache.ignite.internal.util.typedef.internal.S; /** - * Marker that we start memory recovering. - * - * @deprecated Previously, used to track node started\stopped states. But in fact only - * mark files created by method GridCacheDatabaseSharedManager#nodeStart(WALPointer) - * used. Should be removed in 3.0 release. + * Marker indicates that binary memory recovery has finished. */ -@Deprecated public class MemoryRecoveryRecord extends WALRecord { /** Create timestamp, millis */ - private long time; + private final long time; /** * Default constructor. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MetastoreDataRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MetastoreDataRecord.java index e269de2adc012..9e734244c7b84 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MetastoreDataRecord.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MetastoreDataRecord.java @@ -18,13 +18,14 @@ package org.apache.ignite.internal.pagemem.wal.record; +import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage; import org.apache.ignite.internal.util.typedef.internal.S; import org.jetbrains.annotations.Nullable; /** * */ -public class MetastoreDataRecord extends WALRecord { +public class MetastoreDataRecord extends WALRecord implements WalRecordCacheGroupAware { /** */ private final String key; @@ -59,4 +60,9 @@ public String key() { @Override public String toString() { return S.toString(MetastoreDataRecord.class, this, "super", super.toString()); } + + /** {@inheritDoc} */ + @Override public int groupId() { + return MetaStorage.METASTORAGE_CACHE_ID; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java index e5bd343bcb5ee..28294a9f2cd5b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java @@ -55,7 +55,8 @@ public MetaPageUpdatePartitionDataRecord( long updateCntr, long globalRmvId, int partSize, - long cntrsPageId, byte state, + long cntrsPageId, + byte state, int allocatedIdxCandidate ) { super(grpId, pageId); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java index 6e10c0009887f..1283696393a87 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java @@ -17,7 +17,6 @@ package org.apache.ignite.internal.processors.cache; -import javax.cache.CacheException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -32,6 +31,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.stream.Collectors; +import javax.cache.CacheException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteSystemProperties; @@ -362,11 +362,7 @@ public Set waitGroups() { void onCacheGroupCreated(CacheGroupContext grp) { if (!grpHolders.containsKey(grp.groupId())) { cctx.io().addCacheGroupHandler(grp.groupId(), GridDhtAffinityAssignmentResponse.class, - new IgniteBiInClosure() { - @Override public void apply(UUID nodeId, GridDhtAffinityAssignmentResponse res) { - processAffinityAssignmentResponse(nodeId, res); - } - }); + (IgniteBiInClosure) this::processAffinityAssignmentResponse); } } @@ -1281,6 +1277,7 @@ private void forAllCacheGroups(boolean crd, IgniteInClosureX !grp.isLocal()) + .filter(grp -> !grp.isRecoveryMode()) .map(CacheGroupContext::affinity) .collect(Collectors.toList()); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java index a546a369f099e..b0d6c07acc27a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java @@ -22,8 +22,10 @@ import java.util.Collection; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.affinity.AffinityFunction; @@ -39,12 +41,18 @@ import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtAffinityAssignmentRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtAffinityAssignmentResponse; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader; +import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopologyImpl; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader; import org.apache.ignite.internal.processors.cache.persistence.DataRegion; import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager; import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList; +import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx; +import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId; +import org.apache.ignite.internal.processors.cache.persistence.partstate.PartitionRecoverState; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO; import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList; import org.apache.ignite.internal.processors.cache.query.continuous.CounterSkipContext; import org.apache.ignite.internal.processors.query.QueryUtils; @@ -78,13 +86,13 @@ public class CacheGroupContext { private final int grpId; /** Node ID cache group was received from. */ - private final UUID rcvdFrom; + private volatile UUID rcvdFrom; /** Flag indicating that this cache group is in a recovery mode due to partitions loss. */ private boolean needsRecovery; /** */ - private final AffinityTopologyVersion locStartVer; + private volatile AffinityTopologyVersion locStartVer; /** */ private final CacheConfiguration ccfg; @@ -93,7 +101,7 @@ public class CacheGroupContext { private final GridCacheSharedContext ctx; /** */ - private final boolean affNode; + private volatile boolean affNode; /** */ private final CacheType cacheType; @@ -117,16 +125,17 @@ public class CacheGroupContext { private final IgniteLogger log; /** */ - private GridAffinityAssignmentCache aff; + private volatile GridAffinityAssignmentCache aff; /** */ - private GridDhtPartitionTopologyImpl top; + private volatile GridDhtPartitionTopologyImpl top; /** */ - private IgniteCacheOffheapManager offheapMgr; + private volatile IgniteCacheOffheapManager offheapMgr; /** */ - private GridCachePreloader preldr; + private volatile GridCachePreloader preldr; + /** */ private final DataRegion dataRegion; @@ -160,6 +169,12 @@ public class CacheGroupContext { /** */ private volatile boolean globalWalEnabled; + /** Flag indicates that cache group is under recovering and not attached to topology. */ + private final AtomicBoolean recoveryMode; + + /** Flag indicates that all group partitions have restored their state from page memory / disk. */ + private volatile boolean partitionStatesRestored; + /** * @param ctx Context. * @param grpId Group ID. @@ -188,7 +203,8 @@ public class CacheGroupContext { ReuseList reuseList, AffinityTopologyVersion locStartVer, boolean persistenceEnabled, - boolean walEnabled + boolean walEnabled, + boolean recoveryMode ) { assert ccfg != null; assert dataRegion != null || !affNode; @@ -208,8 +224,7 @@ public class CacheGroupContext { this.globalWalEnabled = walEnabled; this.persistenceEnabled = persistenceEnabled; this.localWalEnabled = true; - - persistGlobalWalState(walEnabled); + this.recoveryMode = new AtomicBoolean(recoveryMode); ioPlc = cacheType.ioPolicy(); @@ -715,9 +730,11 @@ public boolean sharedGroup() { * */ public void onKernalStop() { - aff.cancelFutures(new IgniteCheckedException("Failed to wait for topology update, node is stopping.")); + if (!isRecoveryMode()) { + aff.cancelFutures(new IgniteCheckedException("Failed to wait for topology update, node is stopping.")); - preldr.onKernalStop(); + preldr.onKernalStop(); + } offheapMgr.onKernalStop(); } @@ -739,6 +756,11 @@ void stopCache(GridCacheContext cctx, boolean destroy) { * */ void stopGroup() { + offheapMgr.stop(); + + if (isRecoveryMode()) + return; + IgniteCheckedException err = new IgniteCheckedException("Failed to wait for topology update, cache (or node) is stopping."); @@ -748,11 +770,204 @@ void stopGroup() { preldr.onKernalStop(); - offheapMgr.stop(); - ctx.io().removeCacheGroupHandlers(grpId); } + /** + * Finishes recovery for current cache group. + * Attaches topology version and initializes I/O. + * + * @param startVer Cache group start version. + * @param originalReceivedFrom UUID of node that was first who initiated cache group creating. + * This is needed to decide should node calculate affinity locally or fetch from other nodes. + * @param affinityNode Flag indicates, is local node affinity node or not. This may be calculated only after node joined to topology. + * @throws IgniteCheckedException If failed. + */ + public void finishRecovery( + AffinityTopologyVersion startVer, + UUID originalReceivedFrom, + boolean affinityNode + ) throws IgniteCheckedException { + if (recoveryMode.compareAndSet(true, false)) { + affNode = affinityNode; + + rcvdFrom = originalReceivedFrom; + + locStartVer = startVer; + + persistGlobalWalState(globalWalEnabled); + + initializeIO(); + + ctx.affinity().onCacheGroupCreated(this); + } + } + + /** + * Pre-create partitions that resides in page memory or WAL and restores their state. + */ + public long restorePartitionStates(Map partitionRecoveryStates) throws IgniteCheckedException { + if (isLocal() || !affinityNode() || !dataRegion().config().isPersistenceEnabled()) + return 0; + + if (partitionStatesRestored) + return 0; + + long processed = 0; + + PageMemoryEx pageMem = (PageMemoryEx)dataRegion().pageMemory(); + + for (int p = 0; p < affinity().partitions(); p++) { + PartitionRecoverState recoverState = partitionRecoveryStates.get(new GroupPartitionId(grpId, p)); + + if (ctx.pageStore().exists(grpId, p)) { + ctx.pageStore().ensure(grpId, p); + + if (ctx.pageStore().pages(grpId, p) <= 1) { + if (log.isDebugEnabled()) + log.debug("Skipping partition on recovery (pages less than 1) " + + "[grp=" + cacheOrGroupName() + ", p=" + p + "]"); + + continue; + } + + if (log.isDebugEnabled()) + log.debug("Creating partition on recovery (exists in page store) " + + "[grp=" + cacheOrGroupName() + ", p=" + p + "]"); + + processed++; + + GridDhtLocalPartition part = topology().forceCreatePartition(p); + + offheap().onPartitionInitialCounterUpdated(p, 0); + + ctx.database().checkpointReadLock(); + + try { + long partMetaId = pageMem.partitionMetaPageId(grpId, p); + long partMetaPage = pageMem.acquirePage(grpId, partMetaId); + + try { + long pageAddr = pageMem.writeLock(grpId, partMetaId, partMetaPage); + + boolean changed = false; + + try { + PagePartitionMetaIO io = PagePartitionMetaIO.VERSIONS.forPage(pageAddr); + + if (recoverState != null) { + io.setPartitionState(pageAddr, (byte) recoverState.stateId()); + + changed = updateState(part, recoverState.stateId()); + + if (recoverState.stateId() == GridDhtPartitionState.OWNING.ordinal() + || (recoverState.stateId() == GridDhtPartitionState.MOVING.ordinal() + && part.initialUpdateCounter() < recoverState.updateCounter())) { + part.initialUpdateCounter(recoverState.updateCounter()); + + changed = true; + } + + if (log.isInfoEnabled()) + log.warning("Restored partition state (from WAL) " + + "[grp=" + cacheOrGroupName() + ", p=" + p + ", state=" + part.state() + + ", updCntr=" + part.initialUpdateCounter() + "]"); + } + else { + int stateId = (int) io.getPartitionState(pageAddr); + + changed = updateState(part, stateId); + + if (log.isDebugEnabled()) + log.debug("Restored partition state (from page memory) " + + "[grp=" + cacheOrGroupName() + ", p=" + p + ", state=" + part.state() + + ", updCntr=" + part.initialUpdateCounter() + ", stateId=" + stateId + "]"); + } + } + finally { + pageMem.writeUnlock(grpId, partMetaId, partMetaPage, null, changed); + } + } + finally { + pageMem.releasePage(grpId, partMetaId, partMetaPage); + } + } + finally { + ctx.database().checkpointReadUnlock(); + } + } + else if (recoverState != null) { + GridDhtLocalPartition part = topology().forceCreatePartition(p); + + offheap().onPartitionInitialCounterUpdated(p, recoverState.updateCounter()); + + updateState(part, recoverState.stateId()); + + processed++; + + if (log.isDebugEnabled()) + log.debug("Restored partition state (from WAL) " + + "[grp=" + cacheOrGroupName() + ", p=" + p + ", state=" + part.state() + + ", updCntr=" + part.initialUpdateCounter() + "]"); + } + else { + if (log.isDebugEnabled()) + log.debug("Skipping partition on recovery (no page store OR wal state) " + + "[grp=" + cacheOrGroupName() + ", p=" + p + "]"); + } + } + + partitionStatesRestored = true; + + return processed; + } + + /** + * @param part Partition to restore state for. + * @param stateId State enum ordinal. + * @return Updated flag. + */ + private boolean updateState(GridDhtLocalPartition part, int stateId) { + if (stateId != -1) { + GridDhtPartitionState state = GridDhtPartitionState.fromOrdinal(stateId); + + assert state != null; + + part.restoreState(state == GridDhtPartitionState.EVICTED ? GridDhtPartitionState.RENTING : state); + + return true; + } + + return false; + } + + /** + * @return {@code True} if current cache group is in recovery mode. + */ + public boolean isRecoveryMode() { + return recoveryMode.get(); + } + + /** + * Initializes affinity and rebalance I/O handlers. + */ + private void initializeIO() throws IgniteCheckedException { + assert !recoveryMode.get() : "Couldn't initialize I/O handlers, recovery mode is on for group " + this; + + if (ccfg.getCacheMode() != LOCAL) { + if (!ctx.kernalContext().clientNode()) { + ctx.io().addCacheGroupHandler(groupId(), GridDhtAffinityAssignmentRequest.class, + (IgniteBiInClosure) this::processAffinityAssignmentRequest); + } + + preldr = new GridDhtPreloader(this); + + preldr.start(); + } + else + preldr = new GridCachePreloaderAdapter(this); + } + /** * @return IDs of caches in this group. */ @@ -777,7 +992,7 @@ public List caches() { /** * @return {@code True} if group contains caches. */ - boolean hasCaches() { + public boolean hasCaches() { List caches = this.caches; return !caches.isEmpty(); @@ -904,39 +1119,25 @@ public void start() throws IgniteCheckedException { ccfg.getCacheMode() == LOCAL, persistenceEnabled()); - if (ccfg.getCacheMode() != LOCAL) { + if (ccfg.getCacheMode() != LOCAL) top = new GridDhtPartitionTopologyImpl(ctx, this); - if (!ctx.kernalContext().clientNode()) { - ctx.io().addCacheGroupHandler(groupId(), GridDhtAffinityAssignmentRequest.class, - new IgniteBiInClosure() { - @Override public void apply(UUID nodeId, GridDhtAffinityAssignmentRequest msg) { - processAffinityAssignmentRequest(nodeId, msg); - } - }); - } - - preldr = new GridDhtPreloader(this); - - preldr.start(); + try { + offheapMgr = persistenceEnabled + ? new GridCacheOffheapManager() + : new IgniteCacheOffheapManagerImpl(); } - else - preldr = new GridCachePreloaderAdapter(this); - - if (persistenceEnabled()) { - try { - offheapMgr = new GridCacheOffheapManager(); - } - catch (Exception e) { - throw new IgniteCheckedException("Failed to initialize offheap manager", e); - } + catch (Exception e) { + throw new IgniteCheckedException("Failed to initialize offheap manager", e); } - else - offheapMgr = new IgniteCacheOffheapManagerImpl(); offheapMgr.start(ctx, this); - ctx.affinity().onCacheGroupCreated(this); + if (!isRecoveryMode()) { + initializeIO(); + + ctx.affinity().onCacheGroupCreated(this); + } } /** @@ -950,8 +1151,7 @@ public boolean persistenceEnabled() { * @param nodeId Node ID. * @param req Request. */ - private void processAffinityAssignmentRequest(final UUID nodeId, - final GridDhtAffinityAssignmentRequest req) { + private void processAffinityAssignmentRequest(UUID nodeId, GridDhtAffinityAssignmentRequest req) { if (log.isDebugEnabled()) log.debug("Processing affinity assignment request [node=" + nodeId + ", req=" + req + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java index 7eea905966b63..ff430b6eeea91 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java @@ -226,7 +226,7 @@ public class GridCacheContext implements Externalizable { private CacheWeakQueryIteratorsHolder> itHolder; /** Affinity node. */ - private boolean affNode; + private volatile boolean affNode; /** Conflict resolver. */ private CacheVersionConflictResolver conflictRslvr; @@ -238,7 +238,7 @@ public class GridCacheContext implements Externalizable { private CountDownLatch startLatch = new CountDownLatch(1); /** Topology version when cache was started on local node. */ - private AffinityTopologyVersion locStartTopVer; + private volatile AffinityTopologyVersion locStartTopVer; /** Dynamic cache deployment ID. */ private IgniteUuid dynamicDeploymentId; @@ -271,7 +271,10 @@ public class GridCacheContext implements Externalizable { private boolean readFromBackup = CacheConfiguration.DFLT_READ_FROM_BACKUP; /** Local node's MAC address. */ - private String locMacs; + private volatile String locMacs; + + /** Recovery mode flag. */ + private volatile boolean recoveryMode; /** * Empty constructor required for {@link Externalizable}. @@ -311,6 +314,7 @@ public GridCacheContext( AffinityTopologyVersion locStartTopVer, boolean affNode, boolean updatesAllowed, + boolean recoveryMode, /* * Managers in starting order! @@ -395,11 +399,45 @@ public GridCacheContext( readFromBackup = cacheCfg.isReadFromBackup(); + this.recoveryMode = recoveryMode; + + assert kernalContext().recoveryMode() == recoveryMode; + + if (!recoveryMode) { + locMacs = localNode().attribute(ATTR_MACS); + + assert locMacs != null; + } + } + + /** + * Called when cache was restored during recovery and node has joined to topology. + * + * @param topVer Cache topology join version. + * @param statisticsEnabled Flag indicates is statistics enabled or not for that cache. + * Value may be changed after node joined to topology. + */ + public void finishRecovery(AffinityTopologyVersion topVer, boolean statisticsEnabled) { + assert recoveryMode : this; + + recoveryMode = false; + + locStartTopVer = topVer; + locMacs = localNode().attribute(ATTR_MACS); + this.statisticsEnabled = statisticsEnabled; + assert locMacs != null; } + /** + * @return {@code True} if cache is in recovery mode. + */ + public boolean isRecoveryMode() { + return recoveryMode; + } + /** * @return Cache group ID. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java index 3c5cf1e944ad1..ef350160eb11a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java @@ -372,6 +372,10 @@ public void addEvent( public boolean isRecordable(int type) { GridCacheContext cctx0 = cctx; + // Event recording is impossible in recovery mode. + if (cctx0 != null && cctx0.kernalContext().recoveryMode()) + return false; + return cctx0 != null && cctx0.userCache() && cctx0.gridEvents().isRecordable(type) && !cctx0.config().isEventsDisabled(); } 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 88cc6272a3837..71b1bae47a782 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 @@ -99,6 +99,7 @@ import org.apache.ignite.internal.processors.cache.local.atomic.GridLocalAtomicCache; import org.apache.ignite.internal.processors.cache.mvcc.MvccCachingManager; import org.apache.ignite.internal.processors.cache.persistence.DataRegion; +import org.apache.ignite.internal.processors.cache.persistence.DatabaseLifecycleListener; import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener; import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager; @@ -106,7 +107,6 @@ import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList; 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.metastorage.ReadWriteMetastorage; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager; import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotDiscoveryMessage; import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList; @@ -201,7 +201,7 @@ * Cache processor. */ @SuppressWarnings({"unchecked", "TypeMayBeWeakened", "deprecation"}) -public class GridCacheProcessor extends GridProcessorAdapter implements MetastorageLifecycleListener { +public class GridCacheProcessor extends GridProcessorAdapter { /** Template of message of conflicts during configuration merge*/ private static final String MERGE_OF_CONFIG_CONFLICTS_MESSAGE = "Conflicts during configuration merge for cache '%s' : \n%s"; @@ -276,6 +276,9 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor /** Protector of initialization of specific value. */ private final InitializationProtector initializationProtector = new InitializationProtector(); + /** Cache recovery lifecycle state and actions. */ + private final CacheRecoveryLifecycle recovery = new CacheRecoveryLifecycle(); + /** * @param ctx Kernal context. */ @@ -742,41 +745,34 @@ private void cleanup(CacheConfiguration cfg, @Nullable Object rsrc, boolean near } } - /** {@inheritDoc} */ - @Override public void onReadyForRead(ReadOnlyMetastorage metastorage) throws IgniteCheckedException { - startCachesOnStart(); - } - - /** {@inheritDoc} */ - @Override public void onReadyForReadWrite(ReadWriteMetastorage metastorage) throws IgniteCheckedException { - } - /** * @throws IgniteCheckedException If failed. */ - private void startCachesOnStart() throws IgniteCheckedException { - if (!ctx.isDaemon()) { - Map caches = new HashMap<>(); + private void restoreCacheConfigurations() throws IgniteCheckedException { + if (ctx.isDaemon()) + return; - Map templates = new HashMap<>(); + Map caches = new HashMap<>(); - addCacheOnJoinFromConfig(caches, templates); + Map templates = new HashMap<>(); - CacheJoinNodeDiscoveryData discoData = new CacheJoinNodeDiscoveryData( + addCacheOnJoinFromConfig(caches, templates); + + CacheJoinNodeDiscoveryData discoData = new CacheJoinNodeDiscoveryData( IgniteUuid.randomUuid(), caches, templates, startAllCachesOnClientStart() - ); + ); - cachesInfo.onStart(discoData); - } + cachesInfo.onStart(discoData); } /** {@inheritDoc} */ @SuppressWarnings({"unchecked"}) @Override public void start() throws IgniteCheckedException { - ctx.internalSubscriptionProcessor().registerMetastorageListener(this); + ctx.internalSubscriptionProcessor().registerMetastorageListener(recovery); + ctx.internalSubscriptionProcessor().registerDatabaseListener(recovery); cachesInfo = new ClusterCachesInfo(ctx); @@ -802,7 +798,7 @@ private void startCachesOnStart() throws IgniteCheckedException { mgr.start(sharedCtx); if (!ctx.isDaemon() && (!CU.isPersistenceEnabled(ctx.config())) || ctx.config().isClientMode()) - startCachesOnStart(); + restoreCacheConfigurations(); if (log.isDebugEnabled()) log.debug("Started cache processor."); @@ -1461,7 +1457,7 @@ private void onKernalStop(GridCacheAdapter cache, boolean cancel) { cache.onKernalStop(); - if (ctx.events().isRecordable(EventType.EVT_CACHE_STOPPED)) + if (!ctx.isRecoveryMode() && ctx.events().isRecordable(EventType.EVT_CACHE_STOPPED)) ctx.events().addEvent(EventType.EVT_CACHE_STOPPED); } @@ -1479,7 +1475,8 @@ private void onKernalStop(GridCacheAdapter cache, boolean cancel) { * @return Cache context. * @throws IgniteCheckedException If failed to create cache. */ - private GridCacheContext createCache(CacheConfiguration cfg, + private GridCacheContext createCacheContext( + CacheConfiguration cfg, CacheGroupContext grp, @Nullable CachePluginManager pluginMgr, DynamicCacheDescriptor desc, @@ -1487,8 +1484,9 @@ private GridCacheContext createCache(CacheConfiguration cfg, CacheObjectContext cacheObjCtx, boolean affNode, boolean updatesAllowed, - boolean disabledAfterStart) - throws IgniteCheckedException { + boolean disabledAfterStart, + boolean recoveryMode + ) throws IgniteCheckedException { assert cfg != null; if (cfg.getCacheStoreFactory() instanceof GridCacheLoaderWriterStoreFactory) { @@ -1562,6 +1560,7 @@ private GridCacheContext createCache(CacheConfiguration cfg, locStartTopVer, affNode, updatesAllowed, + recoveryMode, /* * Managers in starting order! * =========================== @@ -1697,6 +1696,7 @@ private GridCacheContext createCache(CacheConfiguration cfg, locStartTopVer, affNode, true, + recoveryMode, /* * Managers in starting order! * =========================== @@ -2131,12 +2131,16 @@ private void prepareStartCaches( cacheStartFailHandler.handle( startCacheInfo, cacheInfo -> { - ctx.query().onCacheStart( - cacheContexts.get(cacheInfo), - cacheInfo.getCacheDescriptor().schema() != null - ? cacheInfo.getCacheDescriptor().schema() - : new QuerySchema() - ); + GridCacheContext cctx = cacheContexts.get(cacheInfo); + + if (!cctx.isRecoveryMode()) { + ctx.query().onCacheStart( + cctx, + cacheInfo.getCacheDescriptor().schema() != null + ? cacheInfo.getCacheDescriptor().schema() + : new QuerySchema() + ); + } context().exchange().exchangerUpdateHeartbeat(); } @@ -2151,7 +2155,12 @@ private void prepareStartCaches( cacheStartFailHandler.handle( cacheCtxEntry.getKey(), cacheInfo -> { - onCacheStarted(cacheCtxEntry.getValue()); + GridCacheContext cacheContext = cacheCtxEntry.getValue(); + + if (cacheContext.isRecoveryMode()) + finishRecovery(cacheInfo.getExchangeTopVer(), cacheContext); + else + onCacheStarted(cacheCtxEntry.getValue()); context().exchange().exchangerUpdateHeartbeat(); } @@ -2169,7 +2178,7 @@ private void prepareStartCaches( * change state of proxies to restarting * @throws IgniteCheckedException If failed. */ - void prepareCacheStart( + public void prepareCacheStart( CacheConfiguration startCfg, DynamicCacheDescriptor desc, @Nullable NearCacheConfiguration reqNearCfg, @@ -2180,7 +2189,10 @@ void prepareCacheStart( ctx.query().onCacheStart(cacheCtx, desc.schema() != null ? desc.schema() : new QuerySchema()); - onCacheStarted(cacheCtx); + if (cacheCtx.isRecoveryMode()) + finishRecovery(exchTopVer, cacheCtx); + else + onCacheStarted(cacheCtx); } /** @@ -2202,6 +2214,24 @@ private GridCacheContext prepareCacheContext( AffinityTopologyVersion exchTopVer, boolean disabledAfterStart ) throws IgniteCheckedException { + if (caches.containsKey(startCfg.getName())) { + GridCacheAdapter existingCache = caches.get(startCfg.getName()); + + GridCacheContext cctx = existingCache.context(); + + assert cctx.isRecoveryMode(); + + QuerySchema localSchema = recovery.querySchemas.get(desc.cacheId()); + + QuerySchemaPatch localSchemaPatch = localSchema.makePatch(desc.schema().entities()); + + // Cache schema is changed after restart, workaround is stop existing cache and start new. + if (!localSchemaPatch.isEmpty() || localSchemaPatch.hasConflicts()) + stopCacheSafely(cctx); + else + return existingCache.context(); + } + assert !caches.containsKey(startCfg.getName()) : startCfg.getName(); CacheConfiguration ccfg = new CacheConfiguration(startCfg); @@ -2210,9 +2240,9 @@ private GridCacheContext prepareCacheContext( boolean affNode = checkForAffinityNode(desc, reqNearCfg, ccfg); - CacheGroupContext grp = prepareCacheGroup(desc, exchTopVer, cacheObjCtx, affNode, startCfg.getGroupName()); + CacheGroupContext grp = getOrCreateCacheGroupContext(desc, exchTopVer, cacheObjCtx, affNode, startCfg.getGroupName(), false); - GridCacheContext cacheCtx = createCache(ccfg, + GridCacheContext cacheCtx = createCacheContext(ccfg, grp, null, desc, @@ -2220,7 +2250,8 @@ private GridCacheContext prepareCacheContext( cacheObjCtx, affNode, true, - disabledAfterStart + disabledAfterStart, + false ); initCacheContext(cacheCtx, ccfg, desc.deploymentId()); @@ -2228,6 +2259,74 @@ private GridCacheContext prepareCacheContext( return cacheCtx; } + /** + * Stops cache under checkpoint lock. + * @param cctx Cache context. + */ + private void stopCacheSafely(GridCacheContext cctx) { + sharedCtx.database().checkpointReadLock(); + + try { + prepareCacheStop(cctx.name(), false); + + if (!cctx.group().hasCaches()) + stopCacheGroup(cctx.group().groupId()); + } + finally { + sharedCtx.database().checkpointReadUnlock(); + } + + } + + /** + * Finishes recovery for given cache context. + * + * @param cacheStartVer Cache join to topology version. + * @param cacheContext Cache context. + * @throws IgniteCheckedException If failed. + */ + private void finishRecovery(AffinityTopologyVersion cacheStartVer, GridCacheContext cacheContext) throws IgniteCheckedException { + CacheGroupContext groupContext = cacheContext.group(); + + // Take cluster-wide cache descriptor and try to update local cache and cache group parameters. + DynamicCacheDescriptor updatedDescriptor = cacheDescriptor(cacheContext.cacheId()); + + groupContext.finishRecovery( + cacheStartVer, + updatedDescriptor.receivedFrom(), + isLocalAffinity(updatedDescriptor.cacheConfiguration()) + ); + + cacheContext.finishRecovery(cacheStartVer, updatedDescriptor.cacheConfiguration().isStatisticsEnabled()); + + onKernalStart(cacheContext.cache()); + + if (log.isInfoEnabled()) + log.info("Finished recovery for cache [cache=" + cacheContext.name() + + ", grp=" + groupContext.cacheOrGroupName() + ", startVer=" + cacheStartVer + "]"); + } + + /** + * Stops all caches and groups, that was recovered, but not activated on node join. + * Such caches can remain only if it was filtered by node filter on current node. + * It's impossible to check whether current node is affinity node for given cache before join to topology. + */ + public void shutdownNotFinishedRecoveryCaches() { + for (GridCacheAdapter cacheAdapter : caches.values()) { + GridCacheContext cacheContext = cacheAdapter.context(); + + if (cacheContext.isLocal()) + continue; + + if (cacheContext.isRecoveryMode()) { + assert !isLocalAffinity(cacheContext.config()) + : "Cache " + cacheAdapter.context() + " is still in recovery mode after start, but not activated."; + + stopCacheSafely(cacheContext); + } + } + } + /** * Check for affinity node and customize near configuration if needed. * @@ -2281,12 +2380,13 @@ public void preparePageStore(DynamicCacheDescriptor desc, boolean affNode) throw * @return Prepared cache group context. * @throws IgniteCheckedException if failed. */ - private CacheGroupContext prepareCacheGroup( + private CacheGroupContext getOrCreateCacheGroupContext( DynamicCacheDescriptor desc, AffinityTopologyVersion exchTopVer, CacheObjectContext cacheObjCtx, boolean affNode, - String grpName + String grpName, + boolean recoveryMode ) throws IgniteCheckedException { if (grpName != null) { return initializationProtector.protect( @@ -2297,7 +2397,8 @@ private CacheGroupContext prepareCacheGroup( desc.cacheType(), affNode, cacheObjCtx, - exchTopVer + exchTopVer, + recoveryMode ) ); } @@ -2306,7 +2407,8 @@ private CacheGroupContext prepareCacheGroup( desc.cacheType(), affNode, cacheObjCtx, - exchTopVer + exchTopVer, + recoveryMode ); } @@ -2398,6 +2500,67 @@ private void onCacheStarted(GridCacheContext cacheCtx) throws IgniteCheckedExcep onKernalStart(cache); } + /** + * @param desc Cache descriptor. + * @throws IgniteCheckedException If failed. + */ + private GridCacheContext startCacheInRecoveryMode( + DynamicCacheDescriptor desc + ) throws IgniteCheckedException { + CacheConfiguration cfg = desc.cacheConfiguration(); + + CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(cfg); + + preparePageStore(desc, true); + + CacheGroupContext grp = getOrCreateCacheGroupContext( + desc, + AffinityTopologyVersion.NONE, + cacheObjCtx, + true, + cfg.getGroupName(), + true + ); + + GridCacheContext cacheCtx = createCacheContext(cfg, + grp, + null, + desc, + AffinityTopologyVersion.NONE, + cacheObjCtx, + true, + true, + false, + true + ); + + initCacheContext(cacheCtx, cfg, desc.deploymentId()); + + cacheCtx.onStarted(); + + String dataRegion = cfg.getDataRegionName(); + + if (dataRegion == null && ctx.config().getDataStorageConfiguration() != null) + dataRegion = ctx.config().getDataStorageConfiguration().getDefaultDataRegionConfiguration().getName(); + + grp.onCacheStarted(cacheCtx); + + ctx.query().onCacheStart(cacheCtx, desc.schema() != null ? desc.schema() : new QuerySchema()); + + if (log.isInfoEnabled()) { + log.info("Started cache in recovery mode [name=" + cfg.getName() + + ", id=" + cacheCtx.cacheId() + + (cfg.getGroupName() != null ? ", group=" + cfg.getGroupName() : "") + + ", dataRegionName=" + dataRegion + + ", mode=" + cfg.getCacheMode() + + ", atomicity=" + cfg.getAtomicityMode() + + ", backups=" + cfg.getBackups() + + ", mvcc=" + cacheCtx.mvccEnabled() + ']'); + } + + return cacheCtx; + } + /** * @param grpName Group name. * @return Found group or null. @@ -2448,8 +2611,9 @@ private CacheGroupContext startCacheGroup( CacheType cacheType, boolean affNode, CacheObjectContext cacheObjCtx, - AffinityTopologyVersion exchTopVer) - throws IgniteCheckedException { + AffinityTopologyVersion exchTopVer, + boolean recoveryMode + ) throws IgniteCheckedException { CacheConfiguration cfg = new CacheConfiguration(desc.config()); String memPlcName = cfg.getDataRegionName(); @@ -2458,7 +2622,7 @@ private CacheGroupContext startCacheGroup( FreeList freeList = sharedCtx.database().freeList(memPlcName); ReuseList reuseList = sharedCtx.database().reuseList(memPlcName); - boolean persistenceEnabled = sharedCtx.localNode().isClient() ? desc.persistenceEnabled() : + boolean persistenceEnabled = recoveryMode || sharedCtx.localNode().isClient() ? desc.persistenceEnabled() : dataRegion != null && dataRegion.config().isPersistenceEnabled(); CacheGroupContext grp = new CacheGroupContext(sharedCtx, @@ -2473,7 +2637,8 @@ private CacheGroupContext startCacheGroup( reuseList, exchTopVer, persistenceEnabled, - desc.walEnabled() + desc.walEnabled(), + recoveryMode ); for (Object obj : grp.configuredUserObjects()) @@ -2566,7 +2731,7 @@ private void stopGateway(DynamicCacheChangeRequest req) { * @param destroy Cache data destroy flag. Setting to true will remove all cache data. * @return Stopped cache context. */ - private GridCacheContext prepareCacheStop(String cacheName, boolean destroy) { + public GridCacheContext prepareCacheStop(String cacheName, boolean destroy) { assert sharedCtx.database().checkpointLockIsHeldByThread(); GridCacheAdapter cache = caches.remove(cacheName); @@ -2668,17 +2833,7 @@ private void closeCache(GridCacheContext cctx) { jCacheProxies.remove(cctx.name()); - sharedCtx.database().checkpointReadLock(); - - try { - prepareCacheStop(cctx.name(), false); - } - finally { - sharedCtx.database().checkpointReadUnlock(); - } - - if (!cctx.group().hasCaches()) - stopCacheGroup(cctx.group().groupId()); + stopCacheSafely(cctx); } finally { sharedCtx.io().writeUnlock(); @@ -4571,6 +4726,26 @@ public Map cacheDescriptors() { return cachesInfo.registeredCaches(); } + /** + * @return Collection of persistent cache descriptors. + */ + public Collection persistentCaches() { + return cachesInfo.registeredCaches().values() + .stream() + .filter(desc -> isPersistentCache(desc.cacheConfiguration(), ctx.config().getDataStorageConfiguration())) + .collect(Collectors.toList()); + } + + /** + * @return Collection of persistent cache group descriptors. + */ + public Collection persistentGroups() { + return cachesInfo.registeredCacheGroups().values() + .stream() + .filter(CacheGroupDescriptor::persistenceEnabled) + .collect(Collectors.toList()); + } + /** * @return Cache group descriptors. */ @@ -5254,6 +5429,38 @@ public T clone(final T obj) throws IgniteCheckedException { }); } + /** + * Recovery lifecycle for caches. + */ + private class CacheRecoveryLifecycle implements MetastorageLifecycleListener, DatabaseLifecycleListener { + /** Set of QuerySchema's saved on recovery. It's needed if cache query schema has changed after node joined to topology.*/ + private final Map querySchemas = new ConcurrentHashMap<>(); + + /** {@inheritDoc} */ + @Override public void onReadyForRead(ReadOnlyMetastorage metastorage) throws IgniteCheckedException { + restoreCacheConfigurations(); + } + + /** {@inheritDoc} */ + @Override public void beforeBinaryMemoryRestore(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException { + for (DynamicCacheDescriptor cacheDescriptor : persistentCaches()) + preparePageStore(cacheDescriptor, true); + } + + /** {@inheritDoc} */ + @Override public void afterBinaryMemoryRestore(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException { + for (DynamicCacheDescriptor cacheDescriptor : persistentCaches()) { + // Skip MVCC caches. + if (cacheDescriptor.cacheConfiguration().getAtomicityMode() == TRANSACTIONAL_SNAPSHOT) + continue; + + startCacheInRecoveryMode(cacheDescriptor); + + querySchemas.put(cacheDescriptor.cacheId(), cacheDescriptor.schema().copy()); + } + } + } + /** * Handle of fail during cache start. * 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 3d27e8ac3edde..d13a7a6faf1dc 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 @@ -869,10 +869,15 @@ else if (msg instanceof WalStateAbstractMessage) * @throws IgniteCheckedException If failed. */ private IgniteInternalFuture initCachesOnLocalJoin() throws IgniteCheckedException { - if (!isLocalNodeInBaseline()) { + if (!cctx.kernalContext().clientNode() && !isLocalNodeInBaseline()) { cctx.exchange().exchangerBlockingSectionBegin(); try { + // Stop all recovered caches and groups. + cctx.cache().onKernalStopCaches(true); + + cctx.cache().stopCaches(true); + cctx.database().cleanupRestoredCaches(); // Set initial node started marker. @@ -892,12 +897,12 @@ private IgniteInternalFuture initCachesOnLocalJoin() throws IgniteCheckedExce cctx.exchange().exchangerBlockingSectionEnd(); } - if (!cctx.kernalContext().clientNode()) - cctx.database().onDoneRestoreBinaryMemory(); - IgniteInternalFuture cachesRegistrationFut = cctx.cache().startCachesOnLocalJoin(initialVersion(), exchActions == null ? null : exchActions.localJoinContext()); + if (!cctx.kernalContext().clientNode()) + cctx.cache().shutdownNotFinishedRecoveryCaches(); + ensureClientCachesStarted(); return cachesRegistrationFut; @@ -1057,15 +1062,15 @@ private ExchangeType onClusterStateChangeRequest(boolean crd) { cctx.exchange().exchangerBlockingSectionEnd(); } - if (!cctx.kernalContext().clientNode()) - cctx.database().onDoneRestoreBinaryMemory(); - assert registerCachesFuture == null : "No caches registration should be scheduled before new caches have started."; cctx.exchange().exchangerBlockingSectionBegin(); try { registerCachesFuture = cctx.affinity().onCacheChangeRequest(this, crd, exchActions); + + if (!cctx.kernalContext().clientNode()) + cctx.cache().shutdownNotFinishedRecoveryCaches(); } finally { cctx.exchange().exchangerBlockingSectionEnd(); @@ -1396,16 +1401,15 @@ private void distributedExchange() throws IgniteCheckedException { } } - /* It is necessary to run database callback before all topology callbacks. - In case of persistent store is enabled we first restore partitions presented on disk. - We need to guarantee that there are no partition state changes logged to WAL before this callback - to make sure that we correctly restored last actual states. */ - boolean restored; - cctx.exchange().exchangerBlockingSectionBegin(); try { - restored = cctx.database().beforeExchange(this); + /* It is necessary to run database callback before all topology callbacks. + In case of persistent store is enabled we first restore partitions presented on disk. + We need to guarantee that there are no partition state changes logged to WAL before this callback + to make sure that we correctly restored last actual states. */ + + cctx.database().beforeExchange(this); } finally { cctx.exchange().exchangerBlockingSectionEnd(); @@ -1432,11 +1436,11 @@ private void distributedExchange() throws IgniteCheckedException { } // After all partitions have been restored and pre-created it's safe to make first checkpoint. - if (restored) { + if (localJoinExchange() || activateCluster()) { cctx.exchange().exchangerBlockingSectionBegin(); try { - cctx.database().onStateRestored(); + cctx.database().onStateRestored(initialVersion()); } finally { cctx.exchange().exchangerBlockingSectionEnd(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java index 1f338d393c40f..b109e3440dd6c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java @@ -885,8 +885,6 @@ private GridDhtLocalPartition getOrCreatePartition(int p) { locParts.set(p, part); - ctx.pageStore().onPartitionCreated(grp.groupId(), p); - return part; } finally { 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 bf51103a1e752..e2f802b386a32 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 @@ -319,18 +319,13 @@ public MvccProcessorImpl(GridKernalContext ctx) { } /** {@inheritDoc} */ - @Override public void afterInitialise(IgniteCacheDatabaseSharedManager mgr) { - // No-op. - } - - /** {@inheritDoc} */ - @Override public void beforeBinaryMemoryRestore(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException { + @Override public void beforeResumeWalLogging(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException { + // In case of blt changed we should re-init TX_LOG cache. txLogPageStoreInit(mgr); } /** {@inheritDoc} */ - @Override public void beforeResumeWalLogging(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException { - // In case of blt changed we should re-init TX_LOG cache. + @Override public void beforeBinaryMemoryRestore(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException { txLogPageStoreInit(mgr); } @@ -345,11 +340,6 @@ private void txLogPageStoreInit(IgniteCacheDatabaseSharedManager mgr) throws Ign TX_LOG_CACHE_NAME, mgr.dataRegion(TX_LOG_CACHE_NAME).memoryMetrics()); } - /** {@inheritDoc} */ - @Override public void afterMemoryRestore(IgniteCacheDatabaseSharedManager mgr) { - // No-op. - } - /** {@inheritDoc} */ @Override public void onDiscoveryEvent(int evtType, Collection nodes, long topVer, @Nullable DiscoveryCustomMessage customMsg) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DatabaseLifecycleListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DatabaseLifecycleListener.java index ae65c77cc6e96..1f7ba84a05d02 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DatabaseLifecycleListener.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DatabaseLifecycleListener.java @@ -24,36 +24,43 @@ */ public interface DatabaseLifecycleListener { /** - * @param mgr Database shared manager. + * Callback executed when data regions become to start-up. * + * @param mgr Database shared manager. + * @throws IgniteCheckedException If failed. */ - void onInitDataRegions(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException; + default void onInitDataRegions(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException {}; /** + * Callback executed right before node become perform binary recovery. + * * @param mgr Database shared manager. * @throws IgniteCheckedException If failed. */ - public void beforeBinaryMemoryRestore(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException; + default void beforeBinaryMemoryRestore(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException {}; /** + * Callback executed when binary memory has fully restored and WAL logging is resumed. + * * @param mgr Database shared manager. * @throws IgniteCheckedException If failed. */ - public void beforeResumeWalLogging(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException; + default void afterBinaryMemoryRestore(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException {}; /** - * @param mgr Database shared manager. * + * @param mgr + * @throws IgniteCheckedException */ - void afterMemoryRestore(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException; + default void beforeResumeWalLogging(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException {}; /** * @param mgr Database shared manager. */ - void afterInitialise(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException; + default void afterInitialise(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException {}; /** * @param mgr Database shared manager. */ - void beforeStop(IgniteCacheDatabaseSharedManager mgr); + default void beforeStop(IgniteCacheDatabaseSharedManager mgr) {}; } 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 fb4ec1e890e54..ce014318309b1 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 @@ -70,8 +70,6 @@ import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; -import org.apache.ignite.events.DiscoveryEvent; -import org.apache.ignite.events.EventType; import org.apache.ignite.failure.FailureContext; import org.apache.ignite.failure.FailureType; import org.apache.ignite.internal.GridKernalContext; @@ -82,9 +80,8 @@ import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; import org.apache.ignite.internal.mem.DirectMemoryProvider; import org.apache.ignite.internal.mem.DirectMemoryRegion; -import org.apache.ignite.internal.mem.file.MappedFileMemoryProvider; -import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider; import org.apache.ignite.internal.pagemem.FullPageId; +import org.apache.ignite.internal.pagemem.PageIdAllocator; import org.apache.ignite.internal.pagemem.PageIdUtils; import org.apache.ignite.internal.pagemem.PageMemory; import org.apache.ignite.internal.pagemem.PageUtils; @@ -96,19 +93,21 @@ import org.apache.ignite.internal.pagemem.wal.record.CheckpointRecord; import org.apache.ignite.internal.pagemem.wal.record.DataEntry; import org.apache.ignite.internal.pagemem.wal.record.DataRecord; +import org.apache.ignite.internal.pagemem.wal.record.MemoryRecoveryRecord; import org.apache.ignite.internal.pagemem.wal.record.MetastoreDataRecord; 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.WalRecordCacheGroupAware; import org.apache.ignite.internal.pagemem.wal.record.delta.PageDeltaRecord; import org.apache.ignite.internal.pagemem.wal.record.delta.PartitionDestroyRecord; import org.apache.ignite.internal.pagemem.wal.record.delta.PartitionMetaStateRecord; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor; import org.apache.ignite.internal.processors.cache.ExchangeActions; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; -import org.apache.ignite.internal.processors.cache.StoredCacheData; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState; @@ -125,17 +124,16 @@ 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.partstate.GroupPartitionId; import org.apache.ignite.internal.processors.cache.persistence.partstate.PartitionAllocationMap; +import org.apache.ignite.internal.processors.cache.persistence.partstate.PartitionRecoverState; 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.tree.io.PageIO; -import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer; -import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc; import org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDataIntegrityViolationException; import org.apache.ignite.internal.processors.port.GridPortRecord; import org.apache.ignite.internal.util.GridMultiCollectionWrapper; -import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.future.CountDownFuture; import org.apache.ignite.internal.util.future.GridCompoundFuture; @@ -167,11 +165,9 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_CHECKPOINT_READ_LOCK_TIMEOUT; import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_SKIP_CRC; import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD; -import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_DATA_REG_DEFAULT_NAME; import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR; import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION; import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.CHECKPOINT_RECORD; -import static org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage.METASTORAGE_CACHE_ID; import static org.apache.ignite.internal.util.IgniteUtils.checkpointBufferSize; /** @@ -182,6 +178,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan /** */ public static final String IGNITE_PDS_CHECKPOINT_TEST_SKIP_SYNC = "IGNITE_PDS_CHECKPOINT_TEST_SKIP_SYNC"; + /** */ + public static final String IGNITE_PDS_SKIP_CHECKPOINT_ON_NODE_STOP = "IGNITE_PDS_SKIP_CHECKPOINT_ON_NODE_STOP"; + /** MemoryPolicyConfiguration name reserved for meta store. */ public static final String METASTORE_DATA_REGION_NAME = "metastoreMemPlc"; @@ -199,6 +198,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan private final String throttlingPolicyOverride = IgniteSystemProperties.getString( IgniteSystemProperties.IGNITE_OVERRIDE_WRITE_THROTTLING_ENABLED); + /** */ + private final boolean skipCheckpointOnNodeStop = IgniteSystemProperties.getBoolean(IGNITE_PDS_SKIP_CHECKPOINT_ON_NODE_STOP, false); + /** Checkpoint lock hold count. */ private static final ThreadLocal CHECKPOINT_LOCK_HOLD_COUNT = new ThreadLocal() { @Override protected Integer initialValue() { @@ -454,6 +456,7 @@ private DataRegionConfiguration createMetastoreDataRegionConfig(DataStorageConfi cfg.setInitialSize(storageCfg.getSystemRegionInitialSize()); cfg.setMaxSize(storageCfg.getSystemRegionMaxSize()); cfg.setPersistenceEnabled(true); + return cfg; } @@ -496,15 +499,7 @@ private DataRegionConfiguration createMetastoreDataRegionConfig(DataStorageConfi .resolveFolders() .getLockedFileLockHolder(); - fileLockHolder = preLocked == null ? - new FileLockHolder(storeMgr.workDir().getPath(), kernalCtx, log) : preLocked; - - if (log.isDebugEnabled()) - log.debug("Try to capture file lock [nodeId=" + - cctx.localNodeId() + " path=" + fileLockHolder.lockPath() + "]"); - - if (!fileLockHolder.isLocked()) - fileLockHolder.tryLock(lockWaitTime); + acquireFileLock(preLocked); cleanupTempCheckpointDirectory(); @@ -532,17 +527,20 @@ private DataRegionConfiguration createMetastoreDataRegionConfig(DataStorageConfi /** {@inheritDoc} */ @Override public void cleanupRestoredCaches() { - if (dataRegionMap == null) + if (dataRegionMap.isEmpty()) return; for (CacheGroupDescriptor grpDesc : cctx.cache().cacheGroupDescriptors().values()) { String regionName = grpDesc.config().getDataRegionName(); - DataRegion region = dataRegionMap.get(regionName == null ? DFLT_DATA_REG_DEFAULT_NAME : regionName); + DataRegion region = regionName != null ? dataRegionMap.get(regionName) : dfltDataRegion; if (region == null) continue; + if (log.isInfoEnabled()) + log.info("Page memory " + region + " for " + grpDesc + " has invalidated."); + int partitions = grpDesc.config().getAffinity().partitions(); if (region.pageMemory() instanceof PageMemoryEx) { @@ -550,6 +548,8 @@ private DataRegionConfiguration createMetastoreDataRegionConfig(DataStorageConfi for (int partId = 0; partId < partitions; partId++) memEx.invalidate(grpDesc.groupId(), partId); + + memEx.invalidate(grpDesc.groupId(), PageIdAllocator.INDEX_PARTITION); } } @@ -578,6 +578,39 @@ private DataRegionConfiguration createMetastoreDataRegionConfig(DataStorageConfi } } + /** + * @param preLocked Pre-locked file lock holder. + */ + private void acquireFileLock(FileLockHolder preLocked) throws IgniteCheckedException { + if (cctx.kernalContext().clientNode()) + return; + + fileLockHolder = preLocked == null ? + new FileLockHolder(storeMgr.workDir().getPath(), cctx.kernalContext(), log) : preLocked; + + if (!fileLockHolder.isLocked()) { + if (log.isDebugEnabled()) + log.debug("Try to capture file lock [nodeId=" + + cctx.localNodeId() + " path=" + fileLockHolder.lockPath() + "]"); + + fileLockHolder.tryLock(lockWaitTime); + } + } + + /** + * + */ + private void releaseFileLock() { + if (cctx.kernalContext().clientNode() || fileLockHolder == null) + return; + + if (log.isDebugEnabled()) + log.debug("Release file lock [nodeId=" + + cctx.localNodeId() + " path=" + fileLockHolder.lockPath() + "]"); + + fileLockHolder.close(); + } + /** * Retreives checkpoint history form specified {@code dir}. * @@ -661,40 +694,18 @@ private void removeCheckpointFiles(CheckpointEntry cpEntry) throws IgniteChecked /** */ private void readMetastore() throws IgniteCheckedException { try { - DataStorageConfiguration memCfg = cctx.kernalContext().config().getDataStorageConfiguration(); - - DataRegionConfiguration plcCfg = createMetastoreDataRegionConfig(memCfg); - - File allocPath = buildAllocPath(plcCfg); - - DirectMemoryProvider memProvider = allocPath == null ? - new UnsafeMemoryProvider(log) : - new MappedFileMemoryProvider( - log, - allocPath); - - DataRegionMetricsImpl memMetrics = new DataRegionMetricsImpl(plcCfg); - - PageMemoryEx storePageMem = (PageMemoryEx)createPageMemory(memProvider, memCfg, plcCfg, memMetrics, false); - - DataRegion regCfg = new DataRegion(storePageMem, plcCfg, memMetrics, createPageEvictionTracker(plcCfg, storePageMem)); - CheckpointStatus status = readCheckpointStatus(); - cctx.pageStore().initializeForMetastorage(); - - storePageMem.start(); - checkpointReadLock(); try { - restoreMemory(status, true, storePageMem, Collections.emptySet()); + dataRegion(METASTORE_DATA_REGION_NAME).pageMemory().start(); - metaStorage = new MetaStorage(cctx, regCfg, memMetrics, true); + performBinaryMemoryRestore(status, g -> MetaStorage.METASTORAGE_CACHE_ID == g, false); - metaStorage.init(this); + metaStorage = createMetastorage(true); - applyLastUpdates(status, true); + applyLogicalUpdates(status, g -> MetaStorage.METASTORAGE_CACHE_ID == g, false); fillWalDisabledGroups(); @@ -703,7 +714,7 @@ private void readMetastore() throws IgniteCheckedException { finally { metaStorage = null; - storePageMem.stop(true); + dataRegion(METASTORE_DATA_REGION_NAME).pageMemory().stop(false); cctx.pageStore().cleanupPageStoreIfMatch(new Predicate() { @Override public boolean test(Integer grpId) { @@ -729,21 +740,16 @@ private void readMetastore() throws IgniteCheckedException { snapshotMgr = cctx.snapshot(); - if (!cctx.kernalContext().clientNode() && persistenceCfg.getCheckpointThreads() > 1) { - asyncRunner = new IgniteThreadPoolExecutor( - CHECKPOINT_RUNNER_THREAD_PREFIX, - cctx.igniteInstanceName(), - persistenceCfg.getCheckpointThreads(), - persistenceCfg.getCheckpointThreads(), - 30_000, - new LinkedBlockingQueue<>() - ); - } - - if (checkpointer == null) + if (!cctx.kernalContext().clientNode() && checkpointer == null) checkpointer = new Checkpointer(cctx.igniteInstanceName(), "db-checkpoint-thread", log); super.onActivate(ctx); + + if (!cctx.kernalContext().clientNode()) { + initializeCheckpointPool(); + + finishRecovery(); + } } /** {@inheritDoc} */ @@ -760,6 +766,21 @@ private void readMetastore() throws IgniteCheckedException { stopping = false; } + /** + * + */ + private void initializeCheckpointPool() { + if (persistenceCfg.getCheckpointThreads() > 1) + asyncRunner = new IgniteThreadPoolExecutor( + CHECKPOINT_RUNNER_THREAD_PREFIX, + cctx.igniteInstanceName(), + persistenceCfg.getCheckpointThreads(), + persistenceCfg.getCheckpointThreads(), + 30_000, + new LinkedBlockingQueue() + ); + } + /** {@inheritDoc} */ @Override protected void registerMetricsMBeans(IgniteConfiguration cfg) { super.registerMetricsMBeans(cfg); @@ -798,8 +819,13 @@ private void readMetastore() throws IgniteCheckedException { }; } - /** {@inheritDoc} */ - @Override public void onDoneRestoreBinaryMemory() throws IgniteCheckedException { + /** + * Restores last valid WAL pointer and resumes logging from that pointer. + * Re-creates metastorage if needed. + * + * @throws IgniteCheckedException If failed. + */ + private void finishRecovery() throws IgniteCheckedException { assert !cctx.kernalContext().clientNode(); long time = System.currentTimeMillis(); @@ -810,37 +836,24 @@ private void readMetastore() throws IgniteCheckedException { for (DatabaseLifecycleListener lsnr : getDatabaseListeners(cctx.kernalContext())) lsnr.beforeResumeWalLogging(this); - cctx.pageStore().initializeForMetastorage(); - - CheckpointStatus status = readCheckpointStatus(); - - // Binary memory should be recovered at startup. - assert !status.needRestoreMemory() : status; - - WALPointer statusEndPtr = CheckpointStatus.NULL_PTR.equals(status.endPtr) ? null : status.endPtr; + // Try to resume logging since last finished checkpoint if possible. + if (walTail == null) { + CheckpointStatus status = readCheckpointStatus(); - // If binary memory recovery occurs resume from the last walTail in the other case from END checkpoint. - WALPointer walPtr = walTail == null ? statusEndPtr : walTail; + walTail = CheckpointStatus.NULL_PTR.equals(status.endPtr) ? null : status.endPtr; + } - cctx.wal().resumeLogging(walPtr); + cctx.wal().resumeLogging(walTail); walTail = null; - metaStorage = new MetaStorage( - cctx, - dataRegionMap.get(METASTORE_DATA_REGION_NAME), - (DataRegionMetricsImpl)memMetricsMap.get(METASTORE_DATA_REGION_NAME), - false - ); - - // Init metastore only after WAL logging resumed. Can't do it earlier because - // MetaStorage first initialization also touches WAL, look at #isWalDeltaRecordNeeded. - metaStorage.init(this); + // Recreate metastorage to refresh page memory state after deactivation. + if (metaStorage == null) + metaStorage = createMetastorage(false); notifyMetastorageReadyForReadWrite(); - for (DatabaseLifecycleListener lsnr : getDatabaseListeners(cctx.kernalContext())) - lsnr.afterMemoryRestore(this); + U.log(log, "Finish recovery performed in " + (System.currentTimeMillis() - time) + " ms."); } catch (IgniteCheckedException e) { if (X.hasCause(e, StorageException.class, IOException.class)) @@ -850,17 +863,35 @@ private void readMetastore() throws IgniteCheckedException { } finally { checkpointReadUnlock(); - - U.log(log, "Resume logging performed in " + (System.currentTimeMillis() - time) + " ms."); } } /** - * @param cacheGrps Cache groups to restore. + * @param readOnly Metastorage read-only mode. + * @return Instance of Metastorage. + * @throws IgniteCheckedException If failed to create metastorage. + */ + private MetaStorage createMetastorage(boolean readOnly) throws IgniteCheckedException { + cctx.pageStore().initializeForMetastorage(); + + MetaStorage storage = new MetaStorage( + cctx, + dataRegion(METASTORE_DATA_REGION_NAME), + (DataRegionMetricsImpl) memMetricsMap.get(METASTORE_DATA_REGION_NAME), + readOnly + ); + + storage.init(this); + + return storage; + } + + /** + * @param cacheGroupsPredicate Cache groups to restore. * @return Last seen WAL pointer during binary memory recovery. * @throws IgniteCheckedException If failed. */ - protected WALPointer restoreBinaryMemory(Set cacheGrps) throws IgniteCheckedException { + private WALPointer restoreBinaryMemory(Predicate cacheGroupsPredicate) throws IgniteCheckedException { assert !cctx.kernalContext().clientNode(); long time = System.currentTimeMillis(); @@ -868,28 +899,23 @@ protected WALPointer restoreBinaryMemory(Set cacheGrps) throws IgniteCh checkpointReadLock(); try { - for (DatabaseLifecycleListener lsnr : getDatabaseListeners(cctx.kernalContext())) - lsnr.beforeBinaryMemoryRestore(this); - - cctx.pageStore().initializeForMetastorage(); - CheckpointStatus status = readCheckpointStatus(); // First, bring memory to the last consistent checkpoint state if needed. // This method should return a pointer to the last valid record in the WAL. - WALPointer tailWalPtr = restoreMemory(status, - false, - (PageMemoryEx)dataRegionMap.get(METASTORE_DATA_REGION_NAME).pageMemory(), - cacheGrps); + WALPointer restored = performBinaryMemoryRestore(status, cacheGroupsPredicate, true); - if (tailWalPtr == null && !status.endPtr.equals(CheckpointStatus.NULL_PTR)) { + if (restored == null && !status.endPtr.equals(CheckpointStatus.NULL_PTR)) { throw new StorageException("The memory cannot be restored. The critical part of WAL archive is missing " + - "[tailWalPtr=" + tailWalPtr + ", endPtr=" + status.endPtr + ']'); + "[tailWalPtr=" + restored + ", endPtr=" + status.endPtr + ']'); } - nodeStart(tailWalPtr); + nodeStart(restored); + + if (log.isInfoEnabled()) + log.info("Binary recovery performed in " + (System.currentTimeMillis() - time) + " ms."); - return tailWalPtr; + return restored; } catch (IgniteCheckedException e) { if (X.hasCause(e, StorageException.class, IOException.class)) @@ -899,9 +925,6 @@ protected WALPointer restoreBinaryMemory(Set cacheGrps) throws IgniteCh } finally { checkpointReadUnlock(); - - if (log.isInfoEnabled()) - log.info("Binary recovery performed in " + (System.currentTimeMillis() - time) + " ms."); } } @@ -1015,21 +1038,15 @@ public List> nodeStartedPointers() throws IgniteCheckedExce MBEAN_GROUP, MBEAN_NAME ); + + metaStorage = null; } /** {@inheritDoc} */ @Override protected void stop0(boolean cancel) { super.stop0(cancel); - if (!cctx.kernalContext().clientNode()) { - if (fileLockHolder != null) { - if (log.isDebugEnabled()) - log.debug("Release file lock [nodeId=" + - cctx.localNodeId() + " path=" + fileLockHolder.lockPath() + "]"); - - fileLockHolder.close(); - } - } + releaseFileLock(); } /** */ @@ -1322,34 +1339,30 @@ private void shutdownCheckpointer(boolean cancel) { } /** {@inheritDoc} */ - @Override public boolean beforeExchange(GridDhtPartitionsExchangeFuture fut) throws IgniteCheckedException { - DiscoveryEvent discoEvt = fut.firstEvent(); + @Override public void beforeExchange(GridDhtPartitionsExchangeFuture fut) throws IgniteCheckedException { + // Try to restore partition states. + if (fut.localJoinExchange() || fut.activateCluster() + || (fut.exchangeActions() != null && !F.isEmpty(fut.exchangeActions().cacheGroupsToStart()))) { + U.doInParallel( + cctx.kernalContext().getSystemExecutorService(), + cctx.cache().cacheGroups(), + cacheGroup -> { + if (cacheGroup.isLocal()) + return; + + cctx.database().checkpointReadLock(); - boolean joinEvt = discoEvt.type() == EventType.EVT_NODE_JOINED; - - boolean locNode = discoEvt.eventNode().isLocal(); - - boolean isSrvNode = !cctx.kernalContext().clientNode(); - - boolean clusterInTransitionStateToActive = fut.activateCluster(); - - boolean restored = false; - - long time = System.currentTimeMillis(); - - // In case of cluster activation or local join restore, restore whole manager state. - if (clusterInTransitionStateToActive || (joinEvt && locNode && isSrvNode)) { - restoreState(); - - restored = true; - } - // In case of starting groups, restore partition states only for these groups. - else if (fut.exchangeActions() != null && !F.isEmpty(fut.exchangeActions().cacheGroupsToStart())) { - Set restoreGroups = fut.exchangeActions().cacheGroupsToStart().stream() - .map(actionData -> actionData.descriptor().groupId()) - .collect(Collectors.toSet()); + try { + cacheGroup.restorePartitionStates(Collections.emptyMap()); - restorePartitionStates(Collections.emptyMap(), restoreGroups); + if (cacheGroup.localStartVersion().equals(fut.initialVersion())) + cacheGroup.topology().afterStateRestored(fut.initialVersion()); + } + finally { + cctx.database().checkpointReadUnlock(); + } + } + ); } if (cctx.kernalContext().query().moduleEnabled()) { @@ -1366,11 +1379,6 @@ else if (acts.localJoinContext() != null && !F.isEmpty(acts.localJoinContext().c } } } - - if (log.isInfoEnabled()) - log.info("Logical recovery performed in " + (System.currentTimeMillis() - time) + " ms."); - - return restored; } /** @@ -1661,52 +1669,6 @@ private boolean safeToUpdatePageMemories() { CHECKPOINT_LOCK_HOLD_COUNT.set(CHECKPOINT_LOCK_HOLD_COUNT.get() - 1); } - /** - * Restores from last checkpoint and applies WAL changes since this checkpoint. - * - * @throws IgniteCheckedException If failed to restore database status from WAL. - */ - private void restoreState() throws IgniteCheckedException { - try { - CheckpointStatus status = readCheckpointStatus(); - - checkpointReadLock(); - - try { - applyLastUpdates(status, false); - } - finally { - checkpointReadUnlock(); - } - - snapshotMgr.restoreState(); - } - catch (StorageException e) { - throw new IgniteCheckedException(e); - } - } - - /** - * Called when all partitions have been fully restored and pre-created on node start. - * - * Starts checkpointing process and initiates first checkpoint. - * - * @throws IgniteCheckedException If first checkpoint has failed. - */ - @Override public void onStateRestored() throws IgniteCheckedException { - long time = System.currentTimeMillis(); - - new IgniteThread(cctx.igniteInstanceName(), "db-checkpoint-thread", checkpointer).start(); - - CheckpointProgressSnapshot chp = checkpointer.wakeupForCheckpoint(0, "node started"); - - if (chp != null) - chp.cpBeginFut.get(); - - if (log.isInfoEnabled()) - log.info("Checkpointer initilialzation performed in " + (System.currentTimeMillis() - time) + " ms."); - } - /** {@inheritDoc} */ @Override public synchronized Map> reserveHistoryForExchange() { assert reservedForExchange == null : reservedForExchange; @@ -2005,39 +1967,146 @@ private WALPointer readPointer(File cpMarkerFile, ByteBuffer buf) throws IgniteC if (kctx.clientNode()) return; - // Preform early regions startup before restoring state. - initAndStartRegions(kctx.config().getDataStorageConfiguration()); + checkpointReadLock(); + + try { + // Preform early regions startup before restoring state. + initAndStartRegions(kctx.config().getDataStorageConfiguration()); + + for (DatabaseLifecycleListener lsnr : getDatabaseListeners(kctx)) + lsnr.beforeBinaryMemoryRestore(this); + + log.info("Starting binary memory restore for: " + cctx.cache().cacheGroupDescriptors().keySet()); + + cctx.pageStore().initializeForMetastorage(); + + // Restore binary memory for all not WAL disabled cache groups. + WALPointer restored = restoreBinaryMemory( + g -> !initiallyGlobalWalDisabledGrps.contains(g) && !initiallyLocalWalDisabledGrps.contains(g) + ); + + if (restored != null) + U.log(log, "Binary memory state restored at node startup [restoredPtr=" + restored + ']'); + + for (DatabaseLifecycleListener lsnr : getDatabaseListeners(kctx)) + lsnr.afterBinaryMemoryRestore(this); + + cctx.wal().resumeLogging(restored); + + // We should log this record to ensure that node start marker pointer will be found in compacted segment. + cctx.wal().log(new MemoryRecoveryRecord(System.currentTimeMillis())); - // Only presistence caches to start. - for (DynamicCacheDescriptor desc : cctx.cache().cacheDescriptors().values()) { - if (CU.isPersistentCache(desc.cacheConfiguration(), cctx.gridConfig().getDataStorageConfiguration())) - storeMgr.initializeForCache(desc.groupDescriptor(), new StoredCacheData(desc.cacheConfiguration())); + assert metaStorage == null; + + metaStorage = createMetastorage(false); + + CheckpointStatus status = readCheckpointStatus(); + + RestoreLogicalState logicalState = applyLogicalUpdates( + status, + g -> !initiallyGlobalWalDisabledGrps.contains(g) && !initiallyLocalWalDisabledGrps.contains(g), + true + ); + + // Restore state for all groups. + restorePartitionStates(cctx.cache().cacheGroups(), logicalState.partitionRecoveryStates); + + walTail = tailPointer(logicalState.lastRead); + + cctx.wal().onDeActivate(kctx); } + catch (IgniteCheckedException e) { + releaseFileLock(); - final WALPointer restoredPtr = restoreBinaryMemory(cctx.cache().cacheGroupDescriptors().keySet()); + throw e; + } + finally { + checkpointReadUnlock(); + } + } - walTail = restoredPtr; + /** + * Calculates tail pointer for WAL at the end of logical recovery. + * + * @param from Start replay WAL from. + * @return Tail pointer. + * @throws IgniteCheckedException If failed. + */ + private WALPointer tailPointer(WALPointer from) throws IgniteCheckedException { + WALPointer lastRead = from; + + try (WALIterator it = cctx.wal().replay(from)) { + while (it.hasNextX()) { + IgniteBiTuple rec = it.nextX(); - if (restoredPtr != null) - U.log(log, "Binary memory state restored at node startup [restoredPtr=" + restoredPtr + ']'); + if (rec == null) + break; + + lastRead = rec.get1(); + } + } + + return lastRead != null ? lastRead.next() : null; + } + + /** + * @param forGroups Cache groups. + * @param partitionStates Partition states. + * @throws IgniteCheckedException If failed. + */ + private void restorePartitionStates( + Collection forGroups, + Map partitionStates + ) throws IgniteCheckedException { + long startRestorePart = U.currentTimeMillis(); + + if (log.isInfoEnabled()) + log.info("Restoring partition state for local groups."); + + long totalProcessed = 0; + + for (CacheGroupContext grp : forGroups) + totalProcessed += grp.restorePartitionStates(partitionStates); + + if (log.isInfoEnabled()) + log.info("Finished restoring partition state for local groups [" + + "groupsProcessed" + forGroups.size() + + "partitionsProcessed=" + totalProcessed + + ", time=" + (U.currentTimeMillis() - startRestorePart) + "ms]"); + } + + /** + * Called when all partitions have been fully restored and pre-created on node start. + * + * Starts checkpointing process and initiates first checkpoint. + * + * @throws IgniteCheckedException If first checkpoint has failed. + */ + @Override public void onStateRestored(AffinityTopologyVersion topVer) throws IgniteCheckedException { + long time = System.currentTimeMillis(); + + new IgniteThread(cctx.igniteInstanceName(), "db-checkpoint-thread", checkpointer).start(); + + CheckpointProgressSnapshot chp = checkpointer.wakeupForCheckpoint(0, "node started"); + + if (chp != null) + chp.cpBeginFut.get(); + + if (log.isInfoEnabled()) + log.info("Checkpointer initilialzation performed in " + (System.currentTimeMillis() - time) + " ms."); } /** * @param status Checkpoint status. - * @param metastoreOnly If {@code True} restores Metastorage only. - * @param storePageMem Metastore page memory. - * @param cacheGrps Cache groups to restore. + * @param cacheGroupsPredicate Cache groups to restore. * @throws IgniteCheckedException If failed. * @throws StorageException In case I/O error occurred during operations with storage. */ - @Nullable private WALPointer restoreMemory( + private WALPointer performBinaryMemoryRestore( CheckpointStatus status, - boolean metastoreOnly, - PageMemoryEx storePageMem, - Set cacheGrps + Predicate cacheGroupsPredicate, + boolean finalizeState ) throws IgniteCheckedException { - assert !metastoreOnly || storePageMem != null; - if (log.isInfoEnabled()) log.info("Checking memory state [lastValidPos=" + status.endPtr + ", lastMarked=" + status.startPtr + ", lastCheckpointId=" + status.cpStartId + ']'); @@ -2045,8 +2114,9 @@ private WALPointer readPointer(File cpMarkerFile, ByteBuffer buf) throws IgniteC boolean apply = status.needRestoreMemory(); if (apply) { - U.quietAndWarn(log, "Ignite node stopped in the middle of checkpoint. Will restore memory state and " + - "finish checkpoint on node start."); + if (finalizeState) + U.quietAndWarn(log, "Ignite node stopped in the middle of checkpoint. Will restore memory state and " + + "finish checkpoint on node start."); cctx.pageStore().beginRecover(); } @@ -2057,16 +2127,7 @@ private WALPointer readPointer(File cpMarkerFile, ByteBuffer buf) throws IgniteC long lastArchivedSegment = cctx.wal().lastArchivedSegment(); - RestoreBinaryState restoreBinaryState = new RestoreBinaryState(status, lastArchivedSegment, log); - - // Always perform recovery at least meta storage cache. - Set restoreGrps = new HashSet<>(Collections.singletonList(METASTORAGE_CACHE_ID)); - - if (!metastoreOnly && !F.isEmpty(cacheGrps)) { - restoreGrps.addAll(cacheGrps.stream() - .filter(g -> !initiallyGlobalWalDisabledGrps.contains(g) && !initiallyLocalWalDisabledGrps.contains(g)) - .collect(Collectors.toSet())); - } + RestoreBinaryState restoreBinaryState = new RestoreBinaryState(status, lastArchivedSegment, cacheGroupsPredicate); int applied = 0; @@ -2086,29 +2147,30 @@ private WALPointer readPointer(File cpMarkerFile, ByteBuffer buf) throws IgniteC // several repetitive restarts and the same pages may have changed several times. int grpId = pageRec.fullPageId().groupId(); - if (restoreGrps.contains(grpId)) { - long pageId = pageRec.fullPageId().pageId(); + long pageId = pageRec.fullPageId().pageId(); - PageMemoryEx pageMem = grpId == METASTORAGE_CACHE_ID ? storePageMem : getPageMemoryForCacheGroup(grpId); + PageMemoryEx pageMem = getPageMemoryForCacheGroup(grpId); - long page = pageMem.acquirePage(grpId, pageId, true); + if (pageMem == null) + break; - try { - long pageAddr = pageMem.writeLock(grpId, pageId, page); + long page = pageMem.acquirePage(grpId, pageId, true); - try { - PageUtils.putBytes(pageAddr, 0, pageRec.pageData()); - } - finally { - pageMem.writeUnlock(grpId, pageId, page, null, true, true); - } + try { + long pageAddr = pageMem.writeLock(grpId, pageId, page); + + try { + PageUtils.putBytes(pageAddr, 0, pageRec.pageData()); } finally { - pageMem.releasePage(grpId, pageId, page); + pageMem.writeUnlock(grpId, pageId, page, null, true, true); } - - applied++; } + finally { + pageMem.releasePage(grpId, pageId, page); + } + + applied++; } break; @@ -2119,9 +2181,6 @@ private WALPointer readPointer(File cpMarkerFile, ByteBuffer buf) throws IgniteC { int grpId = metaStateRecord.groupId(); - if (!restoreGrps.contains(grpId)) - continue; - int partId = metaStateRecord.partitionId(); GridDhtPartitionState state = GridDhtPartitionState.fromOrdinal(metaStateRecord.state()); @@ -2140,10 +2199,10 @@ private WALPointer readPointer(File cpMarkerFile, ByteBuffer buf) throws IgniteC { int grpId = destroyRecord.groupId(); - if (!restoreGrps.contains(grpId)) - continue; + PageMemoryEx pageMem = getPageMemoryForCacheGroup(grpId); - PageMemoryEx pageMem = grpId == METASTORAGE_CACHE_ID ? storePageMem : getPageMemoryForCacheGroup(grpId); + if (pageMem == null) + break; pageMem.invalidate(grpId, destroyRecord.partitionId()); @@ -2158,37 +2217,38 @@ private WALPointer readPointer(File cpMarkerFile, ByteBuffer buf) throws IgniteC int grpId = r.groupId(); - if (restoreGrps.contains(grpId)) { - long pageId = r.pageId(); + long pageId = r.pageId(); - PageMemoryEx pageMem = grpId == METASTORAGE_CACHE_ID ? storePageMem : getPageMemoryForCacheGroup(grpId); + PageMemoryEx pageMem = getPageMemoryForCacheGroup(grpId); - // Here we do not require tag check because we may be applying memory changes after - // several repetitive restarts and the same pages may have changed several times. - long page = pageMem.acquirePage(grpId, pageId, true); + if (pageMem == null) + break; - try { - long pageAddr = pageMem.writeLock(grpId, pageId, page); + // Here we do not require tag check because we may be applying memory changes after + // several repetitive restarts and the same pages may have changed several times. + long page = pageMem.acquirePage(grpId, pageId, true); - try { - r.applyDelta(pageMem, pageAddr); - } - finally { - pageMem.writeUnlock(grpId, pageId, page, null, true, true); - } + try { + long pageAddr = pageMem.writeLock(grpId, pageId, page); + + try { + r.applyDelta(pageMem, pageAddr); } finally { - pageMem.releasePage(grpId, pageId, page); + pageMem.writeUnlock(grpId, pageId, page, null, true, true); } - - applied++; } + finally { + pageMem.releasePage(grpId, pageId, page); + } + + applied++; } } } } - if (metastoreOnly) + if (!finalizeState) return null; WALPointer lastReadPtr = restoreBinaryState.lastReadRecordPointer(); @@ -2208,7 +2268,7 @@ private WALPointer readPointer(File cpMarkerFile, ByteBuffer buf) throws IgniteC cpHistory.initialize(retreiveHistory()); - return lastReadPtr == null ? null : lastReadPtr.next(); + return lastReadPtr != null ? lastReadPtr.next() : null; } /** @@ -2219,6 +2279,9 @@ private WALPointer readPointer(File cpMarkerFile, ByteBuffer buf) throws IgniteC * @throws IgniteCheckedException if no DataRegion is configured for a name obtained from cache descriptor. */ private PageMemoryEx getPageMemoryForCacheGroup(int grpId) throws IgniteCheckedException { + if (grpId == MetaStorage.METASTORAGE_CACHE_ID) + return (PageMemoryEx)dataRegion(METASTORE_DATA_REGION_NAME).pageMemory(); + // TODO IGNITE-7792 add generic mapping. if (grpId == TxLog.TX_LOG_CACHE_ID) return (PageMemoryEx)dataRegion(TxLog.TX_LOG_CACHE_NAME).pageMemory(); @@ -2229,7 +2292,7 @@ private PageMemoryEx getPageMemoryForCacheGroup(int grpId) throws IgniteCheckedE CacheGroupDescriptor desc = sharedCtx.cache().cacheGroupDescriptors().get(grpId); if (desc == null) - throw new IgniteCheckedException("Failed to find cache group descriptor [grpId=" + grpId + ']'); + return null; String memPlcName = desc.config().getDataRegionName(); @@ -2242,13 +2305,13 @@ private PageMemoryEx getPageMemoryForCacheGroup(int grpId) throws IgniteCheckedE * @param it WalIterator. * @param recPredicate Wal record filter. * @param entryPredicate Entry filter. - * @param partStates Partition to restore state. + * @param partitionRecoveryStates Partition to restore state. */ public void applyUpdatesOnRecovery( @Nullable WALIterator it, IgnitePredicate> recPredicate, IgnitePredicate entryPredicate, - Map, T2> partStates + Map partitionRecoveryStates ) throws IgniteCheckedException { cctx.walState().runWithOutWAL(() -> { if (it != null) { @@ -2306,7 +2369,7 @@ else if (log != null) checkpointReadLock(); try { - restorePartitionStates(partStates, null); + restorePartitionStates(cctx.cache().cacheGroups(), partitionRecoveryStates); } catch (IgniteCheckedException e) { throw new IgniteException(e); @@ -2319,31 +2382,30 @@ else if (log != null) /** * @param status Last registered checkpoint status. - * @param metastoreOnly If {@code True} only records related to metastorage will be processed. * @throws IgniteCheckedException If failed to apply updates. * @throws StorageException If IO exception occurred while reading write-ahead log. */ - private void applyLastUpdates(CheckpointStatus status, boolean metastoreOnly) throws IgniteCheckedException { + private RestoreLogicalState applyLogicalUpdates( + CheckpointStatus status, + Predicate cacheGroupsPredicate, + boolean skipFieldLookup + ) throws IgniteCheckedException { if (log.isInfoEnabled()) log.info("Applying lost cache updates since last checkpoint record [lastMarked=" + status.startPtr + ", lastCheckpointId=" + status.cpStartId + ']'); - if (!metastoreOnly) + if (skipFieldLookup) cctx.kernalContext().query().skipFieldLookup(true); long lastArchivedSegment = cctx.wal().lastArchivedSegment(); - RestoreLogicalState restoreLogicalState = new RestoreLogicalState(lastArchivedSegment, log); + RestoreLogicalState restoreLogicalState = new RestoreLogicalState(lastArchivedSegment, cacheGroupsPredicate); long start = U.currentTimeMillis(); - int applied = 0; - Collection ignoreGrps = metastoreOnly ? Collections.emptySet() : - F.concat(false, initiallyGlobalWalDisabledGrps, initiallyLocalWalDisabledGrps); + int applied = 0; try (WALIterator it = cctx.wal().replay(status.startPtr)) { - Map, T2> partStates = new HashMap<>(); - while (it.hasNextX()) { WALRecord rec = restoreLogicalState.next(it); @@ -2352,9 +2414,6 @@ private void applyLastUpdates(CheckpointStatus status, boolean metastoreOnly) th switch (rec.type()) { case DATA_RECORD: - if (metastoreOnly) - continue; - DataRecord dataRec = (DataRecord)rec; for (DataEntry dataEntry : dataRec.writeEntries()) { @@ -2366,27 +2425,27 @@ private void applyLastUpdates(CheckpointStatus status, boolean metastoreOnly) th if (cacheDesc == null) continue; - if (!ignoreGrps.contains(cacheDesc.groupId())) { - GridCacheContext cacheCtx = cctx.cacheContext(cacheId); + GridCacheContext cacheCtx = cctx.cacheContext(cacheId); - applyUpdate(cacheCtx, dataEntry); + applyUpdate(cacheCtx, dataEntry); - applied++; - } + applied++; } break; case PART_META_UPDATE_STATE: - if (metastoreOnly) - continue; - PartitionMetaStateRecord metaStateRecord = (PartitionMetaStateRecord)rec; - if (!ignoreGrps.contains(metaStateRecord.groupId())) { - partStates.put(new T2<>(metaStateRecord.groupId(), metaStateRecord.partitionId()), - new T2<>((int)metaStateRecord.state(), metaStateRecord.updateCounter())); - } + GroupPartitionId groupPartitionId = new GroupPartitionId( + metaStateRecord.groupId(), metaStateRecord.partitionId() + ); + + PartitionRecoverState state = new PartitionRecoverState( + (int)metaStateRecord.state(), metaStateRecord.updateCounter() + ); + + restoreLogicalState.partitionRecoveryStates.put(groupPartitionId, state); break; @@ -2400,13 +2459,13 @@ private void applyLastUpdates(CheckpointStatus status, boolean metastoreOnly) th case META_PAGE_UPDATE_NEXT_SNAPSHOT_ID: case META_PAGE_UPDATE_LAST_SUCCESSFUL_SNAPSHOT_ID: case META_PAGE_UPDATE_LAST_SUCCESSFUL_FULL_SNAPSHOT_ID: - if (metastoreOnly) - continue; - PageDeltaRecord rec0 = (PageDeltaRecord) rec; PageMemoryEx pageMem = getPageMemoryForCacheGroup(rec0.groupId()); + if (pageMem == null) + break; + long page = pageMem.acquirePage(rec0.groupId(), rec0.pageId(), true); try { @@ -2429,166 +2488,17 @@ private void applyLastUpdates(CheckpointStatus status, boolean metastoreOnly) th // Skip other records. } } - - if (!metastoreOnly) { - long startRestorePart = U.currentTimeMillis(); - - if (log.isInfoEnabled()) - log.info("Restoring partition state for local groups [cntPartStateWal=" - + partStates.size() + ", lastCheckpointId=" + status.cpStartId + ']'); - - long proc = restorePartitionStates(partStates, null); - - if (log.isInfoEnabled()) - log.info("Finished restoring partition state for local groups [cntProcessed=" + proc + - ", cntPartStateWal=" + partStates.size() + - ", time=" + (U.currentTimeMillis() - startRestorePart) + "ms]"); - } } finally { - if (!metastoreOnly) + if (skipFieldLookup) cctx.kernalContext().query().skipFieldLookup(false); } if (log.isInfoEnabled()) log.info("Finished applying WAL changes [updatesApplied=" + applied + ", time=" + (U.currentTimeMillis() - start) + " ms]"); - } - /** - * Initializes not empty partitions and restores their state from page memory or WAL. - * Partition states presented in page memory may be overriden by states restored from WAL {@code partStates}. - * - * @param partStates Partition states restored from WAL. - * @param onlyForGroups If not {@code null} restore states only for specified cache groups. - * @return cntParts Count of partitions processed. - * @throws IgniteCheckedException If failed to restore partition states. - */ - private long restorePartitionStates( - Map, T2> partStates, - @Nullable Set onlyForGroups - ) throws IgniteCheckedException { - long cntParts = 0; - - for (CacheGroupContext grp : cctx.cache().cacheGroups()) { - if (grp.isLocal() || !grp.affinityNode()) { - // Local cache has no partitions and its states. - continue; - } - - if (!grp.dataRegion().config().isPersistenceEnabled()) - continue; - - if (onlyForGroups != null && !onlyForGroups.contains(grp.groupId())) - continue; - - int grpId = grp.groupId(); - - PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory(); - - for (int i = 0; i < grp.affinity().partitions(); i++) { - T2 restore = partStates.get(new T2<>(grpId, i)); - - if (storeMgr.exists(grpId, i)) { - storeMgr.ensure(grpId, i); - - if (storeMgr.pages(grpId, i) <= 1) - continue; - - if (log.isDebugEnabled()) - log.debug("Creating partition on recovery (exists in page store) " + - "[grp=" + grp.cacheOrGroupName() + ", p=" + i + "]"); - - GridDhtLocalPartition part = grp.topology().forceCreatePartition(i); - - assert part != null; - - // TODO: https://issues.apache.org/jira/browse/IGNITE-6097 - grp.offheap().onPartitionInitialCounterUpdated(i, 0); - - checkpointReadLock(); - - try { - long partMetaId = pageMem.partitionMetaPageId(grpId, i); - long partMetaPage = pageMem.acquirePage(grpId, partMetaId); - - try { - long pageAddr = pageMem.writeLock(grpId, partMetaId, partMetaPage); - - boolean changed = false; - - try { - PagePartitionMetaIO io = PagePartitionMetaIO.VERSIONS.forPage(pageAddr); - - if (restore != null) { - int stateId = restore.get1(); - - io.setPartitionState(pageAddr, (byte)stateId); - - changed = updateState(part, stateId); - - if (stateId == GridDhtPartitionState.OWNING.ordinal() - || (stateId == GridDhtPartitionState.MOVING.ordinal() - && part.initialUpdateCounter() < restore.get2())) { - part.initialUpdateCounter(restore.get2()); - - changed = true; - } - - if (log.isDebugEnabled()) - log.debug("Restored partition state (from WAL) " + - "[grp=" + grp.cacheOrGroupName() + ", p=" + i + ", state=" + part.state() + - "updCntr=" + part.initialUpdateCounter() + "]"); - } - else { - changed = updateState(part, (int) io.getPartitionState(pageAddr)); - - if (log.isDebugEnabled()) - log.debug("Restored partition state (from page memory) " + - "[grp=" + grp.cacheOrGroupName() + ", p=" + i + ", state=" + part.state() + - "updCntr=" + part.initialUpdateCounter() + "]"); - } - } - finally { - pageMem.writeUnlock(grpId, partMetaId, partMetaPage, null, changed); - } - } - finally { - pageMem.releasePage(grpId, partMetaId, partMetaPage); - } - } - finally { - checkpointReadUnlock(); - } - } - else if (restore != null) { - if (log.isDebugEnabled()) - log.debug("Creating partition on recovery (exists in WAL) " + - "[grp=" + grp.cacheOrGroupName() + ", p=" + i + "]"); - - GridDhtLocalPartition part = grp.topology().forceCreatePartition(i); - - assert part != null; - - // TODO: https://issues.apache.org/jira/browse/IGNITE-6097 - grp.offheap().onPartitionInitialCounterUpdated(i, 0); - - updateState(part, restore.get1()); - - if (log.isDebugEnabled()) - log.debug("Restored partition state (from WAL) " + - "[grp=" + grp.cacheOrGroupName() + ", p=" + i + ", state=" + part.state() + - "updCntr=" + part.initialUpdateCounter() + "]"); - } - - cntParts++; - } - - // After partition states are restored, it is necessary to update internal data structures in topology. - grp.topology().afterStateRestored(grp.topology().lastTopologyChangeVersion()); - } - - return cntParts; + return restoreLogicalState; } /** @@ -2603,25 +2513,6 @@ public void onWalTruncated(WALPointer highBound) throws IgniteCheckedException { removeCheckpointFiles(cp); } - /** - * @param part Partition to restore state for. - * @param stateId State enum ordinal. - * @return Updated flag. - */ - private boolean updateState(GridDhtLocalPartition part, int stateId) { - if (stateId != -1) { - GridDhtPartitionState state = GridDhtPartitionState.fromOrdinal(stateId); - - assert state != null; - - part.restoreState(state == GridDhtPartitionState.EVICTED ? GridDhtPartitionState.RENTING : state); - - return true; - } - - return false; - } - /** * @param cacheCtx Cache context to apply an update. * @param dataEntry Data entry to apply. @@ -3143,6 +3034,13 @@ protected Checkpointer(@Nullable String gridName, String name, IgniteLogger log) while (!isCancelled()) { waitCheckpointEvent(); + if (skipCheckpointOnNodeStop && (isStopping() || shutdownNow)) { + if (log.isInfoEnabled()) + log.warning("Skipping last checkpoint because node is stopping."); + + return; + } + GridFutureAdapter enableChangeApplied = GridCacheDatabaseSharedManager.this.enableChangeApplied; if (enableChangeApplied != null) { @@ -3574,7 +3472,7 @@ private void cancelOrWaitPartitionDestroy(int grpId, int partId) throws IgniteCh if (req != null) req.waitCompleted(); - if (log.isDebugEnabled()) + if (req != null && log.isDebugEnabled()) log.debug("Partition file destroy has cancelled [grpId=" + grpId + ", partId=" + partId + "]"); } @@ -4137,7 +4035,7 @@ else if (grpId == TxLog.TX_LOG_CACHE_ID) else { CacheGroupContext grp = context().cache().cacheGroup(grpId); - DataRegion region = grp != null ?grp .dataRegion() : null; + DataRegion region = grp != null ? grp.dataRegion() : null; if (region == null || !region.config().isPersistenceEnabled()) continue; @@ -4645,10 +4543,10 @@ public boolean isCheckpointInapplicableForWalRebalance(Long cpTs, int grpId) thr * */ private void fillWalDisabledGroups() { - MetaStorage meta = cctx.database().metaStorage(); + assert metaStorage != null; try { - Set keys = meta.readForPredicate(WAL_KEY_PREFIX_PRED).keySet(); + Set keys = metaStorage.readForPredicate(WAL_KEY_PREFIX_PRED).keySet(); if (keys.isEmpty()) return; @@ -4713,23 +4611,26 @@ else if (key.startsWith(WAL_GLOBAL_KEY_PREFIX)) /** * Abstract class for create restore context. */ - public abstract static class RestoreStateContext { - /** */ - protected final IgniteLogger log; - + private abstract class RestoreStateContext { /** Last archived segment. */ protected final long lastArchivedSegment; /** Last read record WAL pointer. */ protected FileWALPointer lastRead; + /** Only {@link WalRecordCacheGroupAware} records satisfied this predicate will be applied. */ + private final Predicate cacheGroupPredicate; + + /** Set to {@code true} if data records should be skipped. */ + private final boolean skipDataRecords; + /** * @param lastArchivedSegment Last archived segment index. - * @param log Ignite logger. */ - public RestoreStateContext(long lastArchivedSegment, IgniteLogger log) { + public RestoreStateContext(long lastArchivedSegment, Predicate cacheGroupPredicate, boolean skipDataRecords) { this.lastArchivedSegment = lastArchivedSegment; - this.log = log; + this.cacheGroupPredicate = cacheGroupPredicate; + this.skipDataRecords = skipDataRecords; } /** @@ -4741,17 +4642,63 @@ public RestoreStateContext(long lastArchivedSegment, IgniteLogger log) { */ public WALRecord next(WALIterator it) throws IgniteCheckedException { try { - IgniteBiTuple tup = it.nextX(); + for (;;) { + if (!it.hasNextX()) + return null; - WALRecord rec = tup.get2(); + IgniteBiTuple tup = it.nextX(); - WALPointer ptr = tup.get1(); + if (tup == null) + return null; - lastRead = (FileWALPointer)ptr; + WALRecord rec = tup.get2(); - rec.position(ptr); + WALPointer ptr = tup.get1(); - return rec; + lastRead = (FileWALPointer)ptr; + + rec.position(ptr); + + // Filter out records. + if (rec instanceof WalRecordCacheGroupAware) { + WalRecordCacheGroupAware groupAwareRecord = (WalRecordCacheGroupAware) rec; + + if (!cacheGroupPredicate.test(groupAwareRecord.groupId())) + continue; + } + + switch (rec.type()) { + case METASTORE_DATA_RECORD: + case DATA_RECORD: + if (skipDataRecords) + continue; + + if (rec instanceof DataRecord) { + DataRecord dataRecord = (DataRecord) rec; + + // Filter data entries by group id. + List filteredEntries = dataRecord.writeEntries().stream() + .filter(entry -> { + if (entry == null) + return false; + + int cacheId = entry.cacheId(); + + return cctx != null && cctx.cacheContext(cacheId) != null && cacheGroupPredicate.test(cctx.cacheContext(cacheId).groupId()); + }) + .collect(Collectors.toList()); + + dataRecord.setWriteEntries(filteredEntries); + } + + break; + + default: + break; + } + + return rec; + } } catch (IgniteCheckedException e) { boolean throwsCRCError = throwsCRCError(); @@ -4791,7 +4738,7 @@ public boolean throwsCRCError(){ /** * Restore memory context. Tracks the safety of binary recovery. */ - public static class RestoreBinaryState extends RestoreStateContext { + private class RestoreBinaryState extends RestoreStateContext { /** Checkpoint status. */ private final CheckpointStatus status; @@ -4801,13 +4748,12 @@ public static class RestoreBinaryState extends RestoreStateContext { /** * @param status Checkpoint status. * @param lastArchivedSegment Last archived segment index. - * @param log Ignite logger. */ - public RestoreBinaryState(CheckpointStatus status, long lastArchivedSegment, IgniteLogger log) { - super(lastArchivedSegment, log); + public RestoreBinaryState(CheckpointStatus status, long lastArchivedSegment, Predicate cacheGroupsPredicate) { + super(lastArchivedSegment, cacheGroupsPredicate, true); this.status = status; - needApplyBinaryUpdates = status.needRestoreMemory(); + this.needApplyBinaryUpdates = status.needRestoreMemory(); } /** @@ -4867,13 +4813,15 @@ public boolean needApplyBinaryUpdate() { /** * Restore logical state context. Tracks the safety of logical recovery. */ - public static class RestoreLogicalState extends RestoreStateContext { + private class RestoreLogicalState extends RestoreStateContext { + /** States of partitions recovered during applying logical updates. */ + private final Map partitionRecoveryStates = new HashMap<>(); + /** * @param lastArchivedSegment Last archived segment index. - * @param log Ignite logger. */ - public RestoreLogicalState(long lastArchivedSegment, IgniteLogger log) { - super(lastArchivedSegment, log); + public RestoreLogicalState(long lastArchivedSegment, Predicate cacheGroupsPredicate) { + super(lastArchivedSegment, cacheGroupsPredicate, false); } } 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 21bd45482cd14..4966bcaa0290d 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 @@ -25,6 +25,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import javax.management.InstanceNotFoundException; import org.apache.ignite.DataRegionMetrics; import org.apache.ignite.DataStorageMetrics; @@ -45,6 +46,7 @@ import org.apache.ignite.internal.pagemem.PageMemory; import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl; import org.apache.ignite.internal.pagemem.wal.WALPointer; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; @@ -91,13 +93,19 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap private static final long MAX_PAGE_MEMORY_INIT_SIZE_32_BIT = 2L * 1024 * 1024 * 1024; /** {@code True} to reuse memory on deactive. */ - private final boolean reuseMemory = IgniteSystemProperties.getBoolean(IGNITE_REUSE_MEMORY_ON_DEACTIVATE); + protected final boolean reuseMemory = IgniteSystemProperties.getBoolean(IGNITE_REUSE_MEMORY_ON_DEACTIVATE); + + /** */ + protected final Map dataRegionMap = new ConcurrentHashMap<>(); + + /** Stores memory providers eligible for reuse. */ + private final Map memProviderMap = new ConcurrentHashMap<>(); /** */ private static final String MBEAN_GROUP_NAME = "DataRegionMetrics"; /** */ - protected volatile Map dataRegionMap; + protected final Map memMetricsMap = new ConcurrentHashMap<>(); /** */ private volatile boolean dataRegionsInitialized; @@ -105,9 +113,6 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap /** */ private volatile boolean dataRegionsStarted; - /** */ - protected Map memMetricsMap; - /** */ protected DataRegion dfltDataRegion; @@ -123,8 +128,6 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap /** First eviction was warned flag. */ private volatile boolean firstEvictWarn; - /** Stores memory providers eligible for reuse. */ - private Map memProviderMap; /** {@inheritDoc} */ @Override protected void start0() throws IgniteCheckedException { @@ -266,6 +269,17 @@ public int pageSize() { return pageSize; } + /** + * + */ + private void startDataRegions() { + for (DataRegion region : dataRegionMap.values()) { + region.pageMemory().start(); + + region.evictionTracker().start(); + } + } + /** * @param memCfg Database config. * @throws IgniteCheckedException If failed to initialize swap path. @@ -288,12 +302,6 @@ protected void initDataRegions(DataStorageConfiguration memCfg) throws IgniteChe protected void initDataRegions0(DataStorageConfiguration memCfg) throws IgniteCheckedException { DataRegionConfiguration[] dataRegionCfgs = memCfg.getDataRegionConfigurations(); - int dataRegions = dataRegionCfgs == null ? 0 : dataRegionCfgs.length; - - dataRegionMap = U.newHashMap(3 + dataRegions); - memMetricsMap = U.newHashMap(3 + dataRegions); - memProviderMap = reuseMemory ? U.newHashMap(3 + dataRegions) : null; - if (dataRegionCfgs != null) { for (DataRegionConfiguration dataRegionCfg : dataRegionCfgs) addDataRegion(memCfg, dataRegionCfg, dataRegionCfg.isPersistenceEnabled()); @@ -346,14 +354,14 @@ public void addDataRegion( DataRegionMetricsImpl memMetrics = new DataRegionMetricsImpl(dataRegionCfg, freeSpaceProvider(dataRegionCfg)); - DataRegion memPlc = initMemory(dataStorageCfg, dataRegionCfg, memMetrics, trackable); + DataRegion region = initMemory(dataStorageCfg, dataRegionCfg, memMetrics, trackable); - dataRegionMap.put(dataRegionName, memPlc); + dataRegionMap.put(dataRegionName, region); memMetricsMap.put(dataRegionName, memMetrics); if (dataRegionName.equals(dfltMemPlcName)) - dfltDataRegion = memPlc; + dfltDataRegion = region; else if (dataRegionName.equals(DFLT_DATA_REG_DEFAULT_NAME)) U.warn(log, "Data Region with name 'default' isn't used as a default. " + "Please, check Data Region configuration."); @@ -688,13 +696,6 @@ public DataStorageMetrics persistentStoreMetrics() { return null; } - /** - * @throws IgniteCheckedException If fails. - */ - public void onDoneRestoreBinaryMemory() throws IgniteCheckedException { - // No-op. - } - /** * Creates file with current timestamp and specific "node-started.bin" suffix * and writes into memory recovery pointer. @@ -729,7 +730,7 @@ public DataRegion dataRegion(String memPlcName) throws IgniteCheckedException { if (memPlcName == null) return dfltDataRegion; - if (dataRegionMap == null) + if (dataRegionMap.isEmpty()) return null; DataRegion plc; @@ -856,11 +857,9 @@ public void waitForCheckpoint(String reason) throws IgniteCheckedException { /** * @param discoEvt Before exchange for the given discovery event. - * - * @return {@code True} if partitions have been restored from persistent storage. */ - public boolean beforeExchange(GridDhtPartitionsExchangeFuture discoEvt) throws IgniteCheckedException { - return false; + public void beforeExchange(GridDhtPartitionsExchangeFuture discoEvt) throws IgniteCheckedException { + } /** @@ -878,7 +877,7 @@ public void startMemoryRestore(GridKernalContext kctx) throws IgniteCheckedExcep * * @throws IgniteCheckedException If failed. */ - public void onStateRestored() throws IgniteCheckedException { + public void onStateRestored(AffinityTopologyVersion topVer) throws IgniteCheckedException { // No-op. } @@ -1026,7 +1025,7 @@ private DirectMemoryProvider createOrReuseMemoryProvider(DataRegionConfiguration * * @return {@code True} if policy supports memory reuse. */ - private boolean supportsMemoryReuse(DataRegionConfiguration plcCfg) { + public boolean supportsMemoryReuse(DataRegionConfiguration plcCfg) { return reuseMemory && plcCfg.getSwapPath() == null; } @@ -1209,11 +1208,7 @@ private void startDataRegions(DataStorageConfiguration cfg) throws IgniteChecked registerMetricsMBeans(cctx.gridConfig()); - for (DataRegion memPlc : dataRegionMap.values()) { - memPlc.pageMemory().start(); - - memPlc.evictionTracker().start(); - } + startDataRegions(); initPageMemoryDataStructures(cfg); @@ -1234,33 +1229,26 @@ private void onDeActivate(boolean shutdown) { for (DatabaseLifecycleListener lsnr : getDatabaseListeners(cctx.kernalContext())) lsnr.beforeStop(this); - if (dataRegionMap != null) { - for (DataRegion memPlc : dataRegionMap.values()) { - memPlc.pageMemory().stop(shutdown); + for (DataRegion region : dataRegionMap.values()) { + region.pageMemory().stop(shutdown); - memPlc.evictionTracker().stop(); + region.evictionTracker().stop(); - unregisterMetricsMBean( - cctx.gridConfig(), - MBEAN_GROUP_NAME, - memPlc.memoryMetrics().getName() - ); + unregisterMetricsMBean( + cctx.gridConfig(), + MBEAN_GROUP_NAME, + region.memoryMetrics().getName() + ); } - dataRegionMap.clear(); + dataRegionMap.clear(); - dataRegionMap = null; + if (shutdown && memProviderMap != null) + memProviderMap.clear(); - if (shutdown && memProviderMap != null) { - memProviderMap.clear(); + dataRegionsInitialized = false; - memProviderMap = null; - } - - dataRegionsInitialized = false; - - dataRegionsStarted = false; - } + dataRegionsStarted = false; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIOFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIOFactory.java index 104697e810e58..e0c545b8a87c0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIOFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIOFactory.java @@ -22,10 +22,6 @@ import java.nio.channels.AsynchronousFileChannel; import java.nio.file.OpenOption; -import static java.nio.file.StandardOpenOption.CREATE; -import static java.nio.file.StandardOpenOption.READ; -import static java.nio.file.StandardOpenOption.WRITE; - /** * File I/O factory which uses {@link AsynchronousFileChannel} based implementation of FileIO. */ @@ -36,11 +32,6 @@ public class AsyncFileIOFactory implements FileIOFactory { /** Thread local channel future holder. */ private transient volatile ThreadLocal holder = initHolder(); - /** {@inheritDoc} */ - @Override public FileIO create(File file) throws IOException { - return create(file, CREATE, READ, WRITE); - } - /** {@inheritDoc} */ @Override public FileIO create(File file, OpenOption... modes) throws IOException { if (holder == null) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/EncryptedFileIOFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/EncryptedFileIOFactory.java index 336aab6fa85dd..b4b0389ef2c39 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/EncryptedFileIOFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/EncryptedFileIOFactory.java @@ -75,13 +75,6 @@ public class EncryptedFileIOFactory implements FileIOFactory { this.encSpi = encSpi; } - /** {@inheritDoc} */ - @Override public FileIO create(File file) throws IOException { - FileIO io = plainIOFactory.create(file); - - return new EncryptedFileIO(io, groupId, pageSize, headerSize, encMgr, encSpi); - } - /** {@inheritDoc} */ @Override public FileIO create(File file, OpenOption... modes) throws IOException { FileIO io = plainIOFactory.create(file, modes); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIOFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIOFactory.java index 27351853269db..b236000d07c72 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIOFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIOFactory.java @@ -22,6 +22,10 @@ import java.io.Serializable; import java.nio.file.OpenOption; +import static java.nio.file.StandardOpenOption.CREATE; +import static java.nio.file.StandardOpenOption.READ; +import static java.nio.file.StandardOpenOption.WRITE; + /** * {@link FileIO} factory definition. */ @@ -33,7 +37,9 @@ public interface FileIOFactory extends Serializable { * @return File I/O interface. * @throws IOException If I/O interface creation was failed. */ - public FileIO create(File file) throws IOException; + default FileIO create(File file) throws IOException{ + return create(file, CREATE, READ, WRITE); + }; /** * Creates I/O interface for file with specified mode. @@ -43,5 +49,5 @@ public interface FileIOFactory extends Serializable { * @return File I/O interface. * @throws IOException If I/O interface creation was failed. */ - public FileIO create(File file, OpenOption... modes) throws IOException; + FileIO create(File file, OpenOption... modes) throws IOException; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java index fdf4705b5b177..16d74c33c66e6 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java @@ -258,6 +258,10 @@ private long checkFile(FileIO fileIO) throws IOException { + ", delete=" + delete + "]", e); } finally { + allocatedTracker.updateTotalAllocatedPages(-1L * allocated.getAndSet(0) / pageSize); + + inited = false; + lock.writeLock().unlock(); } } @@ -542,7 +546,8 @@ private void reinit(FileIO fileIO) throws IOException { long off = pageOffset(pageId); assert (off >= 0 && off <= allocated.get()) || recover : - "off=" + U.hexLong(off) + ", allocated=" + U.hexLong(allocated.get()) + ", pageId=" + U.hexLong(pageId); + "off=" + U.hexLong(off) + ", allocated=" + U.hexLong(allocated.get()) + + ", pageId=" + U.hexLong(pageId) + ", file=" + cfgFile.getPath(); assert pageBuf.capacity() == pageSize; assert pageBuf.position() == 0; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIOFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIOFactory.java index 856ba1c749b45..3fa3e2dded52d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIOFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIOFactory.java @@ -21,10 +21,6 @@ import java.io.IOException; import java.nio.file.OpenOption; -import static java.nio.file.StandardOpenOption.CREATE; -import static java.nio.file.StandardOpenOption.READ; -import static java.nio.file.StandardOpenOption.WRITE; - /** * File I/O factory which provides RandomAccessFileIO implementation of FileIO. */ @@ -32,11 +28,6 @@ public class RandomAccessFileIOFactory implements FileIOFactory { /** */ private static final long serialVersionUID = 0L; - /** {@inheritDoc} */ - @Override public FileIO create(File file) throws IOException { - return create(file, CREATE, READ, WRITE); - } - /** {@inheritDoc} */ @Override public FileIO create(File file, OpenOption... modes) throws IOException { return new RandomAccessFileIO(file, modes); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageLifecycleListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageLifecycleListener.java index 8ab418c4f4d5e..12cc4468fc010 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageLifecycleListener.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageLifecycleListener.java @@ -33,7 +33,7 @@ public interface MetastorageLifecycleListener { * * @param metastorage Read-only meta storage. */ - public void onReadyForRead(ReadOnlyMetastorage metastorage) throws IgniteCheckedException; + default void onReadyForRead(ReadOnlyMetastorage metastorage) throws IgniteCheckedException { }; /** * Fully functional metastore capable of performing reading and writing operations. @@ -43,5 +43,5 @@ public interface MetastorageLifecycleListener { * * @param metastorage Fully functional meta storage. */ - public void onReadyForReadWrite(ReadWriteMetastorage metastorage) throws IgniteCheckedException; + default void onReadyForReadWrite(ReadWriteMetastorage metastorage) throws IgniteCheckedException { }; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/partstate/PartitionRecoverState.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/partstate/PartitionRecoverState.java new file mode 100644 index 0000000000000..4c7e4d7323011 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/partstate/PartitionRecoverState.java @@ -0,0 +1,52 @@ +/* + * 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.partstate; + +/** + * Class holds state of partition during recovery process. + */ +public class PartitionRecoverState { + /** State id. */ + private final int stateId; + + /** Update counter. */ + private final long updateCounter; + + /** + * @param stateId State id. + * @param updateCounter Update counter. + */ + public PartitionRecoverState(int stateId, long updateCounter) { + this.stateId = stateId; + this.updateCounter = updateCounter; + } + + /** + * + */ + public int stateId() { + return stateId; + } + + /** + * + */ + public long updateCounter() { + return updateCounter; + } +} 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 index 16cc8f560c278..75aa983abe62d 100644 --- 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 @@ -94,10 +94,6 @@ public boolean partitionsAreFrozen(CacheGroupContext grp) { /** * */ - public void restoreState() throws IgniteCheckedException { - // No-op. - } - public boolean snapshotOperationInProgress(){ return false; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java index 61b1f657911f1..1cc2bf3473c2f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java @@ -81,9 +81,12 @@ import org.apache.ignite.internal.pagemem.wal.WALPointer; import org.apache.ignite.internal.pagemem.wal.record.CheckpointRecord; import org.apache.ignite.internal.pagemem.wal.record.MarshalledRecord; +import org.apache.ignite.internal.pagemem.wal.record.MemoryRecoveryRecord; +import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot; import org.apache.ignite.internal.pagemem.wal.record.RolloverType; import org.apache.ignite.internal.pagemem.wal.record.SwitchSegmentRecord; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +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.GridCacheSharedManagerAdapter; import org.apache.ignite.internal.processors.cache.WalStateManager.WALDisableContext; @@ -528,6 +531,23 @@ public void setFileIOFactory(FileIOFactory ioFactory) { } } + /** + * + */ + private void startArchiverAndCompressor() { + if (isArchiverEnabled()) { + assert archiver != null; + + new IgniteThread(archiver).start(); + } + + if (walSegmentSyncWorker != null) + new IgniteThread(walSegmentSyncWorker).start(); + + if (compressor != null) + compressor.start(); + } + /** * Archiver can be not created, all files will be written to WAL folder, using absolute segment index. * @@ -627,11 +647,17 @@ private void checkWalConfiguration() throws IgniteCheckedException { if (archiver != null) archiver.shutdown(); - if (compressor != null) + if (compressor != null) { compressor.shutdown(); - if (decompressor != null) + compressor = null; + } + + if (decompressor != null) { decompressor.shutdown(); + + decompressor = null; + } } catch (Exception e) { U.error(log, "Failed to gracefully close WAL segment: " + this.currHnd.fileIO, e); @@ -645,20 +671,6 @@ private void checkWalConfiguration() throws IgniteCheckedException { " topVer=" + cctx.discovery().topologyVersionEx() + " ]"); start0(); - - if (!cctx.kernalContext().clientNode()) { - if (isArchiverEnabled()) { - assert archiver != null; - - new IgniteThread(archiver).start(); - } - - if (walSegmentSyncWorker != null) - new IgniteThread(walSegmentSyncWorker).start(); - - if (compressor != null) - compressor.start(); - } } /** {@inheritDoc} */ @@ -686,6 +698,9 @@ private void checkWalConfiguration() throws IgniteCheckedException { @Override public void resumeLogging(WALPointer lastPtr) throws IgniteCheckedException { assert currHnd == null; assert lastPtr == null || lastPtr instanceof FileWALPointer; + + startArchiverAndCompressor(); + assert (isArchiverEnabled() && archiver != null) || (!isArchiverEnabled() && archiver == null) : "Trying to restore FileWriteHandle on deactivated write ahead log manager"; @@ -809,6 +824,11 @@ private void checkWalRolloverRequiredDuringInactivityPeriod() { if (serializer == null || mode == WALMode.NONE) return null; + // Only delta-records, page snapshots and memory recovery are allowed to write in recovery mode. + if (cctx.kernalContext().recoveryMode() && + !(rec instanceof PageDeltaRecord || rec instanceof PageSnapshot || rec instanceof MemoryRecoveryRecord)) + return null; + FileWriteHandle currWrHandle = currentHandle(); WALDisableContext isDisable = walDisableContext; @@ -1772,7 +1792,7 @@ private void shutdown() throws IgniteInterruptedCheckedException { blockingSectionEnd(); } - if (evt.isRecordable(EVT_WAL_SEGMENT_ARCHIVED)) { + if (evt.isRecordable(EVT_WAL_SEGMENT_ARCHIVED) && !cctx.kernalContext().recoveryMode()) { evt.record(new WalSegmentArchivedEvent( cctx.discovery().localNode(), res.getAbsIdx(), @@ -2011,6 +2031,8 @@ private void shutdown() throws IgniteInterruptedCheckedException { for (FileCompressorWorker worker: workers) U.join(worker); + workers.clear(); + U.cancel(this); } @@ -2084,7 +2106,7 @@ private void body0() { f0.force(); } - if (evt.isRecordable(EVT_WAL_SEGMENT_COMPACTED)) { + if (evt.isRecordable(EVT_WAL_SEGMENT_COMPACTED) && !cctx.kernalContext().recoveryMode()) { evt.record(new WalSegmentCompactedEvent( cctx.localNode(), segIdx, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java index def9bf20a7c3b..c74c7eff8d34e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FsyncModeFileWriteAheadLogManager.java @@ -76,9 +76,12 @@ import org.apache.ignite.internal.pagemem.wal.WALIterator; import org.apache.ignite.internal.pagemem.wal.WALPointer; import org.apache.ignite.internal.pagemem.wal.record.MarshalledRecord; +import org.apache.ignite.internal.pagemem.wal.record.MemoryRecoveryRecord; +import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot; import org.apache.ignite.internal.pagemem.wal.record.RolloverType; import org.apache.ignite.internal.pagemem.wal.record.SwitchSegmentRecord; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +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.GridCacheSharedManagerAdapter; import org.apache.ignite.internal.processors.cache.WalStateManager.WALDisableContext; @@ -442,6 +445,20 @@ public void setFileIOFactory(FileIOFactory ioFactory) { } } + /** + * + */ + private void startArchiverAndCompressor() { + if (isArchiverEnabled()) { + assert archiver != null; + + new IgniteThread(archiver).start(); + } + + if (compressor != null) + compressor.start(); + } + /** * @throws IgniteCheckedException if WAL store path is configured and archive path isn't (or vice versa) */ @@ -481,11 +498,17 @@ private void checkWalConfiguration() throws IgniteCheckedException { if (archiver != null) archiver.shutdown(); - if (compressor != null) + if (compressor != null) { compressor.shutdown(); - if (decompressor != null) + compressor = null; + } + + if (decompressor != null) { decompressor.shutdown(); + + decompressor = null; + } } catch (Exception e) { U.error(log, "Failed to gracefully close WAL segment: " + currentHnd.fileIO, e); @@ -499,17 +522,6 @@ private void checkWalConfiguration() throws IgniteCheckedException { " topVer=" + cctx.discovery().topologyVersionEx() + " ]"); start0(); - - if (!cctx.kernalContext().clientNode()) { - if (isArchiverEnabled()) { - assert archiver != null; - - new IgniteThread(archiver).start(); - } - - if (compressor != null) - compressor.start(); - } } /** {@inheritDoc} */ @@ -537,6 +549,9 @@ private void checkWalConfiguration() throws IgniteCheckedException { @Override public void resumeLogging(WALPointer lastPtr) throws IgniteCheckedException { assert currentHnd == null; assert lastPtr == null || lastPtr instanceof FileWALPointer; + + startArchiverAndCompressor(); + assert (isArchiverEnabled() && archiver != null) || (!isArchiverEnabled() && archiver == null) : "Trying to restore FileWriteHandle on deactivated write ahead log manager"; @@ -701,6 +716,11 @@ private void checkWalRolloverRequiredDuringInactivityPeriod() { if (serializer == null || mode == WALMode.NONE) return null; + // Only delta-records, page snapshots and memory recovery are allowed to write in recovery mode. + if (cctx.kernalContext().recoveryMode() && + !(record instanceof PageDeltaRecord || record instanceof PageSnapshot || record instanceof MemoryRecoveryRecord)) + return null; + FileWriteHandle currWrHandle = currentHandle(); WALDisableContext isDisable = walDisableContext; @@ -1649,7 +1669,7 @@ private synchronized void release(long absIdx) { notifyAll(); } - if (evt.isRecordable(EventType.EVT_WAL_SEGMENT_ARCHIVED)) { + if (evt.isRecordable(EventType.EVT_WAL_SEGMENT_ARCHIVED) && !cctx.kernalContext().recoveryMode()) { evt.record(new WalSegmentArchivedEvent(cctx.discovery().localNode(), res.getAbsIdx(), res.getDstArchiveFile())); } @@ -2015,7 +2035,7 @@ private void deleteObsoleteRawSegments() { f0.force(); } - if (evt.isRecordable(EVT_WAL_SEGMENT_COMPACTED)) { + if (evt.isRecordable(EVT_WAL_SEGMENT_COMPACTED) && !cctx.kernalContext().recoveryMode()) { evt.record(new WalSegmentCompactedEvent( cctx.discovery().localNode(), currReservedSegment, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java index 6d379a9381b15..e70a02709f4f8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java @@ -652,6 +652,11 @@ protected IgniteConfiguration prepareIgniteConfiguration() { return null; } + /** {@inheritDoc} */ + @Override public boolean recoveryMode() { + return false; + } + /** {@inheritDoc} */ @Override public PdsFoldersResolver pdsFolderResolver() { return new PdsFoldersResolver() { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java index 2c8f03fc9dde0..ee40039994548 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java @@ -221,7 +221,7 @@ public RecordDataV1Serializer(GridCacheSharedContext cctx) { * @return {@code True} if this record should be encrypted. */ private boolean needEncryption(WALRecord rec) { - if (!(rec instanceof WalRecordCacheGroupAware)) + if (!(rec instanceof WalRecordCacheGroupAware) || rec instanceof MetastoreDataRecord) return false; return needEncryption(((WalRecordCacheGroupAware)rec).groupId()); @@ -1925,7 +1925,7 @@ RecordType recordType(WALRecord rec) { */ boolean isDataRecordEncrypted(DataRecord rec) { for (DataEntry e : rec.writeEntries()) { - if(needEncryption(cctx.cacheContext(e.cacheId()).groupId())) + if (cctx.cacheContext(e.cacheId()) != null && needEncryption(cctx.cacheContext(e.cacheId()).groupId())) return true; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java index 87d7d16cb0bf0..90d4e4edb344a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java @@ -1378,7 +1378,7 @@ final void checkNoCaches(int nodes) { GridCacheProcessor cache = ((IgniteEx)ignite(i)).context().cache(); assertTrue(cache.caches().isEmpty()); - assertTrue(cache.internalCaches().isEmpty()); + assertTrue(cache.internalCaches().stream().allMatch(c -> c.context().isRecoveryMode())); } } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsBinaryMetadataOnClusterRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsBinaryMetadataOnClusterRestartTest.java index e9d9b9e28a167..9121cca5ad8e1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsBinaryMetadataOnClusterRestartTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsBinaryMetadataOnClusterRestartTest.java @@ -72,6 +72,8 @@ public class IgnitePdsBinaryMetadataOnClusterRestartTest extends GridCommonAbstr @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); + cfg.setConsistentId(gridName); + if (customWorkSubDir != null) cfg.setWorkDirectory(Paths.get(U.defaultWorkDirectory(), customWorkSubDir).toString()); @@ -356,8 +358,8 @@ private void copyIncompatibleBinaryMetadata(String fromWorkDir, ) throws Exception { String workDir = U.defaultWorkDirectory(); - Path fromFile = Paths.get(workDir, fromWorkDir, "binary_meta", "node00-" + fromConsId, fileName); - Path toFile = Paths.get(workDir, toWorkDir, "binary_meta", "node00-" + toConsId, fileName); + Path fromFile = Paths.get(workDir, fromWorkDir, "binary_meta", fromConsId, fileName); + Path toFile = Paths.get(workDir, toWorkDir, "binary_meta", toConsId, fileName); Files.copy(fromFile, toFile, StandardCopyOption.REPLACE_EXISTING); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedIndexTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedIndexTest.java index 14d0fb6d4dfb7..da19285d6301d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedIndexTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedIndexTest.java @@ -317,16 +317,6 @@ private static boolean isPartitionFile(File file) { return file.getName().contains("part") && file.getName().endsWith("bin"); } - /** {@inheritDoc} */ - @Override public FileIO create(File file) throws IOException { - FileIO delegate = delegateFactory.create(file); - - if (isPartitionFile(file)) - return new HaltOnTruncateFileIO(delegate, file); - - return delegate; - } - /** {@inheritDoc} */ @Override public FileIO create(File file, OpenOption... modes) throws IOException { FileIO delegate = delegateFactory.create(file, modes); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedStoreTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedStoreTest.java index 059b5eefdcf73..39faf5a1179ff 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedStoreTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedStoreTest.java @@ -54,9 +54,6 @@ import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import static java.nio.file.StandardOpenOption.CREATE; -import static java.nio.file.StandardOpenOption.READ; -import static java.nio.file.StandardOpenOption.WRITE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_SKIP_CRC; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; import static org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage.METASTORAGE_CACHE_ID; @@ -180,6 +177,9 @@ public void testNodeInvalidatedWhenPersistenceIsCorrupted() throws Exception { startGrid(0); } catch (IgniteCheckedException ex) { + if (X.hasCause(ex, StorageException.class, IOException.class)) + return; // Success; + throw ex; } @@ -458,11 +458,6 @@ private static class FailingFileIOFactory implements FileIOFactory { /** Create FileIO closure. */ private volatile IgniteBiClosure createClo; - /** {@inheritDoc} */ - @Override public FileIO create(File file) throws IOException { - return create(file, CREATE, READ, WRITE); - } - /** {@inheritDoc} */ @Override public FileIO create(File file, OpenOption... openOption) throws IOException { FileIO fileIO = null; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesDestroyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesDestroyTest.java index 3605700d739fc..578692c97e840 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesDestroyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesDestroyTest.java @@ -449,16 +449,6 @@ private static boolean isPartitionFile(File file) { return file.getName().contains("part") && file.getName().endsWith("bin"); } - /** {@inheritDoc} */ - @Override public FileIO create(File file) throws IOException { - FileIO delegate = delegateFactory.create(file); - - if (isPartitionFile(file)) - return new FailingFileIO(delegate); - - return delegate; - } - /** {@inheritDoc} */ @Override public FileIO create(File file, OpenOption... modes) throws IOException { FileIO delegate = delegateFactory.create(file, modes); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsTaskCancelingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsTaskCancelingTest.java index b36bac0123d6e..6b457ef47196f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsTaskCancelingTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsTaskCancelingTest.java @@ -287,11 +287,6 @@ private static class SlowIOFactory implements FileIOFactory { /** */ private final FileIOFactory delegateFactory = new RandomAccessFileIOFactory(); - /** {@inheritDoc} */ - @Override public FileIO create(File file) throws IOException { - return create(file, CREATE, READ, WRITE); - } - /** {@inheritDoc} */ @Override public FileIO create(File file, OpenOption... openOption) throws IOException { final FileIO delegate = delegateFactory.create(file, openOption); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java index 4dd5f51b28772..48b60d48c9981 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java @@ -545,11 +545,6 @@ private static class TestFileIOFactory implements FileIOFactory { this.delegate = delegate; } - /** {@inheritDoc} */ - @Override public FileIO create(File file) throws IOException { - return new TestFileIO(delegate.create(file)); - } - /** {@inheritDoc} */ @Override public FileIO create(File file, OpenOption... modes) throws IOException { return new TestFileIO(delegate.create(file, modes)); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/CheckpointBufferDeadlockTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/CheckpointBufferDeadlockTest.java index 3afafe691f462..f58d02d93e944 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/CheckpointBufferDeadlockTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/CheckpointBufferDeadlockTest.java @@ -335,11 +335,6 @@ private static class SlowCheckpointFileIOFactory implements FileIOFactory { /** Delegate factory. */ private final FileIOFactory delegateFactory = new RandomAccessFileIOFactory(); - /** {@inheritDoc} */ - @Override public FileIO create(File file) throws IOException { - return create(file, CREATE, READ, WRITE); - } - /** {@inheritDoc} */ @Override public FileIO create(File file, OpenOption... openOption) throws IOException { final FileIO delegate = delegateFactory.create(file, openOption); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteLogicalRecoveryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteLogicalRecoveryTest.java new file mode 100644 index 0000000000000..5c858e145dc94 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgniteLogicalRecoveryTest.java @@ -0,0 +1,556 @@ +/* + * 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.db; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.OpenOption; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicBoolean; +import com.google.common.collect.Lists; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.WALMode; +import org.apache.ignite.failure.FailureHandler; +import org.apache.ignite.failure.StopNodeFailureHandler; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIODecorator; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; +import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; +import org.apache.ignite.internal.processors.cache.persistence.wal.memtracker.PageMemoryTrackerConfiguration; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.Nullable; +import org.junit.Assert; + +/** + * A set of tests that check correctness of logical recovery performed during node start. + */ +public class IgniteLogicalRecoveryTest extends GridCommonAbstractTest { + /** Shared group name. */ + private static final String SHARED_GROUP_NAME = "group"; + + /** Dynamic cache prefix. */ + private static final String DYNAMIC_CACHE_PREFIX = "dynamic-cache-"; + + /** Cache prefix. */ + private static final String CACHE_PREFIX = "cache-"; + + /** Io factory. */ + private FileIOFactory ioFactory; + + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setConsistentId(igniteInstanceName); + + cfg.setCacheConfiguration( + cacheConfiguration(CACHE_PREFIX + 0, CacheMode.PARTITIONED, CacheAtomicityMode.ATOMIC), + cacheConfiguration(CACHE_PREFIX + 1, CacheMode.PARTITIONED, CacheAtomicityMode.TRANSACTIONAL), + cacheConfiguration(CACHE_PREFIX + 2, CacheMode.REPLICATED, CacheAtomicityMode.ATOMIC), + cacheConfiguration(CACHE_PREFIX + 3, CacheMode.REPLICATED, CacheAtomicityMode.TRANSACTIONAL), + cacheConfiguration(CACHE_PREFIX + 4, SHARED_GROUP_NAME, CacheMode.PARTITIONED, CacheAtomicityMode.TRANSACTIONAL), + cacheConfiguration(CACHE_PREFIX + 5, SHARED_GROUP_NAME, CacheMode.PARTITIONED, CacheAtomicityMode.TRANSACTIONAL) + ); + + DataStorageConfiguration dsCfg = new DataStorageConfiguration() + .setAlwaysWriteFullPages(true) + .setWalMode(WALMode.LOG_ONLY) + .setCheckpointFrequency(1024 * 1024 * 1024) // Disable automatic checkpoints. + .setDefaultDataRegionConfiguration( + new DataRegionConfiguration() + .setName("dflt") + .setInitialSize(256 * 1024 * 1024) + .setMaxSize(256 * 1024 * 1024) + .setPersistenceEnabled(true) + ); + + cfg.setDataStorageConfiguration(dsCfg); + + if (ioFactory != null) + dsCfg.setFileIOFactory(ioFactory); + + cfg.setPluginConfigurations(new PageMemoryTrackerConfiguration().setEnabled(false).setCheckPagesOnCheckpoint(true)); + + return cfg; + } + + /** + * @param name Name. + * @param cacheMode Cache mode. + * @param atomicityMode Atomicity mode. + */ + private CacheConfiguration cacheConfiguration(String name, CacheMode cacheMode, CacheAtomicityMode atomicityMode) { + return cacheConfiguration(name, null, cacheMode, atomicityMode); + } + + /** + * @param name Name. + * @param groupName Group name. + * @param cacheMode Cache mode. + * @param atomicityMode Atomicity mode. + */ + private CacheConfiguration cacheConfiguration(String name, @Nullable String groupName, CacheMode cacheMode, CacheAtomicityMode atomicityMode) { + CacheConfiguration cfg = new CacheConfiguration<>(); + + cfg.setGroupName(groupName); + cfg.setName(name); + cfg.setCacheMode(cacheMode); + cfg.setAtomicityMode(atomicityMode); + cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + cfg.setBackups(2); + cfg.setAffinity(new RendezvousAffinityFunction(false, 32)); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + + System.setProperty(GridCacheDatabaseSharedManager.IGNITE_PDS_SKIP_CHECKPOINT_ON_NODE_STOP, "true"); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + + System.clearProperty(GridCacheDatabaseSharedManager.IGNITE_PDS_SKIP_CHECKPOINT_ON_NODE_STOP); + } + + /** + * + */ + public void testRecoveryOnJoinToActiveCluster() throws Exception { + IgniteEx crd = (IgniteEx) startGridsMultiThreaded(3); + + crd.cluster().active(true); + + IgniteEx node = grid(2); + + AggregateCacheLoader aggCacheLoader = new AggregateCacheLoader(node); + + aggCacheLoader.start(); + + U.sleep(3000); + + forceCheckpoint(); + + U.sleep(3000); + + aggCacheLoader.stop(); + + stopGrid(2, true); + + startGrid(2); + + awaitPartitionMapExchange(); + + aggCacheLoader.consistencyCheck(grid(2)); + } + + /** + * + */ + public void testRecoveryOnJoinToInactiveCluster() throws Exception { + IgniteEx crd = (IgniteEx) startGridsMultiThreaded(3); + + crd.cluster().active(true); + + IgniteEx node = grid(2); + + AggregateCacheLoader aggCacheLoader = new AggregateCacheLoader(node); + + aggCacheLoader.start(); + + U.sleep(3000); + + forceCheckpoint(); + + U.sleep(3000); + + aggCacheLoader.stop(); + + stopGrid(2, true); + + crd.cluster().active(false); + + startGrid(2); + + crd.cluster().active(true); + + awaitPartitionMapExchange(); + + aggCacheLoader.consistencyCheck(grid(2)); + } + + /** + * + */ + public void testRecoveryOnDynamicallyStartedCaches() throws Exception { + IgniteEx crd = (IgniteEx) startGridsMultiThreaded(3); + + crd.cluster().active(true); + + IgniteEx node = grid(2); + + List dynamicCaches = Lists.newArrayList( + cacheConfiguration(DYNAMIC_CACHE_PREFIX + 0, CacheMode.PARTITIONED, CacheAtomicityMode.TRANSACTIONAL), + cacheConfiguration(DYNAMIC_CACHE_PREFIX + 1, CacheMode.REPLICATED, CacheAtomicityMode.TRANSACTIONAL), + cacheConfiguration(DYNAMIC_CACHE_PREFIX + 2, CacheMode.PARTITIONED, CacheAtomicityMode.ATOMIC), + cacheConfiguration(DYNAMIC_CACHE_PREFIX + 3, CacheMode.REPLICATED, CacheAtomicityMode.ATOMIC) + ); + + node.getOrCreateCaches(dynamicCaches); + + AggregateCacheLoader aggCacheLoader = new AggregateCacheLoader(node); + + aggCacheLoader.start(); + + U.sleep(3000); + + forceCheckpoint(); + + U.sleep(3000); + + aggCacheLoader.stop(); + + stopGrid(2, true); + + startGrid(2); + + awaitPartitionMapExchange(); + + for (int idx = 0; idx < 3; idx++) + aggCacheLoader.consistencyCheck(grid(idx)); + } + + /** + * + */ + public void testRecoveryOnJoinToDifferentBlt() throws Exception { + IgniteEx crd = (IgniteEx) startGridsMultiThreaded(3); + + crd.cluster().active(true); + + IgniteEx node = grid(2); + + AggregateCacheLoader aggCacheLoader = new AggregateCacheLoader(node); + + aggCacheLoader.start(); + + U.sleep(3000); + + forceCheckpoint(); + + U.sleep(3000); + + aggCacheLoader.stop(); + + stopGrid(2, true); + + resetBaselineTopology(); + + startGrid(2); + + resetBaselineTopology(); + + awaitPartitionMapExchange(); + + for (int idx = 0; idx < 3; idx++) + aggCacheLoader.consistencyCheck(grid(idx)); + } + + /** + * + */ + public void testRecoveryOnCrushDuringCheckpointOnNodeStart() throws Exception { + // Crash recovery fails because of the bug in pages recycling. + // Test passes if don't perform removes in cache loader. + fail("https://issues.apache.org/jira/browse/IGNITE-9303"); + + IgniteEx crd = (IgniteEx) startGridsMultiThreaded(3, false); + + crd.cluster().active(true); + + IgniteEx node = grid(2); + + AggregateCacheLoader aggCacheLoader = new AggregateCacheLoader(node); + + aggCacheLoader.start(); + + U.sleep(3000); + + forceCheckpoint(); + + U.sleep(3000); + + aggCacheLoader.stop(); + + stopGrid(2, false); + + ioFactory = new CheckpointFailIoFactory(); + + IgniteInternalFuture startNodeFut = GridTestUtils.runAsync(() -> startGrid(2)); + + try { + startNodeFut.get(); + } + catch (Exception expected) { } + + ioFactory = null; + + // Start node again and check recovery. + startGrid(2); + + awaitPartitionMapExchange(); + + for (int idx = 0; idx < 3; idx++) + aggCacheLoader.consistencyCheck(grid(idx)); + } + + /** {@inheritDoc} */ + @Override protected FailureHandler getFailureHandler(String igniteInstanceName) { + return new StopNodeFailureHandler(); + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 600 * 1000; + } + + /** + * + */ + private class AggregateCacheLoader { + /** Ignite. */ + IgniteEx ignite; + + /** Stop flag. */ + AtomicBoolean stopFlag; + + /** Cache loaders. */ + Map cacheLoaders; + + /** + * @param ignite Ignite. + */ + public AggregateCacheLoader(IgniteEx ignite) { + this.ignite = ignite; + } + + /** + * + */ + public void start() { + if (stopFlag != null && !stopFlag.get()) + throw new IllegalStateException("Cache loaders must be stopped before start again"); + + stopFlag = new AtomicBoolean(); + cacheLoaders = new HashMap<>(); + + for (String cacheName : ignite.cacheNames()) { + CacheLoader loader = new CacheLoader(ignite, stopFlag, cacheName); + + IgniteInternalFuture loadFuture = GridTestUtils.runAsync(loader); + + cacheLoaders.put(loader, loadFuture); + } + } + + /** + * + */ + public void stop() throws IgniteCheckedException { + if (stopFlag != null) + stopFlag.set(true); + + if (cacheLoaders != null) + for (IgniteInternalFuture loadFuture : cacheLoaders.values()) + loadFuture.get(); + } + + /** + * @param ignite Ignite. + */ + public void consistencyCheck(IgniteEx ignite) throws IgniteCheckedException { + if (cacheLoaders != null) + for (CacheLoader cacheLoader : cacheLoaders.keySet()) + cacheLoader.consistencyCheck(ignite); + } + } + + /** + * + */ + static class CacheLoader implements Runnable { + /** Keys space. */ + static final int KEYS_SPACE = 3096; + + /** Ignite. */ + final IgniteEx ignite; + + /** Stop flag. */ + final AtomicBoolean stopFlag; + + /** Cache name. */ + final String cacheName; + + /** Local cache. */ + final Map locCache = new ConcurrentHashMap<>(); + + /** + * @param ignite Ignite. + * @param stopFlag Stop flag. + * @param cacheName Cache name. + */ + public CacheLoader(IgniteEx ignite, AtomicBoolean stopFlag, String cacheName) { + this.ignite = ignite; + this.stopFlag = stopFlag; + this.cacheName = cacheName; + } + + /** {@inheritDoc} */ + @Override public void run() { + while (!stopFlag.get()) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + int key = rnd.nextInt(KEYS_SPACE); + + boolean remove = rnd.nextInt(100) <= 20; + + IgniteCache cache = ignite.getOrCreateCache(cacheName); + + try { + if (remove) { + cache.remove(key); + + locCache.remove(key); + } + else { + int[] payload = new int[KEYS_SPACE]; + Arrays.fill(payload, key); + + cache.put(key, payload); + + locCache.put(key, payload); + } + } + catch (Exception ignored) { } + } + } + + /** + * + */ + public void consistencyCheck(IgniteEx ignite) { + IgniteCache cache = ignite.getOrCreateCache(cacheName); + + for (int key = 0; key < KEYS_SPACE; key++) { + int[] expectedValue = (int[]) locCache.get(key); + int[] actualValue = (int[]) cache.get(key); + + Assert.assertEquals("Consistency check failed for: " + cache.getName() + ", key=" + key, + arrayToString(expectedValue), arrayToString(actualValue)); + } + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + CacheLoader loader = (CacheLoader) o; + + return Objects.equals(cacheName, loader.cacheName); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return Objects.hash(cacheName); + } + } + + /** + * + */ + static class CheckpointFailIoFactory implements FileIOFactory { + /** {@inheritDoc} */ + @Override public FileIO create(File file, OpenOption... modes) throws IOException { + FileIO delegate = new RandomAccessFileIOFactory().create(file, modes); + + if (file.getName().contains("part-")) + return new FileIODecorator(delegate) { + @Override public int write(ByteBuffer srcBuf) throws IOException { + throw new IOException("test"); + } + + @Override public int write(ByteBuffer srcBuf, long position) throws IOException { + throw new IOException("test"); + } + + @Override public int write(byte[] buf, int off, int len) throws IOException { + throw new IOException("test"); + } + }; + + return delegate; + } + } + + /** + * @param arr Array. + */ + static String arrayToString(int[] arr) { + if (arr == null) + return "null"; + + StringBuilder sb = new StringBuilder(); + sb.append('['); + for (int i = 0; i < Math.min(arr.length, 10); i++) + sb.append(i + ","); + sb.append(']'); + + return sb.toString(); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java index 4a22a2bcd7f1d..c05f65c26b6b0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java @@ -323,7 +323,8 @@ private void checkMetricsConsistency(final IgniteEx node, String cacheName) thro .memoryMetrics() .getTotalAllocatedPages(); - assertEquals(totalPersistenceSize / pageStoreMgr.pageSize(), totalAllocatedPagesFromMetrics); + assertEquals("Number of allocated pages is different than in metrics for [node=" + node.name() + ", cache=" + cacheName + "]", + totalPersistenceSize / pageStoreMgr.pageSize(), totalAllocatedPagesFromMetrics); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsDiskErrorsRecoveringTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsDiskErrorsRecoveringTest.java index 2f4e4a33403a2..93c4047a8fae8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsDiskErrorsRecoveringTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsDiskErrorsRecoveringTest.java @@ -38,6 +38,7 @@ import org.apache.ignite.failure.StopNodeFailureHandler; import org.apache.ignite.internal.GridKernalState; import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIODecorator; @@ -50,9 +51,6 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Assert; -import static java.nio.file.StandardOpenOption.CREATE; -import static java.nio.file.StandardOpenOption.READ; -import static java.nio.file.StandardOpenOption.WRITE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_MMAP; /** @@ -95,6 +93,8 @@ public class IgnitePdsDiskErrorsRecoveringTest extends GridCommonAbstractTest { @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + cfg.setConsistentId(igniteInstanceName); + DataStorageConfiguration dsCfg = new DataStorageConfiguration() .setDefaultDataRegionConfiguration( new DataRegionConfiguration().setMaxSize(100L * 1024 * 1024).setPersistenceEnabled(true)) @@ -127,27 +127,36 @@ public void testRecoveringOnCacheInitFail() throws Exception { // Fail to initialize page store. 2 extra pages is needed for MetaStorage. ioFactory = new FilteringFileIOFactory(".bin", new LimitedSizeFileIOFactory(new RandomAccessFileIOFactory(), 2 * PAGE_SIZE)); - final IgniteEx grid = startGrid(0); - boolean failed = false; + + IgniteInternalFuture startGridFut = GridTestUtils.runAsync(() -> { + try { + IgniteEx grid = startGrid(0); + + grid.cluster().active(true); + } + catch (Exception e) { + throw new RuntimeException("Failed to start node.", e); + } + }); + try { - grid.cluster().active(true); - } catch (Exception expected) { - log.warning("Expected cache error", expected); + startGridFut.get(); + } + catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("Failed to start node.")); failed = true; } Assert.assertTrue("Cache initialization must failed", failed); - // Grid should be automatically stopped after checkpoint fail. - awaitStop(grid); - // Grid should be successfully recovered after stopping. ioFactory = null; - IgniteEx recoveredGrid = startGrid(0); - recoveredGrid.active(true); + IgniteEx grid = startGrid(0); + + grid.cluster().active(true); } /** @@ -315,7 +324,9 @@ public void testRecoveringOnCheckpointWriteFail() throws Exception { public void testRecoveringOnWALWritingFail1() throws Exception { // Allow to allocate only 1 wal segment, fail on write to second. ioFactory = new FilteringFileIOFactory(".wal", new LimitedSizeFileIOFactory(new RandomAccessFileIOFactory(), WAL_SEGMENT_SIZE)); + System.setProperty(IGNITE_WAL_MMAP, "true"); + doTestRecoveringOnWALWritingFail(); } @@ -325,7 +336,9 @@ public void testRecoveringOnWALWritingFail1() throws Exception { public void testRecoveringOnWALWritingFail2() throws Exception { // Fail somewhere on the second wal segment. ioFactory = new FilteringFileIOFactory(".wal", new LimitedSizeFileIOFactory(new RandomAccessFileIOFactory(), (long) (1.5 * WAL_SEGMENT_SIZE))); + System.setProperty(IGNITE_WAL_MMAP, "false"); + doTestRecoveringOnWALWritingFail(); } @@ -333,18 +346,23 @@ public void testRecoveringOnWALWritingFail2() throws Exception { * Test node stopping & recovery on WAL writing fail. */ private void doTestRecoveringOnWALWritingFail() throws Exception { - final IgniteEx grid = startGrid(0); + IgniteEx grid = startGrid(0); FileWriteAheadLogManager wal = (FileWriteAheadLogManager)grid.context().cache().context().wal(); + wal.setFileIOFactory(ioFactory); grid.cluster().active(true); int failedPosition = -1; - for (int i = 0; i < 1000; i++) { + final int keysCount = 2000; + + final int dataSize = 2048; + + for (int i = 0; i < keysCount; i++) { byte payload = (byte) i; - byte[] data = new byte[2048]; + byte[] data = new byte[dataSize]; Arrays.fill(data, payload); try { @@ -358,7 +376,7 @@ private void doTestRecoveringOnWALWritingFail() throws Exception { } // We must be able to put something into cache before fail. - Assert.assertTrue(failedPosition > 0); + Assert.assertTrue("One of the cache puts must be failed", failedPosition > 0); // Grid should be automatically stopped after WAL fail. awaitStop(grid); @@ -366,15 +384,16 @@ private void doTestRecoveringOnWALWritingFail() throws Exception { ioFactory = null; // Grid should be successfully recovered after stopping. - IgniteEx recoveredGrid = startGrid(0); - recoveredGrid.cluster().active(true); + grid = startGrid(0); + + grid.cluster().active(true); for (int i = 0; i < failedPosition; i++) { byte payload = (byte) i; - byte[] data = new byte[2048]; + byte[] data = new byte[dataSize]; Arrays.fill(data, payload); - byte[] actualData = (byte[]) recoveredGrid.cache(CACHE_NAME).get(i); + byte[] actualData = (byte[]) grid.cache(CACHE_NAME).get(i); Assert.assertArrayEquals(data, actualData); } } @@ -473,11 +492,6 @@ private static class FilteringFileIOFactory implements FileIOFactory { this.pattern = pattern; } - /** {@inheritDoc} */ - @Override public FileIO create(File file) throws IOException { - return create(file, CREATE, WRITE, READ); - } - /** {@inheritDoc} */ @Override public FileIO create(File file, OpenOption... modes) throws IOException { if (file.getName().endsWith(pattern)) @@ -508,11 +522,6 @@ private LimitedSizeFileIOFactory(FileIOFactory delegate, long fsSpaceBytes) { this.availableSpaceBytes = new AtomicLong(fsSpaceBytes); } - /** {@inheritDoc} */ - @Override public FileIO create(File file) throws IOException { - return new LimitedSizeFileIO(delegate.create(file), availableSpaceBytes); - } - /** {@inheritDoc} */ @Override public FileIO create(File file, OpenOption... modes) throws IOException { return new LimitedSizeFileIO(delegate.create(file, modes), availableSpaceBytes); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteNodeStoppedDuringDisableWALTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteNodeStoppedDuringDisableWALTest.java index a744ab10c1671..906f19199a4b7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteNodeStoppedDuringDisableWALTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteNodeStoppedDuringDisableWALTest.java @@ -48,7 +48,6 @@ import static java.nio.file.FileVisitResult.CONTINUE; import static java.nio.file.Files.walkFileTree; import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.CP_FILE_NAME_PATTERN; - import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.META_STORAGE_NAME; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX; @@ -92,6 +91,8 @@ public class IgniteNodeStoppedDuringDisableWALTest extends GridCommonAbstractTes } /** + * Test checks that after WAL is globally disabled and node is stopped, persistent store is cleaned properly after node restart. + * * @throws Exception If failed. */ public void test() throws Exception { @@ -190,6 +191,8 @@ private void testStopNodeWithDisableWAL(NodeStopPoint nodeStopPoint) throws Exce String msg = nodeStopPoint.toString(); + int pageSize = ig1.configuration().getDataStorageConfiguration().getPageSize(); + if (nodeStopPoint.needCleanUp) { PdsFoldersResolver foldersResolver = ((IgniteEx)ig1).context().pdsFolderResolver(); @@ -215,14 +218,14 @@ private void testStopNodeWithDisableWAL(NodeStopPoint nodeStopPoint) throws Exce if (CP_FILE_NAME_PATTERN.matcher(name).matches()) failed = true; - if (name.startsWith(PART_FILE_PREFIX)) + if (name.startsWith(PART_FILE_PREFIX) && path.toFile().length() > pageSize) failed = true; - if (name.startsWith(INDEX_FILE_NAME)) + if (name.startsWith(INDEX_FILE_NAME) && path.toFile().length() > pageSize) failed = true; if (failed) - fail(msg + " " + filePath); + fail(msg + " " + filePath + " " + path.toFile().length()); return CONTINUE; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java index de654f11a46e9..e6e72e0494a97 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java @@ -184,11 +184,6 @@ private static class FailingFileIOFactory implements FileIOFactory { this.fail = fail; } - /** {@inheritDoc} */ - @Override public FileIO create(File file) throws IOException { - return create(file, CREATE, READ, WRITE); - } - /** {@inheritDoc} */ @Override public FileIO create(File file, OpenOption... modes) throws IOException { final FileIO delegate = delegateFactory.create(file, modes); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java index a28ec5fe1e74c..0134e6974291f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java @@ -256,11 +256,6 @@ private static class FailingFileIOFactory implements FileIOFactory { this.fail = fail; } - /** {@inheritDoc} */ - @Override public FileIO create(File file) throws IOException { - return create(file, CREATE, READ, WRITE); - } - /** {@inheritDoc} */ @Override public FileIO create(File file, OpenOption... modes) throws IOException { final FileIO delegate = delegateFactory.create(file, modes); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFormatFileFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFormatFileFailoverTest.java index 5a1a6fa179654..a77ab989a939b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFormatFileFailoverTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFormatFileFailoverTest.java @@ -112,6 +112,8 @@ public void testNodeStartFailedFsync() throws Exception { * @throws Exception If failed. */ public void testFailureHandlerTriggeredFsync() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-10035"); + fsync = true; failFormatFileOnClusterActivate(); @@ -121,6 +123,8 @@ public void testFailureHandlerTriggeredFsync() throws Exception { * @throws Exception If failed. */ public void testFailureHandlerTriggered() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-10035"); + fsync = false; failFormatFileOnClusterActivate(); @@ -224,11 +228,6 @@ private static class FailingFileIOFactory implements FileIOFactory { this.failMtdNameRef = failMtdNameRef; } - /** {@inheritDoc} */ - @Override public FileIO create(File file) throws IOException { - return create(file, CREATE, READ, WRITE); - } - /** {@inheritDoc} */ @Override public FileIO create(File file, OpenOption... modes) throws IOException { final FileIO delegate = delegateFactory.create(file, modes); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java index 6d2b1f79227b5..4c34093a10544 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java @@ -114,7 +114,7 @@ public void testReservedOnExchange() throws Exception { final IgniteEx ig0 = (IgniteEx)startGrids(initGridCnt + 1); - ig0.active(true); + ig0.cluster().active(true); stopGrid(initGridCnt); @@ -256,7 +256,7 @@ public void testRemovesArePreloadedIfHistoryIsAvailable() throws Exception { IgniteEx ig0 = (IgniteEx)startGrids(2); - ig0.active(true); + ig0.cluster().active(true); IgniteCache cache = ig0.cache("cache1"); @@ -270,6 +270,8 @@ public void testRemovesArePreloadedIfHistoryIsAvailable() throws Exception { IgniteEx ig1 = startGrid(1); + awaitPartitionMapExchange(); + IgniteCache cache1 = ig1.cache("cache1"); assertEquals(entryCnt / 2, cache.size()); @@ -286,8 +288,6 @@ public void testRemovesArePreloadedIfHistoryIsAvailable() throws Exception { } } - cache.rebalance().get(); - for (int p = 0; p < ig1.affinity("cache1").partitions(); p++) { GridDhtLocalPartition p0 = ig0.context().cache().cache("cache1").context().topology().localPartition(p); GridDhtLocalPartition p1 = ig1.context().cache().cache("cache1").context().topology().localPartition(p); @@ -305,7 +305,7 @@ public void testNodeIsClearedIfHistoryIsUnavailable() throws Exception { IgniteEx ig0 = (IgniteEx)startGrids(2); - ig0.active(true); + ig0.cluster().active(true); IgniteCache cache = ig0.cache("cache1"); @@ -330,6 +330,8 @@ public void testNodeIsClearedIfHistoryIsUnavailable() throws Exception { IgniteEx ig1 = startGrid(1); + awaitPartitionMapExchange(); + IgniteCache cache1 = ig1.cache("cache1"); assertEquals(entryCnt / 2, cache.size()); @@ -346,8 +348,6 @@ public void testNodeIsClearedIfHistoryIsUnavailable() throws Exception { } } - cache.rebalance().get(); - for (int p = 0; p < ig1.affinity("cache1").partitions(); p++) { GridDhtLocalPartition p0 = ig0.context().cache().cache("cache1").context().topology().localPartition(p); GridDhtLocalPartition p1 = ig1.context().cache().cache("cache1").context().topology().localPartition(p); @@ -407,7 +407,7 @@ public void testNodeLeftDuringExchange() throws Exception { final Ignite ig0 = startGrids(initGridCnt); - ig0.active(true); + ig0.cluster().active(true); IgniteCache cache = ig0.cache("cache1"); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java index 57565bfd58c48..0409a297be1b1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java @@ -612,11 +612,6 @@ static class FailingIOFactory implements FileIOFactory { this.delegate = delegate; } - /** {@inheritDoc} */ - @Override public FileIO create(File file) throws IOException { - return create(file, CREATE, WRITE, READ); - } - /** {@inheritDoc} */ @Override public FileIO create(File file, OpenOption... modes) throws IOException { FileIO delegateIO = delegate.create(file, modes); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRecoveryTxLogicalRecordsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRecoveryTxLogicalRecordsTest.java index 261167a7d6170..d2ea4f114d372 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRecoveryTxLogicalRecordsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalRecoveryTxLogicalRecordsTest.java @@ -49,8 +49,8 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager; import org.apache.ignite.internal.processors.cache.IgniteRebalanceIterator; -import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtDemandedPartitionsMap; +import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; @@ -584,6 +584,8 @@ public void testRecoveryNoPageLost2() throws Exception { } /** + * Test checks that the number of pages per each page store are not changing before and after node restart. + * * @throws Exception If failed. */ public void testRecoveryNoPageLost3() throws Exception { @@ -679,7 +681,7 @@ private void recoveryNoPageLost(boolean checkpoint) throws Exception { pages = allocatedPages(ignite, CACHE2_NAME); - ignite.close(); + stopGrid(0, true); } } finally { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java index 7ff134850387b..b05d5db75a197 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java @@ -287,11 +287,6 @@ private static class SlowCheckpointFileIOFactory implements FileIOFactory { /** Delegate factory. */ private final FileIOFactory delegateFactory = new RandomAccessFileIOFactory(); - /** {@inheritDoc} */ - @Override public FileIO create(File file) throws IOException { - return create(file, CREATE, READ, WRITE); - } - /** {@inheritDoc} */ @Override public FileIO create(File file, OpenOption... openOption) throws IOException { final FileIO delegate = delegateFactory.create(file, openOption); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java index 219db8d166433..a888ecb2607c8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java @@ -786,7 +786,9 @@ public Runnable checkCacheEmpty() { Map caches = caches(ig); - Assert.assertEquals(0, caches.size()); + for (GridCacheAdapter cacheAdapter : caches.values()) + Assert.assertTrue("Cache should be in recovery mode: " + cacheAdapter.context(), + cacheAdapter.context().isRecoveryMode()); } }); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalDeltaConsistencyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalDeltaConsistencyTest.java index 7f872fd005cb7..e0057b0c75ed1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalDeltaConsistencyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalDeltaConsistencyTest.java @@ -76,6 +76,8 @@ protected boolean checkPagesOnCheckpoint() { protected DataStorageConfiguration getDataStorageConfiguration() { return new DataStorageConfiguration() .setDefaultDataRegionConfiguration(new DataRegionConfiguration() + .setInitialSize(256 * 1024 * 1024) + .setMaxSize(256 * 1024 * 1024) .setPersistenceEnabled(true) .setName("dflt-plc")); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/ExplicitWalDeltaConsistencyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/ExplicitWalDeltaConsistencyTest.java index 1b9a18a8de1e6..e9512e111ec0f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/ExplicitWalDeltaConsistencyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/ExplicitWalDeltaConsistencyTest.java @@ -72,6 +72,8 @@ public final void testNotEmptyPds() throws Exception { for (int i = 0; i < 3_000; i++) cache.put(i, "Cache value " + i); + forceCheckpoint(); + stopGrid(0); ignite = startGrid(0); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/memtracker/PageMemoryTracker.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/memtracker/PageMemoryTracker.java index 09cacfbb2c5cc..864d2cc0137e8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/memtracker/PageMemoryTracker.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/memtracker/PageMemoryTracker.java @@ -475,18 +475,7 @@ else if (record instanceof InitNewPageRecord) return; // Increment statistics. - AtomicInteger statCnt = stats.get(record.type()); - - if (statCnt == null) { - statCnt = new AtomicInteger(); - - AtomicInteger oldCnt = stats.putIfAbsent(record.type(), statCnt); - - if (oldCnt != null) - statCnt = oldCnt; - } - - statCnt.incrementAndGet(); + stats.computeIfAbsent(record.type(), r -> new AtomicInteger()).incrementAndGet(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/memtracker/PageMemoryTrackerPluginProvider.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/memtracker/PageMemoryTrackerPluginProvider.java index c5f83b5ef68dd..cad3b0269dd75 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/memtracker/PageMemoryTrackerPluginProvider.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/memtracker/PageMemoryTrackerPluginProvider.java @@ -20,12 +20,16 @@ import java.io.Serializable; import java.util.UUID; import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager; import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; +import org.apache.ignite.internal.processors.cache.persistence.DatabaseLifecycleListener; +import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.plugin.CachePluginContext; @@ -43,7 +47,7 @@ * PageMemory tracker plugin provider. */ public class PageMemoryTrackerPluginProvider implements PluginProvider, - IgniteChangeGlobalStateSupport { + IgniteChangeGlobalStateSupport, DatabaseLifecycleListener { /** System property name to implicitly enable page memory tracker . */ public static final String IGNITE_ENABLE_PAGE_MEMORY_TRACKER = "IGNITE_ENABLE_PAGE_MEMORY_TRACKER"; @@ -132,7 +136,7 @@ else if (IgnitePageStoreManager.class.equals(cls)) /** {@inheritDoc} */ @Override public void start(PluginContext ctx) { - // No-op + ((IgniteEx)ctx.grid()).context().internalSubscriptionProcessor().registerDatabaseListener(this); } /** {@inheritDoc} */ @@ -197,4 +201,15 @@ public static PageMemoryTracker tracker(Ignite ignite) { return null; } } + + @Override public void beforeBinaryMemoryRestore(IgniteCacheDatabaseSharedManager mgr) throws IgniteCheckedException { + if (plugin != null) { + try { + plugin.start(); + } + catch (Exception e) { + log.error("Can't start plugin", e); + } + } + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIteratorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIteratorTest.java index 4d2bdcfcc82fb..6aa2a33e77142 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIteratorTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIteratorTest.java @@ -161,11 +161,6 @@ private String getArchiveWalDirPath(Ignite ignite) throws IgniteCheckedException * */ private static class CountedFileIOFactory extends RandomAccessFileIOFactory { - /** {@inheritDoc} */ - @Override public FileIO create(File file) throws IOException { - return create(file, StandardOpenOption.CREATE, StandardOpenOption.READ, StandardOpenOption.WRITE); - } - /** {@inheritDoc} */ @Override public FileIO create(File file, OpenOption... modes) throws IOException { return new CountedFileIO(file, modes); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TransactionIntegrityWithPrimaryIndexCorruptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TransactionIntegrityWithPrimaryIndexCorruptionTest.java index 473eaf5ce97bc..fd31ee5e78f56 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TransactionIntegrityWithPrimaryIndexCorruptionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TransactionIntegrityWithPrimaryIndexCorruptionTest.java @@ -181,9 +181,6 @@ class IndexCorruptionFailoverScenario implements FailoverScenario { return hnd; if (treeCorruptionPred.apply(hnd, tree)) { - log.info("Created corrupted tree handler [nodeOrder=" + locIgnite.localNode().order() + ", hnd=" + hnd + - ", tree=" + tree + ']'); - PageHandler delegate = (PageHandler)hnd; return new PageHandler() { 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 c8cf9aae7bb08..4159f6ed742a4 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 @@ -90,6 +90,7 @@ public GridCacheTestContext(GridTestKernalContext ctx) throws Exception { AffinityTopologyVersion.ZERO, true, true, + false, new GridCacheEventManager(), new CacheOsStoreManager(null, new CacheConfiguration()), new GridCacheEvictionManager(), diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java index 057087e0b369b..af573fe81bb6d 100755 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java @@ -184,9 +184,6 @@ public abstract class GridAbstractTest extends TestCase { /** Timestamp for tests. */ private static long ts = System.currentTimeMillis(); - /** Starting Ignite instance name. */ - protected static final ThreadLocal startingIgniteInstanceName = new ThreadLocal<>(); - /** Force failure flag. */ private boolean forceFailure; @@ -923,7 +920,7 @@ protected Ignite startGrid(String igniteInstanceName, IgniteConfiguration cfg, G checkConfiguration(cfg); if (!isRemoteJvm(igniteInstanceName)) { - startingIgniteInstanceName.set(igniteInstanceName); + IgniteUtils.setCurrentIgniteName(igniteInstanceName); try { String cfgProcClsName = System.getProperty(IGNITE_CFG_PREPROCESSOR_CLS); @@ -959,7 +956,7 @@ protected Ignite startGrid(String igniteInstanceName, IgniteConfiguration cfg, G return node; } finally { - startingIgniteInstanceName.set(null); + IgniteUtils.setCurrentIgniteName(null); } } else diff --git a/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIOFactory.java b/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIOFactory.java index 8a28e9eac1c47..ea555a0c583f8 100644 --- a/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIOFactory.java +++ b/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIOFactory.java @@ -144,11 +144,6 @@ public AlignedBuffersDirectFileIOFactory( return allocate; } - /** {@inheritDoc} */ - @Override public FileIO create(File file) throws IOException { - return create(file, CREATE, READ, WRITE); - } - /** {@inheritDoc} */ @Override public FileIO create(File file, OpenOption... modes) throws IOException { if (useBackupFactory) diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java index e9159bf7cd553..214a541a17214 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java @@ -27,6 +27,7 @@ import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsTxHistoricalRebalancingTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePersistentStoreCacheGroupsTest; import org.apache.ignite.internal.processors.cache.persistence.PersistenceDirectoryWarningLoggingTest; +import org.apache.ignite.internal.processors.cache.persistence.db.IgniteLogicalRecoveryTest; import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsMultiNodePutGetRestartTest; import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsPageEvictionTest; import org.apache.ignite.internal.processors.cache.persistence.db.file.IgnitePdsCacheDestroyDuringCheckpointTest; @@ -88,6 +89,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgnitePdsCorruptedIndexTest.class); + suite.addTestSuite(IgniteLogicalRecoveryTest.class); + return suite; } }