diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index 66c381185c2fe..e185f8dcfbf27 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -745,6 +745,16 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_PDS_WAL_REBALANCE_THRESHOLD = "IGNITE_PDS_WAL_REBALANCE_THRESHOLD"; + /** + * Flag to enable file rebalancing. + */ + public static final String IGNITE_FILE_REBALANCE_ENABLED = "IGNITE_FILE_REBALANCE_ENABLED"; + + /** + * File rebalancing is applicable only if the number of entries in the partition is greater than this threshold. + */ + public static final String IGNITE_FILE_REBALANCE_THRESHOLD = "IGNITE_PDS_FILE_REBALANCE_THRESHOLD"; + /** Ignite page memory concurrency level. */ public static final String IGNITE_OFFHEAP_LOCK_CONCURRENCY_LEVEL = "IGNITE_OFFHEAP_LOCK_CONCURRENCY_LEVEL"; diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java index a705e496feacf..a4bfcaee31ce9 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java @@ -81,6 +81,9 @@ public class DataStorageConfiguration implements Serializable { (long)(DFLT_DATA_REGION_FRACTION * U.getTotalMemoryAvailable()), DFLT_DATA_REGION_INITIAL_SIZE); + /** Default working directory for a snapshot final result. */ + public static final String DFLT_LOCAL_SNAPSHOT_DIRECTORY = "snapshots"; + /** Default initial size of a memory chunk for the system cache (40 MB). */ private static final long DFLT_SYS_REG_INIT_SIZE = 40L * 1024 * 1024; @@ -159,6 +162,9 @@ public class DataStorageConfiguration implements Serializable { /** Default wal archive directory. */ public static final String DFLT_WAL_ARCHIVE_PATH = "db/wal/archive"; + /** Default working directory for backup temporary files. */ + public static final String DFLT_BACKUP_DIRECTORY = "db/backup"; + /** Default write throttling enabled. */ public static final boolean DFLT_WRITE_THROTTLING_ENABLED = false; @@ -194,6 +200,9 @@ public class DataStorageConfiguration implements Serializable { /** Directory where index and partition files are stored. */ private String storagePath; + /** Directory where will be stored all results of snapshot operations. */ + private String locSnapshotPath = DFLT_LOCAL_SNAPSHOT_DIRECTORY; + /** Checkpoint frequency. */ private long checkpointFreq = DFLT_CHECKPOINT_FREQ; @@ -472,6 +481,24 @@ public DataStorageConfiguration setStoragePath(String persistenceStorePath) { return this; } + /** + * @return Relative path where will be stored all local snapshot results. + */ + public String getLocalSnapshotPath() { + return locSnapshotPath; + } + + /** + * @param locSnapshotPath Relative path to store all local snapshot results. By default + * {@code DFLT_LOCAL_SNAPSHOT_DIRECTORY} is used. + * @return Data storage configurations for chaining. + */ + public DataStorageConfiguration setLocalSnapshotPath(String locSnapshotPath) { + this.locSnapshotPath = locSnapshotPath; + + return this; + } + /** * Gets checkpoint frequency. * diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java index 313d462e35d83..93c3b7719c5cc 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java @@ -176,6 +176,9 @@ public class IgniteConfiguration { /** Default rebalance batch size in bytes (value is {@code 512Kb}). */ public static final int DFLT_REBALANCE_BATCH_SIZE = 512 * 1024; // 512K + /** Default WAL rebalance threshold (value is 500 000 cache entries). */ + public static final long DFLT_PDS_WAL_REBALANCE_THRESHOLD = 500_000; + /** Default size of system thread pool. */ public static final int DFLT_SYSTEM_CORE_THREAD_CNT = DFLT_PUBLIC_THREAD_CNT; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java index 0bba3f2c77ca4..0149d8affa9e2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java @@ -99,7 +99,10 @@ public enum IgniteFeatures { PERSISTENCE_CACHE_SNAPSHOT(23), /** Long operations dump timeout. */ - LONG_OPERATIONS_DUMP_TIMEOUT(30); + LONG_OPERATIONS_DUMP_TIMEOUT(30), + + /** */ + CACHE_PARTITION_FILE_REBALANCE(24); /** * Unique feature identifier. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractTransmission.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractTransmission.java index bd1da546c0cf5..aed45f47b4737 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractTransmission.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/AbstractTransmission.java @@ -57,9 +57,9 @@ protected AbstractTransmission( int chunkSize ) { A.notNull(meta, "Initial file meta cannot be null"); - A.notNullOrEmpty(meta.name(), "Trasmisson name cannot be empty or null"); + A.notNullOrEmpty(meta.name(), "Transmission name cannot be empty or null"); A.ensure(meta.offset() >= 0, "File start position cannot be negative"); - A.ensure(meta.count() > 0, "Total number of bytes to transfer must be greater than zero"); + A.ensure(meta.count() >= 0, "Total number of bytes to transfer can't be less than zero"); A.notNull(stopChecker, "Process stop checker cannot be null"); A.ensure(chunkSize > 0, "Size of chunks to transfer data must be positive"); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java index 6af3ca4286f1e..c826e4de574a8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/FileReceiver.java @@ -82,7 +82,8 @@ public FileReceiver( fileIo.position(meta.offset()); } catch (IOException e) { - throw new IgniteException("Unable to open destination file. Receiver will will be stopped", e); + throw new IgniteException("Unable to open destination file. Receiver will be stopped: " + + file.getAbsolutePath(), e); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 3ece5c9fb2c84..ce69c4e4c5e1a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -257,6 +257,9 @@ public class GridIoManager extends GridManagerAdapter CUR_PLC = new ThreadLocal<>(); @@ -2711,9 +2714,15 @@ private void interruptRecevier(ReceiverContext rctx, Exception ex) { rctx.lastState = rctx.lastState == null ? new TransmissionMeta(ex) : rctx.lastState.error(ex); - rctx.hnd.onException(rctx.rmtNodeId, ex); + if (X.hasCause(ex, TransmissionCancelledException.class)) { + if (log.isInfoEnabled()) + log.info("Transmission receiver has been cancelled [rctx=" + rctx + ']'); + } + else { + rctx.hnd.onException(rctx.rmtNodeId, ex); - U.error(log, "Receiver has been interrupted due to an exception occurred [ctx=" + rctx + ']', ex); + U.error(log, "Receiver has been interrupted due to an exception occurred [rctx=" + rctx + ']', ex); + } } } @@ -2767,7 +2776,8 @@ private void processOpenedChannel(Object topic, UUID rmtNodeId, SessionChannelMe "It's not allowed to process different sessions over the same topic simultaneously. " + "Channel will be closed [initMsg=" + initMsg + ", channel=" + ch + ", nodeId=" + rmtNodeId + ']'); - U.error(log, err); + U.error(log, "Error has been sent back to remote node. Receiver holds the local topic " + + "[topic=" + topic + ", rmtNodeId=" + rmtNodeId + ", ctx=" + rcvCtx + ']', err); out.writeObject(new TransmissionMeta(err)); @@ -2799,8 +2809,6 @@ private void processOpenedChannel(Object topic, UUID rmtNodeId, SessionChannelMe } } catch (Throwable t) { - U.error(log, "Download session cannot be finished due to an unexpected error [ctx=" + rcvCtx + ']', t); - // Do not remove receiver context here, since sender will recconect to get this error. interruptRecevier(rcvCtx, new IgniteCheckedException("Channel processing error [nodeId=" + rmtNodeId + ']', t)); } @@ -2898,7 +2906,7 @@ private void receiveFromChannel( } @Override public long endTime() { - return startTs + netTimeoutMs; + return startTs + CHANNEL_SESSION_TIMEOUT_MS; } @Override public void onTimeout() { @@ -3174,6 +3182,13 @@ private TransmissionMeta connect() throws IgniteCheckedException, IOException { return syncMeta; } + /** + * @return {@code true} if sender's session is currently active. + */ + public boolean opened() { + return senderStopFlags.containsKey(sesKey); + } + /** * @param file Source file to send to remote. * @param params Additional file params. @@ -3312,6 +3327,11 @@ public void send( if (stopping) throw new NodeStoppingException("Operation has been cancelled (node is stopping)"); + if (senderStopFlags.get(sesKey) == null) + e.printStackTrace(); + + assert senderStopFlags.get(sesKey) != null : "key=" + sesKey + ", flags=" + senderStopFlags.keySet(); + if (senderStopFlags.get(sesKey).get()) throw new ClusterTopologyCheckedException("Remote node left the cluster: " + rmtId, e); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java index 0d7976e877fdf..038126d03c7f7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java @@ -138,6 +138,8 @@ import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccTxSnapshotRequest; import org.apache.ignite.internal.processors.cache.mvcc.msg.PartitionCountersNeighborcastRequest; import org.apache.ignite.internal.processors.cache.mvcc.msg.PartitionCountersNeighborcastResponse; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRequestMessage; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotResponseMessage; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryRequest; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryResponse; import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery; @@ -401,6 +403,8 @@ public class GridIoMessageFactory implements MessageFactoryProvider { factory.register(GridIoSecurityAwareMessage.TYPE_CODE, GridIoSecurityAwareMessage::new); factory.register(SessionChannelMessage.TYPE_CODE, SessionChannelMessage::new); factory.register(SingleNodeMessage.TYPE_CODE, SingleNodeMessage::new); + factory.register(SnapshotRequestMessage.TYPE_CODE, SnapshotRequestMessage::new); + factory.register(SnapshotResponseMessage.TYPE_CODE, SnapshotResponseMessage::new); // [-3..119] [124..129] [-23..-28] [-36..-55] - this // [120..123] - DR diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java index 243d99d93e46f..5094013c00ff8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.pagemem.store; +import java.io.File; import java.nio.ByteBuffer; import java.util.Map; import java.util.function.Predicate; @@ -117,9 +118,8 @@ public void initializeForCache(CacheGroupDescriptor grpDesc, StoredCacheData cac * @param grpId Cache group ID. * @param partId Partition ID. * @return {@code True} if partition store exists. - * @throws IgniteCheckedException If failed. */ - public boolean exists(int grpId, int partId) throws IgniteCheckedException; + public boolean exists(int grpId, int partId); /** * Reads a header of a page store. @@ -266,4 +266,14 @@ public void initializeForCache(CacheGroupDescriptor grpDesc, StoredCacheData cac * @throws IgniteCheckedException If failed. */ public boolean checkAndInitCacheWorkDir(CacheConfiguration cacheCfg) throws IgniteCheckedException; + + /** + * Restore partition file. + * + * @param grpId Cache group ID. + * @param partId Partition ID. + * @param src Partition file. + * @throws IgniteCheckedException If the page store doesn't exists or the file couldn't be moved. + */ + public void restore(int grpId, int partId, File src) throws IgniteCheckedException; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/IgniteWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/IgniteWriteAheadLogManager.java index dee9c5a704881..f7fd815c89309 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/IgniteWriteAheadLogManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/IgniteWriteAheadLogManager.java @@ -119,6 +119,7 @@ public WALPointer log(WALRecord entry, RolloverType rolloverType) * Invoke this method to iterate over the written log entries. * * @param start Optional WAL pointer from which to start iteration. + * @param limited Limit iterator to current segment. * @param recordDeserializeFilter Specify a filter to skip WAL records. Those records will not be explicitly deserialized. * @return Records iterator. * @throws IgniteException If failed to start iteration. @@ -126,6 +127,7 @@ public WALPointer log(WALRecord entry, RolloverType rolloverType) */ public WALIterator replay( WALPointer start, + boolean limited, @Nullable IgniteBiPredicate recordDeserializeFilter ) throws IgniteCheckedException, StorageException; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index b1ebd2cbec17c..54cb0e6ea870c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java @@ -104,7 +104,6 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridClientPartitionTopology; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; -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.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; @@ -1026,6 +1025,11 @@ public AffinityTopologyVersion rebalanceTopologyVersion() { return rebTopVer; } + /** */ + public void resetRebalanceVersion() { + rebTopVer = NONE; + } + /** * @return Last initialized topology future. */ @@ -3185,7 +3189,7 @@ private void body0() throws InterruptedException, IgniteCheckedException { busy = true; - Map assignsMap = null; + Map> assignsMap = null; boolean forcePreload = false; @@ -3341,9 +3345,6 @@ else if (task instanceof ForceRebalanceExchangeTask) { if (grp.isLocal()) continue; - if (grp.preloader().rebalanceRequired(rebTopVer, exchFut)) - rebTopVer = NONE; - changed |= grp.topology().afterExchange(exchFut); } @@ -3357,36 +3358,14 @@ else if (task instanceof ForceRebalanceExchangeTask) { // Schedule rebalance if force rebalance or force reassign occurs. if (exchFut == null) - rebTopVer = NONE; + resetRebalanceVersion();; if (!cctx.kernalContext().clientNode() && rebTopVer.equals(NONE)) { if (rebalanceDelay > 0) U.sleep(rebalanceDelay); - assignsMap = new HashMap<>(); - - IgniteCacheSnapshotManager snp = cctx.snapshot(); - - for (final CacheGroupContext grp : cctx.cache().cacheGroups()) { - long delay = grp.config().getRebalanceDelay(); - - boolean disableRebalance = snp.partitionsAreFrozen(grp); - - GridDhtPreloaderAssignments assigns = null; - - // Don't delay for dummy reassigns to avoid infinite recursion. - if ((delay == 0 || forcePreload) && !disableRebalance) - assigns = grp.preloader().generateAssignments(exchId, exchFut); - - assignsMap.put(grp.groupId(), assigns); - - if (resVer == null && !grp.isLocal()) - resVer = grp.topology().readyTopologyVersion(); - } + assignsMap = cctx.preloader().generateAssignments(exchId, exchFut, cnt, forcePreload); } - - if (resVer == null) - resVer = exchId.topologyVersion(); } finally { // Must flip busy flag before assignments are given to demand workers. @@ -3398,9 +3377,7 @@ else if (task instanceof ForceRebalanceExchangeTask) { NavigableMap> orderMap = new TreeMap<>(); - for (Map.Entry e : assignsMap.entrySet()) { - int grpId = e.getKey(); - + for (Integer grpId : assignsMap.keySet()) { CacheGroupContext grp = cctx.cache().cacheGroup(grpId); CacheRebalanceOrder order = new CacheRebalanceOrder( @@ -3411,14 +3388,18 @@ else if (task instanceof ForceRebalanceExchangeTask) { orderMap.put(order, new ArrayList(size)); orderMap.get(order).add(grpId); + + if (resVer == null && !grp.isLocal()) + resVer = grp.topology().readyTopologyVersion(); } + if (resVer == null) + resVer = exchId.topologyVersion(); + Runnable r = null; List rebList = new LinkedList<>(); - boolean assignsCancelled = false; - GridCompoundFuture forcedRebFut = null; if (task instanceof ForceRebalanceExchangeTask) @@ -3428,12 +3409,11 @@ else if (task instanceof ForceRebalanceExchangeTask) { for (Integer grpId : orderMap.get(order)) { CacheGroupContext grp = cctx.cache().cacheGroup(grpId); - GridDhtPreloaderAssignments assigns = assignsMap.get(grpId); + IgniteInternalFuture fut = assignsMap.get(grpId); - if (assigns != null) - assignsCancelled |= assigns.cancelled(); + assert fut != null : "grp=" + grp.cacheOrGroupName(); - Runnable cur = grp.preloader().addAssignments(assigns, + Runnable cur = grp.preloader().addAssignments(fut, forcePreload, cnt, r, @@ -3450,7 +3430,7 @@ else if (task instanceof ForceRebalanceExchangeTask) { if (forcedRebFut != null) forcedRebFut.markInitialized(); - if (assignsCancelled || hasPendingServerExchange()) { + if (hasPendingServerExchange()) { U.log(log, "Skipping rebalancing (obsolete exchange ID) " + "[top=" + resVer + ", evt=" + exchId.discoveryEventName() + ", node=" + exchId.nodeId() + ']'); @@ -3478,7 +3458,7 @@ else if (r != null) { } else U.log(log, "Skipping rebalancing (no affinity changes) " + - "[top=" + resVer + + "[top=" + resVer != null ? resVer : exchId.topologyVersion() + ", rebTopVer=" + rebTopVer + ", evt=" + exchId.discoveryEventName() + ", evtNode=" + exchId.nodeId() + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java index 1b6aca103d055..33f622091a6af 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java @@ -65,11 +65,13 @@ public interface GridCachePreloader { public void onInitialExchangeComplete(@Nullable Throwable err); /** - * @param rebTopVer Previous rebalance topology version or {@code NONE} if there is no info. - * @param exchFut Completed exchange future. - * @return {@code True} if rebalance should be started (previous will be interrupted). + * Set rebalance version to {@link AffinityTopologyVersion#NONE} if rebalancing should be restarted. + * + * @param exchFut Exchange future. + * @param resVer Exchange result version. + * @return {@code True} if the version has been updated or it's already set to {@link AffinityTopologyVersion#NONE}. */ - public boolean rebalanceRequired(AffinityTopologyVersion rebTopVer, GridDhtPartitionsExchangeFuture exchFut); + public boolean updateRebalanceVersion(GridDhtPartitionsExchangeFuture exchFut, AffinityTopologyVersion resVer); /** * @param exchId Exchange ID. @@ -89,7 +91,7 @@ public interface GridCachePreloader { * @param forcedRebFut External future for forced rebalance. * @return Rebalancing runnable. */ - public Runnable addAssignments(GridDhtPreloaderAssignments assignments, + public Runnable addAssignments(IgniteInternalFuture assignments, boolean forcePreload, long rebalanceId, Runnable next, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java index ff5e321727c91..2abb95b1e9bfa 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java @@ -138,8 +138,7 @@ public GridCachePreloaderAdapter(CacheGroupContext grp) { } /** {@inheritDoc} */ - @Override public boolean rebalanceRequired(AffinityTopologyVersion rebTopVer, - GridDhtPartitionsExchangeFuture exchFut) { + @Override public boolean updateRebalanceVersion(GridDhtPartitionsExchangeFuture fut, AffinityTopologyVersion ver) { return true; } @@ -150,7 +149,7 @@ public GridCachePreloaderAdapter(CacheGroupContext grp) { } /** {@inheritDoc} */ - @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments, + @Override public Runnable addAssignments(IgniteInternalFuture assignments, boolean forcePreload, long rebalanceId, Runnable next, 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 326f24d1f012b..eb83a3608b997 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 @@ -86,6 +86,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache; import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedCache; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgnitePartitionPreloadManager; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.StopCachesOnClientReconnectExchangeTask; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.PartitionDefferedDeleteQueueCleanupTask; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.PartitionsEvictManager; @@ -196,6 +197,7 @@ import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isNearEnabled; import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isPersistentCache; import static org.apache.ignite.internal.processors.cache.ValidationOnNodeJoinUtils.validateHashIdResolvers; +import static org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager.INTERNAL_DATA_REGION_NAMES; import static org.apache.ignite.internal.util.IgniteUtils.doInParallel; /** @@ -2959,6 +2961,7 @@ private GridCacheSharedContext createSharedContext( IgniteCacheDatabaseSharedManager dbMgr; IgnitePageStoreManager pageStoreMgr = null; IgniteWriteAheadLogManager walMgr = null; + IgnitePartitionPreloadManager preloadMgr = null; if (CU.isPersistenceEnabled(ctx.config()) && !ctx.clientNode()) { dbMgr = new GridCacheDatabaseSharedManager(ctx); @@ -2982,6 +2985,9 @@ private GridCacheSharedContext createSharedContext( dbMgr = new IgniteCacheDatabaseSharedManager(); } + if (!ctx.clientNode()) + preloadMgr = new IgnitePartitionPreloadManager(); + WalStateManager walStateMgr = new WalStateManager(ctx); IgniteSnapshotManager snapshotMgr = new IgniteSnapshotManager(ctx); @@ -3024,7 +3030,8 @@ private GridCacheSharedContext createSharedContext( storeSesLsnrs, mvccCachingMgr, deadlockDetectionMgr, - diagnosticMgr + diagnosticMgr, + preloadMgr ); } @@ -5196,6 +5203,15 @@ private class CacheRecoveryLifecycle implements MetastorageLifecycleListener, Da @Override public void onBaselineChange() { onKernalStopCaches(true); + for (DataRegion region : sharedCtx.database().dataRegions()) { + if (!region.config().isPersistenceEnabled() || + INTERNAL_DATA_REGION_NAMES.contains(region.config().getName())) + continue; + + region.pageMemory().stop(false); + region.pageMemory().start(); + } + stopCaches(true); sharedCtx.coordinators().stopTxLog(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java index ab496bec393cd..7ed960d6eead2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java @@ -44,6 +44,7 @@ import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager; import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgnitePartitionPreloadManager; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.PartitionsEvictManager; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; @@ -192,6 +193,9 @@ public class GridCacheSharedContext { /** Cluster is in read-only mode. */ private volatile boolean readOnlyMode; + /** Partition files preloader. */ + private IgnitePartitionPreloadManager preloadMgr; + /** * @param kernalCtx Context. * @param txMgr Transaction manager. @@ -234,7 +238,8 @@ public GridCacheSharedContext( Collection storeSesLsnrs, MvccCachingManager mvccCachingMgr, DeadlockDetectionManager deadlockDetectionMgr, - CacheDiagnosticManager diagnosticMgr + CacheDiagnosticManager diagnosticMgr, + IgnitePartitionPreloadManager preloadMgr ) { this.kernalCtx = kernalCtx; @@ -251,6 +256,7 @@ public GridCacheSharedContext( snapshotMgr, snpMgr, depMgr, + preloadMgr, exchMgr, affMgr, ioMgr, @@ -431,6 +437,7 @@ void onReconnected(boolean active) throws IgniteCheckedException { snapshotMgr, snpMgr, new GridCacheDeploymentManager(), + preloadMgr, new GridCachePartitionExchangeManager(), affMgr, ioMgr, @@ -481,6 +488,7 @@ private void setManagers( IgniteSnapshotManager snapshotMgr, IgniteCacheSnapshotManager snpMgr, GridCacheDeploymentManager depMgr, + IgnitePartitionPreloadManager preloadMgr, GridCachePartitionExchangeManager exchMgr, CacheAffinitySharedManager affMgr, GridCacheIoManager ioMgr, @@ -502,6 +510,7 @@ private void setManagers( this.snpMgr = add(mgrs, snpMgr); this.jtaMgr = add(mgrs, jtaMgr); this.depMgr = add(mgrs, depMgr); + this.preloadMgr = add(mgrs, preloadMgr); this.exchMgr = add(mgrs, exchMgr); this.affMgr = add(mgrs, affMgr); this.ioMgr = add(mgrs, ioMgr); @@ -735,6 +744,13 @@ public IgniteCacheDatabaseSharedManager database() { return dbMgr; } + /** + * @return Partition files preloader. + */ + public IgnitePartitionPreloadManager preloader() { + return preloadMgr; + } + /** * @return Snapshot manager. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java index 8c0f43a38575d..f2eedd57bcb44 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java @@ -1104,5 +1104,24 @@ public GridCursor cursor(int cacheId, KeyCacheObject low * Partition storage. */ public PartitionMetaStorage partStorage(); + + /** + * @return {@code True} if data store is in active mode and is processing updates. + */ + public boolean active(); + + /** + * Change current cache data store mode.to enable updates. + * + * @return {@code True} if partition mode was changed. + */ + public boolean enable(); + + /** + * Change current cache data store mode.to disable updates. + * + * @return {@code True} if partition mode was changed. + */ + public boolean disable(); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java index daf0f26c92b9f..24e27698859b7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java @@ -153,7 +153,7 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager public static final int PRELOAD_SIZE_UNDER_CHECKPOINT_LOCK = 100; /** */ - private final boolean failNodeOnPartitionInconsistency = Boolean.getBoolean( + protected final boolean failNodeOnPartitionInconsistency = Boolean.getBoolean( IgniteSystemProperties.IGNITE_FAIL_NODE_ON_UNRECOVERABLE_PARTITION_INCONSISTENCY); /** Batch size for cache removals during destroy. */ @@ -1317,7 +1317,7 @@ protected CacheDataStore createCacheDataStore0(int p) throws IgniteCheckedExcept } /** {@inheritDoc} */ - @Override public final void destroyCacheDataStore(CacheDataStore store) throws IgniteCheckedException { + @Override public final void destroyCacheDataStore(CacheDataStore store) { int p = store.partId(); partStoreLock.lock(p); @@ -1451,7 +1451,7 @@ protected class CacheDataStoreImpl implements CacheDataStore { private final CacheDataTree dataTree; /** Update counter. */ - protected final PartitionUpdateCounter pCntr; + private final PartitionUpdateCounter pCntr; /** Partition size. */ private final AtomicLong storageSize = new AtomicLong(); @@ -3063,6 +3063,21 @@ public void restoreState(long size, long updCntr, @Nullable Map c return null; } + /** {@inheritDoc} */ + @Override public boolean enable() { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public boolean disable() { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public boolean active() { + return true; + } + /** * @param cctx Cache context. * @param key Key. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounter.java index d0f05020ba2b9..a5ea8dce9c1f7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounter.java @@ -143,4 +143,9 @@ public interface PartitionUpdateCounter extends Iterable { * @return Cache group context. */ public CacheGroupContext context(); + + /** + * @return Highest applied counter. + */ + public long highestAppliedCounter(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounterDebugWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounterDebugWrapper.java index 556e329018358..a775b5c4f0113 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounterDebugWrapper.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounterDebugWrapper.java @@ -267,6 +267,11 @@ public PartitionUpdateCounterDebugWrapper(int partId, PartitionUpdateCounter del return delegate.context(); } + /** {@inheritDoc} */ + @Override public long highestAppliedCounter() { + return delegate.highestAppliedCounter(); + } + /** {@inheritDoc} */ @Override public String toString() { return delegate.toString(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounterTrackingImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounterTrackingImpl.java index 9a5f1b263aed1..a40d5709dcf08 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounterTrackingImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounterTrackingImpl.java @@ -114,8 +114,8 @@ public PartitionUpdateCounterTrackingImpl(CacheGroupContext grp) { return cntr.get(); } - /** */ - protected synchronized long highestAppliedCounter() { + /** {@inheritDoc} */ + @Override public synchronized long highestAppliedCounter() { return queue.isEmpty() ? cntr.get() : queue.lastEntry().getValue().absolute(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounterVolatileImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounterVolatileImpl.java index f25f45606f4b5..ebc1f2bf62545 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounterVolatileImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounterVolatileImpl.java @@ -158,6 +158,11 @@ public PartitionUpdateCounterVolatileImpl(CacheGroupContext grp) { return new GridEmptyIterator<>(); } + /** {@inheritDoc} */ + @Override public long highestAppliedCounter() { + return get(); + } + /** {@inheritDoc} */ @Override public String toString() { return "Counter [init=" + initCntr + ", val=" + get() + ']'; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java index ca768fc7473b7..dae45e99ad117 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java @@ -315,8 +315,23 @@ Runnable addAssignments( if (!grp.localWalEnabled()) fut.listen(new IgniteInClosureX>() { @Override public void applyx(IgniteInternalFuture future) throws IgniteCheckedException { - if (future.get()) - ctx.walState().onGroupRebalanceFinished(grp.groupId(), assignments.topologyVersion()); + if (future.get()) { + GridCompoundFuture idxsFut = new GridCompoundFuture<>(); + + for (GridCacheContext ctx : grp.caches()) { + IgniteInternalFuture idxFut = + (IgniteInternalFuture)ctx.shared().database().indexRebuildFuture(ctx.cacheId()); + + if (idxFut != null) + idxsFut.add(idxFut); + } + + idxsFut.markInitialized(); + + idxsFut.listen(f -> + ctx.walState().onGroupRebalanceFinished(grp.groupId(), assignments.topologyVersion()) + ); + } } }); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java index 8dfb7c201e2d0..56edabc7dad46 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java @@ -460,7 +460,7 @@ else if (iter.isPartitionMissing(p)) { try { if (sctx != null) clearContext(sctx, log); - else if (iter != null) + else if (iter != null && !iter.isClosed()) iter.close(); } catch (Throwable t1) { 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 e907c202fda25..53086d86884a2 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 @@ -1509,8 +1509,6 @@ private void distributedExchange() throws IgniteCheckedException { cctx.exchange().exchangerBlockingSectionBegin(); try { - cctx.database().releaseHistoryForPreloading(); - // To correctly rebalance when persistence is enabled, it is necessary to reserve history within exchange. partHistReserved = cctx.database().reserveHistoryForExchange(); } @@ -2353,10 +2351,8 @@ private String exchangeTimingsLogMessage(String header, List timings) { boolean success = cctx.database().reserveHistoryForPreloading( e.getKey().get1(), e.getKey().get2(), e.getValue()); - if (!success) { - // TODO: how to handle? - err = new IgniteCheckedException("Could not reserve history"); - } + // We can't fail here since history is reserved for exchange. + assert success : "History was not reserved"; } } @@ -3186,6 +3182,8 @@ private void assignPartitionSizes(GridDhtPartitionTopology top) { * @param top Topology to assign. */ private void assignPartitionStates(GridDhtPartitionTopology top) { + CacheGroupContext grp = cctx.cache().cacheGroup(top.groupId()); + Map maxCntrs = new HashMap<>(); Map minCntrs = new HashMap<>(); @@ -3224,6 +3222,18 @@ private void assignPartitionStates(GridDhtPartitionTopology top) { else if (cntr == maxCntr.cnt) maxCntr.nodes.add(uuid); } + + if (nodeCntrs.isEmpty()) { + // We should provide the supplier for file rebalancing even if the counters were not sent. + for (int p = 0; p < top.partitions(); p++) { + GridDhtPartitionState state = top.partitionState(e.getKey(), p); + + if (state != GridDhtPartitionState.OWNING && state != GridDhtPartitionState.MOVING) + continue; + + minCntrs.put(p, 0L); + } + } } // Also must process counters from the local node. @@ -3261,6 +3271,19 @@ else if (cntr == maxCntr.cnt) maxCntr.nodes.add(cctx.localNodeId()); } + Map> ownersByUpdCounters = new HashMap<>(maxCntrs.size()); + for (Map.Entry e : maxCntrs.entrySet()) + ownersByUpdCounters.put(e.getKey(), e.getValue().nodes); + + Map partSizes = new HashMap<>(maxCntrs.size()); + for (Map.Entry e : maxCntrs.entrySet()) + partSizes.put(e.getKey(), e.getValue().size); + + top.globalPartSizes(partSizes); + + boolean fileRebalancingSupported = grp != null && + cctx.preloader().supports(grp, cctx.discovery().aliveServerNodes()); + Map> partHistReserved0 = partHistReserved; Map localReserved = partHistReserved0 != null ? partHistReserved0.get(top.groupId()) : null; @@ -3275,27 +3298,22 @@ else if (cntr == maxCntr.cnt) long maxCntr = maxCntrObj != null ? maxCntrObj.cnt : 0; - // If minimal counter is zero, do clean preloading. - if (minCntr == 0 || minCntr == maxCntr) + if (maxCntr == 0) continue; - if (localReserved != null) { - Long localHistCntr = localReserved.get(p); + Map msgCntrs = F.viewReadOnly(msgs, v -> v.partitionHistoryCounters(top.groupId()).get(p)); - if (localHistCntr != null && localHistCntr <= minCntr && maxCntrObj.nodes.contains(cctx.localNodeId())) { - partHistSuppliers.put(cctx.localNodeId(), top.groupId(), p, localHistCntr); - - haveHistory.add(p); - - continue; - } - } + Collection> cntrsView = localReserved == null ? msgCntrs.entrySet() : + F.concat(false, new T2<>(cctx.localNodeId(), localReserved.get(p)), msgCntrs.entrySet()); - for (Map.Entry e0 : msgs.entrySet()) { - Long histCntr = e0.getValue().partitionHistoryCounters(top.groupId()).get(p); + for (Map.Entry e0 : cntrsView) { + Long histCntr = e0.getValue(); + UUID nodeId = e0.getKey(); - if (histCntr != null && histCntr <= minCntr && maxCntrObj.nodes.contains(e0.getKey())) { - partHistSuppliers.put(e0.getKey(), top.groupId(), p, histCntr); + if (histCntr != null && + maxCntrObj.nodes.contains(nodeId) && + ((minCntr != 0 && histCntr <= minCntr) || (minCntr == 0 && fileRebalancingSupported))) { + partHistSuppliers.put(nodeId, top.groupId(), p, minCntr == 0 ? maxCntr : histCntr); haveHistory.add(p); @@ -3304,16 +3322,6 @@ else if (cntr == maxCntr.cnt) } } - Map> ownersByUpdCounters = new HashMap<>(maxCntrs.size()); - for (Map.Entry e : maxCntrs.entrySet()) - ownersByUpdCounters.put(e.getKey(), e.getValue().nodes); - - Map partSizes = new HashMap<>(maxCntrs.size()); - for (Map.Entry e : maxCntrs.entrySet()) - partSizes.put(e.getKey(), e.getValue().size); - - top.globalPartSizes(partSizes); - Map> partitionsToRebalance = top.resetOwners(ownersByUpdCounters, haveHistory, this); for (Map.Entry> e : partitionsToRebalance.entrySet()) { @@ -3642,8 +3650,20 @@ else if (discoveryCustomMessage instanceof SnapshotDiscoveryMessage } for (CacheGroupContext grpCtx : cctx.cache().cacheGroups()) { - if (!grpCtx.isLocal()) - grpCtx.topology().applyUpdateCounters(); + if (grpCtx.isLocal()) + continue; + + boolean rebalanceRequired = grpCtx.preloader().updateRebalanceVersion(this, resTopVer); + + if (rebalanceRequired && grpCtx.persistenceEnabled()) { + CachePartitionFullCountersMap cntrs = grpCtx.topology().fullUpdateCounters(); + + Map partSizes = grpCtx.topology().globalPartSizes(); + + cctx.preloader().onExchange(resTopVer, grpCtx, cntrs, partSizes, partHistSuppliers); + } + + grpCtx.topology().applyUpdateCounters(); } timeBag.finishGlobalStage("Apply update counters"); @@ -3755,6 +3775,23 @@ else if (forceAffReassignment) } } + boolean hasMoving = !partsToReload.isEmpty(); + + Set waitGrps = cctx.affinity().waitGroups(); + + if (!hasMoving) { + for (CacheGroupContext grpCtx : cctx.cache().cacheGroups()) { + if (waitGrps.contains(grpCtx.groupId()) && grpCtx.topology().hasMovingPartitions()) { + hasMoving = true; + + break; + } + } + } + + if (!hasMoving) + cctx.database().releaseHistoryForPreloading(); + if (stateChangeExchange()) { StateChangeRequest req = exchActions.stateChangeRequest(); @@ -3767,24 +3804,8 @@ else if (forceAffReassignment) cctx.kernalContext().state().onStateChangeError(exchangeGlobalExceptions, req); } - else { - boolean hasMoving = !partsToReload.isEmpty(); - - Set waitGrps = cctx.affinity().waitGroups(); - - if (!hasMoving) { - for (CacheGroupContext grpCtx : cctx.cache().cacheGroups()) { - if (waitGrps.contains(grpCtx.groupId()) && grpCtx.topology().hasMovingPartitions()) { - hasMoving = true; - - break; - } - - } - } - + else cctx.kernalContext().state().onExchangeFinishedOnCoordinator(this, hasMoving); - } if (!cctx.kernalContext().state().clusterState().localBaselineAutoAdjustment()) { ClusterState state = stateChangeErr ? ClusterState.INACTIVE : req.state(); @@ -4426,8 +4447,18 @@ private void updatePartitionFullMap(AffinityTopologyVersion resTopVer, GridDhtPa CacheGroupContext grp = cctx.cache().cacheGroup(grpId); if (grp != null) { - CachePartitionFullCountersMap cntrMap = msg.partitionUpdateCounters(grpId, - grp.topology().partitions()); + boolean rebalanceRequired = grp.preloader().updateRebalanceVersion(this, resTopVer); + + CachePartitionFullCountersMap cntrMap = + msg.partitionUpdateCounters(grpId, grp.topology().partitions()); + + if (rebalanceRequired && grp.persistenceEnabled()) { + cctx.preloader().onExchange(resTopVer, + grp, + cntrMap, + msg.partitionSizes(cctx).get(grp.groupId()), + partHistSuppliers); + } grp.topology().update(resTopVer, msg.partitions().get(grpId), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java index 67ba01804aae1..6beb438d95ffc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java @@ -44,8 +44,12 @@ import org.apache.ignite.internal.util.typedef.CI1; import org.jetbrains.annotations.Nullable; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD; +import static org.apache.ignite.IgniteSystemProperties.getLong; +import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_PDS_WAL_REBALANCE_THRESHOLD; import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST; import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_UNLOADED; +import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.NONE; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.EVICTED; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.LOST; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.MOVING; @@ -59,6 +63,10 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter { /** Default preload resend timeout. */ public static final long DFLT_PRELOAD_RESEND_TIMEOUT = 1500; + /** */ + private final long walRebalanceThreshold = + getLong(IGNITE_PDS_WAL_REBALANCE_THRESHOLD, DFLT_PDS_WAL_REBALANCE_THRESHOLD); + /** */ private GridDhtPartitionTopology top; @@ -145,11 +153,38 @@ private IgniteCheckedException stopError() { } /** {@inheritDoc} */ - @Override public boolean rebalanceRequired(AffinityTopologyVersion rebTopVer, - GridDhtPartitionsExchangeFuture exchFut) { - if (ctx.kernalContext().clientNode() || rebTopVer.equals(AffinityTopologyVersion.NONE)) + @Override public boolean updateRebalanceVersion( + GridDhtPartitionsExchangeFuture exchFut, + AffinityTopologyVersion resVer + ) { + AffinityTopologyVersion rebTopVer = ctx.exchange().rebalanceTopologyVersion(); + + if (ctx.kernalContext().clientNode()) return false; // No-op. + if (rebTopVer.equals(NONE)) + return true; + + if (rebalanceRequired(rebTopVer, resVer, exchFut)) { + ctx.exchange().resetRebalanceVersion(); + + return true; + } + + return false; + } + + /** + * @param rebTopVer Current rebalance topology version. + * @param resVer Exchange result version. + * @param exchFut Exchange future. + * @return {@code True} if rebalance should be restarted. + */ + private boolean rebalanceRequired( + AffinityTopologyVersion rebTopVer, + AffinityTopologyVersion resVer, + GridDhtPartitionsExchangeFuture exchFut + ) { if (exchFut.resetLostPartitionFor(grp.cacheOrGroupName())) return true; @@ -164,13 +199,12 @@ private IgniteCheckedException stopError() { return true; // Required, since no history info available. } - final IgniteInternalFuture rebFut = rebalanceFuture(); + IgniteInternalFuture rebFut = rebalanceFuture(); - if (rebFut.isDone() && !rebFut.result()) + if ((rebFut.isDone() && !rebFut.result()) || ctx.preloader().incompleteRebalance(grp)) return true; // Required, previous rebalance cancelled. - AffinityTopologyVersion lastAffChangeTopVer = - ctx.exchange().lastAffinityChangedTopologyVersion(exchFut.topologyVersion()); + AffinityTopologyVersion lastAffChangeTopVer = ctx.exchange().lastAffinityChangedTopologyVersion(resVer); return lastAffChangeTopVer.compareTo(rebTopVer) > 0; } @@ -201,6 +235,8 @@ private IgniteCheckedException stopError() { CachePartitionFullCountersMap countersMap = grp.topology().fullUpdateCounters(); + boolean fileRebalanceSupported = ctx.preloader().supports(grp); + for (int p = 0; p < partitions; p++) { if (ctx.exchange().hasPendingServerExchange()) { if (log.isDebugEnabled()) @@ -249,7 +285,9 @@ private IgniteCheckedException stopError() { ClusterNode histSupplier = null; - if (grp.persistenceEnabled() && exchFut != null) { + if (grp.persistenceEnabled() && exchFut != null && + countersMap.updateCounter(p) != part.initialUpdateCounter() && + part.fullSize() > walRebalanceThreshold) { UUID nodeId = exchFut.partitionHistorySupplier(grp.groupId(), p, part.initialUpdateCounter()); if (nodeId != null) @@ -257,6 +295,8 @@ private IgniteCheckedException stopError() { } if (histSupplier != null && !exchFut.isClearingPartition(grp, p)) { + assert !part.isEmpty() : "grp=" + grp.cacheOrGroupName() + ", p=" + p; + assert !part.isClearing() : "grp=" + grp.cacheOrGroupName() + ", p=" + p; assert grp.persistenceEnabled(); assert remoteOwners(p, topVer).contains(histSupplier) : remoteOwners(p, topVer); @@ -276,7 +316,7 @@ private IgniteCheckedException stopError() { else { // If for some reason (for example if supplier fails and new supplier is elected) partition is // assigned for full rebalance force clearing if not yet set. - if (grp.persistenceEnabled() && exchFut != null && !exchFut.isClearingPartition(grp, p)) + if (grp.persistenceEnabled() && exchFut != null && !exchFut.isClearingPartition(grp, p) && part.active()) part.clearAsync(); List picked = remoteOwners(p, topVer); @@ -296,7 +336,27 @@ private IgniteCheckedException stopError() { log.debug("Owning partition as there are no other owners: " + part); } else { - ClusterNode n = picked.get(p % picked.size()); + ClusterNode n = null; + + if (fileRebalanceSupported && exchFut != null) { + // Pick a supplier to preload the partition file. + UUID nodeId = + exchFut.partitionHistorySupplier(grp.groupId(), p, countersMap.updateCounter(p)); + + if (nodeId != null) { + n = ctx.discovery().node(nodeId); + + assert picked.contains(n); + + if (log.isDebugEnabled()) { + log.debug("Partition file supplier [node=" + nodeId + ", grp=" + + grp.cacheOrGroupName() + ", p=" + p + "]"); + } + } + } + + if (n == null) + n = picked.get(p % picked.size()); GridDhtPartitionDemandMessage msg = assignments.get(n); @@ -381,13 +441,27 @@ private List remoteOwners(int p, AffinityTopologyVersion topVer) { /** {@inheritDoc} */ @Override public Runnable addAssignments( - GridDhtPreloaderAssignments assignments, + IgniteInternalFuture assignsFut, boolean forceRebalance, long rebalanceId, Runnable next, @Nullable GridCompoundFuture forcedRebFut ) { - return demander.addAssignments(assignments, forceRebalance, rebalanceId, next, forcedRebFut); + if (assignsFut.isDone()) + return demander.addAssignments(assignsFut.result(), forceRebalance, rebalanceId, next, forcedRebFut); + + return () -> { + assignsFut.listen(f -> { + GridDhtPreloaderAssignments assigns = f.result(); + + Runnable rebRunner = demander.addAssignments(assigns, forceRebalance, rebalanceId, next, forcedRebFut); + + if (rebRunner != null) + rebRunner.run(); + else if (next != null) + next.run(); + }); + }; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/IgniteDhtPartitionHistorySuppliersMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/IgniteDhtPartitionHistorySuppliersMap.java index 6755f287b6974..45171964fb80c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/IgniteDhtPartitionHistorySuppliersMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/IgniteDhtPartitionHistorySuppliersMap.java @@ -52,7 +52,7 @@ public static IgniteDhtPartitionHistorySuppliersMap empty() { * @param cntrSince Partition update counter since history supplying is requested. * @return Supplier UUID. */ - @Nullable public synchronized UUID getSupplier(int grpId, int partId, long cntrSince) { + public synchronized @Nullable UUID getSupplier(int grpId, int partId, long cntrSince) { if (map == null) return null; @@ -72,7 +72,7 @@ public static IgniteDhtPartitionHistorySuppliersMap empty() { * @param nodeId Node ID to check. * @return Reservations for the given node. */ - @Nullable public synchronized Map, Long> getReservations(UUID nodeId) { + public synchronized @Nullable Map, Long> getReservations(UUID nodeId) { if (map == null) return null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/IgnitePartitionPreloadManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/IgnitePartitionPreloadManager.java new file mode 100644 index 0000000000000..8c4cdd55b5d82 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/IgnitePartitionPreloadManager.java @@ -0,0 +1,337 @@ +/* + * 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.distributed.dht.preloader; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.cache.CacheRebalanceMode; +import org.apache.ignite.cluster.BaselineNode; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteFeatures; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.affinity.AffinityAssignment; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; +import org.apache.ignite.internal.util.future.GridFinishedFuture; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.CU; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISABLE_WAL_DURING_REBALANCING; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_FILE_REBALANCE_ENABLED; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_FILE_REBALANCE_THRESHOLD; +import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; +import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_PDS_WAL_REBALANCE_THRESHOLD; +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.UTILITY_CACHE_NAME; +import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.MOVING; + +/** + * DHT cache partition files preloader. + */ +public class IgnitePartitionPreloadManager extends GridCacheSharedManagerAdapter { + /** */ + private final boolean fileRebalanceEnabled = + IgniteSystemProperties.getBoolean(IGNITE_FILE_REBALANCE_ENABLED, true); + + /** */ + private final long fileRebalanceThreshold = + IgniteSystemProperties.getLong(IGNITE_FILE_REBALANCE_THRESHOLD, DFLT_PDS_WAL_REBALANCE_THRESHOLD); + + /** Lock. */ + private final Lock lock = new ReentrantLock(); + + /** Partition File rebalancing routine. */ + private volatile PartitionPreloadingRoutine partPreloadingRoutine; + + /** {@inheritDoc} */ + @Override protected void stop0(boolean cancel) { + lock.lock(); + + try { + if (partPreloadingRoutine != null) + partPreloadingRoutine.onDone(false); + } + finally { + lock.unlock(); + } + } + + /** + * @param exchId Exchange ID. + * @param exchFut Completed exchange future. Can be {@code null} if forced or reassigned generation occurs. + * @param rebalanceId Rebalance id. + * @param forcePreload {@code True} if preload requested by {@link ForceRebalanceExchangeTask}. + * @return Cache group identifiers with future assignments that will be ready when partitions are preloaded. + */ + public Map> generateAssignments( + GridDhtPartitionExchangeId exchId, + GridDhtPartitionsExchangeFuture exchFut, + long rebalanceId, + boolean forcePreload + ) { + Map>> filePreloadingAssignments = new HashMap<>(); + Map> futAssigns = new HashMap<>(); + + for (final CacheGroupContext grp : cctx.cache().cacheGroups()) { + long delay = grp.config().getRebalanceDelay(); + + GridDhtPreloaderAssignments assigns = null; + + // Don't delay for dummy reassigns to avoid infinite recursion. + if ((delay == 0 || forcePreload) && !cctx.snapshot().partitionsAreFrozen(grp)) + assigns = grp.preloader().generateAssignments(exchId, exchFut); + + GridDhtLocalPartition anyPart; + + if (F.isEmpty(assigns) || assigns.cancelled() || !supports(grp) || forcePreload || + (anyPart = F.first(grp.topology().currentLocalPartitions())) == null || anyPart.active()) { + futAssigns.put(grp.groupId(), new GridFinishedFuture<>(assigns)); + + continue; + } + + for (Map.Entry e0 : assigns.entrySet()) { + Map> grpAssigns = + filePreloadingAssignments.computeIfAbsent(e0.getKey().id(), v -> new HashMap<>()); + + grpAssigns.put(grp.groupId(), e0.getValue().partitions().fullSet()); + } + } + + if (!filePreloadingAssignments.isEmpty()) + futAssigns.putAll(preloadAsync(exchFut, rebalanceId, filePreloadingAssignments)); + + return futAssigns; + } + + /** + * Initiates new partitions preload process from given {@code assignments}. + * + * @param exchFut Exchange future. + * @param rebalanceId Rebalance id. + * @param assignments A map of cache assignments grouped by node. + * @return Cache group identifiers with future assignments that will be ready when partitions are preloaded. + */ + private Map> preloadAsync( + GridDhtPartitionsExchangeFuture exchFut, + long rebalanceId, + Map>> assignments + ) { + lock.lock(); + + try { + if (isStopping()) + return Collections.emptyMap(); + + assert partPreloadingRoutine == null || partPreloadingRoutine.isDone(); + + // Start new rebalance session. + partPreloadingRoutine = new PartitionPreloadingRoutine(exchFut, cctx, rebalanceId, assignments); + + return partPreloadingRoutine.startPartitionsPreloading(); + } + finally { + lock.unlock(); + } + } + + /** + * Callback on exchange done, should be invoked before initialize file page store. + * + * @param resVer Exchange result version. + * @param grp Cache group. + * @param cntrs Partition counters. + * @param globalSizes Global partition sizes. + * @param suppliers Historical suppliers. + */ + public void onExchange( + AffinityTopologyVersion resVer, + CacheGroupContext grp, + CachePartitionFullCountersMap cntrs, + Map globalSizes, + IgniteDhtPartitionHistorySuppliersMap suppliers + ) { + assert !cctx.kernalContext().clientNode() : "File preloader should not be created on the client node"; + + PartitionPreloadingRoutine preloadRoutine = partPreloadingRoutine; + + // Abort the current parttition preloading if it is still in progress. + if (preloadRoutine != null && !preloadRoutine.isDone()) + preloadRoutine.cancel(); + + if (!supports(grp)) + return; + + boolean disable = filePreloadingApplicable(resVer, grp, cntrs, globalSizes, suppliers); + + // Should rebuild indexes if the last partition files preloading routine was incomplete. + boolean rebuildIdx = false; + + // At this point, cache updates are queued, and we can safely + // switch partitions to inactive mode and vice versa. + for (GridDhtLocalPartition part : grp.topology().currentLocalPartitions()) { + if (disable) + part.disable(); + else if (part.enable() && cctx.pageStore().exists(grp.groupId(), part.id())) + rebuildIdx = true; + } + + if (rebuildIdx) + cctx.database().rebuildIndexes(grp); + } + + /** + * Check whether file rebalancing is supported for the cache group. + * + * @param grp Cache group. + * @param nodes List of Nodes. + * @return {@code True} if file rebalancing is applicable for specified cache group and all nodes supports it. + */ + public boolean supports(CacheGroupContext grp, Collection nodes) { + assert nodes != null && !nodes.isEmpty(); + + if (!supports(grp)) + return false; + + if (!IgniteFeatures.allNodesSupports(nodes, IgniteFeatures.CACHE_PARTITION_FILE_REBALANCE)) + return false; + + Map globalSizes = grp.topology().globalPartSizes(); + + if (globalSizes.isEmpty()) + return false; + + for (int p = 0; p < grp.affinity().partitions(); p++) { + Long size = globalSizes.get(p); + + if (size != null && size > fileRebalanceThreshold) + return true; + } + + return false; + } + + /** + * Check whether file rebalancing is supported for the cache group. + * + * @param grp Cache group. + * @return {@code True} if file rebalancing is applicable for specified cache group. + */ + public boolean supports(CacheGroupContext grp) { + if (grp == null || !fileRebalanceEnabled || !grp.persistenceEnabled() || grp.isLocal()) + return false; + + if (!IgniteSystemProperties.getBoolean(IGNITE_DISABLE_WAL_DURING_REBALANCING, true)) + return false; + + if (grp.config().getRebalanceDelay() == -1 || grp.config().getRebalanceMode() != CacheRebalanceMode.ASYNC) + return false; + + // Do not rebalance system cache with files as they are not exists. + assert grp.groupId() != CU.cacheId(UTILITY_CACHE_NAME) : "Should not preload utility cache partitions"; + + for (GridCacheContext ctx : grp.caches()) { + if (ctx.config().getAtomicityMode() == ATOMIC) + return false; + } + + return !grp.mvccEnabled(); + } + + /** + * @param grp Group. + * @return {@code True} If the last rebalance attempt was incomplete for specified cache group. + */ + public boolean incompleteRebalance(CacheGroupContext grp) { + PartitionPreloadingRoutine routine = partPreloadingRoutine; + + return routine != null && routine.isDone() && routine.remainingGroups().contains(grp.groupId()); + } + + /** + * @param resVer Exchange result version. + * @param grp Cache group. + * @param cntrs Partition counters. + * @param globalSizes Global partition sizes. + * @param suppliers Historical suppliers. + * @return {@code True} if file preloading is applicable for specified cache group. + */ + private boolean filePreloadingApplicable( + AffinityTopologyVersion resVer, + CacheGroupContext grp, + CachePartitionFullCountersMap cntrs, + Map globalSizes, + IgniteDhtPartitionHistorySuppliersMap suppliers + ) { + Collection bltNodes = cctx.discovery().baselineNodes(resVer); + + if (bltNodes == null || !bltNodes.contains(cctx.localNode())) + return false; + + if (grp.topology().stopping()) + return false; + + AffinityAssignment aff = grp.affinity().readyAffinity(resVer); + + assert aff != null; + + boolean hasApplicablePart = false; + + for (int p = 0; p < grp.affinity().partitions(); p++) { + if (!aff.get(p).contains(cctx.localNode())) { + // Should not start when a partition is evicting + if (grp.topology().localPartition(p) != null) + return false; + + continue; + } + + GridDhtLocalPartition part = grp.topology().localPartition(p); + + assert part != null : "grp=" + grp.cacheOrGroupName() + ", p=" + part.id(); + + if (part.state() != MOVING) + return false; + + if (part.dataStore().rowStore() != null || cctx.pageStore().exists(grp.groupId(), part.id())) + return false; + + if (!hasApplicablePart) { + Long partSize = globalSizes.get(p); + + if (partSize != null && partSize > fileRebalanceThreshold) + hasApplicablePart = true; + } + + // Should have partition file supplier for all partitions to start file preloading. + if (suppliers.getSupplier(grp.groupId(), p, cntrs.updateCounter(p)) == null) + return false; + } + + return hasApplicablePart; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/PartitionPreloadingRoutine.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/PartitionPreloadingRoutine.java new file mode 100644 index 0000000000000..4285f621ef036 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/PartitionPreloadingRoutine.java @@ -0,0 +1,564 @@ +/* + * 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.distributed.dht.preloader; + +import java.io.File; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Queue; +import java.util.Set; +import java.util.TreeMap; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.ReentrantLock; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.PartitionUpdateCounter; +import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; +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.GridCacheOffheapManager; +import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId; +import org.apache.ignite.internal.util.GridConcurrentHashSet; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lifecycle.LifecycleAware; +import org.jetbrains.annotations.Nullable; + +/** + * Partition File preloading routine. + */ +public class PartitionPreloadingRoutine extends GridFutureAdapter { + /** Rebalance topology version. */ + private final AffinityTopologyVersion topVer; + + /** Logger. */ + private final IgniteLogger log; + + /** Cache shared context. */ + private final GridCacheSharedContext cctx; + + /** Unique (per demander) rebalance id. */ + private final long rebalanceId; + + /** Lock. */ + private final ReentrantLock lock = new ReentrantLock(); + + /** Partition restore handler. */ + private final PartitionRestoreHandler restoreHnd; + + /** Exchange ID. */ + private final GridDhtPartitionExchangeId exchId; + + /** Remaining nodes with groups and the number of partitions. */ + @GridToStringInclude + private final Map>> remaining; + + /** Count of partition snapshots received. */ + private final AtomicLong receivedCnt = new AtomicLong(); + + /** Cache group with restored partition snapshots and HWM value of update counter mapped to node identifier. */ + @GridToStringInclude + private final Map>> restored = new ConcurrentHashMap<>(); + + /** + * Cache group identifiers with historical assignments future that will be completed when partition files are + * preloaded. + */ + private final Map> futAssigns = new ConcurrentHashMap<>(); + + /** Total number of partitions. */ + private final long totalPartitionsCnt; + + /** Snapshot future. */ + private IgniteInternalFuture snapshotFut; + + /** + * @param exchFut Exchange future. + * @param cctx Cache shared context. + * @param rebalanceId Rebalance ID + * @param assignments Assignments mapped by node ID. + */ + public PartitionPreloadingRoutine( + GridDhtPartitionsExchangeFuture exchFut, + GridCacheSharedContext cctx, + long rebalanceId, + Map>> assignments + ) { + long totalParts = 0; + + // Copy contents. + Map>> remaining0 = U.newHashMap(assignments.size()); + + for (Map.Entry>> nodeAssign : assignments.entrySet()) { + Map> nodeAssign0 = new ConcurrentHashMap<>(nodeAssign.getValue().size()); + + remaining0.put(nodeAssign.getKey(), nodeAssign0); + + for (Map.Entry> grpAssign : nodeAssign.getValue().entrySet()) { + nodeAssign0.put(grpAssign.getKey(), new GridConcurrentHashSet<>(grpAssign.getValue())); + futAssigns.put(grpAssign.getKey(), new GridFutureAdapter<>()); + + totalParts += grpAssign.getValue().size(); + } + } + + this.cctx = cctx; + this.rebalanceId = rebalanceId; + + exchId = exchFut.exchangeId(); + topVer = exchFut.topologyVersion(); + log = cctx.kernalContext().log(getClass()); + totalPartitionsCnt = totalParts; + remaining = Collections.unmodifiableMap(remaining0); + restoreHnd = new PartitionRestoreHandler(cctx); + } + + /** + * Start partitions preloading. + * + * @return Cache group identifiers with futures that will be completed when partitions are preloaded. + */ + public Map> startPartitionsPreloading() { + assert !remaining.isEmpty(); + + restoreHnd.start(); + + requestPartitionsSnapshot(remaining.entrySet().iterator()); + + return Collections.unmodifiableMap(futAssigns); + } + + /** + * @param it Iterator on node assignments. + */ + private void requestPartitionsSnapshot(Iterator>>> it) { + if (!it.hasNext()) + return; + + Map.Entry>> nodeAssigns = it.next(); + + UUID nodeId = nodeAssigns.getKey(); + Map> assigns = nodeAssigns.getValue(); + + Set currGroups = new HashSet<>(); + + for (Integer grpId : assigns.keySet()) { + CacheGroupContext grp = cctx.cache().cacheGroup(grpId); + + currGroups.add(grp.cacheOrGroupName()); + } + + lock.lock(); + + try { + if (isDone()) + return; + + if (log.isInfoEnabled()) + log.info("Preloading partition files [supplier=" + nodeId + ", groups=" + currGroups + "]"); + + assert snapshotFut == null || snapshotFut.isDone() : snapshotFut; + + (snapshotFut = cctx.snapshotMgr() + .requestRemoteSnapshot(nodeId, + assigns, + (file, uniquePartId) -> onPartitionSnapshotReceived(nodeId, uniquePartId, file))) + .chain(f -> { + if (!f.isCancelled() && f.error() == null) + requestPartitionsSnapshot(it); + else { + if (!onDone(f.error()) && log.isDebugEnabled()) + log.debug("Stale error (ignored): " + f.error().getMessage()); + } + + return null; + } + ); + } + finally { + lock.unlock(); + } + } + + /** + * @return Set of identifiers of the remaining groups. + */ + public Set remainingGroups() { + return futAssigns.keySet(); + } + + /** + * @param nodeId Node id. + * @param uniquePartId Pair of cache group ID with partition ID. + * @param file Partition snapshot file. + */ + public void onPartitionSnapshotReceived(UUID nodeId, GroupPartitionId uniquePartId, File file) { + int grpId = uniquePartId.getGroupId(); + int partId = uniquePartId.getPartitionId(); + + CacheGroupContext grp = cctx.cache().cacheGroup(grpId); + + if (grp == null) { + log.warning("Partition snapshot initialization skipped, cache group not found [grpId=" + grpId + "]"); + + return; + } + + lock.lock(); + + try { + // Ensure that we are not stopping when getting checkpoint read lock. + if (isDone()) + return; + + GridDhtLocalPartition part = grp.topology().localPartition(partId); + + assert part != null; + + IgniteInternalFuture restoreFut = restoreHnd.schedule(grp, part, file); + + restoreFut.listen(f -> { + try { + if (!f.isCancelled() && !isDone()) + onPartitionSnapshotRestored(nodeId, grpId, partId, f.get()); + } + catch (IgniteCheckedException e) { + log.error("Unable to restore partition snapshot [grpId=" + grpId + ", p=" + partId + "]"); + + onDone(e); + } + }); + } catch (IgniteCheckedException e) { + log.error("Unable to restore partition snapshot " + + "[grp=" + grp.cacheOrGroupName() + + ", p=" + partId + + ", file=" + file + "]", e); + + onDone(e); + } finally { + lock.unlock(); + } + + if (receivedCnt.incrementAndGet() == totalPartitionsCnt) { + if (log.isInfoEnabled()) + log.info("All partition files are received - triggering checkpoint to complete rebalancing."); + + cctx.database().wakeupForCheckpoint("Partition files preload complete."); + } + } + + /** + * @param grpId Cache group ID. + * @param partId Partition ID. + * @param cntr The highest value of the update counter before this partition began to process updates. + */ + private void onPartitionSnapshotRestored(UUID nodeId, int grpId, int partId, long cntr) { + Map> grpParts = remaining.get(nodeId); + + assert grpParts != null : "nodeId=" + nodeId + ", grpId=" + grpId + ", p=" + partId; + + Set parts = grpParts.get(grpId); + + boolean rmvd = parts.remove(partId); + + assert rmvd : "nodeId=" + nodeId + ", grpId=" + grpId + ", p=" + partId; + + Map> grpCntrs = restored.computeIfAbsent(grpId, v -> new ConcurrentHashMap<>()); + + grpCntrs.computeIfAbsent(nodeId, v -> new ConcurrentHashMap<>()).put(partId, cntr); + + GridFutureAdapter resFut; + + if (!parts.isEmpty() || + grpParts.remove(grpId) == null || + remaining.values().stream().map(Map::keySet).anyMatch(grps -> grps.contains(grpId)) || + (resFut = futAssigns.remove(grpId)) == null) + return; + + CacheGroupContext grp = cctx.cache().cacheGroup(grpId); + + assert !grp.localWalEnabled() : "grp=" + grp.cacheOrGroupName(); + + IgniteInternalFuture idxFut = cctx.database().rebuildIndexes(grp); + + GridDhtPreloaderAssignments histAssignments = makeHistoricalAssignments(grp, grpCntrs); + + resFut.onDone(histAssignments); + + if (histAssignments.isEmpty()) + idxFut.listen(f -> cctx.walState().onGroupRebalanceFinished(grp.groupId(), topVer)); + + boolean finalPreloading = futAssigns.isEmpty() && onDone(true); + + if (log.isInfoEnabled()) { + log.info("Completed" + (finalPreloading ? " (final)" : "") + + " partition files preloading [grp=" + grp.cacheOrGroupName() + "]"); + } + } + + /** + * Prepare assignments for historical rebalancing. + * + * @param grp Cache group. + * @param cntrs Partition set with HWM update counter value for hstorical rebalance. + * @return Partition to node assignments. + */ + private GridDhtPreloaderAssignments makeHistoricalAssignments( + CacheGroupContext grp, + Map> cntrs + ) { + GridDhtPreloaderAssignments histAssigns = new GridDhtPreloaderAssignments(exchId, topVer); + + int parts = grp.topology().partitions(); + + for (Map.Entry> e : cntrs.entrySet()) { + ClusterNode node = cctx.discovery().node(e.getKey()); + + assert node != null : e.getKey(); + + Map orderedCntrs = new TreeMap<>(e.getValue()); + + for (Map.Entry partCntr : orderedCntrs.entrySet()) { + int partId = partCntr.getKey(); + + long from = grp.topology().localPartition(partId).initialUpdateCounter(); + long to = partCntr.getValue(); + + if (from == to) + continue; + + assert to > from : "from=" + from + ", to=" + to; + + GridDhtPartitionDemandMessage msg = histAssigns. + computeIfAbsent(node, v -> new GridDhtPartitionDemandMessage(rebalanceId, topVer, grp.groupId())); + + msg.partitions().addHistorical(partId, from, to, parts); + } + } + + return histAssigns; + } + + /** {@inheritDoc} */ + @Override public boolean cancel() { + return onDone(false, null, true); + } + + /** {@inheritDoc} */ + @Override protected boolean onDone(@Nullable Boolean res, @Nullable Throwable err, boolean cancel) { + lock.lock(); + + try { + if (!super.onDone(res, err, cancel)) + return false; + + if (!(cctx.database() instanceof GridCacheDatabaseSharedManager)) + return true; + + restoreHnd.stop(); + + if (!isCancelled() && !isFailed()) + return true; + + if (log.isInfoEnabled()) + log.info("Cancelling File preloading [topVer=" + topVer + "]"); + + if (snapshotFut != null && !snapshotFut.isDone()) { + if (log.isDebugEnabled()) + log.debug("Cancelling snapshot creation [fut=" + snapshotFut + "]"); + + snapshotFut.cancel(); + } + + for (GridFutureAdapter fut : futAssigns.values()) + fut.onDone(); + + if (isFailed()) + log.error("File preloading failed [topVer=" + topVer + "]", err); + + return true; + } + catch (IgniteCheckedException e) { + if (err != null) + e.addSuppressed(err); + + log.error("Failed to cancel File preloading.", e); + } + finally { + lock.unlock(); + } + + return false; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(PartitionPreloadingRoutine.class, this); + } + + /** + * + */ + private static class PartitionRestoreHandler implements DbCheckpointListener, LifecycleAware { + /** Cache shared context. */ + private final GridCacheSharedContext cctx; + + /** Lock. */ + private final ReentrantLock lock = new ReentrantLock(); + + /** Checkpoint request queue. */ + private final Queue checkpointRequests = new ConcurrentLinkedQueue<>(); + + /** + * @param cctx Cache shared context. + */ + private PartitionRestoreHandler(GridCacheSharedContext cctx) { + this.cctx = cctx; + } + + /** {@inheritDoc} */ + @Override public void start() { + assert checkpointRequests.isEmpty(); + + ((GridCacheDatabaseSharedManager)cctx.database()).addCheckpointListener(this); + } + + /** {@inheritDoc} */ + @Override public void stop() { + lock.lock(); + + try { + checkpointRequests.clear(); + + ((GridCacheDatabaseSharedManager)cctx.database()).removeCheckpointListener(this); + } + finally { + lock.unlock(); + } + } + + /** {@inheritDoc} */ + @Override public void onMarkCheckpointBegin(Context ctx) { + lock.lock(); + + try { + Runnable r; + + while ((r = checkpointRequests.poll()) != null) + r.run(); + } + finally { + lock.unlock(); + } + } + + /** + * Schedule partition snapshot hot restore. + * + * @param grp Cache group. + * @param part Local partition. + * @param file Snapshot file. + * @return Future with max update counter as a result of restore. + * @throws IgniteCheckedException If failed. + */ + public IgniteInternalFuture schedule( + CacheGroupContext grp, + GridDhtLocalPartition part, + File file + ) throws IgniteCheckedException { + GridFutureAdapter res = new GridFutureAdapter<>(); + + cctx.pageStore().restore(grp.groupId(), part.id(), file); + + boolean initialized = part.dataStore().init(); + + assert initialized; + + checkpointRequests.offer(() -> enablePartition(grp, part, res)); + + return res; + } + + /** + * @param grp Cache group. + * @param part Local partition. + * @param res Future with max update counter as a result. + */ + @SuppressWarnings("unchecked") + private void enablePartition(CacheGroupContext grp, GridDhtLocalPartition part, GridFutureAdapter res) { + if (grp.topology().stopping()) { + res.onCancelled(); + + return; + } + + assert !part.active() : "grp=" + grp.cacheOrGroupName() + " p=" + part.id(); + + // Save current counter. + PartitionUpdateCounter cntr = + ((GridCacheOffheapManager.GridCacheDataStore)part.dataStore()).inactivePartUpdateCounter(); + + // Save current update counter. + PartitionUpdateCounter snapshotCntr = part.dataStore().partUpdateCounter(); + + part.enable(); + + AffinityTopologyVersion infinTopVer = new AffinityTopologyVersion(Long.MAX_VALUE, 0); + GridCompoundFuture partReleaseFut = new GridCompoundFuture(); + + partReleaseFut.add(cctx.mvcc().finishAtomicUpdates(infinTopVer)); + partReleaseFut.add(cctx.mvcc().finishDataStreamerUpdates(infinTopVer)); + partReleaseFut.add(cctx.tm().finishLocalTxs(infinTopVer, null)); + + partReleaseFut.markInitialized(); + + // Local updates that are in progress now will be lost and should be included in historical rebalancing. + // These operations can update the old update counter or the new update counter, so the maximum applied + // counter is used after all updates are completed. + partReleaseFut.listen(c -> { + long hwm = Math.max(cntr.highestAppliedCounter(), snapshotCntr.highestAppliedCounter()); + + cctx.kernalContext().getSystemExecutorService().submit(() -> res.onDone(hwm)); + } + ); + } + + /** {@inheritDoc} */ + @Override public void onCheckpointBegin(Context ctx) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void beforeCheckpointBegin(Context ctx) { + // No-op. + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java index 3e187379c7f0b..6e08b64cc4e25 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java @@ -46,6 +46,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; import org.apache.ignite.internal.processors.cache.GridCacheMapEntryFactory; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry; import org.apache.ignite.internal.processors.cache.distributed.dht.GridReservable; @@ -74,7 +75,6 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_CACHE_REMOVED_ENTRIES_TTL; import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_OBJECT_UNLOADED; -import static org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.EVICTED; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.LOST; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.MOVING; @@ -442,6 +442,55 @@ public void onDeferredDelete(int cacheId, KeyCacheObject key, GridCacheVersion v rmvQueue.add(new RemovedEntryHolder(cacheId, key, ver, rmvdEntryTtl)); } + /** + * @return {@code True} if data store is in active mode and is processing updates. + */ + public boolean active() { + return store.active(); + } + + /** + * Change current cache data store mode.to enable updates on current partition. + * + * @return {@code True} if partition mode was changed, otherwise updates already enabled. + */ + public boolean enable() { + if (store.active()) + return false; + + if (state() != MOVING) + throw new IgniteException("Expected MOVING partition [p=" + id() + ", state=" + state() + "]"); + + if (store.enable()) { + // Clear all on-heap entries before start processing updates. + if (grp.sharedGroup()) { + for (GridCacheContext ctx : grp.caches()) + entriesMap(ctx).map.clear(); + } + else + entriesMap(null).map.clear(); + + return true; + } + + return false; + } + + /** + * Change current cache data store mode.to disable updates on current partition. + * + * @return {@code True} if partition mode was changed, otherwise updates already disabled. + */ + public boolean disable() { + if (!store.active()) + return false; + + if (state() != MOVING) + throw new IgniteException("Expected MOVING partition [p=" + id() + ", state=" + state() + "]"); + + return store.disable(); + } + /** * Reserves a partition so it won't be cleared or evicted. * @@ -620,7 +669,8 @@ public void moving() { GridDhtPartitionState partState = getPartState(state); - assert partState == OWNING || partState == RENTING : "Only partitions in state OWNING or RENTING can be moved to MOVING state"; + assert partState == OWNING || partState == RENTING : + "Only partitions in state OWNING or RENTING can be moved to MOVING state [state=" + partState + "]"; if (casState(state, MOVING)) break; @@ -770,6 +820,8 @@ private void clearAsync0(boolean updateSeq) { * This enforces clearing happens before sending demand requests. */ public void clearAsync() { + assert active() : "grp=" + grp.cacheOrGroupName() + ", p=" + id; + GridDhtPartitionState state0 = state(); if (state0 != MOVING && state0 != RENTING) 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 1a94737114cae..898d401d7588c 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 @@ -2676,6 +2676,8 @@ private void removeNode(UUID nodeId) { /** {@inheritDoc} */ @Override public boolean own(GridDhtLocalPartition part) { + assert part.active() : "grp=" + grp.cacheOrGroupName() + " p=" + part.id(); + lock.writeLock().lock(); try { 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 9d9433b6b9275..1fc33f001aa88 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 @@ -197,12 +197,15 @@ import static java.util.Objects.isNull; import static java.util.Objects.nonNull; import static org.apache.ignite.IgniteSystemProperties.IGNITE_CHECKPOINT_READ_LOCK_TIMEOUT; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_FILE_REBALANCE_THRESHOLD; import static org.apache.ignite.IgniteSystemProperties.IGNITE_JVM_PAUSE_DETECTOR_THRESHOLD; import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD; import static org.apache.ignite.IgniteSystemProperties.IGNITE_RECOVERY_SEMAPHORE_PERMITS; import static org.apache.ignite.IgniteSystemProperties.getBoolean; import static org.apache.ignite.IgniteSystemProperties.getInteger; +import static org.apache.ignite.IgniteSystemProperties.getLong; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT; +import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_PDS_WAL_REBALANCE_THRESHOLD; import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR; import static org.apache.ignite.failure.FailureType.SYSTEM_CRITICAL_OPERATION_TIMEOUT; import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION; @@ -256,7 +259,12 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan private final boolean skipSync = getBoolean(IGNITE_PDS_CHECKPOINT_TEST_SKIP_SYNC); /** */ - private final int walRebalanceThreshold = getInteger(IGNITE_PDS_WAL_REBALANCE_THRESHOLD, 500_000); + private final long walRebalanceThreshold = + getLong(IGNITE_PDS_WAL_REBALANCE_THRESHOLD, DFLT_PDS_WAL_REBALANCE_THRESHOLD); + + /** */ + private final long fileRebalanceThreshold = + getLong(IGNITE_FILE_REBALANCE_THRESHOLD, DFLT_PDS_WAL_REBALANCE_THRESHOLD); /** Value of property for throttling policy override. */ private final String throttlingPolicyOverride = IgniteSystemProperties.getString( @@ -459,7 +467,7 @@ public GridCacheDatabaseSharedManager(GridKernalContext ctx) { ? ctx.config().getDataStorageConfiguration().getCheckpointReadLockTimeout() : null; - checkpointReadLockTimeout = IgniteSystemProperties.getLong(IGNITE_CHECKPOINT_READ_LOCK_TIMEOUT, + checkpointReadLockTimeout = getLong(IGNITE_CHECKPOINT_READ_LOCK_TIMEOUT, cfgCheckpointReadLockTimeout != null ? cfgCheckpointReadLockTimeout : (ctx.workersRegistry() != null @@ -1510,11 +1518,14 @@ else if (acts.localJoinContext() != null && !F.isEmpty(acts.localJoinContext().c * * @param cacheId Cache ID. */ - private void prepareIndexRebuildFuture(int cacheId) { - GridFutureAdapter old = idxRebuildFuts.put(cacheId, new GridFutureAdapter<>()); + private GridFutureAdapter prepareIndexRebuildFuture(int cacheId) { + GridFutureAdapter newFut = new GridFutureAdapter<>(); + GridFutureAdapter oldFut = idxRebuildFuts.put(cacheId, newFut); - if (old != null) - old.onDone(); + if (oldFut != null) + oldFut.onDone(); + + return newFut; } /** {@inheritDoc} */ @@ -1579,6 +1590,44 @@ else if (nonNull(usrFut)) { } } + /** + * @param grp Cache group. + * @return Future that will be completed when indexes for all caches in the specified group are ready for use. + */ + @Override public IgniteInternalFuture rebuildIndexes(CacheGroupContext grp) { + if (!cctx.kernalContext().query().moduleEnabled()) + return new GridFinishedFuture<>(); + + GridCompoundFuture idxsFut = new GridCompoundFuture<>(); + + for (GridCacheContext ctx : grp.caches()) { + IgniteInternalFuture fut = cctx.kernalContext().query().rebuildIndexesFromHash(ctx); + + if (fut != null) { + idxsFut.add(((IgniteInternalFuture)fut)); + + if (log.isInfoEnabled()) + log.info("Starting index rebuild [cache=" + ctx.cache().name() + "]"); + + GridFutureAdapter usrFut = + ((GridCacheDatabaseSharedManager)cctx.database()).prepareIndexRebuildFuture(ctx.cacheId()); + + fut.listen(f -> { + assert !f.isCancelled(); + + log.info("Finished index rebuild [cache=" + ctx.cache().name() + + ", success=" + (f.error() == null) + "]"); + + usrFut.onDone(); + }); + } + } + + idxsFut.markInitialized(); + + return idxsFut; + } + /** * Return short information about cache. * @@ -1804,7 +1853,7 @@ private boolean safeToUpdatePageMemories() { reservedForExchange = new HashMap<>(); - Map> applicableGroupsAndPartitions = partitionsApplicableForWalRebalance(); + Map> applicableGroupsAndPartitions = partitionsApplicableForWalOrFileRebalance(); Map> earliestValidCheckpoints; @@ -1847,15 +1896,18 @@ private boolean safeToUpdatePageMemories() { /** * @return Map of group id -> Set of partitions which can be used as suppliers for WAL rebalance. */ - private Map> partitionsApplicableForWalRebalance() { + private Map> partitionsApplicableForWalOrFileRebalance() { Map> res = new HashMap<>(); for (CacheGroupContext grp : cctx.cache().cacheGroups()) { if (grp.isLocal()) continue; + boolean fileRebalanceSupported = cctx.preloader().supports(grp); + for (GridDhtLocalPartition locPart : grp.topology().currentLocalPartitions()) { - if (locPart.state() == GridDhtPartitionState.OWNING && locPart.fullSize() > walRebalanceThreshold) + if (locPart.state() == GridDhtPartitionState.OWNING && (locPart.fullSize() > walRebalanceThreshold || + (fileRebalanceSupported && locPart.fullSize() > fileRebalanceThreshold))) res.computeIfAbsent(grp.groupId(), k -> new HashSet<>()).add(locPart.id()); } } @@ -1897,26 +1949,58 @@ private Map> partitionsApplicableForWalRebalance() { /** {@inheritDoc} */ @Override public boolean reserveHistoryForPreloading(int grpId, int partId, long cntr) { + T2 saved = reservedForPreloading.get(new T2<>(grpId, partId)); + + if (saved != null && saved.get1() <= cntr) + return true; + + if (log.isDebugEnabled()) { + log.debug("Reserve history for preloading [grp=" + + cctx.cache().cacheGroup(grpId).cacheOrGroupName() + ", p=" + partId + ", cntr=" + cntr + "]"); + } + CheckpointEntry cpEntry = cpHistory.searchCheckpointEntry(grpId, partId, cntr); - if (cpEntry == null) + if (cpEntry == null) { + log.error("Unable to reserve history for preloading, checkpoint entry not found [grp=" + + cctx.cache().cacheGroup(grpId).cacheOrGroupName() + ", p=" + partId + ", cntr=" + cntr + "]"); + return false; + } WALPointer ptr = cpEntry.checkpointMark(); - if (ptr == null) + if (ptr == null) { + log.error("Unable to reserve history for preloading, checkpoint end mark is undefined [grp=" + + cctx.cache().cacheGroup(grpId).cacheOrGroupName() + ", p=" + partId + ", cntr=" + cntr + "]"); + return false; + } boolean reserved = cctx.wal().reserve(ptr); - if (reserved) + if (reserved) { + try { + if (saved != null) + cctx.wal().release(saved.get2()); + } + catch (IgniteCheckedException e) { + log.error("Could not release WAL reservation", e); + + throw new IgniteException(e); + } + reservedForPreloading.put(new T2<>(grpId, partId), new T2<>(cntr, ptr)); + } return reserved; } /** {@inheritDoc} */ @Override public void releaseHistoryForPreloading() { + if (log.isDebugEnabled()) + log.debug("Release history for preloading"); + releaseHistForPreloadingLock.lock(); try { @@ -1938,6 +2022,29 @@ private Map> partitionsApplicableForWalRebalance() { } } + /** + * Get reserved WAL pointer for preloading. + * + * @param grpId Group ID. + * @param partId Part ID. + * @param initCntr Initial update counter. + * @return Reserved WAL pointer for preloading. + */ + public FileWALPointer searchWALPointer(int grpId, int partId, long initCntr) { + assert reservedForPreloading != null; + + T2 reserved = reservedForPreloading.get(new T2<>(grpId, partId)); + + if (reserved == null) + return (FileWALPointer)checkpointHistory().searchPartitionCounter(grpId, partId, initCntr); + + long cntr = reserved.get1(); + + assert cntr <= initCntr : "reserved=" + cntr + ", init=" + initCntr; + + return (FileWALPointer)reserved.get2(); + } + /** * */ @@ -2324,7 +2431,7 @@ private RestoreBinaryState performBinaryMemoryRestore( long lastArchivedSegment = cctx.wal().lastArchivedSegment(); - WALIterator it = cctx.wal().replay(recPtr, recordTypePredicate); + WALIterator it = cctx.wal().replay(recPtr, true, recordTypePredicate); RestoreBinaryState restoreBinaryState = new RestoreBinaryState(status, it, lastArchivedSegment, cacheGroupsPredicate); @@ -2810,7 +2917,7 @@ private RestoreLogicalState applyLogicalUpdates( Map partitionRecoveryStates = new HashMap<>(); - WALIterator it = cctx.wal().replay(status.startPtr, recordTypePredicate); + WALIterator it = cctx.wal().replay(status.startPtr, true, recordTypePredicate); RestoreLogicalState restoreLogicalState = new RestoreLogicalState(status, it, lastArchivedSegment, cacheGroupsPredicate, partitionRecoveryStates); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java index c88bb5c194474..a1f0db1ceeced 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java @@ -67,6 +67,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.PartitionUpdateCounter; +import org.apache.ignite.internal.processors.cache.PartitionUpdateCounterTrackingImpl; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionPartialCountersMap; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteHistoricalIterator; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; @@ -94,6 +95,7 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer; import org.apache.ignite.internal.processors.cache.tree.CacheDataRowStore; import org.apache.ignite.internal.processors.cache.tree.CacheDataTree; +import org.apache.ignite.internal.processors.cache.tree.DataRow; import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree; import org.apache.ignite.internal.processors.cache.tree.PendingRow; import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccUpdateResult; @@ -239,11 +241,11 @@ private void syncMetadata(Context ctx) throws IgniteCheckedException { if (needSnapshot) { if (execSvc == null) - addPartitions(ctx); + addIndexPartition(ctx); else { execSvc.execute(() -> { try { - addPartitions(ctx); + addIndexPartition(ctx); } catch (IgniteCheckedException e) { throw new IgniteException(e); @@ -300,6 +302,12 @@ private void saveStoreMetadata( boolean beforeDestroy, boolean needSnapshot ) throws IgniteCheckedException { + if (!store.active()) { + assert !needSnapshot : "You should not request partition while store is inactive: " + store; + + return; + } + RowStore rowStore0 = store.rowStore(); if (rowStore0 != null) { @@ -459,6 +467,10 @@ else if (state == MOVING || state == RENTING) { "a partition not in " + OWNING + " state [grp=" + grp.cacheOrGroupName() + ", partId=" + store.partId() + ", state=" + state + ']'); } + + U.warn(log,"Partition Will not be included to snapshot because it is not in " + + "OWNING state [grp=" + grp.cacheOrGroupName() + ", partId=" + store.partId() + + ", state=" + state + ']'); } changed = true; @@ -770,7 +782,7 @@ private static long writeSharedGroupCacheSizes(PageMemory pageMem, int grpId, /** * @param ctx Context. */ - private void addPartitions(Context ctx) throws IgniteCheckedException { + private void addIndexPartition(Context ctx) throws IgniteCheckedException { int grpId = grp.groupId(); PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory(); @@ -820,15 +832,6 @@ private static boolean addPartition( final int currAllocatedPageCnt, final long partSize ) { - if (part != null) { - boolean reserved = part.reserve(); - - if (!reserved) - return false; - } - else - assert partId == PageIdAllocator.INDEX_PARTITION : partId; - assert PageIO.getPageId(metaPageAddr) != 0; int lastAllocatedPageCnt = io.getLastAllocatedPageCount(metaPageAddr); @@ -1017,8 +1020,7 @@ private Metas getOrAllocateCacheMetas() throws IgniteCheckedException { int p = partCntrs.partitionAt(i); long initCntr = partCntrs.initialUpdateCounterAt(i); - FileWALPointer startPtr = (FileWALPointer)database.checkpointHistory().searchPartitionCounter( - grp.groupId(), p, initCntr); + FileWALPointer startPtr = database.searchWALPointer(grp.groupId(), p, initCntr); if (startPtr == null) throw new IgniteCheckedException("Could not find start pointer for partition [part=" + p + ", partCntrSince=" + initCntr + "]"); @@ -1027,7 +1029,7 @@ private Metas getOrAllocateCacheMetas() throws IgniteCheckedException { minPtr = startPtr; } - WALIterator it = grp.shared().wal().replay(minPtr); + WALIterator it = grp.shared().wal().replay(minPtr, false, null); WALHistoricalIterator iterator = new WALHistoricalIterator(log, grp, partCntrs, it); @@ -1344,7 +1346,7 @@ private void releasePartitions() { GridDhtLocalPartition part = grp.topology().localPartition(p); assert part != null && part.state() == OWNING && part.reservations() > 0 - : "Partition should in OWNING state and has at least 1 reservation"; + : "Partition should in OWNING state and has at least 1 reservation [state=" + part.state() + "]"; part.release(); } @@ -1437,6 +1439,8 @@ else if (rec.get2() instanceof RollbackRecord) { } } + assert false; + return; } } @@ -1635,6 +1639,12 @@ public class GridCacheDataStore implements CacheDataStore { /** */ private final CountDownLatch latch = new CountDownLatch(1); + /** Currently used data storage state. */ + private final AtomicBoolean active = new AtomicBoolean(true); + + /** Update counter used when data store does not process updates. */ + private volatile PartitionUpdateCounter noopModeCntr; + /** * @param partId Partition. * @param exists {@code True} if store exists. @@ -2044,7 +2054,7 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public boolean init() { try { - return init0(true) != null; + return init0(active()) != null; } catch (IgniteCheckedException e) { throw new IgniteException(e); @@ -2126,6 +2136,9 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public long updateCounter() { + if (!active()) + return 0; + try { CacheDataStore delegate0 = init0(true); @@ -2138,6 +2151,9 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public long reservedCounter() { + if (!active()) + return noopModeCntr.reserved(); + try { CacheDataStore delegate0 = init0(true); @@ -2160,8 +2176,18 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { } } + /** + * @return Update counter for inactive mode. + */ + public PartitionUpdateCounter inactivePartUpdateCounter() { + return noopModeCntr; + } + /** {@inheritDoc} */ @Override public long getAndIncrementUpdateCounter(long delta) { + if (!active()) + return noopModeCntr.reserve(delta); + try { CacheDataStore delegate0 = init0(false); @@ -2174,6 +2200,9 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public long reserve(long delta) { + if (!active()) + return noopModeCntr.reserve(delta); + try { CacheDataStore delegate0 = init0(false); @@ -2189,6 +2218,23 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public void updateCounter(long val) { + if (!active()) { + try { + noopModeCntr.update(val); + + return; + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to update partition counter. " + + "Most probably a node with most actual data is out of topology or data streamer is used " + + "in preload mode (allowOverride=false) concurrently with cache transactions [grpName=" + + grp.name() + ", partId=" + partId + ']', e); + + if (failNodeOnPartitionInconsistency) + ctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e)); + } + } + try { CacheDataStore delegate0 = init0(false); @@ -2202,6 +2248,9 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public boolean updateCounter(long start, long delta) { + if (!active()) + return noopModeCntr.update(start, delta); + try { CacheDataStore delegate0 = init0(false); @@ -2214,6 +2263,9 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public GridLongList finalizeUpdateCounters() { + if (!active()) + return noopModeCntr.finalizeUpdateCounters(); + try { CacheDataStore delegate0 = init0(true); @@ -2226,6 +2278,9 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public long nextUpdateCounter() { + if (!active()) + return noopModeCntr.next(); + try { CacheDataStore delegate0 = init0(false); @@ -2241,6 +2296,10 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public long initialUpdateCounter() { + //todo can be removed? + if (!active()) + return 0; + try { CacheDataStore delegate0 = init0(true); @@ -2253,6 +2312,10 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public void updateInitialCounter(long start, long delta) { + // todo can be removed? + if (!active()) + noopModeCntr.updateInitial(start, delta); + try { CacheDataStore delegate0 = init0(false); @@ -2288,6 +2351,9 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { ) throws IgniteCheckedException { assert ctx.database().checkpointLockIsHeldByThread(); + if (!active()) + return; + CacheDataStore delegate = init0(false); delegate.update(cctx, key, val, ver, expireTime, oldRow); @@ -2418,6 +2484,15 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { @Nullable CacheDataRow oldRow) throws IgniteCheckedException { assert ctx.database().checkpointLockIsHeldByThread(); + if (!active()) { + assert oldRow == null; + + if (key.partition() < 0) + key.partition(partId); + + return new DataRow(key, val, ver, partId, expireTime, cctx.cacheId()); + } + CacheDataStore delegate = init0(false); return delegate.createRow(cctx, key, val, ver, expireTime, oldRow); @@ -2426,6 +2501,9 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public void insertRows(Collection rows, IgnitePredicateX initPred) throws IgniteCheckedException { + if (!active()) + return; + CacheDataStore delegate = init0(false); delegate.insertRows(rows, initPred); @@ -2441,6 +2519,9 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public void updateTxState(GridCacheContext cctx, CacheSearchRow row) throws IgniteCheckedException { + if (!active()) + return; + CacheDataStore delegate = init0(false); delegate.updateTxState(cctx, row); @@ -2451,6 +2532,14 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { throws IgniteCheckedException { assert ctx.database().checkpointLockIsHeldByThread(); + if (!active()) { + // Assume we've performed an invoke operation on the B+ Tree and find nothing. + // Emulating that always inserting/removing a new value. + c.call(null); + + return; + } + CacheDataStore delegate = init0(false); delegate.invoke(cctx, key, c); @@ -2461,6 +2550,9 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { throws IgniteCheckedException { assert ctx.database().checkpointLockIsHeldByThread(); + if (!active()) + return; + CacheDataStore delegate = init0(false); delegate.remove(cctx, key, partId); @@ -2468,6 +2560,9 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public CacheDataRow find(GridCacheContext cctx, KeyCacheObject key) throws IgniteCheckedException { + if (!active()) + return null; + CacheDataStore delegate = init0(true); if (delegate != null) @@ -2619,6 +2714,8 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public void clear(int cacheId) throws IgniteCheckedException { + assert active.get() : "grp=" + grp.cacheOrGroupName() + ", p=" + partId; + assert ctx.database().checkpointLockIsHeldByThread(); CacheDataStore delegate0 = init0(true); @@ -2829,9 +2926,53 @@ private int purgeExpiredInternal( } } + /** {@inheritDoc} */ @Override public PartitionMetaStorage partStorage() { return partStorage; } + + /** {@inheritDoc} */ + @Override public boolean active() { + return active.get(); + } + + /** {@inheritDoc} */ + @Override public boolean enable() { + return changeMode(true); + } + + /** {@inheritDoc} */ + @Override public boolean disable() { + assert delegate == null && !ctx.pageStore().exists(grp.groupId(), partId) : + "grp=" + grp.cacheOrGroupName() + " p=" + partId; + + return changeMode(false); + } + + /** + * Change current cache data store mode. + * + * @param activeMode Active mode flag. + * @return {@code True} if partition mode was changed, otherwise partition already in the specified mode. + */ + private boolean changeMode(boolean activeMode) { + if (active.compareAndSet(!activeMode, activeMode)) { + if (log.isInfoEnabled()) { + log.info("Partition data store mode changed [grp=" + grp.cacheOrGroupName() + + ", p=" + partId + + ", cntr=" + updateCounter() + + ", size=" + fullSize() + + ", mode=" + (activeMode ? "ACTIVE" : "DISABLED") + "]"); + } + + if (!activeMode) + noopModeCntr = new PartitionUpdateCounterTrackingImpl(grp); + + return true; + } + + return 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 2ab24d3b5ad13..b550db9aa79a0 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 @@ -77,6 +77,7 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageLockListener; import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport; import org.apache.ignite.internal.util.TimeBag; +import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.LT; @@ -156,7 +157,6 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap /** First eviction was warned flag. */ private volatile boolean firstEvictWarn; - /** {@inheritDoc} */ @Override protected void start0() throws IgniteCheckedException { if (cctx.kernalContext().clientNode() && cctx.kernalContext().config().getDataStorageConfiguration() == null) @@ -982,13 +982,21 @@ public void onStateRestored(AffinityTopologyVersion topVer) throws IgniteChecked // No-op. } - /** - * @param fut Partition exchange future. - */ + /** + * @param fut Partition exchange future. + */ public void rebuildIndexesIfNeeded(GridDhtPartitionsExchangeFuture fut) { // No-op. } + /** + * @param grp Cache group. + * @return Future that will be completed when the indexes rebuild for specified cache group is finished. + */ + public IgniteInternalFuture rebuildIndexes(CacheGroupContext grp) { + return new GridFinishedFuture<>(); + } + /** * Needed action before any cache will stop */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java index 2ee7e4dabfe8a..974e60e281de1 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java @@ -565,10 +565,13 @@ public void read(int grpId, long pageId, ByteBuffer pageBuf, boolean keepCrc) th } /** {@inheritDoc} */ - @Override public boolean exists(int grpId, int partId) throws IgniteCheckedException { - PageStore store = getStore(grpId, partId); - - return store.exists(); + @Override public boolean exists(int grpId, int partId) { + try { + return getStore(grpId, partId).exists(); + } + catch (IgniteCheckedException ignore) { + return false; + } } /** {@inheritDoc} */ @@ -1334,6 +1337,25 @@ public PageStore getStore(int grpId, int partId) throws IgniteCheckedException { } } + /** {@inheritDoc} */ + @Override public void restore(int grpId, int partId, File src) throws IgniteCheckedException { + FilePageStore pageStore = (FilePageStore)getStore(grpId, partId); + + File dest = new File(pageStore.getFileAbsolutePath()); + + assert !dest.exists() : "dest=" + dest; + + if (log.isDebugEnabled()) + log.debug("Moving snapshot [from=" + src + " , to=" + dest + " , size=" + src.length() + "]"); + + try { + Files.move(src.toPath(), dest.toPath()); + } + catch (IOException e) { + throw new IgniteCheckedException("Unable to move partition file [from=" + src + ", to=" + dest + "]", e); + } + } + /** * @param pageStoreFileIoFactory File IO factory to override default, may be used for blocked read-write. * @param pageStoreV1FileIoFactory File IO factory for reading V1 page store and for fast touching page files diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotMessage.java new file mode 100644 index 0000000000000..35bd54af63131 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotMessage.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.ignite.internal.processors.cache.persistence.snapshot; + +import java.io.Externalizable; +import java.nio.ByteBuffer; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; + +/** + * + */ +abstract class AbstractSnapshotMessage implements Message { + /** Unique snapshot name. */ + private String snpName; + + /** + * Empty constructor required for {@link Externalizable}. + */ + protected AbstractSnapshotMessage() { + // No-op. + } + + /** + * @param snpName Unique snapshot name. + */ + protected AbstractSnapshotMessage(String snpName) { + assert U.alphanumericUnderscore(snpName) : snpName; + + this.snpName = snpName; + } + + /** + * @return Unique snapshot name. + */ + public String snapshotName() { + return snpName; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + if (writer.state() == 0) { + if (!writer.writeString("snpName", snpName)) + return false; + + writer.incrementState(); + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + if (reader.state() == 0) { + snpName = reader.readString("snpName"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + } + + return reader.afterMessageRead(AbstractSnapshotMessage.class); + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(AbstractSnapshotMessage.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java index ae8203a04160b..031a20931c701 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java @@ -30,12 +30,15 @@ import java.nio.file.Paths; import java.nio.file.SimpleFileVisitor; import java.nio.file.attribute.BasicFileAttributes; +import java.util.ArrayDeque; import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; +import java.util.Queue; import java.util.Set; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; @@ -43,22 +46,39 @@ import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.atomic.LongAdder; +import java.util.function.BiConsumer; import java.util.function.BiFunction; +import java.util.function.Consumer; import java.util.function.Function; +import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; import org.apache.ignite.IgniteSnapshot; import org.apache.ignite.binary.BinaryType; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.events.DiscoveryEvent; +import org.apache.ignite.failure.FailureContext; +import org.apache.ignite.failure.FailureType; import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.GridTopic; import org.apache.ignite.internal.IgniteFeatures; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.events.DiscoveryCustomEvent; +import org.apache.ignite.internal.managers.communication.GridIoManager; +import org.apache.ignite.internal.managers.communication.GridMessageListener; +import org.apache.ignite.internal.managers.communication.TransmissionCancelledException; +import org.apache.ignite.internal.managers.communication.TransmissionHandler; +import org.apache.ignite.internal.managers.communication.TransmissionMeta; +import org.apache.ignite.internal.managers.communication.TransmissionPolicy; import org.apache.ignite.internal.managers.eventstorage.DiscoveryEventListener; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; @@ -66,6 +86,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware; +import org.apache.ignite.internal.processors.cache.persistence.StorageException; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStore; @@ -107,6 +128,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; import static org.apache.ignite.internal.IgniteFeatures.PERSISTENCE_CACHE_SNAPSHOT; +import static org.apache.ignite.internal.IgniteFeatures.nodeSupports; import static org.apache.ignite.internal.MarshallerContextImpl.mappingFileStoreWorkDir; import static org.apache.ignite.internal.MarshallerContextImpl.saveMappings; import static org.apache.ignite.internal.events.DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT; @@ -117,6 +139,7 @@ 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.PART_FILE_TEMPLATE; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.getPartitionFile; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.getPartitionFileName; import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.DB_DEFAULT_FOLDER; import static org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId.getFlagByPartId; import static org.apache.ignite.internal.util.IgniteUtils.isLocalNodeCoordinator; @@ -126,9 +149,11 @@ /** * Internal implementation of snapshot operations over persistence caches. *

- * These major actions available: + * There are two major actions available: *

    *
  • Create snapshot of the whole cluster cache groups by triggering PME to achieve consistency.
  • + *
  • Create local snapshot of requested cache groups and send it to the node which request this operation. + * Cache groups will be transmitted using internal API for transferring files. See {@link TransmissionHandler}.
  • *
*/ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter @@ -145,9 +170,15 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter /** Text Reason for checkpoint to start snapshot operation. */ public static final String CP_SNAPSHOT_REASON = "Checkpoint started to enforce snapshot operation: %s"; + /** Name prefix for each remote snapshot operation. */ + public static final String RMT_SNAPSHOT_PREFIX = "snapshot_"; + /** Default snapshot directory for loading remote snapshots. */ public static final String DFLT_SNAPSHOT_TMP_DIR = "snp"; + /** Timeout in millisecond for snapshot operations. */ + public static final long DFLT_SNAPSHOT_TIMEOUT = 15_000L; + /** Snapshot in progress error message. */ public static final String SNP_IN_PROGRESS_ERR_MSG = "Operation rejected due to the snapshot operation in progress."; @@ -167,6 +198,27 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter /** Total number of thread to perform local snapshot. */ private static final int SNAPSHOT_THREAD_POOL_SIZE = 4; + /** Default snapshot topic to receive snapshots from remote node. */ + private static final Object DFLT_INITIAL_SNAPSHOT_TOPIC = GridTopic.TOPIC_SNAPSHOT.topic("rmt_snp"); + + /** File transmission parameter of cache group id. */ + private static final String SNP_GRP_ID_PARAM = "grpId"; + + /** File transmission parameter of cache partition id. */ + private static final String SNP_PART_ID_PARAM = "partId"; + + /** File transmission parameter of node-sender directory path with its consistentId (e.g. db/IgniteNode0). */ + private static final String SNP_DB_NODE_PATH_PARAM = "dbNodePath"; + + /** File transmission parameter of a cache directory with is currently sends its partitions. */ + private static final String SNP_CACHE_DIR_NAME_PARAM = "cacheDirName"; + + /** Snapshot parameter name for a file transmission. */ + private static final String SNP_NAME_PARAM = "snpName"; + + /** Total snapshot files count which receiver should expect to receive. */ + private static final String SNP_PARTITIONS_CNT = "partsCnt"; + /** * Local buffer to perform copy-on-write operations with pages for {@code SnapshotFutureTask.PageStoreSerialWriter}s. * It is important to have only only buffer per thread (instead of creating each buffer per @@ -181,6 +233,9 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter /** Lock to protect the resources is used. */ private final GridBusyLock busyLock = new GridBusyLock(); + /** Requested snapshot from remote node. */ + private final AtomicReference rmtSnpReq = new AtomicReference<>(); + /** Mutex used to order cluster snapshot operation progress. */ private final Object snpOpMux = new Object(); @@ -321,6 +376,86 @@ public static String partDeltaFileName(int partId) { cctx.exchange().registerExchangeAwareComponent(this); ctx.internalSubscriptionProcessor().registerMetastorageListener(this); + // Receive remote snapshots requests. + cctx.gridIO().addMessageListener(DFLT_INITIAL_SNAPSHOT_TOPIC, new GridMessageListener() { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { + if (!busyLock.enterBusy()) + return; + + try { + if (msg instanceof SnapshotRequestMessage) { + SnapshotRequestMessage reqMsg0 = (SnapshotRequestMessage)msg; + String snpName = reqMsg0.snapshotName(); + + synchronized (this) { + SnapshotFutureTask task = lastScheduledRemoteSnapshotTask(nodeId); + + if (task != null) { + // Task will also be removed from local map due to the listener on future done. + task.cancel(); + + log.info("Snapshot request has been cancelled due to another request received " + + "[prevSnpResp=" + task + ", msg0=" + reqMsg0 + ']'); + } + } + + SnapshotFutureTask task = registerSnapshotTask(snpName, + nodeId, + reqMsg0.parts(), + remoteSnapshotSender(snpName, nodeId)); + + task.listen(f -> { + if (f.error() == null) + return; + + U.error(log, "Failed to process request of creating a snapshot " + + "[from=" + nodeId + ", msg=" + reqMsg0 + ']', f.error()); + + try { + cctx.gridIO().sendToCustomTopic(nodeId, + DFLT_INITIAL_SNAPSHOT_TOPIC, + new SnapshotResponseMessage(reqMsg0.snapshotName(), f.error().getMessage()), + SYSTEM_POOL); + } + catch (IgniteCheckedException ex0) { + U.error(log, "Fail to send the response message with processing snapshot request " + + "error [request=" + reqMsg0 + ", nodeId=" + nodeId + ']', ex0); + } + }); + + task.start(); + } + else if (msg instanceof SnapshotResponseMessage) { + SnapshotResponseMessage respMsg0 = (SnapshotResponseMessage)msg; + + RemoteSnapshotFuture fut0 = rmtSnpReq.get(); + + if (fut0 == null || !fut0.snpName.equals(respMsg0.snapshotName())) { + if (log.isInfoEnabled()) { + log.info("A stale snapshot response message has been received. Will be ignored " + + "[fromNodeId=" + nodeId + ", response=" + respMsg0 + ']'); + } + + return; + } + + if (respMsg0.errorMessage() != null) { + fut0.onDone(new IgniteCheckedException("Request cancelled. The snapshot operation stopped " + + "on the remote node with an error: " + respMsg0.errorMessage())); + } + } + } + catch (Throwable e) { + U.error(log, "Processing snapshot request from remote node fails with an error", e); + + cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e)); + } + finally { + busyLock.leaveBusy(); + } + } + }); + cctx.gridEvents().addDiscoveryEventListener(discoLsnr = (evt, discoCache) -> { if (!busyLock.enterBusy()) return; @@ -340,12 +475,213 @@ public static String partDeltaFileName(int partId) { "One of baseline nodes left the cluster: " + leftNodeId)); } } + + RemoteSnapshotFuture snpTrFut = rmtSnpReq.get(); + + if (snpTrFut != null && snpTrFut.rmtNodeId.equals(leftNodeId)) { + snpTrFut.onDone(new ClusterTopologyCheckedException("The node from which a snapshot has been " + + "requested left the grid")); + } } } finally { busyLock.leaveBusy(); } }, EVT_NODE_LEFT, EVT_NODE_FAILED); + + // Remote snapshot handler. + cctx.kernalContext().io().addTransmissionHandler(DFLT_INITIAL_SNAPSHOT_TOPIC, new TransmissionHandler() { + @Override public void onEnd(UUID nodeId) { + RemoteSnapshotFuture snpTrFut = rmtSnpReq.get(); + + assert snpTrFut.stores.isEmpty() : snpTrFut.stores.entrySet(); + assert snpTrFut.partsLeft == 0 : snpTrFut; + + snpTrFut.onDone(); + + log.info("Requested snapshot from remote node has been fully received " + + "[snpName=" + snpTrFut.snpName + ", snpTrans=" + snpTrFut + ']'); + } + + /** {@inheritDoc} */ + @Override public void onException(UUID nodeId, Throwable err) { + RemoteSnapshotFuture fut = rmtSnpReq.get(); + + if (fut == null) + return; + + if (fut.rmtNodeId.equals(nodeId)) + fut.onDone(err); + } + + /** {@inheritDoc} */ + @Override public String filePath(UUID nodeId, TransmissionMeta fileMeta) { + Integer partId = (Integer)fileMeta.params().get(SNP_PART_ID_PARAM); + String rmtDbNodePath = (String)fileMeta.params().get(SNP_DB_NODE_PATH_PARAM); + String cacheDirName = (String)fileMeta.params().get(SNP_CACHE_DIR_NAME_PARAM); + + RemoteSnapshotFuture transFut = resolve(nodeId, fileMeta); + + try { + File cacheDir = U.resolveWorkDirectory(tmpWorkDir.getAbsolutePath(), + Paths.get(transFut.snpName, rmtDbNodePath, cacheDirName).toString(), + false); + + return new File(cacheDir, getPartitionFileName(partId)).getAbsolutePath(); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + + /** + * @param nodeId Remote node id. + * @param meta Transmission meta. + * @return Resolved transmission future. + */ + private RemoteSnapshotFuture resolve(UUID nodeId, TransmissionMeta meta) { + String snpName = (String)meta.params().get(SNP_NAME_PARAM); + Integer partsCnt = (Integer)meta.params().get(SNP_PARTITIONS_CNT); + + RemoteSnapshotFuture snpTrFut = rmtSnpReq.get(); + + if (snpTrFut == null || !snpTrFut.snpName.equals(snpName)) { + throw new TransmissionCancelledException("Stale snapshot transmission will be ignored " + + "[snpName=" + snpName + ", meta=" + meta + ", snpTrFut=" + snpTrFut + ']'); + } + + assert snpTrFut.snpName.equals(snpName) && snpTrFut.rmtNodeId.equals(nodeId) : + "Another transmission in progress [snpTrFut=" + snpTrFut + ", nodeId=" + snpName + ']'; + + if (snpTrFut.partsLeft == -1) + snpTrFut.partsLeft = partsCnt; + + return snpTrFut; + } + + /** + * @param snpTrans Current snapshot transmission. + * @param grpPartId Pair of group id and its partition id. + */ + private void finishRecover(RemoteSnapshotFuture snpTrans, GroupPartitionId grpPartId) { + FilePageStore pageStore = null; + + try { + pageStore = snpTrans.stores.remove(grpPartId); + + pageStore.finishRecover(); + + snpTrans.partConsumer.accept(new File(pageStore.getFileAbsolutePath()), grpPartId); + + snpTrans.partsLeft--; + } + catch (StorageException e) { + throw new IgniteException(e); + } + finally { + U.closeQuiet(pageStore); + } + } + + /** {@inheritDoc} */ + @Override public Consumer chunkHandler(UUID nodeId, TransmissionMeta initMeta) { + Integer grpId = (Integer)initMeta.params().get(SNP_GRP_ID_PARAM); + Integer partId = (Integer)initMeta.params().get(SNP_PART_ID_PARAM); + + RemoteSnapshotFuture snpTrFut = resolve(nodeId, initMeta); + + GroupPartitionId grpPartId = new GroupPartitionId(grpId, partId); + FilePageStore pageStore = snpTrFut.stores.get(grpPartId); + + if (pageStore == null) { + throw new IgniteException("Partition must be loaded before applying snapshot delta pages " + + "[snpName=" + snpTrFut.snpName + ", grpId=" + grpId + ", partId=" + partId + ']'); + } + + pageStore.beginRecover(); + + // No snapshot delta pages received. Finalize recovery. + if (initMeta.count() == 0) + finishRecover(snpTrFut, grpPartId); + + return new Consumer() { + final LongAdder transferred = new LongAdder(); + + @Override public void accept(ByteBuffer buff) { + try { + assert initMeta.count() != 0 : initMeta; + + RemoteSnapshotFuture fut0 = rmtSnpReq.get(); + + if (fut0 == null || !fut0.equals(snpTrFut) || fut0.isCancelled()) { + throw new TransmissionCancelledException("Snapshot request is cancelled " + + "[snpName=" + snpTrFut.snpName + ", grpId=" + grpId + ", partId=" + partId + ']'); + } + + pageStore.write(PageIO.getPageId(buff), buff, 0, false); + + transferred.add(buff.capacity()); + + if (transferred.longValue() == initMeta.count()) + finishRecover(snpTrFut, grpPartId); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + }; + } + + /** {@inheritDoc} */ + @Override public Consumer fileHandler(UUID nodeId, TransmissionMeta initMeta) { + Integer grpId = (Integer)initMeta.params().get(SNP_GRP_ID_PARAM); + Integer partId = (Integer)initMeta.params().get(SNP_PART_ID_PARAM); + String snpName = (String)initMeta.params().get(SNP_NAME_PARAM); + + assert grpId != null; + assert partId != null; + assert snpName != null; + assert storeFactory != null; + + RemoteSnapshotFuture transFut = rmtSnpReq.get(); + + if (transFut == null) { + throw new IgniteException("Snapshot transmission with given name doesn't exists " + + "[snpName=" + snpName + ", grpId=" + grpId + ", partId=" + partId + ']'); + } + + return new Consumer() { + @Override public void accept(File file) { + RemoteSnapshotFuture fut0 = rmtSnpReq.get(); + + if (fut0 == null || !fut0.equals(transFut) || fut0.isCancelled()) { + throw new TransmissionCancelledException("Snapshot request is cancelled [snpName=" + snpName + + ", grpId=" + grpId + ", partId=" + partId + ']'); + } + + busyLock.enterBusy(); + + try { + FilePageStore pageStore = (FilePageStore)storeFactory + .apply(grpId, false) + .createPageStore(getFlagByPartId(partId), + file::toPath, + new LongAdderMetric("NO_OP", null)); + + transFut.stores.put(new GroupPartitionId(grpId, partId), pageStore); + + pageStore.init(); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + finally { + busyLock.leaveBusy(); + } + } + }; + } + }); } /** {@inheritDoc} */ @@ -359,6 +695,11 @@ public static String partDeltaFileName(int partId) { locSnpTasks.clear(); + RemoteSnapshotFuture snpTrFut = rmtSnpReq.get(); + + if (snpTrFut != null) + snpTrFut.cancel(); + synchronized (snpOpMux) { if (clusterSnpFut != null) { clusterSnpFut.onDone(new NodeStoppingException(SNP_NODE_STOPPING_ERR_MSG)); @@ -370,6 +711,9 @@ public static String partDeltaFileName(int partId) { if (snpRunner != null) snpRunner.shutdownNow(); + cctx.kernalContext().io().removeMessageListener(DFLT_INITIAL_SNAPSHOT_TOPIC); + cctx.kernalContext().io().removeTransmissionHandler(DFLT_INITIAL_SNAPSHOT_TOPIC); + if (discoLsnr != null) cctx.kernalContext().event().removeDiscoveryEventListener(discoLsnr); @@ -764,6 +1108,78 @@ public static boolean isSnapshotOperation(DiscoveryEvent evt) { } } + /** + * @param parts Collection of pairs group and appropriate cache partition to be snapshot. + * @param rmtNodeId The remote node to connect to. + * @param partConsumer Received partition handler. + * @return Future which will be completed when requested snapshot fully received. + */ + public IgniteInternalFuture requestRemoteSnapshot( + UUID rmtNodeId, + Map> parts, + BiConsumer partConsumer + ) { + assert partConsumer != null; + + ClusterNode rmtNode = cctx.discovery().node(rmtNodeId); + + if (!nodeSupports(rmtNode, PERSISTENCE_CACHE_SNAPSHOT)) + return new GridFinishedFuture<>(new IgniteCheckedException("Snapshot on remote node is not supported: " + rmtNode.id())); + + if (rmtNode == null) { + return new GridFinishedFuture<>(new ClusterTopologyCheckedException("Snapshot request cannot be performed. " + + "Remote node left the grid [rmtNodeId=" + rmtNodeId + ']')); + } + + String snpName = RMT_SNAPSHOT_PREFIX + U.maskForFileName(UUID.randomUUID().toString()); + + RemoteSnapshotFuture snpTransFut = new RemoteSnapshotFuture(rmtNodeId, snpName, partConsumer); + + busyLock.enterBusy(); + SnapshotRequestMessage msg0; + + try { + msg0 = new SnapshotRequestMessage(snpName, parts); + + RemoteSnapshotFuture fut = rmtSnpReq.get(); + + try { + if (fut != null) + fut.get(DFLT_SNAPSHOT_TIMEOUT, TimeUnit.MILLISECONDS); + } + catch (IgniteCheckedException e) { + if (log.isInfoEnabled()) + log.info("The previous snapshot request finished with an exception:" + e.getMessage()); + } + + try { + if (rmtSnpReq.compareAndSet(null, snpTransFut)) { + cctx.gridIO().sendOrderedMessage(rmtNode, DFLT_INITIAL_SNAPSHOT_TOPIC, msg0, SYSTEM_POOL, + Long.MAX_VALUE, true); + } + else + return new GridFinishedFuture<>(new IgniteCheckedException("Snapshot request has been concurrently interrupted.")); + + } + catch (IgniteCheckedException e) { + rmtSnpReq.compareAndSet(snpTransFut, null); + + return new GridFinishedFuture<>(e); + } + } + finally { + busyLock.leaveBusy(); + } + + if (log.isInfoEnabled()) { + log.info("Snapshot request is sent to the remote node [rmtNodeId=" + rmtNodeId + + ", msg0=" + msg0 + ", snpTransFut=" + snpTransFut + + ", topVer=" + cctx.discovery().topologyVersionEx() + ']'); + } + + return snpTransFut; + } + /** * @param grps List of cache groups which will be destroyed. */ @@ -842,6 +1258,20 @@ Function localSnapshotSenderFactory() { return locSndrFactory; } + /** + * @param snpName Snapshot name. + * @param rmtNodeId Remote node id to send snapshot to. + * @return Snapshot sender instance. + */ + SnapshotSender remoteSnapshotSender(String snpName, UUID rmtNodeId) { + // Remote snapshots can be send only by single threaded executor since only one transmissionSender created. + return new RemoteSnapshotSender(log, + new SequentialExecutorWrapper(log, snpRunner), + () -> databaseRelativePath(pdsSettings.folderName()), + cctx.gridIO().openTransmissionSender(rmtNodeId, DFLT_INITIAL_SNAPSHOT_TOPIC), + snpName); + } + /** Snapshot finished successfully or already restored. Key can be removed. */ private void removeLastMetaStorageKey() throws IgniteCheckedException { cctx.database().checkpointReadLock(); @@ -870,6 +1300,17 @@ void ioFactory(FileIOFactory ioFactory) { this.ioFactory = ioFactory; } + /** + * @param nodeId Remote node id on which requests has been registered. + * @return Snapshot future related to given node id. + */ + SnapshotFutureTask lastScheduledRemoteSnapshotTask(UUID nodeId) { + return locSnpTasks.values().stream() + .filter(t -> t.type() == RemoteSnapshotSender.class && t.sourceNodeId().equals(nodeId)) + .findFirst() + .orElse(null); + } + /** * @return Relative configured path of persistence data storage directory for the local node. * Example: {@code snapshotWorkDir/db/IgniteNodeName0} @@ -917,6 +1358,282 @@ static void copy(FileIOFactory factory, File from, File to, long length) { } } + /** Remote snapshot future which tracks remote snapshot transmission result. */ + private class RemoteSnapshotFuture extends GridFutureAdapter { + /** Snapshot name to create. */ + private final String snpName; + + /** Remote node id to request snapshot from. */ + private final UUID rmtNodeId; + + /** Collection of partition to be received. */ + private final Map stores = new ConcurrentHashMap<>(); + + /** Partition handler given by request initiator. */ + private final BiConsumer partConsumer; + + /** Counter which show how many partitions left to be received. */ + private int partsLeft = -1; + + /** + * @param partConsumer Received partition handler. + */ + public RemoteSnapshotFuture(UUID rmtNodeId, String snpName, BiConsumer partConsumer) { + this.snpName = snpName; + this.rmtNodeId = rmtNodeId; + this.partConsumer = partConsumer; + } + + /** {@inheritDoc} */ + @Override public boolean cancel() { + return onCancelled(); + } + + /** {@inheritDoc} */ + @Override protected boolean onDone(@Nullable Void res, @Nullable Throwable err, boolean cancel) { + assert err != null || cancel || stores.isEmpty() : "Not all file storage processed: " + stores; + + rmtSnpReq.compareAndSet(this, null); + + if (err != null || cancel) { + // Close non finished file storage. + for (Map.Entry entry : stores.entrySet()) { + FilePageStore store = entry.getValue(); + + try { + store.stop(true); + } + catch (StorageException e) { + log.warning("Error stopping received file page store", e); + } + } + } + + U.delete(Paths.get(tmpWorkDir.getAbsolutePath(), snpName)); + + return super.onDone(res, err, cancel); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + RemoteSnapshotFuture fut = (RemoteSnapshotFuture)o; + + return rmtNodeId.equals(fut.rmtNodeId) && + snpName.equals(fut.snpName); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return Objects.hash(rmtNodeId, snpName); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(RemoteSnapshotFuture.class, this); + } + } + + /** + * Such an executor can executes tasks not in a single thread, but executes them + * on different threads sequentially. It's important for some {@link SnapshotSender}'s + * to process sub-task sequentially due to all these sub-tasks may share a single socket + * channel to send data to. + */ + private static class SequentialExecutorWrapper implements Executor { + /** Ignite logger. */ + private final IgniteLogger log; + + /** Queue of task to execute. */ + private final Queue tasks = new ArrayDeque<>(); + + /** Delegate executor. */ + private final Executor executor; + + /** Currently running task. */ + private volatile Runnable active; + + /** If wrapped executor is shutting down. */ + private volatile boolean stopping; + + /** + * @param executor Executor to run tasks on. + */ + public SequentialExecutorWrapper(IgniteLogger log, Executor executor) { + this.log = log.getLogger(SequentialExecutorWrapper.class); + this.executor = executor; + } + + /** {@inheritDoc} */ + @Override public synchronized void execute(final Runnable r) { + assert !stopping : "Task must be cancelled prior to the wrapped executor is shutting down."; + + tasks.offer(() -> { + try { + r.run(); + } + finally { + scheduleNext(); + } + }); + + if (active == null) + scheduleNext(); + } + + /** */ + protected synchronized void scheduleNext() { + if ((active = tasks.poll()) != null) { + try { + executor.execute(active); + } + catch (RejectedExecutionException e) { + tasks.clear(); + + stopping = true; + + log.warning("Task is outdated. Wrapped executor is shutting down.", e); + } + } + } + } + + /** + * + */ + private static class RemoteSnapshotSender extends SnapshotSender { + /** The sender which sends files to remote node. */ + private final GridIoManager.TransmissionSender sndr; + + /** Relative node path initializer. */ + private final Supplier initPath; + + /** Snapshot name. */ + private final String snpName; + + /** Local node persistent directory with consistent id. */ + private String relativeNodePath; + + /** The number of cache partition files expected to be processed. */ + private int partsCnt; + + /** + * @param log Ignite logger. + * @param sndr File sender instance. + * @param snpName Snapshot name. + */ + public RemoteSnapshotSender( + IgniteLogger log, + Executor exec, + Supplier initPath, + GridIoManager.TransmissionSender sndr, + String snpName + ) { + super(log, exec); + + this.sndr = sndr; + this.snpName = snpName; + this.initPath = initPath; + } + + /** {@inheritDoc} */ + @Override protected void init(int partsCnt) { + this.partsCnt = partsCnt; + + relativeNodePath = initPath.get(); + + if (relativeNodePath == null) + throw new IgniteException("Relative node path cannot be empty."); + } + + /** {@inheritDoc} */ + @Override public void sendPart0(File part, String cacheDirName, GroupPartitionId pair, Long len) { + try { + assert part.exists(); + assert len > 0 : "Requested partitions has incorrect file length " + + "[pair=" + pair + ", cacheDirName=" + cacheDirName + ']'; + + sndr.send(part, 0, len, transmissionParams(snpName, cacheDirName, pair), TransmissionPolicy.FILE); + + if (log.isInfoEnabled()) { + log.info("Partition file has been send [part=" + part.getName() + ", pair=" + pair + + ", length=" + len + ']'); + } + } + catch (TransmissionCancelledException e) { + if (log.isInfoEnabled()) { + log.info("Transmission partition file has been interrupted [part=" + part.getName() + + ", pair=" + pair + ']'); + } + } + catch (IgniteCheckedException | InterruptedException | IOException e) { + U.error(log, "Error sending partition file [part=" + part.getName() + ", pair=" + pair + + ", length=" + len + ']', e); + + throw new IgniteException(e); + } + } + + /** {@inheritDoc} */ + @Override public void sendDelta0(File delta, String cacheDirName, GroupPartitionId pair) { + try { + sndr.send(delta, transmissionParams(snpName, cacheDirName, pair), TransmissionPolicy.CHUNK); + + if (log.isInfoEnabled()) + log.info("Delta pages storage has been send [part=" + delta.getName() + ", pair=" + pair + ']'); + } + catch (TransmissionCancelledException e) { + if (log.isInfoEnabled()) { + log.info("Transmission delta pages has been interrupted [part=" + delta.getName() + + ", pair=" + pair + ']'); + } + } + catch (IgniteCheckedException | InterruptedException | IOException e) { + U.error(log, "Error sending delta file [part=" + delta.getName() + ", pair=" + pair + ']', e); + + throw new IgniteException(e); + } + } + + /** + * @param cacheDirName Cache directory name. + * @param pair Cache group id with corresponding partition id. + * @return Map of params. + */ + private Map transmissionParams(String snpName, String cacheDirName, + GroupPartitionId pair) { + Map params = new HashMap<>(); + + params.put(SNP_GRP_ID_PARAM, pair.getGroupId()); + params.put(SNP_PART_ID_PARAM, pair.getPartitionId()); + params.put(SNP_DB_NODE_PATH_PARAM, relativeNodePath); + params.put(SNP_CACHE_DIR_NAME_PARAM, cacheDirName); + params.put(SNP_NAME_PARAM, snpName); + params.put(SNP_PARTITIONS_CNT, partsCnt); + + return params; + } + + /** {@inheritDoc} */ + @Override public void close0(@Nullable Throwable th) { + U.closeQuiet(sndr); + + if (th == null) { + if (log.isInfoEnabled()) + log.info("The remote snapshot sender closed normally [snpName=" + snpName + ']'); + } + else { + U.warn(log, "The remote snapshot sender closed due to an error occurred while processing " + + "snapshot operation [snpName=" + snpName + ']', th); + } + } + } + /** * Snapshot sender which writes all data to local directory. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRequestMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRequestMessage.java new file mode 100644 index 0000000000000..4797266814e6a --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRequestMessage.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.ignite.internal.processors.cache.persistence.snapshot; + +import java.io.Externalizable; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.ignite.internal.GridDirectMap; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; + +/** + * + */ +public class SnapshotRequestMessage extends AbstractSnapshotMessage { + /** Snapshot request message type (value is {@code 177}). */ + public static final short TYPE_CODE = 177; + + /** Serialization version. */ + private static final long serialVersionUID = 0L; + + /** Map of cache group ids and corresponding set of its partition ids. */ + @GridDirectMap(keyType = Integer.class, valueType = int[].class) + private Map parts; + + /** + * Empty constructor required for {@link Externalizable}. + */ + public SnapshotRequestMessage() { + // No-op. + } + + /** + * @param snpName Unique snapshot name. + * @param parts Map of cache group ids and corresponding set of its partition ids to be snapshot. + */ + public SnapshotRequestMessage(String snpName, Map> parts) { + super(snpName); + + assert parts != null && !parts.isEmpty(); + + this.parts = new HashMap<>(); + + for (Map.Entry> e : parts.entrySet()) + this.parts.put(e.getKey(), U.toIntArray(e.getValue())); + } + + /** + * @return The demanded cache group partitions per each cache group. + */ + public Map> parts() { + Map> res = new HashMap<>(); + + for (Map.Entry e : parts.entrySet()) { + res.put(e.getKey(), e.getValue().length == 0 ? null : Arrays.stream(e.getValue()) + .boxed() + .collect(Collectors.toSet())); + } + + return res; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!super.writeTo(buf, writer)) + return false; + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + if (writer.state() == 1) { + if (!writer.writeMap("parts", parts, MessageCollectionItemType.INT, MessageCollectionItemType.INT_ARR)) + return false; + + writer.incrementState(); + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + if (!super.readFrom(buf, reader)) + return false; + + if (reader.state() == 1) { + parts = reader.readMap("parts", MessageCollectionItemType.INT, MessageCollectionItemType.INT_ARR, false); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + } + + return reader.afterMessageRead(SnapshotRequestMessage.class); + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 2; + } + + /** {@inheritDoc} */ + @Override public short directType() { + return TYPE_CODE; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(SnapshotRequestMessage.class, this, super.toString()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotResponseMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotResponseMessage.java new file mode 100644 index 0000000000000..66118225a21a0 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotResponseMessage.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.ignite.internal.processors.cache.persistence.snapshot; + +import java.io.Externalizable; +import java.nio.ByteBuffer; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; + +/** + * + */ +public class SnapshotResponseMessage extends AbstractSnapshotMessage { + /** Snapshot response message type (value is {@code 178}). */ + public static final short TYPE_CODE = 178; + + /** Serialization version. */ + private static final long serialVersionUID = 0L; + + /** Exception occurred during snapshot processing. */ + private String errMsg; + + /** + * Empty constructor required for {@link Externalizable}. + */ + public SnapshotResponseMessage() { + // No-op. + } + + /** + * @param snpName Snapshot name to which response related to. + * @param errMsg Response error message. + */ + public SnapshotResponseMessage(String snpName, String errMsg) { + super(snpName); + + this.errMsg = errMsg; + } + + /** + * @return Response error message. + */ + public String errorMessage() { + return errMsg; + } + + /** + * @param errMsg Response error message. + * @return {@code this} for chaining. + */ + public SnapshotResponseMessage errorMessage(String errMsg) { + this.errMsg = errMsg; + + return this; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!super.writeTo(buf, writer)) + return false; + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + if (writer.state() == 1) { + if (!writer.writeString("errMsg", errMsg)) + return false; + + writer.incrementState(); + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + if (!super.readFrom(buf, reader)) + return false; + + if (reader.state() == 1) { + errMsg = reader.readString("errMsg"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + } + + return reader.afterMessageRead(SnapshotResponseMessage.class); + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 2; + } + + /** {@inheritDoc} */ + @Override public short directType() { + return TYPE_CODE; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(SnapshotResponseMessage.class, this, super.toString()); + } +} 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 cfdb30c700931..87a3cf417e4c5 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 @@ -917,22 +917,25 @@ private FileWriteHandle closeBufAndRollover( /** {@inheritDoc} */ @Override public WALIterator replay(WALPointer start) throws IgniteCheckedException, StorageException { - return replay(start, null); + return replay(start, true, null); } /** {@inheritDoc} */ @Override public WALIterator replay( WALPointer start, + boolean limited, @Nullable IgniteBiPredicate recordDeserializeFilter ) throws IgniteCheckedException, StorageException { assert start == null || start instanceof FileWALPointer : "Invalid start pointer: " + start; - FileWriteHandle hnd = currentHandle(); - FileWALPointer end = null; - if (hnd != null) - end = hnd.position(); + if (limited) { + FileWriteHandle hnd = currentHandle(); + + if (hnd != null) + end = hnd.position(); + } RecordsIterator iter = new RecordsIterator( cctx, @@ -999,13 +1002,14 @@ private boolean hasIndex(long absIdx) { String zipSegmentName = FileDescriptor.fileName(absIdx) + FilePageStoreManager.ZIP_SUFFIX; + boolean wasInArchive = absIdx <= lastArchivedIndex(); + boolean inArchive = new File(walArchiveDir, segmentName).exists() || new File(walArchiveDir, zipSegmentName).exists(); if (inArchive) return true; - - if (absIdx <= lastArchivedIndex()) + else if (wasInArchive) return false; FileWriteHandle cur = currHnd; @@ -1369,6 +1373,13 @@ private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws St } } + /** + * @return WAL working directory. + */ + public File walWorkDir() { + return walWorkDir; + } + /** * Fills the file header for a new segment. Calling this method signals we are done with the segment and it can be * archived. If we don't have prepared file yet and achiever is busy this method blocks. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java index cb5bfc1217f9b..c9820004d3e42 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java @@ -378,7 +378,7 @@ private FileDescriptor readFileDescriptor(File file, FileIOFactory ioFactory) { kernalCtx, null, null, null, null, null, null, dbMgr, null, null, null, null, null, null, null, - null, null, null, null, null, null + null, null, null, null, null, null, null ); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/BaselineTopologyHistoryItem.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/BaselineTopologyHistoryItem.java index 4b2f0b5f2fbb9..2cddcfb103d1a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/BaselineTopologyHistoryItem.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/BaselineTopologyHistoryItem.java @@ -76,4 +76,11 @@ public int id() { public List branchingHistory() { return branchingHistory; } + + /** + * @return Collection of consistent IDs. + */ + public Collection consistentIds() { + return consIds; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 3539093829cdd..cbd816ee7dc0f 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -4517,12 +4517,18 @@ assert nodeSupports(remote, CHANNEL_COMMUNICATION) : "Node doesn't support direc } addTimeoutObject(new IgniteSpiTimeoutObject() { + /** */ + private final long endTime = U.currentTimeMillis() + connTimeout; + + /** */ + private final IgniteUuid id = IgniteUuid.randomUuid(); + @Override public IgniteUuid id() { - return IgniteUuid.randomUuid(); + return id; } @Override public long endTime() { - return U.currentTimeMillis() + connTimeout; + return endTime; } @Override public void onTimeout() { diff --git a/modules/core/src/test/config/log4j-test.xml b/modules/core/src/test/config/log4j-test.xml index 0df622f461c40..5e3d0680d4032 100755 --- a/modules/core/src/test/config/log4j-test.xml +++ b/modules/core/src/test/config/log4j-test.xml @@ -31,7 +31,7 @@ - + @@ -40,7 +40,7 @@ - + @@ -65,7 +65,7 @@ Logs all output to specified file. --> - + @@ -75,6 +75,14 @@ + + + + + + + + @@ -106,6 +114,34 @@ --> + + diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java index abb5fb749f901..58b60df0132e2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridIoManagerFileTransmissionSelfTest.java @@ -182,6 +182,8 @@ public void testFileHandlerBase() throws Exception { awaitPartitionMapExchange(); + forceCheckpoint(); + Map fileSizes = new HashMap<>(); Map fileCrcs = new HashMap<>(); Map fileParams = new HashMap<>(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheFileRebalancingAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheFileRebalancingAbstractTest.java new file mode 100644 index 0000000000000..680fcda8a1f57 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheFileRebalancingAbstractTest.java @@ -0,0 +1,1080 @@ +/* + * 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; + +import java.io.File; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cluster.BaselineNode; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.cluster.ClusterState; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.TestRecordingCommunicationSpi; +import org.apache.ignite.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheGroupIdMessage; +import org.apache.ignite.internal.processors.cache.IgniteInternalCache; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStore; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRequestMessage; +import org.apache.ignite.internal.util.GridConcurrentHashSet; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.internal.util.typedef.P2; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiInClosure; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.WithSystemProperty; +import org.junit.Test; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_FILE_REBALANCE_ENABLED; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_FILE_REBALANCE_THRESHOLD; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD; + +/** + * File rebalancing tests. + */ +@WithSystemProperty(key = IGNITE_FILE_REBALANCE_ENABLED, value = "true") +@WithSystemProperty(key = IGNITE_FILE_REBALANCE_THRESHOLD, value = "0") +public abstract class IgniteCacheFileRebalancingAbstractTest extends IgnitePdsCacheRebalancingCommonAbstractTest { + /** Initial entries count. */ + private static final int INITIAL_ENTRIES_COUNT = 100_000; + + /** */ + private static final long AWAIT_TIME_MILLIS = 15_000; + + /** */ + private static final int DFLT_LOADER_THREADS = Math.max(2, Runtime.getRuntime().availableProcessors() / 2); + + /** */ + private static final String SHARED_GROUP = "shared_group"; + + /** */ + private static final String SHARED1 = "shared1"; + + /** */ + private static final String SHARED2 = "shared2"; + + /** */ + private final Function testValProducer = n -> new TestValue(n, n, n); + + /** */ + private final Set requestedGroups = new GridConcurrentHashSet<>(); + + /** {@inheritDoc} */ + @Override protected long checkpointFrequency() { + return 5_000; + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 2 * 60 * 1000; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + assertFalse("File rebalance hasn't been triggered.", requestedGroups.isEmpty()); + + super.afterTest(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setCommunicationSpi(new TestRecordingCommunicationSpi() { + @Override public void sendMessage(ClusterNode node, Message msg, + IgniteInClosure ackC) throws IgniteSpiException { + if (((GridIoMessage)msg).message() instanceof SnapshotRequestMessage) { + SnapshotRequestMessage msg0 = ((SnapshotRequestMessage)((GridIoMessage)msg).message()); + + requestedGroups.addAll(msg0.parts().keySet()); + } + + super.sendMessage(node, msg, ackC); + } + }); + + // Add 2 more cache (shared cache group). + CacheConfiguration[] ccfgs = cfg.getCacheConfiguration(); + + int len = ccfgs.length; + int rebalanceOrder = 10; + + CacheConfiguration[] ccfgs0 = Arrays.copyOf(ccfgs, ccfgs.length + 2); + + ccfgs0[len] = cacheConfiguration(SHARED1).setGroupName(SHARED_GROUP).setRebalanceOrder(rebalanceOrder); + ccfgs0[len + 1] = cacheConfiguration(SHARED2).setGroupName(SHARED_GROUP).setRebalanceOrder(rebalanceOrder); + + cfg.setCacheConfiguration(ccfgs0); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + @Test + public void testSimpleRebalancing() throws Exception { + IgniteEx ignite0 = startGrid(0, true); + + LoadParameters idxCache = testValuesLoader(false, DFLT_LOADER_THREADS).loadData(ignite0); + + LoadParameters replicatedCache = new DataLoader<>( + grid(0).cache(CACHE), + INITIAL_ENTRIES_COUNT, + n -> n, + true, + DFLT_LOADER_THREADS + ).loadData(ignite0); + + forceCheckpoint(ignite0); + + IgniteEx ignite1 = startGrid(1); + + ignite0.cluster().setBaselineTopology(2); + + awaitPartitionMapExchange(); + + int expParts = ignite0.cachex(INDEXED_CACHE).context().affinity().partitions() + + ignite0.cachex(CACHE).context().affinity().partitions(); + + assertTrue(requestedGroups.contains(CU.cacheId(INDEXED_CACHE))); + assertTrue(requestedGroups.contains(CU.cacheId(CACHE))); + + verifyCache(ignite1, idxCache); + verifyCache(ignite1, replicatedCache); + } + + /** + * @throws Exception If failed. + */ + @Test + @WithSystemProperty(key = IGNITE_PDS_WAL_REBALANCE_THRESHOLD, value = "0") + public void testHistoricalStartsAfterFilesPreloading() throws Exception { + assert backups() > 0 : backups(); + + IgniteEx ignite0 = startGrid(0, true); + + DataLoader ldr = testValuesLoader(false, DFLT_LOADER_THREADS).loadData(ignite0); + + ldr.start(); + + forceCheckpoint(ignite0); + + IgniteEx ignite1 = startGrid(1); + + IgniteInternalCache cache = ignite0.cachex(INDEXED_CACHE); + + int totalPartitions = cache.affinity().partitions(); + + TestRecordingCommunicationSpi recCommSpi = TestRecordingCommunicationSpi.spi(ignite1); + + P2 msgPred = + (node, msg) -> (msg instanceof GridDhtPartitionDemandMessage) && + ((GridCacheGroupIdMessage)msg).groupId() == cache.context().groupId() && + ((GridDhtPartitionDemandMessage)msg).partitions().historicalSet().size() == totalPartitions; + + // Recording all historical demand messages. + recCommSpi.record(msgPred); + + IgniteInternalFuture loadPartsFut = + waitForPartitions(ignite1.cachex(INDEXED_CACHE).context(), totalPartitions); + + recCommSpi.blockMessages(msgPred); + + // After baseline has changed.file rebalance should start. + ignite0.cluster().setBaselineTopology(2); + + // Wait until partition files received. + loadPartsFut.get(); + + // File rebalancing should request historiacal rebalance for loaded partitions. + recCommSpi.waitForBlocked(); + + // Changing minor topology version to interrupt rebalancing routine. + ignite1.getOrCreateCache(new CacheConfiguration<>("tmp-cache")); + + // Current rebalance routine should be restarted. + recCommSpi.stopBlock(); + + awaitPartitionMapExchange(); + + ldr.stop(); + + verifyCache(ignite1, ldr); + + List msgs = recCommSpi.recordedMessages(true); + + assertEquals("Expecting specified count demand messages for historical rebalance.", 2, msgs.size()); + } + + /** + * @throws Exception If failed. + */ + @Test + public void testConsistencyOnFilesPreloadingInterruption() throws Exception { + assert backups() > 0 : backups(); + + IgniteEx ignite0 = startGrid(0, true); + + DataLoader ldr = testValuesLoader(false, DFLT_LOADER_THREADS).loadData(ignite0); + + ldr.start(); + + forceCheckpoint(ignite0); + + IgniteEx ignite1 = startGrid(1); + + IgniteInternalCache cache = ignite0.cachex(INDEXED_CACHE); + + int totalPartitions = cache.affinity().partitions(); + + IgniteInternalFuture loadPartsFut = + waitForPartitions(ignite1.cachex(INDEXED_CACHE).context(), totalPartitions / 4); + + // After baseline has changed.file rebalance should start. + ignite0.cluster().setBaselineTopology(2); + + // Wait until some partition files received. + loadPartsFut.get(); + + // Switching this partitions from read-only to normal mode. + forceCheckpoint(ignite1); + + // Changing minor topology version to interrupt rebalancing routine. + ignite1.getOrCreateCache(new CacheConfiguration<>("tmp-cache")); + + awaitPartitionMapExchange(); + + ldr.stop(); + + // If historical rebalancing starts after interrupting file preloading, + // we'll get inconsistent indexes and cache verification should fail. + verifyCache(ignite1, ldr); + } + + /** + * @param ctx Cache context. + * @param expectedCnt Expected number of existing partitions. + * @return The future, which will be completed when the number of existing partition files on the disk is greater + * than or equal to the specified. + */ + private IgniteInternalFuture waitForPartitions(GridCacheContext ctx, int expectedCnt) { + return GridTestUtils.runAsync(() -> { + boolean success = GridTestUtils.waitForCondition( + () -> { + int cnt = 0; + + try { + for (GridDhtLocalPartition part : ctx.topology().currentLocalPartitions()) { + FilePageStoreManager storeMgr = (FilePageStoreManager)ctx.shared().pageStore(); + + FilePageStore fileStore = (FilePageStore)storeMgr.getStore(ctx.groupId(), part.id()); + + if (new File(fileStore.getFileAbsolutePath()).exists()) + ++cnt; + } + } + catch (IgniteCheckedException e) { + throw new RuntimeException(e); + } + + return cnt >= expectedCnt; + }, + AWAIT_TIME_MILLIS); + + assertTrue(success); + + return true; + }); + } + + /** + * @throws Exception If failed. + */ + @Test + public void testSimpleRebalancingWithLoad() throws Exception { + boolean checkRemoves = true; + + IgniteEx ignite0 = startGrid(0, true); + + DataLoader idxLdr = testValuesLoader(checkRemoves, DFLT_LOADER_THREADS).loadData(ignite0); + + DataLoader cacheLdr = new DataLoader<>( + grid(0).cache(CACHE), + INITIAL_ENTRIES_COUNT, + n -> n, + true, + DFLT_LOADER_THREADS + ).loadData(ignite0); + + idxLdr.start(); + cacheLdr.start(); + + forceCheckpoint(ignite0); + + U.sleep(1_000); + + IgniteEx ignite1 = startGrid(1); + + ignite0.cluster().setBaselineTopology(2); + + awaitPartitionMapExchange(); + + idxLdr.stop(); + cacheLdr.stop(); + + assertTrue(requestedGroups.contains(CU.cacheId(INDEXED_CACHE))); + assertTrue(requestedGroups.contains(CU.cacheId(CACHE))); + + verifyCache(ignite1, idxLdr); + verifyCache(ignite1, cacheLdr); + } + + /** + * @throws Exception If failed. + */ + @Test + public void testSimpleRebalancingSharedGroupOrdered() throws Exception { + boolean checkRemoves = true; + + IgniteEx ignite0 = startGrid(0, true); + + DataLoader idxLdr = testValuesLoader(checkRemoves, 1).loadData(ignite0); + + DataLoader sharedLdr1 = new DataLoader<>( + grid(0).cache(SHARED1), + INITIAL_ENTRIES_COUNT, + n -> n, + true, + 1 + ).loadData(ignite0); + + DataLoader sharedLdr2 = new DataLoader<>( + grid(0).cache(SHARED2), + INITIAL_ENTRIES_COUNT, + n -> n, + true, + 1 + ).loadData(ignite0); + + idxLdr.start(); + sharedLdr1.start(); + sharedLdr2.start(); + + forceCheckpoint(ignite0); + + U.sleep(1_000); + + IgniteEx ignite1 = startGrid(1); + + ignite0.cluster().setBaselineTopology(2); + + awaitPartitionMapExchange(); + + idxLdr.stop(); + sharedLdr1.stop(); + sharedLdr2.stop(); + + assertTrue(requestedGroups.contains(CU.cacheId(INDEXED_CACHE))); + assertTrue(requestedGroups.contains(CU.cacheId(SHARED_GROUP))); + + verifyCache(ignite1, idxLdr); + verifyCache(ignite1, sharedLdr1); + verifyCache(ignite1, sharedLdr2); + } + + /** + * Check file rebalancing when the coordinator joins the baseline. + * + * @throws Exception If failed. + */ + @Test + public void testCoordinatorJoinsBaselineWithLoad() throws Exception { + boolean checkRemoves = false; + + startGrid(0, true); + + IgniteEx crd = startGrid(1); + + stopGrid(0); + + IgniteEx node = startGrid(0); + + awaitPartitionMapExchange(); + + Collection constIds = + F.viewReadOnly(node.cluster().currentBaselineTopology(), BaselineNode::consistentId); + + // Ensure that coordinator node is not in baseline. + assert U.oldest(crd.cluster().nodes(), null).equals(crd.localNode()); + assert !constIds.contains(crd.localNode().consistentId()) : constIds; + assert constIds.contains(node.localNode().consistentId()) : constIds; + + DataLoader ldr = new DataLoader<>( + node.cache(INDEXED_CACHE), + INITIAL_ENTRIES_COUNT, + testValProducer, + checkRemoves, + DFLT_LOADER_THREADS + ).loadData(node); + + forceCheckpoint(node); + + ldr.start(); + + List blt = new ArrayList<>(); + + blt.add(node.localNode()); + blt.add(crd.localNode()); + + node.cluster().setBaselineTopology(blt); + + awaitPartitionMapExchange(); + + ldr.stop(); + + verifyCache(crd, ldr); + } + + /** + * Check file rebalancing from multiple suppliers. + * + * @throws Exception If failed. + */ + @Test + public void testMultipleSuppliers() throws Exception { + boolean checkRemoves = false; + + int initGrids = Math.max(1, backups()) * 3; + + IgniteEx node0 = (IgniteEx)startGridsMultiThreaded(initGrids); + + node0.cluster().state(ClusterState.ACTIVE); + node0.cluster().baselineAutoAdjustEnabled(false); + + List blt = new ArrayList<>(node0.context().discovery().aliveServerNodes()); + + node0.cluster().setBaselineTopology(blt); + + awaitPartitionMapExchange(); + + DataLoader ldr = testValuesLoader(checkRemoves, DFLT_LOADER_THREADS); + + ldr.loadData(node0); + + forceCheckpoint(); + + ldr.start(); + + IgniteEx node = startGrid(initGrids); + + Set remotes = new GridConcurrentHashSet<>(); + + IgniteBiInClosure c = (rmtNode, msg) -> { + if (msg instanceof SnapshotRequestMessage) + remotes.add(rmtNode.id()); + }; + + TestRecordingCommunicationSpi.spi(node).closure(c); + + blt.add(node.cluster().localNode()); + + node0.cluster().setBaselineTopology(blt); + + awaitPartitionMapExchange(); + + ldr.stop(); + + int rmtCnt = remotes.size(); + + assertTrue("Snapshot was not requested from multiple nodes [count=" + rmtCnt + "]", rmtCnt > 1); + + verifyCache(node, ldr); + } + + /** + * Check that file rebalance can be supplied from the node that was previously rebalanced. + * + * @throws Exception If failed. + */ + @Test + public void testFileRebalanceCanChain() throws Exception { + IgniteEx node0 = startGrid(0, true); + + List blt = new ArrayList<>(node0.context().discovery().aliveServerNodes()); + + DataLoader ldr = testValuesLoader(false, DFLT_LOADER_THREADS); + + ldr.loadData(node0); + + forceCheckpoint(node0); + + IgniteEx node1 = startGrid(1); + + TestRecordingCommunicationSpi spi1 = TestRecordingCommunicationSpi.spi(node1); + + P2 msgPred = (node, msg) -> (msg instanceof SnapshotRequestMessage); + + spi1.blockMessages(msgPred); + + blt.add(node1.cluster().localNode()); + + node0.cluster().setBaselineTopology(blt); + + spi1.waitForBlocked(); + spi1.stopBlock(); + + awaitPartitionMapExchange(); + + ClusterNode clusterNode0 = node0.cluster().localNode(); + + node0.close(); + + blt.remove(clusterNode0); + + awaitPartitionMapExchange(); + + forceCheckpoint(node1); + + ldr.loadData(node1); + + forceCheckpoint(node1); + + IgniteEx node2 = startGrid(2); + + TestRecordingCommunicationSpi spi2 = TestRecordingCommunicationSpi.spi(node2); + + spi2.blockMessages(msgPred); + + blt.add(node2.cluster().localNode()); + + node1.cluster().setBaselineTopology(blt); + + spi2.waitForBlocked(); + spi2.stopBlock(); + + awaitPartitionMapExchange(); + + forceCheckpoint(); + + verifyCache(node2, ldr); + } + + /** + * Ensures that file rebalancing starts every time the baseline changes. + * + * @throws Exception If failed. + */ + @Test + public void testContinuousBaselineChangeUnstableTopology() throws Exception { + boolean checkRemoves = false; + + IgniteEx crd = startGrid(0, true); + + DataLoader ldr = testValuesLoader(checkRemoves, DFLT_LOADER_THREADS).loadData(crd); + + ldr.start(); + + Set blt = new GridConcurrentHashSet<>(); + + blt.add(crd.localNode()); + + long timeout = U.currentTimeMillis() + 30_000; + + AtomicInteger nodes = new AtomicInteger(1); + + int grids = 4; + + int backups = backups(); + + assert backups > 0; + + BlockingQueue queue = new ArrayBlockingQueue<>(4); + + forceCheckpoint(); + + do { + GridTestUtils.runMultiThreadedAsync( () -> { + U.sleep(ThreadLocalRandom.current().nextLong(100)); + + int n = nodes.incrementAndGet(); + + Ignite node = startGrid(n); + + queue.add(n); + + blt.add(node.cluster().localNode()); + + crd.cluster().setBaselineTopology(blt); + + return null; + }, grids, "starter"); + + int stopped = 0; + + do { + Integer n = queue.poll(30, TimeUnit.SECONDS); + + assert n != null; + + ++stopped; + + ClusterNode node = grid(n).cluster().localNode(); + + stopGrid(n); + + blt.remove(node); + + crd.cluster().setBaselineTopology(blt); + + if (stopped % backups == 0) { + awaitPartitionMapExchange(); + + if (stopped != grids) { + ldr.suspend(); + + for (Ignite g : G.allGrids()) { + if (!g.name().equals(crd.name())) { + verifyCache(crd, ldr); + + break; + } + } + + ldr.resume(); + } + } + } while (stopped < grids); + + awaitPartitionMapExchange(); + } while (U.currentTimeMillis() < timeout); + + ldr.stop(); + + verifyCache(crd, ldr); + } + + /** + * @throws Exception If failed. + */ + @Test + public void test4nodesRestartLastNodeWithLoad() throws Exception { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + List blt = new ArrayList<>(); + + boolean checkRemoves = true; + + IgniteEx ignite0 = startGrid(0, true); + + blt.add(ignite0.localNode()); + + ignite0.cluster().setBaselineTopology(blt); + + DataLoader ldr = testValuesLoader(checkRemoves, DFLT_LOADER_THREADS).loadData(ignite0); + + ldr.start(); + + forceCheckpoint(ignite0); + + IgniteEx ignite1 = startGrid(1); + + blt.add(ignite1.localNode()); + + ignite0.cluster().setBaselineTopology(blt); + + long timeout = rnd.nextLong(2000); + + log.info(">>> Starting grid 2 (timeout=" + timeout + ")"); + + U.sleep(timeout); + + IgniteEx ignite2 = startGrid(2); + + blt.add(ignite2.localNode()); + + ignite0.cluster().setBaselineTopology(blt); + + timeout = rnd.nextLong(2000); + + log.info(">>> Starting grid 3 (timeout=" + timeout + ")"); + + U.sleep(timeout); + + IgniteEx ignite3 = startGrid(3); + + ClusterNode node3 = ignite3.localNode(); + + blt.add(ignite3.localNode()); + + ignite0.cluster().setBaselineTopology(blt); + + timeout = rnd.nextLong(2000); + + U.sleep(timeout); + + log.info(">>> Stopping grid 3 (timeout=" + timeout + ")"); + + stopGrid(3); + + blt.remove(node3); + + ignite0.cluster().setBaselineTopology(blt); + + timeout = rnd.nextLong(2000); + + U.sleep(timeout); + + log.info(">>> Starting grid 3 (timeout=" + timeout + ")"); + + ignite3 = startGrid(3); + + blt.add(ignite3.localNode()); + + ignite0.cluster().setBaselineTopology(blt); + + awaitPartitionMapExchange(); + + ldr.stop(); + + verifyCache(ignite3, ldr); + } + + /** + * @param enableRemoves Enabled entries removes. + * @param threadsCnt Threads count. + * @return make loader for indexed cache. + */ + private DataLoader testValuesLoader(boolean enableRemoves, int threadsCnt) { + return new DataLoader<>( + grid(0).cache(INDEXED_CACHE), + INITIAL_ENTRIES_COUNT, + testValProducer, + enableRemoves, + threadsCnt + ); + } + + /** + * @param node Target node. + * @param cfg Testing paramters. + * @param Type of value. + * @throws Exception If failed. + */ + protected void verifyCache(IgniteEx node, LoadParameters cfg) throws Exception { + String name = cfg.cacheName(); + int cnt = cfg.entriesCnt(); + boolean removes = cfg.checkRemoves(); + Function valProducer = cfg.valueProducer(); + + log.info("Verifying cache contents [node=" + node.cluster().localNode().id() + + " cache=" + name + ", size=" + cnt + "]"); + + IgniteCache cache = node.cache(name); + + StringBuilder buf = new StringBuilder(); + + int fails = 0; + + for (int k = 0; k < cnt; k++) { + if (removes && k % 10 == 0) + continue; + + V exp = valProducer.apply(k); + V actual = cache.get(k); + + if (!Objects.equals(exp, actual)) { + if (fails++ < 100) { + buf.append("cache=").append(cache.getName()).append(", key=").append(k).append(", expect="). + append(exp).append(", actual=").append(actual).append('\n'); + } + else { + buf.append("\n... and so on\n"); + + break; + } + } + + if ((k + 1) % (cnt / 10) == 0) + log.info("Verification: " + (k + 1) * 100 / cnt + "%"); + } + + if (!removes && cnt != cache.size()) { + buf.append("\ncache=").append(cache.getName()).append(" size mismatch [expect=").append(cnt). + append(", actual=").append(cache.size()).append('\n'); + } + + // todo remove next block + if (buf.length() > 0) { + long size = 0; + + for (Ignite g : G.allGrids()) { + log.info("Partitions states [node=" + g.name() + ", cache=" + name + "]"); + + CacheGroupContext ctx = ((IgniteEx)g).cachex(name).context().group(); + + for (GridDhtLocalPartition part : ctx.topology().currentLocalPartitions()) { + log.info("\tp=" + part.id() + " size=" + part.fullSize() + " init=" + + part.initialUpdateCounter() + ", cntr=" + part.updateCounter() + ", state=" + + part.state() + " mode=" + (part.primary(ctx.affinity().lastVersion()) ? "PRIMARY" : "BACKUP")); + } + + for (GridDhtLocalPartition part : ctx.topology().currentLocalPartitions()) { + boolean primary = part.primary(ctx.affinity().lastVersion()); + + if (primary) + size += part.fullSize(); + } + } + + log.info("Expected size for " + name + " is " + size); + } + + assertTrue(buf.toString(), buf.length() == 0); + } + + /** + * @param idx Node index. + * @param activate Activate flag. + */ + private IgniteEx startGrid(int idx, boolean activate) throws Exception { + IgniteEx ignite = startGrid(idx); + + if (activate) + ignite.cluster().state(ClusterState.ACTIVE); + + if (idx == 0) + ignite.cluster().baselineAutoAdjustEnabled(false); + + return ignite; + } + + /** */ + interface LoadParameters { + /** */ + public int entriesCnt(); + + /** */ + public String cacheName(); + + /** */ + public Function valueProducer(); + + /** */ + public boolean checkRemoves(); + } + + /** */ + protected static class DataLoader implements Runnable, LoadParameters { + /** */ + private final AtomicInteger cntr; + + /** */ + private final int loadCnt; + + /** */ + private final boolean enableRmv; + + /** */ + private final CyclicBarrier pauseBarrier; + + /** */ + private volatile boolean pause; + + /** */ + private volatile boolean paused; + + /** */ + private volatile boolean stop; + + /** */ + private final IgniteCache cache; + + /** */ + private final Function valFunc; + + /** */ + private final int threadCnt; + + /** */ + private volatile IgniteInternalFuture ldrFut; + + /** */ + public DataLoader( + IgniteCache cache, + int loadCnt, + Function valFunc, + boolean enableRmv, + int threadCnt + ) { + this.cache = cache; + this.enableRmv = enableRmv; + this.threadCnt = threadCnt; + this.valFunc = valFunc; + this.loadCnt = loadCnt; + + pauseBarrier = new CyclicBarrier(threadCnt + 1); // +1 waiter (suspend originator) + cntr = new AtomicInteger(); + + } + + /** {@inheritDoc} */ + @Override public void run() { + while (!stop && !Thread.currentThread().isInterrupted()) { + if (pause) { + if (!paused) { + U.awaitQuiet(pauseBarrier); + + log.info("Async loader paused."); + + paused = true; + } + + // Busy wait for resume. + try { + U.sleep(300); + } + catch (IgniteInterruptedCheckedException e) { + break; + } + + continue; + } + + int from = cntr.getAndAdd(100); + + for (int i = from; i < from + 100; i++) + cache.put(i, valFunc.apply(i)); + + if (!enableRmv) + continue; + + for (int i = from; i < from + 100; i += 10) + cache.remove(i); + } + + log.info("Async loader stopped."); + } + + /** */ + public void start() { + ldrFut = GridTestUtils.runMultiThreadedAsync(this, threadCnt, "thread"); + } + + /** + * Stop loader thread. + */ + public void stop() throws IgniteCheckedException { + stop = true; + + ldrFut.get(10_000); + } + + /** + * Pause loading. + */ + public void suspend() { + pause = true; + + log.info("Suspending loader threads: " + pauseBarrier.getParties()); + + // Wait all workers came to barrier. + U.awaitQuiet(pauseBarrier); + + log.info("Loader suspended"); + } + + /** + * Resume loading. + */ + public void resume() { + pause = paused = false; + } + + /** + * @param node Data originator. + * @return Data loader instance. + */ + public DataLoader loadData(Ignite node) { + int start = cntr.addAndGet(loadCnt) - loadCnt; + + try (IgniteDataStreamer streamer = node.dataStreamer(cache.getName())) { + for (int i = 0; i < loadCnt; i++) { + if ((i + 1) % (loadCnt / 10) == 0) + log.info("Prepared " + ((i + 1) * 100 / loadCnt) + "% entries."); + + int v = i + start; + + streamer.addData(v, valFunc.apply(v)); + } + } + + return this; + } + + /** */ + @Override public int entriesCnt() { + return cntr.get(); + } + + /** */ + @Override public String cacheName() { + return cache.getName(); + } + + /** */ + @Override public Function valueProducer() { + return valFunc; + } + + /** */ + @Override public boolean checkRemoves() { + return enableRmv; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheFileRebalancingTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheFileRebalancingTxTest.java new file mode 100644 index 0000000000000..d6ff8dac027de --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheFileRebalancingTxTest.java @@ -0,0 +1,42 @@ +/* + * 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; + +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; + +/** + * Test for rebalancing and persistence integration. + */ +public class IgniteCacheFileRebalancingTxTest extends IgniteCacheFileRebalancingAbstractTest { + /** {@inheritDoc} */ + @Override protected CacheConfiguration cacheConfiguration(String cacheName) { + CacheConfiguration ccfg = new CacheConfiguration(cacheName); + + ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + ccfg.setCacheMode(CacheMode.PARTITIONED); + ccfg.setBackups(backups()); + ccfg.setAffinity(new RendezvousAffinityFunction(false, 32)); + ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + + return ccfg; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java index 1903c65392b4f..fe2198a0fcd51 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java @@ -18,11 +18,9 @@ package org.apache.ignite.internal.processors.cache.persistence; import com.google.common.collect.Lists; -import java.io.Serializable; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -39,29 +37,14 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteDataStreamer; import org.apache.ignite.cache.CachePeekMode; -import org.apache.ignite.cache.CacheRebalanceMode; -import org.apache.ignite.cache.CacheWriteSynchronizationMode; -import org.apache.ignite.cache.PartitionLossPolicy; -import org.apache.ignite.cache.QueryEntity; -import org.apache.ignite.cache.QueryIndex; -import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.cache.query.ScanQuery; -import org.apache.ignite.cluster.ClusterNode; -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.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.GridTestUtils.SF; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.apache.ignite.transactions.Transaction; import org.junit.Test; @@ -70,154 +53,7 @@ /** * Test for rebalancing and persistence integration. */ -public abstract class IgnitePdsCacheRebalancingAbstractTest extends GridCommonAbstractTest { - /** Default cache. */ - private static final String CACHE = "cache"; - - /** Cache with node filter. */ - private static final String FILTERED_CACHE = "filtered"; - - /** Cache with enabled indexes. */ - private static final String INDEXED_CACHE = "indexed"; - - /** Cache with enabled indexes. */ - private static final String INDEXED_CACHE_IN_MEMORY = "indexed-in-memory"; - - /** In memory region. */ - private static final String IN_MEMORY_REGION = "in-memory-region"; - - /** */ - protected boolean explicitTx; - - /** Set to enable filtered cache on topology. */ - private boolean filteredCacheEnabled; - - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - - cfg.setConsistentId(gridName); - - cfg.setRebalanceThreadPoolSize(2); - - CacheConfiguration ccfg1 = cacheConfiguration(CACHE) - .setPartitionLossPolicy(PartitionLossPolicy.READ_WRITE_SAFE) - .setBackups(2) - .setRebalanceMode(CacheRebalanceMode.ASYNC) - .setIndexedTypes(Integer.class, Integer.class) - .setAffinity(new RendezvousAffinityFunction(false, 32)) - .setRebalanceBatchesPrefetchCount(2) - .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); - - CacheConfiguration ccfg2 = cacheConfiguration(INDEXED_CACHE) - .setBackups(2) - .setAffinity(new RendezvousAffinityFunction(false, 32)) - .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); - - CacheConfiguration ccfg3 = cacheConfiguration(INDEXED_CACHE_IN_MEMORY) - .setDataRegionName(IN_MEMORY_REGION); - - QueryEntity qryEntity = new QueryEntity(Integer.class.getName(), TestValue.class.getName()); - - LinkedHashMap fields = new LinkedHashMap<>(); - - fields.put("v1", Integer.class.getName()); - fields.put("v2", Integer.class.getName()); - - qryEntity.setFields(fields); - - QueryIndex qryIdx = new QueryIndex("v1", true); - - qryEntity.setIndexes(Collections.singleton(qryIdx)); - - ccfg2.setQueryEntities(Collections.singleton(qryEntity)); - ccfg3.setQueryEntities(Collections.singleton(qryEntity)); - - List cacheCfgs = new ArrayList<>(); - cacheCfgs.add(ccfg1); - cacheCfgs.add(ccfg2); - cacheCfgs.add(ccfg3); - - if (filteredCacheEnabled && !gridName.endsWith("0")) { - CacheConfiguration ccfg4 = cacheConfiguration(FILTERED_CACHE) - .setPartitionLossPolicy(PartitionLossPolicy.READ_ONLY_SAFE) - .setBackups(2) - .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC) - .setNodeFilter(new CoordinatorNodeFilter()); - - cacheCfgs.add(ccfg4); - } - - cfg.setCacheConfiguration(asArray(cacheCfgs)); - - DataStorageConfiguration dsCfg = new DataStorageConfiguration() - .setConcurrencyLevel(Runtime.getRuntime().availableProcessors() * 4) - .setCheckpointFrequency(checkpointFrequency()) - .setWalMode(WALMode.LOG_ONLY) - .setPageSize(1024) - .setWalSegmentSize(8 * 1024 * 1024) // For faster node restarts with enabled persistence. - .setDefaultDataRegionConfiguration(new DataRegionConfiguration() - .setName("dfltDataRegion") - .setPersistenceEnabled(true) - .setMaxSize(512 * 1024 * 1024) - ).setDataRegionConfigurations(new DataRegionConfiguration() - .setName(IN_MEMORY_REGION) - .setMaxSize(512 * 1024 * 1024) - ); - - cfg.setDataStorageConfiguration(dsCfg); - - return cfg; - } - - /** - * @param cacheCfgs Cache cfgs. - */ - private static CacheConfiguration[] asArray(List cacheCfgs) { - CacheConfiguration[] res = new CacheConfiguration[cacheCfgs.size()]; - for (int i = 0; i < res.length; i++) - res[i] = cacheCfgs.get(i); - - return res; - } - - /** - * @return Checkpoint frequency; - */ - protected long checkpointFrequency() { - return DataStorageConfiguration.DFLT_CHECKPOINT_FREQ; - } - - /** {@inheritDoc} */ - @Override protected long getTestTimeout() { - return 20 * 60 * 1000; - } - - /** {@inheritDoc} */ - @Override protected long getPartitionMapExchangeTimeout() { - return 60 * 1000; - } - - /** - * @param cacheName Cache name. - * @return Cache configuration. - */ - protected abstract CacheConfiguration cacheConfiguration(String cacheName); - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - stopAllGrids(); - - cleanPersistenceDir(); - } - - /** {@inheritDoc} */ - @Override protected void afterTest() throws Exception { - stopAllGrids(); - - cleanPersistenceDir(); - } - +public abstract class IgnitePdsCacheRebalancingAbstractTest extends IgnitePdsCacheRebalancingCommonAbstractTest { /** * Test that outdated partitions on restarted nodes are correctly replaced with newer versions. * @@ -343,7 +179,7 @@ public void testTopologyChangesWithConstantLoad() throws Exception { final int entriesCnt = 10_000; final int maxNodesCnt = 4; - final int topChanges = SF.applyLB(15, 5); + final int topChanges = 5;//SF.applyLB(15, 5); final boolean allowRemoves = true; final AtomicBoolean stop = new AtomicBoolean(); @@ -735,70 +571,4 @@ public void testRebalancingWithMixedDataRegionConfigurations() throws Exception assertEquals(value, cacheInMem1.localPeek(i, peekAll)); } } - - /** - * - */ - private static class TestValue implements Serializable { - /** Operation order. */ - private final long order; - - /** V 1. */ - private final int v1; - - /** V 2. */ - private final int v2; - - /** Flag indicates that value has removed. */ - private final boolean removed; - - private TestValue(long order, int v1, int v2) { - this(order, v1, v2, false); - } - - private TestValue(long order, int v1, int v2, boolean removed) { - this.order = order; - this.v1 = v1; - this.v2 = v2; - this.removed = removed; - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object o) { - if (this == o) return true; - - if (o == null || getClass() != o.getClass()) return false; - - TestValue testValue = (TestValue) o; - - return order == testValue.order && - v1 == testValue.v1 && - v2 == testValue.v2; - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - return Objects.hash(order, v1, v2); - } - - /** {@inheritDoc} */ - @Override public String toString() { - return "TestValue{" + - "order=" + order + - ", v1=" + v1 + - ", v2=" + v2 + - '}'; - } - } - - /** - * - */ - private static class CoordinatorNodeFilter implements IgnitePredicate { - /** {@inheritDoc} */ - @Override public boolean apply(ClusterNode node) { - // Do not start cache on coordinator. - return !node.attribute(IgniteNodeAttributes.ATTR_IGNITE_INSTANCE_NAME).endsWith("0"); - } - } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingCommonAbstractTest.java new file mode 100644 index 0000000000000..5d98e447c9f10 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingCommonAbstractTest.java @@ -0,0 +1,267 @@ +/* + * 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; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Objects; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CacheRebalanceMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.cache.PartitionLossPolicy; +import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.cache.QueryIndex; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cluster.ClusterNode; +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.internal.IgniteNodeAttributes; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Base prmitives for testing persistence rebalancing. + */ +public abstract class IgnitePdsCacheRebalancingCommonAbstractTest extends GridCommonAbstractTest { + /** Default cache. */ + protected static final String CACHE = "cache"; + + /** Cache with node filter. */ + protected static final String FILTERED_CACHE = "filtered"; + + /** Cache with enabled indexes. */ + protected static final String INDEXED_CACHE = "indexed"; + + /** Cache with enabled indexes. */ + protected static final String INDEXED_CACHE_IN_MEMORY = "indexed-in-memory"; + + /** In memory region. */ + protected static final String IN_MEMORY_REGION = "in-memory-region"; + + /** */ + protected boolean explicitTx; + + /** Set to enable filtered cache on topology. */ + protected boolean filteredCacheEnabled; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setConsistentId(gridName); + + cfg.setRebalanceThreadPoolSize(2); + + CacheConfiguration ccfg1 = cacheConfiguration(CACHE) + .setPartitionLossPolicy(PartitionLossPolicy.READ_WRITE_SAFE) + .setBackups(backups()) + .setCacheMode(CacheMode.REPLICATED) + .setRebalanceMode(CacheRebalanceMode.ASYNC) + .setIndexedTypes(Integer.class, Integer.class) + .setAffinity(new RendezvousAffinityFunction(false, 32)) + .setRebalanceBatchesPrefetchCount(2) + .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + + CacheConfiguration ccfg2 = cacheConfiguration(INDEXED_CACHE) + .setBackups(backups()) + .setAffinity(new RendezvousAffinityFunction(false, 32)) + .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + + CacheConfiguration ccfg3 = cacheConfiguration(INDEXED_CACHE_IN_MEMORY) + .setDataRegionName(IN_MEMORY_REGION); + + QueryEntity qryEntity = new QueryEntity(Integer.class, TestValue.class); + + LinkedHashMap fields = new LinkedHashMap<>(); + + fields.put("v1", Integer.class.getName()); + fields.put("v2", Integer.class.getName()); + + qryEntity.setFields(fields); + + QueryIndex qryIdx = new QueryIndex("v1", true); + + qryEntity.setIndexes(Collections.singleton(qryIdx)); + + ccfg2.setQueryEntities(Collections.singleton(qryEntity)); + ccfg3.setQueryEntities(Collections.singleton(qryEntity)); + + List cacheCfgs = new ArrayList<>(); + cacheCfgs.add(ccfg1); + cacheCfgs.add(ccfg2); + cacheCfgs.add(ccfg3); + + if (filteredCacheEnabled && !gridName.endsWith("0")) { + CacheConfiguration ccfg4 = cacheConfiguration(FILTERED_CACHE) + .setPartitionLossPolicy(PartitionLossPolicy.READ_ONLY_SAFE) + .setBackups(backups()) + .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC) + .setNodeFilter(new CoordinatorNodeFilter()); + + cacheCfgs.add(ccfg4); + } + + cfg.setCacheConfiguration(asArray(cacheCfgs)); + + DataStorageConfiguration dsCfg = new DataStorageConfiguration() + .setConcurrencyLevel(Runtime.getRuntime().availableProcessors() * 4) + .setCheckpointFrequency(checkpointFrequency()) + .setWalMode(WALMode.LOG_ONLY) + .setPageSize(1024) + .setWalSegmentSize(8 * 1024 * 1024) // For faster node restarts with enabled persistence. + .setDefaultDataRegionConfiguration(new DataRegionConfiguration() + .setName("dfltDataRegion") + .setPersistenceEnabled(true) + .setMaxSize(512 * 1024 * 1024) + ).setDataRegionConfigurations(new DataRegionConfiguration() + .setName(IN_MEMORY_REGION) + .setMaxSize(512 * 1024 * 1024) + ); + + cfg.setDataStorageConfiguration(dsCfg); + + return cfg; + } + + /** + * @return Number of backups. + */ + protected int backups() { + return 2; + } + + /** + * @param cacheCfgs Cache cfgs. + */ + private static CacheConfiguration[] asArray(List cacheCfgs) { + CacheConfiguration[] res = new CacheConfiguration[cacheCfgs.size()]; + for (int i = 0; i < res.length; i++) + res[i] = cacheCfgs.get(i); + + return res; + } + + /** + * @return Checkpoint frequency; + */ + protected long checkpointFrequency() { + return DataStorageConfiguration.DFLT_CHECKPOINT_FREQ; + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 20 * 60 * 1000; + } + + /** {@inheritDoc} */ + @Override protected long getPartitionMapExchangeTimeout() { + return 60 * 1000; + } + + /** + * @param cacheName Cache name. + * @return Cache configuration. + */ + protected abstract CacheConfiguration cacheConfiguration(String cacheName); + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** + * + */ + static class TestValue implements Serializable { + /** Operation order. */ + final long order; + + /** V 1. */ + final int v1; + + /** V 2. */ + final int v2; + + /** Flag indicates that value has removed. */ + final boolean removed; + + TestValue(long order, int v1, int v2) { + this(order, v1, v2, false); + } + + TestValue(long order, int v1, int v2, boolean removed) { + this.order = order; + this.v1 = v1; + this.v2 = v2; + this.removed = removed; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) return true; + + if (o == null || getClass() != o.getClass()) return false; + + TestValue testValue = (TestValue) o; + + return order == testValue.order && + v1 == testValue.v1 && + v2 == testValue.v2; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return Objects.hash(order, v1, v2); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "TestValue{" + + "order=" + order + + ", v1=" + v1 + + ", v2=" + v2 + + '}'; + } + } + + /** + * + */ + private static class CoordinatorNodeFilter implements IgnitePredicate { + /** {@inheritDoc} */ + @Override public boolean apply(ClusterNode node) { + // Do not start cache on coordinator. + return !node.attribute(IgniteNodeAttributes.ATTR_IGNITE_INSTANCE_NAME).endsWith("0"); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java index 9ff591495ce2d..2b273a76aa508 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java @@ -151,6 +151,7 @@ private void checkInvariantSwitchSegmentSize(int serVer) throws Exception { null, null, null, + null, null) ).createSerializer(serVer); @@ -479,6 +480,7 @@ private T2 initiate( null, null, null, + null, null ); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java index f530e801f6f82..8aab40ed570f2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java @@ -96,7 +96,8 @@ public class BPlusTreePageMemoryImplTest extends BPlusTreeSelfTest { null, null, null, - new CacheDiagnosticManager() + new CacheDiagnosticManager(), + null ); IgniteOutClosure clo = new IgniteOutClosure() { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java index 106b9ec670485..1aff424dd4aac 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java @@ -95,6 +95,7 @@ public class BPlusTreeReuseListPageMemoryImplTest extends BPlusTreeReuseSelfTest null, null, null, + null, null ); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java index c575f6dbceff7..2a9b5871d4442 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java @@ -111,7 +111,8 @@ public class IndexStoragePageMemoryImplTest extends IndexStorageSelfTest { null, null, null, - new CacheDiagnosticManager() + new CacheDiagnosticManager(), + null ); IgniteOutClosure clo = new IgniteOutClosure() { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java index cc7c6a9c4b519..ad231e2fa6150 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache.persistence.pagemem; +import java.io.File; import java.nio.ByteBuffer; import java.util.Collections; import java.util.Map; @@ -93,7 +94,7 @@ public class NoOpPageStoreManager implements IgnitePageStoreManager { } /** {@inheritDoc} */ - @Override public boolean exists(int cacheId, int partId) throws IgniteCheckedException { + @Override public boolean exists(int cacheId, int partId) { return false; } @@ -249,4 +250,9 @@ public class NoOpPageStoreManager implements IgnitePageStoreManager { @Override public boolean checkAndInitCacheWorkDir(CacheConfiguration cacheCfg) throws IgniteCheckedException { return false; } + + /** {@inheritDoc} */ + @Override public void restore(int grpId, int partId, File src) { + // No-op. + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java index 866b8e69d3499..05a3ad522e5d6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java @@ -80,7 +80,7 @@ public class NoOpWALManager implements IgniteWriteAheadLogManager { } /** {@inheritDoc} */ - @Override public WALIterator replay(WALPointer start, @Nullable IgniteBiPredicate recordDeserializeFilter) throws IgniteCheckedException, StorageException { + @Override public WALIterator replay(WALPointer start, boolean limited, @Nullable IgniteBiPredicate recordDeserializeFilter) { return null; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java index 5105489b1d78b..c58e39ba5c24b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java @@ -101,6 +101,7 @@ public class PageMemoryImplNoLoadTest extends PageMemoryNoLoadSelfTest { null, null, null, + null, null ); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java index 4be75b732cdcf..d030b2fb1141e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java @@ -621,6 +621,7 @@ private PageMemoryImpl createPageMemory( null, null, null, + null, null ); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManagerSelfTest.java index b957d480416ab..e9ea752cc64d3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManagerSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManagerSelfTest.java @@ -30,6 +30,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; import java.util.stream.Collectors; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; @@ -41,7 +42,10 @@ import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.TestRecordingCommunicationSpi; +import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState; import org.apache.ignite.internal.processors.cache.persistence.CheckpointState; import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener; @@ -53,12 +57,15 @@ import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings; import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId; +import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.GridTestUtils; import org.junit.Test; +import static org.apache.ignite.cluster.ClusterState.ACTIVE; import static org.apache.ignite.internal.MarshallerContextImpl.mappingFileStoreWorkDir; import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.resolveBinaryWorkDir; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.cacheDirName; @@ -336,6 +343,306 @@ public void testSnapshotCreateLocalCopyPartitionFail() throws Exception { err_msg); } + /** @throws Exception If fails. */ + @Test + public void testSnapshotRemoteWithNodeFiler() throws Exception { + int grids = 3; + CacheConfiguration ccfg = txCacheConfig(new CacheConfiguration(DEFAULT_CACHE_NAME)) + .setNodeFilter(node -> node.consistentId().toString().endsWith("1")); + + for (int i = 0; i < grids; i++) + startGrid(optimize(getConfiguration(getTestIgniteInstanceName(i)).setCacheConfiguration())); + + IgniteEx ig0 = grid(0); + ig0.cluster().baselineAutoAdjustEnabled(false); + ig0.cluster().state(ACTIVE); + + for (int i = 0; i < CACHE_KEYS_RANGE; i++) + ig0.getOrCreateCache(ccfg).put(i, i); + + Map> parts = new HashMap<>(); + parts.put(CU.cacheId(DEFAULT_CACHE_NAME), null); + + IgniteInternalFuture fut = snp(grid(0)).requestRemoteSnapshot(grid(2).localNode().id(), + parts, + new BiConsumer() { + @Override public void accept(File file, GroupPartitionId gprPartId) { + fail("We should not receive anything."); + } + }); + + assertThrowsAnyCause(log, + fut::get, + IgniteCheckedException.class, + "The snapshot operation stopped"); + } + + /** @throws Exception If fails. */ + @Test + public void testSnapshotRemotePartitionsWithLoad() throws Exception { + IgniteEx ig0 = startGrids(2); + + ig0.cluster().state(ClusterState.ACTIVE); + + AtomicInteger cntr = new AtomicInteger(); + + for (int i = 0; i < CACHE_KEYS_RANGE; i++) + ig0.cache(DEFAULT_CACHE_NAME).put(i, cntr.incrementAndGet()); + + GridCacheSharedContext cctx1 = grid(1).context().cache().context(); + GridCacheDatabaseSharedManager db1 = (GridCacheDatabaseSharedManager)cctx1.database(); + + forceCheckpoint(); + + Map rmtPartCRCs = new HashMap<>(); + CountDownLatch cancelLatch = new CountDownLatch(1); + + db1.addCheckpointListener(new DbCheckpointListener() { + /** {@inheritDoc} */ + @Override public void beforeCheckpointBegin(Context ctx) { + //No-op. + } + + /** {@inheritDoc} */ + @Override public void onMarkCheckpointBegin(Context ctx) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void onCheckpointBegin(Context ctx) { + SnapshotFutureTask task = cctx1.snapshotMgr().lastScheduledRemoteSnapshotTask(grid(0).localNode().id()); + + // Skip first remote snapshot creation due to it will be cancelled. + if (task == null || cancelLatch.getCount() > 0) + return; + + Map> processed = GridTestUtils.getFieldValue(task, + SnapshotFutureTask.class, + "processed"); + + if (!processed.isEmpty()) { + assert rmtPartCRCs.isEmpty(); + + // Calculate actual partition CRCs when the checkpoint will be finished on this node. + ctx.finishedStateFut().listen(f -> { + File cacheWorkDir = ((FilePageStoreManager)grid(1).context().cache().context().pageStore()) + .cacheWorkDir(dfltCacheCfg); + + rmtPartCRCs.putAll(calculateCRC32Partitions(cacheWorkDir)); + }); + } + } + }); + + IgniteSnapshotManager mgr0 = snp(ig0); + + UUID rmtNodeId = grid(1).localNode().id(); + Map snpPartCRCs = new HashMap<>(); + + Map> parts = new HashMap<>(); + parts.put(CU.cacheId(DEFAULT_CACHE_NAME), null); + + IgniteInternalFuture loadFut = GridTestUtils.runMultiThreadedAsync(() -> { + while (!Thread.currentThread().isInterrupted()) + ig0.cache(DEFAULT_CACHE_NAME).put(cntr.incrementAndGet(), cntr.incrementAndGet()); + }, 5, "cache-loader-"); + + try { + // Snapshot must be taken on node1 and transmitted to node0. + IgniteInternalFuture fut = mgr0.requestRemoteSnapshot(rmtNodeId, + parts, + new BiConsumer() { + @Override public void accept(File file, GroupPartitionId gprPartId) { + log.info("Snapshot partition received successfully [rmtNodeId=" + rmtNodeId + + ", part=" + file.getAbsolutePath() + ", gprPartId=" + gprPartId + ']'); + + cancelLatch.countDown(); + } + }); + + cancelLatch.await(); + + fut.cancel(); + + IgniteInternalFuture fut2 = mgr0.requestRemoteSnapshot(rmtNodeId, + parts, + (part, pair) -> { + try { + snpPartCRCs.put(part.getName(), FastCrc.calcCrc(part)); + } + catch (IOException e) { + throw new IgniteException(e); + } + }); + + fut2.get(); + } + finally { + loadFut.cancel(); + } + + assertEquals("Partitions from remote node must have the same CRCs as those which have been received", + rmtPartCRCs, snpPartCRCs); + } + + /** @throws Exception If fails. */ + @Test + public void testSnapshotRemoteOnBothNodes() throws Exception { + IgniteEx ig0 = startGrids(2); + + ig0.cluster().state(ClusterState.ACTIVE); + + for (int i = 0; i < CACHE_KEYS_RANGE; i++) + ig0.cache(DEFAULT_CACHE_NAME).put(i, i); + + forceCheckpoint(ig0); + + IgniteSnapshotManager mgr0 = snp(ig0); + IgniteSnapshotManager mgr1 = snp(grid(1)); + + UUID node0 = grid(0).localNode().id(); + UUID node1 = grid(1).localNode().id(); + + Map> fromNode1 = owningParts(ig0, + new HashSet<>(Collections.singletonList(CU.cacheId(DEFAULT_CACHE_NAME))), + node1); + + Map> fromNode0 = owningParts(grid(1), + new HashSet<>(Collections.singletonList(CU.cacheId(DEFAULT_CACHE_NAME))), + node0); + + // Snapshot must be taken on node1 and transmitted to node0. + IgniteInternalFuture futFrom1To0 = mgr0.requestRemoteSnapshot(node1, fromNode1, + (part, pair) -> assertTrue("Received partition has not been requested", fromNode1.get(pair.getGroupId()) + .remove(pair.getPartitionId()))); + IgniteInternalFuture futFrom0To1 = mgr1.requestRemoteSnapshot(node0, fromNode0, + (part, pair) -> assertTrue("Received partition has not been requested", fromNode0.get(pair.getGroupId()) + .remove(pair.getPartitionId()))); + + futFrom0To1.get(); + futFrom1To0.get(); + + assertTrue("Not all of partitions have been received: " + fromNode1, + fromNode1.get(CU.cacheId(DEFAULT_CACHE_NAME)).isEmpty()); + assertTrue("Not all of partitions have been received: " + fromNode0, + fromNode0.get(CU.cacheId(DEFAULT_CACHE_NAME)).isEmpty()); + } + + /** @throws Exception If fails. */ + @Test(expected = ClusterTopologyCheckedException.class) + public void testRemoteSnapshotRequestedNodeLeft() throws Exception { + IgniteEx ig0 = startGridWithCache(dfltCacheCfg, CACHE_KEYS_RANGE); + IgniteEx ig1 = startGrid(1); + + ig0.cluster().setBaselineTopology(ig0.cluster().forServers().nodes()); + + awaitPartitionMapExchange(); + + CountDownLatch hold = new CountDownLatch(1); + + ((GridCacheDatabaseSharedManager)ig1.context().cache().context().database()) + .addCheckpointListener(new DbCheckpointListener() { + /** {@inheritDoc} */ + @Override public void beforeCheckpointBegin(Context ctx) throws IgniteCheckedException { + // Listener will be executed inside the checkpoint thead. + U.await(hold); + } + + /** {@inheritDoc} */ + @Override public void onMarkCheckpointBegin(Context ctx) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void onCheckpointBegin(Context ctx) { + // No-op. + } + }); + + UUID rmtNodeId = ig1.localNode().id(); + + Map> parts = new HashMap<>(); + parts.put(CU.cacheId(DEFAULT_CACHE_NAME), null); + + snp(ig0).requestRemoteSnapshot(rmtNodeId, parts, (part, grp) -> {}); + + IgniteInternalFuture[] futs = new IgniteInternalFuture[1]; + + assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + IgniteInternalFuture snpFut = snp(ig1) + .lastScheduledRemoteSnapshotTask(ig0.localNode().id()); + + if (snpFut == null) + return false; + else + futs[0] = snpFut; + + return true; + } + }, 5_000L)); + + stopGrid(0); + + hold.countDown(); + + futs[0].get(); + } + + /** + *
+     * 1. Start 2 nodes.
+     * 2. Request snapshot from 2-nd node
+     * 3. Block snapshot-request message.
+     * 4. Start 3-rd node and change BLT.
+     * 5. Stop 3-rd node and change BLT.
+     * 6. 2-nd node now have MOVING partitions to be preloaded.
+     * 7. Release snapshot-request message.
+     * 8. Should get an error of snapshot creation since MOVING partitions cannot be snapshot.
+     * 
+ * + * @throws Exception If fails. + */ + @Test(expected = IgniteCheckedException.class) + public void testRemoteOutdatedSnapshot() throws Exception { + dfltCacheCfg.setBackups(1); + + IgniteEx ig0 = startGridsWithCache(2, dfltCacheCfg, CACHE_KEYS_RANGE); + + TestRecordingCommunicationSpi.spi(ig0) + .blockMessages((node, msg) -> msg instanceof SnapshotRequestMessage); + + UUID rmtNodeId = grid(1).localNode().id(); + + IgniteSnapshotManager mgr0 = snp(ig0); + + // Snapshot must be taken on node1 and transmitted to node0. + IgniteInternalFuture snpFut = mgr0.requestRemoteSnapshot(rmtNodeId, + owningParts(ig0, new HashSet<>(Collections.singletonList(CU.cacheId(DEFAULT_CACHE_NAME))), rmtNodeId), + (part, grp) -> {}); + + TestRecordingCommunicationSpi.spi(ig0) + .waitForBlocked(); + + startGrid(2); + + ig0.cluster().setBaselineTopology(ig0.cluster().forServers().nodes()); + + awaitPartitionMapExchange(); + + stopGrid(2); + + TestRecordingCommunicationSpi.spi(grid(1)) + .blockMessages((node, msg) -> msg instanceof GridDhtPartitionDemandMessage); + + ig0.cluster().setBaselineTopology(ig0.cluster().forServers().nodes()); + + TestRecordingCommunicationSpi.spi(ig0) + .stopBlock(true, obj -> obj.get2().message() instanceof SnapshotRequestMessage); + + snpFut.get(); + } + /** @throws Exception If fails. */ @Test(expected = IgniteCheckedException.class) public void testLocalSnapshotOnCacheStopped() throws Exception { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxCrossCachePartitionConsistencyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxCrossCachePartitionConsistencyTest.java index 826fb899cc59d..afb96731e4b27 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxCrossCachePartitionConsistencyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxCrossCachePartitionConsistencyTest.java @@ -84,7 +84,7 @@ public class TxCrossCachePartitionConsistencyTest extends GridCommonAbstractTest cfg.setDataStorageConfiguration(new DataStorageConfiguration(). setWalSegmentSize(8 * MB).setWalMode(LOG_ONLY).setPageSize(1024). - setCheckpointFrequency(MILLISECONDS.convert(365, DAYS)). + //setCheckpointFrequency(MILLISECONDS.convert(365, DAYS)). setDefaultDataRegionConfiguration(new DataRegionConfiguration().setPersistenceEnabled(persistenceEnabled). setInitialSize(100 * MB).setMaxSize(300 * MB))); 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 38da7574f8838..b2ab902599142 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 @@ -39,6 +39,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheTtlManager; import org.apache.ignite.internal.processors.cache.WalStateManager; import org.apache.ignite.internal.processors.cache.datastructures.CacheDataStructuresManager; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgnitePartitionPreloadManager; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.PartitionsEvictManager; import org.apache.ignite.internal.processors.cache.dr.GridOsCacheDrManager; import org.apache.ignite.internal.processors.cache.jta.CacheNoopJtaManager; @@ -87,7 +88,8 @@ public GridCacheTestContext(GridTestKernalContext ctx) throws Exception { null, null, null, - new CacheDiagnosticManager() + new CacheDiagnosticManager(), + new IgnitePartitionPreloadManager() ), defaultCacheConfiguration(), null, diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IndexedCacheFileRebalancingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IndexedCacheFileRebalancingTest.java new file mode 100644 index 0000000000000..2b1fab82d461a --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IndexedCacheFileRebalancingTest.java @@ -0,0 +1,88 @@ +/* + * 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; + +import java.util.List; +import java.util.UUID; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.query.FieldsQueryCursor; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.query.GridQueryProcessor; +import org.apache.ignite.internal.util.typedef.G; + +import static org.apache.ignite.internal.processors.query.h2.opt.H2TableScanIndex.SCAN_INDEX_NAME_SUFFIX; + +/** + * + */ +public class IndexedCacheFileRebalancingTest extends IgniteCacheFileRebalancingTxTest { + /** {@inheritDoc} */ + @Override protected void verifyCache(IgniteEx node, LoadParameters cfg) throws Exception { + super.verifyCache(node, cfg); + + String name = cfg.cacheName(); + + if (!name.equals(INDEXED_CACHE)) + return; + + assert node.cachex(name).context().isQueryEnabled(); + + int cnt = cfg.entriesCnt(); + boolean removes = cfg.checkRemoves(); + + int expSize = removes ? node.cache(name).size() : cnt; + String tbl = "\"" + name + "\"." + TestValue.class.getSimpleName(); + String sql = "select COUNT(V1) from " + tbl + " where V1 >= 0 and V1 < 2147483647"; + + for (Ignite g : G.allGrids()) { + log.info("Index validation [cache=" + name + ", node=" + g.cluster().localNode().id()); + + g.cache(name).indexReadyFuture().get(15_000); + + UUID nodeId = g.cluster().localNode().id(); + + boolean idxUsed = isIndexUsed(((IgniteEx)g).context().query(), "V1", tbl, "V1"); + + assertTrue("node=" + nodeId, idxUsed); + + IgniteCache cache0 = g.cache(name); + + FieldsQueryCursor> cur = cache0.query(new SqlFieldsQuery(sql)); + + long idxCnt = cur.getAll().get(0).get(0); + + assertEquals("node=" + nodeId, expSize, idxCnt); + } + } + + /** */ + private boolean isIndexUsed(GridQueryProcessor qryProc, String idxName, String tblName, String... reqFlds) { + int len = reqFlds.length; + String sql = "explain select * from " + tblName + " where "; + + for (int i = 0; i < len; ++i) + sql += reqFlds[i] + " > 0 and " + reqFlds[i] + " < 2147483647" + ((i < len - 1) ? " and " : ""); + + String plan = qryProc.querySqlFields(new SqlFieldsQuery(sql), true) + .getAll().get(0).get(0).toString().toUpperCase(); + + return !plan.contains(SCAN_INDEX_NAME_SUFFIX) && plan.contains(idxName.toUpperCase()); + } +} 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 bf772fa1baebc..033ae9831bb63 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 @@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsTxCacheRebalancingTest; 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.IndexedCacheFileRebalancingTest; import org.apache.ignite.internal.processors.cache.persistence.PersistenceDirectoryWarningLoggingTest; import org.apache.ignite.internal.processors.cache.persistence.db.IgniteCacheGroupsWithRestartsTest; import org.apache.ignite.internal.processors.cache.persistence.db.IgniteLogicalRecoveryTest; @@ -46,6 +47,7 @@ import org.apache.ignite.internal.processors.cache.persistence.db.wal.WalRecoveryTxLogicalRecordsTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.WalRolloverRecordLoggingFsyncTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.WalRolloverRecordLoggingLogOnlyTest; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManagerSelfTest; import org.junit.runner.RunWith; import org.junit.runners.Suite; @@ -75,6 +77,7 @@ IgnitePdsTxCacheRebalancingTest.class, IgnitePdsTxHistoricalRebalancingTest.class, + IndexedCacheFileRebalancingTest.class, IgniteWalRecoveryPPCTest.class, @@ -94,7 +97,9 @@ IgniteSequentialNodeCrashRecoveryTest.class, - IgniteCacheGroupsWithRestartsTest.class + IgniteCacheGroupsWithRestartsTest.class, + + IgniteSnapshotManagerSelfTest.class }) public class IgnitePdsWithIndexingCoreTestSuite { } diff --git a/parent/pom.xml b/parent/pom.xml index b6cb2b535a7eb..6a28701f6ed53 100644 --- a/parent/pom.xml +++ b/parent/pom.xml @@ -749,47 +749,42 @@ checkstyle - - - - org.apache.maven.plugins - maven-checkstyle-plugin - ${maven.checkstyle.plugin.version} - - - style - - check - - compile - - true - true - true - true - ${project.build.directory}/checkstyle-result.xml - ../checkstyle/checkstyle.xml - ../checkstyle/checkstyle-suppressions.xml - true - **/generated/**/* - - - - - - org.apache.ignite - ignite-tools - ${project.version} - - - com.puppycrawl.tools - checkstyle - ${checkstyle.puppycrawl.version} - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +