From 8903de5991220c557546aa1fb66e02d97c50076f Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Tue, 8 Oct 2019 13:07:19 +0300 Subject: [PATCH] IGNITE-12069 (PoC) File rebalancing. --- .../apache/ignite/IgniteSystemProperties.java | 5 + .../DataStorageConfiguration.java | 3 + .../org/apache/ignite/internal/GridTopic.java | 5 +- .../ignite/internal/IgniteFeatures.java | 5 +- .../communication/GridIoMessageFactory.java | 6 + .../communication/TransmissionMeta.java | 2 +- .../internal/pagemem/store/PageStore.java | 23 +- .../pagemem/store/PageStoreListener.java | 35 + .../cache/CacheAffinitySharedManager.java | 10 + .../processors/cache/CacheDataStoreEx.java | 75 + .../cache/CacheDataStoreExImpl.java | 496 +++++++ .../processors/cache/GridCacheIoManager.java | 56 + .../processors/cache/GridCacheMapEntry.java | 22 +- .../GridCachePartitionExchangeManager.java | 36 + .../processors/cache/GridCacheProcessor.java | 10 +- .../cache/GridCacheSharedContext.java | 34 +- .../cache/IgniteCacheOffheapManager.java | 5 +- .../cache/IgniteCacheOffheapManagerImpl.java | 44 +- .../processors/cache/WalStateManager.java | 7 +- .../GridCachePreloadSharedManager.java | 1305 +++++++++++++++++ .../preloader/GridDhtPartitionDemander.java | 9 +- .../dht/preloader/GridDhtPartitionMap.java | 4 +- .../preloader/GridDhtPartitionSupplier.java | 5 + .../GridDhtPartitionsExchangeFuture.java | 41 + .../dht/preloader/GridDhtPreloader.java | 5 + .../topology/GridClientPartitionTopology.java | 2 +- .../dht/topology/GridDhtLocalPartition.java | 47 +- .../topology/GridDhtPartitionTopology.java | 3 +- .../GridDhtPartitionTopologyImpl.java | 16 +- .../persistence/DbCheckpointListener.java | 12 + .../GridCacheDatabaseSharedManager.java | 200 ++- .../persistence/GridCacheOffheapManager.java | 73 +- .../IgniteCacheDatabaseSharedManager.java | 35 +- .../ReadOnlyGridCacheDataStore.java | 491 +++++++ .../backup/IgniteBackupManager.java | 793 ++++++++++ .../persistence/backup/IgniteTriClosure.java | 42 + .../cache/persistence/file/FilePageStore.java | 81 +- .../file/FilePageStoreManager.java | 124 +- .../persistence/file/FileSerialPageStore.java | 205 +++ .../persistence/pagemem/PageMemoryImpl.java | 5 +- .../snapshot/CompoundSnapshotOperation.java | 66 + .../snapshot/SnapshotOperationAdapter.java | 45 + .../wal/FileWriteAheadLogManager.java | 4 +- .../cache/persistence/wal/crc/FastCrc.java | 2 +- .../wal/reader/IgniteWalIteratorFactory.java | 4 +- .../cache/preload/FileMetaInfo.java | 19 + .../GridPartitionBatchDemandMessage.java | 178 +++ .../preload/IgniteBackupPageStoreManager.java | 41 + .../cache/preload/PartitionFileMetaInfo.java | 106 ++ .../preload/PartitionSwitchModeManager.java | 132 ++ .../cache/preload/PartitionUploadManager.java | 427 ++++++ .../transactions/IgniteTxLocalAdapter.java | 1 + .../ignite/internal/util/GridIntList.java | 17 + .../internal/util/collection/IntHashMap.java | 5 + .../internal/util/collection/IntMap.java | 2 + .../util/collection/IntRWHashMap.java | 10 + .../internal/util/nio/GridNioSessionImpl.java | 4 + .../discovery/tcp/internal/FutureTask.java | 2 +- modules/core/src/test/config/log4j-test.xml | 9 +- ...GridCachePersistenceRebalanceSelfTest.java | 488 ++++++ .../persistence/WalRebalanceBasicTest.java | 80 + .../backup/IgniteBackupManagerSelfTest.java | 291 ++++ .../IgniteWalIteratorSwitchSegmentTest.java | 4 + .../pagemem/BPlusTreePageMemoryImplTest.java | 4 +- .../BPlusTreeReuseListPageMemoryImplTest.java | 2 + .../IndexStoragePageMemoryImplTest.java | 4 +- .../pagemem/PageMemoryImplNoLoadTest.java | 2 + .../pagemem/PageMemoryImplTest.java | 2 + .../GridCacheReadOnlyStoreSelfTest.java | 447 ++++++ .../hashmap/GridCacheTestContext.java | 5 +- 70 files changed, 6672 insertions(+), 108 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/PageStoreListener.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheDataStoreEx.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheDataStoreExImpl.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridCachePreloadSharedManager.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/ReadOnlyGridCacheDataStore.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/backup/IgniteBackupManager.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/backup/IgniteTriClosure.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileSerialPageStore.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CompoundSnapshotOperation.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationAdapter.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/FileMetaInfo.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/GridPartitionBatchDemandMessage.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/IgniteBackupPageStoreManager.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/PartitionFileMetaInfo.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/PartitionSwitchModeManager.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/PartitionUploadManager.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/GridCachePersistenceRebalanceSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/WalRebalanceBasicTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/backup/IgniteBackupManagerSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/database/GridCacheReadOnlyStoreSelfTest.java 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 ad394e8a367bc..ec29257d72a1a 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -1160,6 +1160,11 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_BASELINE_AUTO_ADJUST_ENABLED = "IGNITE_BASELINE_AUTO_ADJUST_ENABLED"; + /** + * Flag to enable persistence rebalance. + */ + public static final String IGNITE_PERSISTENCE_REBALANCE_ENABLED = "IGNITE_PERSISTENCE_REBALANCE_ENABLED"; + /** * Maximum number of diagnostic warning messages per category, when waiting for PME. */ 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 380595407beae..3d1951b138953 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 @@ -158,6 +158,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; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java b/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java index 5677178568e3e..da71c0e40cb3e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java @@ -142,7 +142,10 @@ public enum GridTopic { TOPIC_SERVICES, /** */ - TOPIC_DEADLOCK_DETECTION; + TOPIC_DEADLOCK_DETECTION, + + /** */ + TOPIC_REBALANCE; /** Enum values. */ private static final GridTopic[] VALS = values(); 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 fad20f17626b5..95f494d9ce2c4 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 @@ -73,7 +73,10 @@ public enum IgniteFeatures { TCP_DISCOVERY_MESSAGE_NODE_COMPACT_REPRESENTATION(14), /** LRT system and user time dump settings. */ - LRT_SYSTEM_USER_TIME_DUMP_SETTINGS(18); + LRT_SYSTEM_USER_TIME_DUMP_SETTINGS(18), + + /** */ + CACHE_PARTITION_FILE_REBALANCE(19); /** * Unique feature identifier. 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 d8d62d4595a96..39244961ba280 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 @@ -141,6 +141,7 @@ 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.preload.GridPartitionBatchDemandMessage; 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; @@ -1166,6 +1167,11 @@ public GridIoMessageFactory(MessageFactory[] ext) { break; + case GridPartitionBatchDemandMessage.TYPE_CODE: + msg = new GridPartitionBatchDemandMessage(); + + break; + // [-3..119] [124..129] [-23..-28] [-36..-55] - this // [120..123] - DR // [-4..-22, -30..-35] - SQL diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionMeta.java index 986bf555565eb..5768d74f30ed4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionMeta.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionMeta.java @@ -30,7 +30,7 @@ * Class represents a file meta information to send to the remote node. Used to initiate a new file transfer * process or to continue the previous unfinished from the last transmitted point. */ -class TransmissionMeta implements Externalizable { +public class TransmissionMeta implements Externalizable { /** Serial version uid. */ private static final long serialVersionUID = 0L; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/PageStore.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/PageStore.java index 7c1e15d09e1e0..04c6fb451e0b1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/PageStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/PageStore.java @@ -25,6 +25,11 @@ * Persistent store of pages. */ public interface PageStore { + /** + * @param lsnr Page store listener to set. + */ + public void setListener(PageStoreListener lsnr); + /** * Checks if page exists. * @@ -55,7 +60,18 @@ public interface PageStore { * @param keepCrc by default reading zeroes CRC which was on file, but you can keep it in pageBuf if set keepCrc * @throws IgniteCheckedException If reading failed (IO error occurred). */ - public void read(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteCheckedException; + public default void read(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteCheckedException { + readPage(pageId, pageBuf, keepCrc); + } + + /** + * @param pageId Page id. + * @param pageBuf Page buffer to read into. + * @param keepCrc by default reading zeroes CRC which was on file, but you can keep it in pageBuf if set keepCrc + * @return Number of read bytes, or negative value if page read the first time. + * @throws IgniteCheckedException If reading failed (IO error occurred). + */ + public int readPage(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteCheckedException; /** * Reads a header. @@ -97,6 +113,11 @@ public interface PageStore { */ public void ensure() throws IgniteCheckedException; + /** + * Size of page store header. + */ + public int headerSize(); + /** * @return Page store version. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/PageStoreListener.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/PageStoreListener.java new file mode 100644 index 0000000000000..eb158935c760e --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/PageStoreListener.java @@ -0,0 +1,35 @@ +/* + * 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.pagemem.store; + +import java.nio.ByteBuffer; + +/** + * + */ +@FunctionalInterface +public interface PageStoreListener { + /** Default handler. */ + public static PageStoreListener NO_OP = (pageId, buff) -> {}; + + /** + * @param pageId Handled page id. + * @param buf Buffer with data. + */ + public void onPageWrite(long pageId, ByteBuffer buf); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java index f00b6b1a85ef1..625f763098eb5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java @@ -287,6 +287,7 @@ void checkRebalanceState(GridDhtPartitionTopology top, Integer checkGrpId) { GridDhtPartitionState state = top.partitionState(waitNode, part); if (state != GridDhtPartitionState.OWNING) { + System.out.println(">xxx> not owning " + part); rebalanced = false; break; @@ -297,13 +298,22 @@ void checkRebalanceState(GridDhtPartitionTopology top, Integer checkGrpId) { } if (rebalanced) { + System.out.println(">>> checkRebaalnceState remove " + checkGrpId); + waitInfo.waitGrps.remove(checkGrpId); if (waitInfo.waitGrps.isEmpty()) { + System.out.println(">>> waitInfo.waitGrps empty "); + msg = affinityChangeMessage(waitInfo); waitInfo = null; } + else { + Map.Entry> e = waitInfo.waitGrps.entrySet().iterator().next(); + + System.out.println(">>> waitInfo.waitGrps=" + waitInfo.waitGrps.keySet().size() + ", first=[name=" + cctx.cache().cacheGroup(e.getKey()).cacheOrGroupName() + ", count=" + e.getValue().size() + "]"); + } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheDataStoreEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheDataStoreEx.java new file mode 100644 index 0000000000000..e033e59874b12 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheDataStoreEx.java @@ -0,0 +1,75 @@ +/* + * 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; + +import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore; + +/** + * + */ +public interface CacheDataStoreEx extends CacheDataStore { + public CacheDataStore store(boolean readOnly); + + public void readOnly(boolean readOnly); + + public boolean readOnly(); + +// /** +// * @param mode The storage mode. +// * @return The storage intance for the given mode. +// */ +// public CacheDataStore store(StorageMode mode); +// +// /** +// * @param mode The mode to switch to. +// */ +// public void storeMode(StorageMode mode); +// +// /** +// * @return The currently used storage mode. Some of the long-running threads will remain to use +// * the old mode until they finish. +// */ +// public StorageMode storeMode(); + +// /** +// * @return The storage is used to expose temporary cache data rows when the LOG_ONLY mode is active. +// */ +// public IgnitePartitionCatchUpLog catchLog(); + // +// /** +// * @param mode The mode to switch to. +// */ +// public IgniteInternalFuture storeModeAsync(StorageMode mode); + +// /** +// * @param mode The mode to associate with data storage instance. +// * @param storage The cache data storage instance to set to. +// */ +// public void store(StorageMode mode, IgniteCacheOffheapManager.CacheDataStore storage); + +// /** +// * +// */ +// public enum StorageMode { +// /** Proxy will normally route all operations to the PageMemrory. */ +// FULL, +// +// /** Proxy will redirect the write operations to the temp-WAL storage. */ +// READ_ONLY; +// } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheDataStoreExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheDataStoreExImpl.java new file mode 100644 index 0000000000000..29a2c353b9f9e --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheDataStoreExImpl.java @@ -0,0 +1,496 @@ +/* + * 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; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import javax.cache.processor.EntryProcessor; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow; +import org.apache.ignite.internal.processors.cache.persistence.DataRowCacheAware; +import org.apache.ignite.internal.processors.cache.persistence.RowStore; +import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow; +import org.apache.ignite.internal.processors.cache.persistence.partstorage.PartitionMetaStorage; +import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree; +import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccUpdateResult; +import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccLinkAwareSearchRow; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner; +import org.apache.ignite.internal.util.GridLongList; +import org.apache.ignite.internal.util.lang.GridCursor; +import org.apache.ignite.internal.util.lang.IgnitePredicateX; +import org.apache.ignite.lang.IgniteBiTuple; +import org.jetbrains.annotations.Nullable; + +/** + *

+ * This is the CacheDataStoreEx implementation. The main purpose is hot switching between different + * modes of cache data storage (e.g. between FULL and LOG_ONLY mode) to guarantee the + * consistency for Checkpointer writes and async cache put operations. + *

+ */ +public class CacheDataStoreExImpl implements CacheDataStoreEx { + /** */ + private final IgniteLogger log; + + /** */ + private final GridCacheSharedContext cctx; + + /** Currently used data storage state. FULL mode is used by default. */ + private final AtomicBoolean readOnly = new AtomicBoolean(); + + /** */ + private final CacheDataStore store; + + /** */ + private final CacheDataStore readOnlyStore; + + /** + * @param primary The main storage to perform full cache operations. + * @param secondary The storage to handle only write operation in temporary mode. + */ + public CacheDataStoreExImpl( + GridCacheSharedContext cctx, + CacheDataStore primary, + CacheDataStore secondary, + IgniteLogger log + ) { + assert primary != null; + + this.cctx = cctx; + this.log = log; + + store = primary; + readOnlyStore = secondary; + } + + /** {@inheritDoc} */ + @Override public CacheDataStore store(boolean readOnly) { + return readOnly ? readOnlyStore : store; + } + + /** {@inheritDoc} */ + @Override public void readOnly(boolean readOnly) { + assert readOnly || cctx.database().checkpointLockIsHeldByThread() : "Changing mode required checkpoint write lock"; + + if (this.readOnly.compareAndSet(!readOnly, readOnly)) + log.info("Changing data store mode to " + (readOnly ? "READ-REMOVE" : "FULL") + " [p=" + partId() + "]"); + } + + /** {@inheritDoc} */ + @Override public boolean readOnly() { + return readOnly.get(); + } + + /** + * @return The currently active cache data storage. + */ + private CacheDataStore activeStorage() { + return store(readOnly.get()); + } + + /** {@inheritDoc} */ + @Override public boolean init() { + return activeStorage().init(); + } + + /** {@inheritDoc} */ + @Override public int partId() { + return activeStorage().partId(); + } + + /** {@inheritDoc} */ + @Override public CacheDataRow createRow( + GridCacheContext cctx, + KeyCacheObject key, + CacheObject val, + GridCacheVersion ver, + long expireTime, + @Nullable CacheDataRow oldRow + ) throws IgniteCheckedException { + return activeStorage().createRow(cctx, key, val, ver, expireTime, oldRow); + } + + /** {@inheritDoc} */ + @Override public void insertRows(Collection rows, + IgnitePredicateX initPred) throws IgniteCheckedException { + activeStorage().insertRows(rows, initPred); + } + + /** {@inheritDoc} */ + @Override public int cleanup( + GridCacheContext cctx, + @Nullable List cleanupRows + ) throws IgniteCheckedException { + return activeStorage().cleanup(cctx, cleanupRows); + } + + /** {@inheritDoc} */ + @Override public void updateTxState(GridCacheContext cctx, CacheSearchRow row) throws IgniteCheckedException { + activeStorage().updateTxState(cctx, row); + } + + /** {@inheritDoc} */ + @Override public void update( + GridCacheContext cctx, + KeyCacheObject key, + CacheObject val, + GridCacheVersion ver, + long expireTime, + @Nullable CacheDataRow oldRow + ) throws IgniteCheckedException { + activeStorage().update(cctx, key, val, ver, expireTime, oldRow); + } + + /** {@inheritDoc} */ + @Override public boolean mvccInitialValue( + GridCacheContext cctx, + KeyCacheObject key, + @Nullable CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccVersion mvccVer, + MvccVersion newMvccVer + ) throws IgniteCheckedException { + return activeStorage().mvccInitialValue(cctx, key, val, ver, expireTime, mvccVer, newMvccVer); + } + + /** {@inheritDoc} */ + @Override public boolean mvccApplyHistoryIfAbsent( + GridCacheContext cctx, + KeyCacheObject key, + List hist + ) throws IgniteCheckedException { + return activeStorage().mvccApplyHistoryIfAbsent(cctx, key, hist); + } + + /** {@inheritDoc} */ + @Override public boolean mvccUpdateRowWithPreloadInfo( + GridCacheContext cctx, + KeyCacheObject key, + @Nullable CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccVersion mvccVer, + MvccVersion newMvccVer, + byte mvccTxState, + byte newMvccTxState + ) throws IgniteCheckedException { + return activeStorage().mvccUpdateRowWithPreloadInfo( + cctx, key, val, ver, expireTime, mvccVer, newMvccVer, mvccTxState, newMvccTxState + ); + } + + /** {@inheritDoc} */ + @Override public MvccUpdateResult mvccUpdate( + GridCacheContext cctx, + KeyCacheObject key, + CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccSnapshot mvccSnapshot, + @Nullable CacheEntryPredicate filter, + EntryProcessor entryProc, + Object[] invokeArgs, + boolean primary, + boolean needHist, + boolean noCreate, + boolean needOldVal, + boolean retVal, + boolean keepBinary + ) throws IgniteCheckedException { + return activeStorage().mvccUpdate( + cctx, key, val, ver, expireTime, mvccSnapshot, filter, entryProc, invokeArgs, primary, needHist, noCreate, + needOldVal, retVal, keepBinary + ); + } + + /** {@inheritDoc} */ + @Override public MvccUpdateResult mvccRemove( + GridCacheContext cctx, + KeyCacheObject key, + MvccSnapshot mvccSnapshot, + @Nullable CacheEntryPredicate filter, + boolean primary, + boolean needHistory, + boolean needOldVal, + boolean retVal + ) throws IgniteCheckedException { + return activeStorage().mvccRemove(cctx, key, mvccSnapshot, filter, primary, needHistory, needOldVal, retVal); + } + + /** {@inheritDoc} */ + @Override public MvccUpdateResult mvccLock( + GridCacheContext cctx, + KeyCacheObject key, + MvccSnapshot mvccSnapshot + ) throws IgniteCheckedException { + return activeStorage().mvccLock(cctx, key, mvccSnapshot); + } + + /** {@inheritDoc} */ + @Override public void mvccRemoveAll(GridCacheContext cctx, KeyCacheObject key) throws IgniteCheckedException { + activeStorage().mvccRemoveAll(cctx, key); + } + + /** {@inheritDoc} */ + @Override public void invoke( + GridCacheContext cctx, + KeyCacheObject key, + IgniteCacheOffheapManager.OffheapInvokeClosure c + ) throws IgniteCheckedException { + // todo should be executed under read lock? + activeStorage().invoke(cctx, key, c); + } + + /** {@inheritDoc} */ + @Override public void mvccApplyUpdate( + GridCacheContext cctx, + KeyCacheObject key, + CacheObject val, + GridCacheVersion ver, + long expireTime, + MvccVersion mvccVer + ) throws IgniteCheckedException { + activeStorage().mvccApplyUpdate(cctx, key, val, ver, expireTime, mvccVer); + } + + /** {@inheritDoc} */ + @Override public void remove(GridCacheContext cctx, KeyCacheObject key, int partId) throws IgniteCheckedException { + activeStorage().remove(cctx, key, partId); + } + + /** {@inheritDoc} */ + @Override public CacheDataRow find(GridCacheContext cctx, KeyCacheObject key) throws IgniteCheckedException { + return activeStorage().find(cctx, key); + } + + /** {@inheritDoc} */ + @Override public GridCursor mvccAllVersionsCursor( + GridCacheContext cctx, + KeyCacheObject key, + Object x + ) throws IgniteCheckedException { + return activeStorage().mvccAllVersionsCursor(cctx, key, x); + } + + /** {@inheritDoc} */ + @Override public CacheDataRow mvccFind( + GridCacheContext cctx, + KeyCacheObject key, + MvccSnapshot snapshot + ) throws IgniteCheckedException { + return activeStorage().mvccFind(cctx, key, snapshot); + } + + /** {@inheritDoc} */ + @Override public List> mvccFindAllVersions( + GridCacheContext cctx, + KeyCacheObject key + ) throws IgniteCheckedException { + return activeStorage().mvccFindAllVersions(cctx, key); + } + + /** {@inheritDoc} */ + @Override public GridCursor cursor() throws IgniteCheckedException { + return activeStorage().cursor(); + } + + /** {@inheritDoc} */ + @Override public GridCursor cursor(Object x) throws IgniteCheckedException { + return activeStorage().cursor(x); + } + + /** {@inheritDoc} */ + @Override public GridCursor cursor( + MvccSnapshot mvccSnapshot + ) throws IgniteCheckedException { + return activeStorage().cursor(mvccSnapshot); + } + + /** {@inheritDoc} */ + @Override public GridCursor cursor(int cacheId) throws IgniteCheckedException { + return activeStorage().cursor(cacheId); + } + + /** {@inheritDoc} */ + @Override public GridCursor cursor( + int cacheId, + MvccSnapshot mvccSnapshot + ) throws IgniteCheckedException { + return activeStorage().cursor(cacheId, mvccSnapshot); + } + + /** {@inheritDoc} */ + @Override public GridCursor cursor( + int cacheId, + KeyCacheObject lower, + KeyCacheObject upper + ) throws IgniteCheckedException { + return activeStorage().cursor(cacheId, lower, upper); + } + + /** {@inheritDoc} */ + @Override public GridCursor cursor( + int cacheId, + KeyCacheObject lower, + KeyCacheObject upper, + Object x + ) throws IgniteCheckedException { + return activeStorage().cursor(cacheId, lower, upper, x); + } + + /** {@inheritDoc} */ + @Override public GridCursor cursor( + int cacheId, + KeyCacheObject lower, + KeyCacheObject upper, + Object x, + MvccSnapshot snapshot + ) throws IgniteCheckedException { + return activeStorage().cursor(cacheId, lower, upper, x, snapshot); + } + + /** {@inheritDoc} */ + @Override public void destroy() throws IgniteCheckedException { + activeStorage().destroy(); + } + + /** {@inheritDoc} */ + @Override public void clear(int cacheId) throws IgniteCheckedException { + activeStorage().clear(cacheId); + } + + /** {@inheritDoc} */ + @Override public RowStore rowStore() { + // Checkpointer must always have assess to the original storage. + return activeStorage().rowStore(); + } + + /** {@inheritDoc} */ + @Override public void updateInitialCounter(long start, long delta) { + activeStorage().updateInitialCounter(start, delta); + } + + /** {@inheritDoc} */ + @Override public void setRowCacheCleaner(GridQueryRowCacheCleaner rowCacheCleaner) { + activeStorage().setRowCacheCleaner(rowCacheCleaner); + } + + /** {@inheritDoc} */ + @Override public PendingEntriesTree pendingTree() { + return activeStorage().pendingTree(); + } + + /** {@inheritDoc} */ + @Override public void preload() throws IgniteCheckedException { + activeStorage().preload(); + } + + /** {@inheritDoc} */ + @Override public void resetUpdateCounter() { + activeStorage().resetUpdateCounter(); + } + + /** {@inheritDoc} */ + @Override public PartitionMetaStorage partStorage() { + return activeStorage().partStorage(); + } + + /** {@inheritDoc} */ + @Override public long cacheSize(int cacheId) { + return activeStorage().cacheSize(cacheId); + } + + /** {@inheritDoc} */ + @Override public Map cacheSizes() { + return activeStorage().cacheSizes(); + } + + /** {@inheritDoc} */ + @Override public long fullSize() { + return activeStorage().fullSize(); + } + + /** {@inheritDoc} */ + @Override public boolean isEmpty() { + return activeStorage().isEmpty(); + } + + /** {@inheritDoc} */ + @Override public void updateSize(int cacheId, long delta) { + activeStorage().updateSize(cacheId, delta); + } + + /** {@inheritDoc} */ + @Override public long updateCounter() { + return activeStorage().updateCounter(); + } + + /** {@inheritDoc} */ + @Override public long reservedCounter() { + return activeStorage().reservedCounter(); + } + + /** {@inheritDoc} */ + @Override public @Nullable PartitionUpdateCounter partUpdateCounter() { + return activeStorage().partUpdateCounter(); + } + + /** {@inheritDoc} */ + @Override public long reserve(long delta) { + return activeStorage().reserve(delta); + } + + /** {@inheritDoc} */ + @Override public void updateCounter(long val) { + activeStorage().updateCounter(val); + } + + /** {@inheritDoc} */ + @Override public boolean updateCounter(long start, long delta) { + return activeStorage().updateCounter(start, delta); + } + + /** {@inheritDoc} */ + @Override public long nextUpdateCounter() { + return activeStorage().nextUpdateCounter(); + } + + /** {@inheritDoc} */ + @Override public long getAndIncrementUpdateCounter(long delta) { + return activeStorage().getAndIncrementUpdateCounter(delta); + } + + /** {@inheritDoc} */ + @Override public long initialUpdateCounter() { + return activeStorage().initialUpdateCounter(); + } + + /** {@inheritDoc} */ + @Override public GridLongList finalizeUpdateCounters() { + return activeStorage().finalizeUpdateCounters(); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java index 7170ac3d059a8..6516fc8934052 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java @@ -304,6 +304,62 @@ else if (desc.receivedFromStartVersion() != null) return; } + else { + IgniteInternalFuture switchFut = cctx.filePreloader().partitionRestoreFuture(nodeId, cacheMsg); + + if (switchFut != null && !switchFut.isDone()) { + System.out.println(">xxx> lock updates " + cacheMsg.getClass().getSimpleName()); + + synchronized (pendingMsgs) { + if (pendingMsgs.size() < MAX_STORED_PENDING_MESSAGES) + pendingMsgs.add(cacheMsg); + } + + Thread curThread = Thread.currentThread(); + + final int stripe = curThread instanceof IgniteThread ? ((IgniteThread)curThread).stripe() : -1; + +// log.info("wait for partition switch"); + + switchFut.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture t) { + Runnable c = new Runnable() { + @Override public void run() { + synchronized (pendingMsgs) { + pendingMsgs.remove(cacheMsg); + } + + IgniteLogger log = cacheMsg.messageLogger(cctx); + + if (log.isInfoEnabled()) { + StringBuilder msg0 = new StringBuilder("Process cache message after wait for " + + "affinity topology version ["); + + appendMessageInfo(cacheMsg, nodeId, msg0).append(']'); + + log.info(msg0.toString()); + } + + handleMessage(nodeId, cacheMsg, plc); + } + }; + + if (stripe >= 0) + cctx.kernalContext().getStripedExecutorService().execute(stripe, c); + else { + try { + cctx.kernalContext().pools().poolForPolicy(plc).execute(c); + } + catch (IgniteCheckedException e) { + U.error(cacheMsg.messageLogger(cctx), "Failed to get pool for policy: " + plc, e); + } + } + } + }); + + return; + } + } handleMessage(nodeId, cacheMsg, plc); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index f9388d24cc2cc..54cc0c4042a2b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -51,6 +51,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry; import org.apache.ignite.internal.processors.cache.extras.GridCacheEntryExtras; @@ -3331,8 +3332,11 @@ private boolean skipInterceptor(@Nullable GridCacheVersion explicitVer) { long expTime = expireTime < 0 ? CU.toExpireTime(ttl) : expireTime; + CacheObject val0 = val; + val = cctx.kernalContext().cacheObjects().prepareForCache(val, cctx); + final boolean unswapped = ((flags & IS_UNSWAPPED_MASK) != 0); boolean update; @@ -3349,8 +3353,14 @@ private boolean skipInterceptor(@Nullable GridCacheVersion explicitVer) { if (!isStartVer) { if (cctx.atomic()) update0 = ATOMIC_VER_COMPARATOR.compare(currentVer, ver) < 0; - else + else { update0 = currentVer.compareTo(ver) < 0; + + if (!update0 && val0 == null) + log.info("TRACE REMOVE " + key + " inputVer=" + ver + ", cached=" + currentVer); + + + } } else update0 = true; @@ -3358,7 +3368,7 @@ private boolean skipInterceptor(@Nullable GridCacheVersion explicitVer) { else update0 = isStartVer; - update0 |= (!preload && deletedUnlocked()); +// update0 |= (!preload && deletedUnlocked()); return update0; } @@ -4339,6 +4349,9 @@ protected WALPointer logTxUpdate(IgniteInternalTx tx, CacheObject val, long expi else op = this.val == null ? GridCacheOperation.CREATE : UPDATE; +// if (cctx.localNodeId().toString().endsWith("0")) +// log.info(">>> wal tx update [p=" + key.partition() + ", key=" + key.value(cctx.cacheObjectContext(), false) + ", cntr=" + updCntr); + return cctx.shared().wal().log(new DataRecord(new DataEntry( cctx.cacheId(), key, @@ -5025,6 +5038,11 @@ private int extrasSize() { lock.lock(); } + /** {@inheritDoc} */ + public boolean isLockedEntry() { + return lock.isLocked(); + } + /** {@inheritDoc} */ @Override public void unlockEntry() { lock.unlock(); 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 19299a4277889..c9ecb6b57a415 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 @@ -72,6 +72,7 @@ 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.GridMessageListener; import org.apache.ignite.internal.managers.discovery.DiscoCache; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.managers.discovery.DiscoveryLocalJoinData; @@ -103,6 +104,7 @@ 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.preload.GridPartitionBatchDemandMessage; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; @@ -158,6 +160,7 @@ import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL; import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.NONE; import static org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionPartialCountersMap.PARTIAL_COUNTERS_MAP_SINCE; +import static org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridCachePreloadSharedManager.rebalanceThreadTopic; import static org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.nextDumpTimeout; import static org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader.DFLT_PRELOAD_RESEND_TIMEOUT; import static org.apache.ignite.internal.processors.metric.GridMetricManager.PME_OPS_BLOCKED_DURATION; @@ -492,6 +495,28 @@ else if (m instanceof GridDhtPartitionDemandLegacyMessage) { } } + // todo + if (cctx.filePreloader() != null && cctx.filePreloader().persistenceRebalanceApplicable()) { + if (log.isDebugEnabled()) + log.debug("Starting file rebalancing messages handler."); + + cctx.gridIO().addMessageListener(rebalanceThreadTopic(), new GridMessageListener() { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { + if (msg instanceof GridPartitionBatchDemandMessage) { + if (!enterBusy()) + return; + + try { + cctx.filePreloader().handleDemandMessage(nodeId, (GridPartitionBatchDemandMessage)msg); + } + finally { + leaveBusy(); + } + } + } + }); + } + MetricRegistry mreg = cctx.kernalContext().metric().registry(PME_METRICS); mreg.register(PME_DURATION, @@ -1246,6 +1271,8 @@ public void refreshPartitions(@NotNull Collection grps) { log.debug("Refreshing local partitions from non-oldest node: " + cctx.localNodeId()); + System.out.println("sending partitions"); + sendLocalPartitions(oldest, null, grps); } } @@ -3321,6 +3348,7 @@ else if (task instanceof ForceRebalanceExchangeTask) { } Runnable r = null; + Runnable loadPartsRun = null; List rebList = new LinkedList<>(); @@ -3331,6 +3359,9 @@ else if (task instanceof ForceRebalanceExchangeTask) { if (task instanceof ForceRebalanceExchangeTask) forcedRebFut = ((ForceRebalanceExchangeTask)task).forcedRebalanceFuture(); + if (cctx.filePreloader() != null) + loadPartsRun = cctx.filePreloader().addNodeAssignments(assignsMap, resVer, forcePreload, cnt); + for (Integer order : orderMap.descendingKeySet()) { for (Integer grpId : orderMap.get(order)) { CacheGroupContext grp = cctx.cache().cacheGroup(grpId); @@ -3340,6 +3371,10 @@ else if (task instanceof ForceRebalanceExchangeTask) { if (assigns != null) assignsCancelled |= assigns.cancelled(); + if (cctx.filePreloader() != null && + cctx.filePreloader().FileRebalanceSupported(grp, assigns.keySet())) + continue; + Runnable cur = grp.preloader().addAssignments(assigns, forcePreload, cnt, @@ -3375,6 +3410,7 @@ else if (r != null) { // Start rebalancing cache groups chain. Each group will be rebalanced // sequentially one by one e.g.: // ignite-sys-cache -> cacheGroupR1 -> cacheGroupP2 -> cacheGroupR3 + loadPartsRun.run(); r.run(); } else 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 18a421276d9eb..780825c8d9e04 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 @@ -102,6 +102,7 @@ import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener; import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.persistence.backup.IgniteBackupManager; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList; import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage; @@ -112,6 +113,7 @@ import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotDiscoveryMessage; import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridCachePreloadSharedManager; import org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager; import org.apache.ignite.internal.processors.cache.query.GridCacheLocalQueryManager; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager; @@ -2926,9 +2928,13 @@ private GridCacheSharedContext createSharedContext( IgniteCacheDatabaseSharedManager dbMgr; IgnitePageStoreManager pageStoreMgr = null; IgniteWriteAheadLogManager walMgr = null; + IgniteBackupManager backupMgr = null; + GridCachePreloadSharedManager preloadMgr = null; if (CU.isPersistenceEnabled(ctx.config()) && !ctx.clientNode()) { dbMgr = new GridCacheDatabaseSharedManager(ctx); + backupMgr = new IgniteBackupManager(ctx); + preloadMgr = new GridCachePreloadSharedManager(ctx); pageStoreMgr = ctx.plugins().createComponent(IgnitePageStoreManager.class); @@ -2978,6 +2984,7 @@ private GridCacheSharedContext createSharedContext( walMgr, walStateMgr, dbMgr, + backupMgr, snpMgr, depMgr, exchMgr, @@ -2989,7 +2996,8 @@ private GridCacheSharedContext createSharedContext( storeSesLsnrs, mvccCachingMgr, deadlockDetectionMgr, - diagnosticMgr + diagnosticMgr, + preloadMgr ); } 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 9cdc8fc68aafc..546a5148c65ca 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 @@ -51,7 +51,9 @@ import org.apache.ignite.internal.processors.cache.mvcc.MvccProcessor; import org.apache.ignite.internal.processors.cache.persistence.DataRegion; import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.persistence.backup.IgniteBackupManager; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridCachePreloadSharedManager; import org.apache.ignite.internal.processors.cache.store.CacheStoreManager; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; @@ -123,6 +125,9 @@ public class GridCacheSharedContext { /** Page store manager. {@code Null} if persistence is not enabled. */ @Nullable private IgnitePageStoreManager pageStoreMgr; + /** Page file snapshot manager. Can be {@code null} if presistence is not enabled. */ + private IgniteBackupManager backupMgr; + /** Affinity manager. */ private CacheAffinitySharedManager affMgr; @@ -186,6 +191,9 @@ public class GridCacheSharedContext { /** Cluster is in read-only mode. */ private volatile boolean readOnlyMode; + /** Manager to preload cache partions. Can be {@code null} if presistence is not enabled. */ + private GridCachePreloadSharedManager preloadMgr; + /** * @param kernalCtx Context. * @param txMgr Transaction manager. @@ -216,6 +224,7 @@ public GridCacheSharedContext( @Nullable IgniteWriteAheadLogManager walMgr, WalStateManager walStateMgr, IgniteCacheDatabaseSharedManager dbMgr, + IgniteBackupManager backupMgr, IgniteCacheSnapshotManager snpMgr, GridCacheDeploymentManager depMgr, GridCachePartitionExchangeManager exchMgr, @@ -227,7 +236,8 @@ public GridCacheSharedContext( Collection storeSesLsnrs, MvccCachingManager mvccCachingMgr, DeadlockDetectionManager deadlockDetectionMgr, - CacheDiagnosticManager diagnosticMgr + CacheDiagnosticManager diagnosticMgr, + GridCachePreloadSharedManager preloadMgr ) { this.kernalCtx = kernalCtx; @@ -241,8 +251,10 @@ public GridCacheSharedContext( walMgr, walStateMgr, dbMgr, + backupMgr, snpMgr, depMgr, + preloadMgr, exchMgr, affMgr, ioMgr, @@ -410,8 +422,10 @@ void onReconnected(boolean active) throws IgniteCheckedException { walMgr, walStateMgr, dbMgr, + backupMgr, snpMgr, new GridCacheDeploymentManager(), + preloadMgr, new GridCachePartitionExchangeManager(), affMgr, ioMgr, @@ -459,8 +473,10 @@ private void setManagers( IgniteWriteAheadLogManager walMgr, WalStateManager walStateMgr, IgniteCacheDatabaseSharedManager dbMgr, + IgniteBackupManager backupMgr, IgniteCacheSnapshotManager snpMgr, GridCacheDeploymentManager depMgr, + GridCachePreloadSharedManager preloadMgr, GridCachePartitionExchangeManager exchMgr, CacheAffinitySharedManager affMgr, GridCacheIoManager ioMgr, @@ -478,9 +494,11 @@ private void setManagers( this.walMgr = add(mgrs, walMgr); this.walStateMgr = add(mgrs, walStateMgr); this.dbMgr = add(mgrs, dbMgr); + this.backupMgr = add(mgrs, backupMgr); 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); @@ -714,6 +732,13 @@ public IgniteCacheDatabaseSharedManager database() { return dbMgr; } + /** + * @return File rebalancing manager. + */ + public GridCachePreloadSharedManager filePreloader() { + return preloadMgr; + } + /** * @return Snapshot manager. */ @@ -728,6 +753,13 @@ public IgniteCacheSnapshotManager snapshot() { return pageStoreMgr; } + /** + * @return Page store backup manager. + */ + public IgniteBackupManager backup() { + return backupMgr; + } + /** * @return Write ahead log 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 e73ad52400451..443e4279ff36d 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 @@ -24,6 +24,7 @@ import javax.cache.Cache; import javax.cache.processor.EntryProcessor; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtDemandedPartitionsMap; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; @@ -142,7 +143,7 @@ public interface IgniteCacheOffheapManager { * @return Data store. * @throws IgniteCheckedException If failed. */ - public CacheDataStore createCacheDataStore(int p) throws IgniteCheckedException; + public CacheDataStoreEx createCacheDataStore(int p) throws IgniteCheckedException; /** * @return Iterable over all existing cache data stores. @@ -159,7 +160,7 @@ public interface IgniteCacheOffheapManager { * @param store Data store. * @throws IgniteCheckedException If failed. */ - public void destroyCacheDataStore(CacheDataStore store) throws IgniteCheckedException; + public IgniteInternalFuture destroyCacheDataStore(CacheDataStore store) throws IgniteCheckedException; /** * TODO: GG-10884, used on only from initialValue. 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 0df7728f5c5db..e3f7f3d6f019a 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 @@ -40,6 +40,7 @@ import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.failure.FailureContext; import org.apache.ignite.failure.FailureType; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.metric.IoStatisticsHolder; import org.apache.ignite.internal.pagemem.FullPageId; @@ -108,6 +109,7 @@ import org.apache.ignite.internal.util.collection.IntMap; import org.apache.ignite.internal.util.collection.IntRWHashMap; import org.apache.ignite.internal.util.collection.IntSet; +import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.lang.GridCloseableIterator; import org.apache.ignite.internal.util.lang.GridCursor; import org.apache.ignite.internal.util.lang.GridIterator; @@ -153,7 +155,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); /** */ @@ -180,6 +182,9 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager /** */ protected final GridSpinBusyLock busyLock = new GridSpinBusyLock(); + /** */ + private final IgniteInternalFuture alwaysDoneFut = new GridFinishedFuture<>(true); + /** */ private int updateValSizeThreshold; @@ -923,7 +928,9 @@ private GridCloseableIterator iterator(final int cacheId, while (true) { if (cur == null) { - if (dataIt.hasNext()) { + boolean hasnext = dataIt.hasNext(); + + if (hasnext) { CacheDataStore ds = dataIt.next(); curPart = ds.partId(); @@ -1248,8 +1255,9 @@ private long allocateForTree() throws IgniteCheckedException { } /** {@inheritDoc} */ - @Override public final CacheDataStore createCacheDataStore(int p) throws IgniteCheckedException { - CacheDataStore dataStore; + @Override public final CacheDataStoreEx createCacheDataStore(int p) throws IgniteCheckedException { +// System.out.println(">xxx> create " + p); + CacheDataStoreEx dataStore; partStoreLock.lock(p); @@ -1272,7 +1280,7 @@ private long allocateForTree() throws IgniteCheckedException { * @return Cache data store. * @throws IgniteCheckedException If failed. */ - protected CacheDataStore createCacheDataStore0(int p) throws IgniteCheckedException { + protected CacheDataStoreEx createCacheDataStore0(int p) throws IgniteCheckedException { final long rootPage = allocateForTree(); CacheDataRowStore rowStore = new CacheDataRowStore(grp, grp.freeList(), p); @@ -1291,7 +1299,17 @@ protected CacheDataStore createCacheDataStore0(int p) throws IgniteCheckedExcept lsnr ); - return new CacheDataStoreImpl(p, rowStore, dataTree); + //return new CacheDataStoreImpl(p, rowStore, dataTree); + String treeName = treeName(p); + //grp, + + return new CacheDataStoreExImpl(grp.shared(), + new CacheDataStoreImpl( + p, + rowStore, + dataTree), + null, + log); } /** {@inheritDoc} */ @@ -1307,7 +1325,7 @@ protected CacheDataStore createCacheDataStore0(int p) throws IgniteCheckedExcept } /** {@inheritDoc} */ - @Override public final void destroyCacheDataStore(CacheDataStore store) throws IgniteCheckedException { + @Override public IgniteInternalFuture destroyCacheDataStore(CacheDataStore store) { int p = store.partId(); partStoreLock.lock(p); @@ -1315,9 +1333,9 @@ protected CacheDataStore createCacheDataStore0(int p) throws IgniteCheckedExcept try { boolean removed = partDataStores.remove(p, store); - assert removed; + assert removed : "cache=" + grp.cacheOrGroupName() + " p=" + p; - destroyCacheDataStore0(store); + return destroyCacheDataStore0(store); } catch (IgniteCheckedException e) { throw new IgniteException(e); @@ -1331,15 +1349,17 @@ protected CacheDataStore createCacheDataStore0(int p) throws IgniteCheckedExcept * @param store Cache data store. * @throws IgniteCheckedException If failed. */ - protected void destroyCacheDataStore0(CacheDataStore store) throws IgniteCheckedException { + protected IgniteInternalFuture destroyCacheDataStore0(CacheDataStore store) throws IgniteCheckedException { store.destroy(); + + return alwaysDoneFut; } /** * @param p Partition. * @return Tree name for given partition. */ - protected final String treeName(int p) { + protected static final String treeName(int p) { return BPlusTree.treeName("p-" + p, "CacheData"); } @@ -1441,7 +1461,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(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java index 4f7ed20b909d1..4e2db9347b7c3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java @@ -475,6 +475,8 @@ else if (!grp.localWalEnabled()) * @param topVer Topology version. */ public void onGroupRebalanceFinished(int grpId, AffinityTopologyVersion topVer) { + System.out.println("onGroupRebalanceFinished " + grpId + " topVer="+ topVer.topologyVersion() + "." + topVer.minorTopologyVersion() + " session topVer=" + tmpDisabledWal.topVer.topologyVersion() + "." + tmpDisabledWal.topVer.minorTopologyVersion()); + TemporaryDisabledWal session0 = tmpDisabledWal; if (session0 == null || session0.topVer.compareTo(topVer) > 0) @@ -522,8 +524,11 @@ public void onGroupRebalanceFinished(int grpId, AffinityTopologyVersion topVer) CacheGroupContext grp = cctx.cache().cacheGroup(grpId0); - if (grp != null) + if (grp != null) { + log.info("own moving " + grp.cacheOrGroupName() + " on topVer="+topVer.topologyVersion() + "." + topVer.minorTopologyVersion()); + grp.topology().ownMoving(topVer); + } else if (log.isDebugEnabled()) log.debug("Cache group was destroyed before checkpoint finished, [grpId=" + grpId0 + ']'); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridCachePreloadSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridCachePreloadSharedManager.java new file mode 100644 index 0000000000000..c976e26e43b88 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridCachePreloadSharedManager.java @@ -0,0 +1,1305 @@ +/* + * 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.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.Files; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Set; +import java.util.TreeMap; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.cache.CacheRebalanceMode; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteFeatures; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.managers.communication.TransmissionHandler; +import org.apache.ignite.internal.managers.communication.TransmissionMeta; +import org.apache.ignite.internal.pagemem.PageIdUtils; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.GridCacheGroupIdMessage; +import org.apache.ignite.internal.processors.cache.GridCacheIdMessage; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +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.ReadOnlyGridCacheDataStore; +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.pagemem.PageMemoryEx; +import org.apache.ignite.internal.processors.cache.preload.GridPartitionBatchDemandMessage; +import org.apache.ignite.internal.processors.cache.preload.PartitionUploadManager; +import org.apache.ignite.internal.util.GridConcurrentHashSet; +import org.apache.ignite.internal.util.GridIntList; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.lang.IgniteInClosureX; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.GridTopic.TOPIC_REBALANCE; +import static org.apache.ignite.internal.managers.communication.GridIoPolicy.PUBLIC_POOL; +import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL; +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.UTILITY_CACHE_NAME; + +/** */ +public class GridCachePreloadSharedManager extends GridCacheSharedManagerAdapter { + /** */ + public static final String REBALANCE_CP_REASON = "Rebalance has been scheduled [grps=%s]"; + + /** */ + private static final Runnable NO_OP = () -> {}; + + /** */ + public static final int REBALANCE_TOPIC_IDX = 0; + + /** todo */ + private static final boolean presistenceRebalanceEnabled = IgniteSystemProperties.getBoolean( + IgniteSystemProperties.IGNITE_PERSISTENCE_REBALANCE_ENABLED, false); + + /** */ + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + + /** Checkpoint listener. */ + private final CheckpointListener cpLsnr = new CheckpointListener(); + + /** */ +// private volatile FileRebalanceSingleNodeFuture headFut = new FileRebalanceSingleNodeFuture(); + + private volatile FileRebalanceFuture mainFut = new FileRebalanceFuture(); + + /** */ + private PartitionUploadManager uploadMgr; + + /** + * @param ktx Kernal context. + */ + public GridCachePreloadSharedManager(GridKernalContext ktx) { + assert CU.isPersistenceEnabled(ktx.config()) : + "Persistence must be enabled to preload any of cache partition files"; + + uploadMgr = new PartitionUploadManager(ktx); + } + + /** + * @return The Rebalance topic to communicate with. + */ + public static Object rebalanceThreadTopic() { + return TOPIC_REBALANCE.topic("Rebalance", REBALANCE_TOPIC_IDX); + } + + public boolean persistenceRebalanceApplicable() { + return !cctx.kernalContext().clientNode() && + CU.isPersistenceEnabled(cctx.kernalContext().config()) && + cctx.isRebalanceEnabled(); + } + + /** {@inheritDoc} */ + @Override protected void start0() throws IgniteCheckedException { + uploadMgr.start0(cctx); + + ((GridCacheDatabaseSharedManager)cctx.database()).addCheckpointListener(cpLsnr); + } + + /** {@inheritDoc} */ + @Override protected void stop0(boolean cancel) { + lock.writeLock().lock(); + + try { + uploadMgr.stop0(cancel); + + ((GridCacheDatabaseSharedManager)cctx.database()).removeCheckpointListener(cpLsnr); + + mainFut.cancel(); + } + finally { + lock.writeLock().unlock(); + } + } + + /** + * @param assignsMap A map of cache assignments grouped by grpId. + * @param force {@code true} if must cancel previous rebalance. + * @param rebalanceId Current rebalance id. + * @return Runnable to execute the chain. + */ + public Runnable addNodeAssignments( + Map assignsMap, + AffinityTopologyVersion topVer, + boolean force, + long rebalanceId + ) { + U.dumpStack(cctx.localNodeId() + ">>> add assignments"); + + NavigableMap>>> nodeOrderAssignsMap = + sliceNodeCacheAssignments(assignsMap); + + if (nodeOrderAssignsMap.isEmpty()) + return NO_OP; + + // Start new rebalance session. + FileRebalanceFuture mainFut0 = mainFut; + + lock.writeLock().lock(); + + try { + if (!mainFut0.isDone()) + mainFut0.cancel(); + + mainFut0 = mainFut = new FileRebalanceFuture(cpLsnr, assignsMap, topVer); + + FileRebalanceSingleNodeFuture rqFut = null; + Runnable rq = NO_OP; + + if (log.isInfoEnabled()) + log.info("Prepare the chain to demand assignments: " + nodeOrderAssignsMap); + + // Clear the previous rebalance futures if exists. +// futMap.clear(); + + for (Map.Entry>>> entry : nodeOrderAssignsMap.descendingMap().entrySet()) { + Map>> descNodeMap = entry.getValue(); + + int order = entry.getKey(); + + for (Map.Entry>> assignEntry : descNodeMap.entrySet()) { + FileRebalanceSingleNodeFuture rebFut = new FileRebalanceSingleNodeFuture(cctx, mainFut, log, assignEntry.getKey(), + order, rebalanceId, assignEntry.getValue(), topVer); + + mainFut0.add(order, rebFut); + + final Runnable nextRq0 = rq; + final FileRebalanceSingleNodeFuture rqFut0 = rqFut; + +// } +// else { + + if (rqFut0 != null) { + // headFut = rebFut; // The first seen rebalance node. + rebFut.listen(f -> { + try { + if (log.isDebugEnabled()) + log.debug("Running next task, last future result is " + f.get()); + + if (f.get()) // Not cancelled. + nextRq0.run(); + // todo check how this chain is cancelling + } + catch (IgniteCheckedException e) { + rqFut0.onDone(e); + } + }); + } + + rq = requestNodePartitions(assignEntry.getKey(), rebFut); + rqFut = rebFut; + } + } + + // create listener + TransmissionHandler hndr = new RebalanceDownloadHandler(); + + cctx.kernalContext().io().addTransmissionHandler(rebalanceThreadTopic(), hndr); + + // todo should be invoked in separated thread + mainFut.enableReadOnlyMode(); + + mainFut0.listen(new IgniteInClosureX>() { + @Override public void applyx(IgniteInternalFuture fut0) throws IgniteCheckedException { + cctx.kernalContext().io().removeTransmissionHandler(rebalanceThreadTopic()); + + if (log.isInfoEnabled()) + log.info("The final persistence rebalance is done [result=" + fut0.get() + ']'); + } + }); + +// mainFut = mainFut0; + + return rq; + } + finally { + lock.writeLock().unlock(); + } + } + + /** + * @param node Clustre node to send inital demand message to. + * @param rebFut The future to handle demand request. + */ + private Runnable requestNodePartitions( + ClusterNode node, + FileRebalanceSingleNodeFuture rebFut + ) { + return new Runnable() { + @Override public void run() { + if (staleFuture(rebFut)) + return; + + if (log.isInfoEnabled()) + log.info("Start partitions preloading [from=" + node.id() + ", fut=" + rebFut + ']'); + + final Map> assigns = rebFut.assigns; + + try { + if (rebFut.initReq.compareAndSet(false, true)) { + if (log.isDebugEnabled()) + log.debug("Prepare demand batch message [rebalanceId=" + rebFut.rebalanceId + "]"); + + GridPartitionBatchDemandMessage msg0 = + new GridPartitionBatchDemandMessage(rebFut.rebalanceId, + rebFut.topVer, + assigns.entrySet() + .stream() + .collect(Collectors.toMap(Map.Entry::getKey, + e -> GridIntList.valueOf(e.getValue())))); + +// futMap.put(node.id(), rebFut); + + cctx.gridIO().sendToCustomTopic(node, rebalanceThreadTopic(), msg0, SYSTEM_POOL); + + if (log.isDebugEnabled()) + log.debug("Demand message is sent to partition supplier [node=" + node.id() + "]"); + } + } + catch (IgniteCheckedException e) { + U.error(log, "Error sending request for demanded cache partitions", e); + +// rebFut.onDone(e); + + mainFut.onDone(e); + } + } + }; + } + + /** + * @param assignsMap The map of cache groups assignments to process. + * @return The map of cache assignments [group_order, [node, [group_id, partitions]]] + */ + private NavigableMap>>> sliceNodeCacheAssignments( + Map assignsMap + ) { + NavigableMap>>> result = new TreeMap<>(); + + for (Map.Entry grpEntry : assignsMap.entrySet()) { + int grpId = grpEntry.getKey(); + + CacheGroupContext grp = cctx.cache().cacheGroup(grpId); + + GridDhtPreloaderAssignments assigns = grpEntry.getValue(); + + if (fileRebalanceRequired(grp, assigns.keySet())) { + int grpOrderNo = grp.config().getRebalanceOrder(); + + result.putIfAbsent(grpOrderNo, new HashMap<>()); + + for (Map.Entry grpAssigns : assigns.entrySet()) { + ClusterNode node = grpAssigns.getKey(); + + result.get(grpOrderNo).putIfAbsent(node, new HashMap<>()); + + result.get(grpOrderNo) + .get(node) + .putIfAbsent(grpId, + grpAssigns.getValue() + .partitions() + .fullSet()); + } + } + } + + return result; + } + + /** + * @param fut The future to check. + * @return true if future can be processed. + */ + private boolean staleFuture(GridFutureAdapter fut) { + return fut == null || fut.isCancelled() || fut.isFailed() || fut.isDone(); + } + + /** + * @param grp The corresponding to assignments cache group context. + * @param nodes Assignment nodes for specified cache group. + * @return {@code True} if cache must be rebalanced by sending files. + */ + public boolean fileRebalanceRequired(CacheGroupContext grp, Collection nodes) { + return FileRebalanceSupported(grp, nodes) && + grp.config().getRebalanceDelay() != -1 && + grp.config().getRebalanceMode() != CacheRebalanceMode.NONE; + } + + /** + * @param grp The corresponding to assignments cache group context. + * @param nodes Assignment nodes for specified cache group. + * @return {@code True} if cache might be rebalanced by sending cache partition files. + */ + public boolean FileRebalanceSupported(CacheGroupContext grp, Collection nodes) { + if (nodes == null || nodes.isEmpty()) + return false; + + // Do not rebalance system cache with files as they are not exists. + if (grp.groupId() == CU.cacheId(UTILITY_CACHE_NAME)) + return false; + + if (grp.mvccEnabled()) + return false; + + Map globalSizes = grp.topology().globalPartSizes(); + + assert !globalSizes.isEmpty() : grp.cacheOrGroupName(); + + boolean notEnoughData = true; + + for (Long partSize : globalSizes.values()) { + if (partSize > 0) { + notEnoughData = false; + + break; + } + } + + if (notEnoughData) + return false; + + return presistenceRebalanceEnabled && + grp.persistenceEnabled() && + IgniteFeatures.allNodesSupports(nodes, IgniteFeatures.CACHE_PARTITION_FILE_REBALANCE); + } + + /** + * @param fut Exchange future. + */ + public void onExchangeDone(GridDhtPartitionsExchangeFuture fut) { + // todo switch to read-only mode after first exchange + //System.out.println(cctx.localNodeId() + " >xxx> process onExchangeDone"); + +// if (!mainFut.isDone() && fut.topologyVersion().equals(mainFut.topVer)) { +// mainFut.switchAllPartitions(); +// } +// else { +// U.dumpStack(cctx.localNodeId() + " skip onExchange done=" + mainFut.isDone() + ", topVer="+fut.topologyVersion() +", rebVer="+mainFut.topVer +", equals="+fut.topologyVersion().equals(mainFut.topVer)); +// } + + // switch partitions without exchange + } + + public void handleDemandMessage(UUID nodeId, GridPartitionBatchDemandMessage msg) { + if (log.isDebugEnabled()) + log.debug("Handling demand request " + msg.rebalanceId()); + + if (msg.rebalanceId() < 0) // Demand node requested context cleanup. + return; + + ClusterNode demanderNode = cctx.discovery().node(nodeId); + + if (demanderNode == null) { + log.error("The demand message rejected (demander node left the cluster) [" + + ", nodeId=" + nodeId + ", topVer=" + msg.topologyVersion() + ']'); + + return; + } + + if (msg.assignments() == null || msg.assignments().isEmpty()) { + log.error("The Demand message rejected. Node assignments cannot be empty [" + + "nodeId=" + nodeId + ", topVer=" + msg.topologyVersion() + ']'); + + return; + } + + uploadMgr.onDemandMessage(nodeId, msg, PUBLIC_POOL); + } + + /** + * Get partition restore future. + * + * @param msg Message. + * @return Partition restore future or {@code null} if no partition currently restored. + */ + public IgniteInternalFuture partitionRestoreFuture(UUID nodeId, GridCacheMessage msg) { + if (!(msg instanceof GridCacheGroupIdMessage) && !(msg instanceof GridCacheIdMessage)) + return null; + + return mainFut.lockMessagesFuture(null, -1, -1); + } + + /** + * Completely destroy the partition without changing its state. + * + * @param part Partition to destroy. + * @return Future that will be completed after removing the partition file. + */ + private IgniteInternalFuture destroyPartitionAsync(GridDhtLocalPartition part) { + GridFutureAdapter fut = new GridFutureAdapter<>(); + + part.clearAsync(); + + part.onClearFinished(c -> { + //todo should prevent any removes on DESTROYED partition. + ReadOnlyGridCacheDataStore store = (ReadOnlyGridCacheDataStore)part.dataStore().store(true); + + store.disableRemoves(); + + try { + part.group().offheap().destroyCacheDataStore(part.dataStore()).listen(f -> { + try { + fut.onDone(f.get()); + } + catch (IgniteCheckedException e) { + fut.onDone(e); + } + } + ); + } + catch (IgniteCheckedException e) { + fut.onDone(e); + } + }); + + return fut; + } + + /** + * Restore partition on new file. Partition should be completely destroyed before restore it with new file. + * + * @param grpId Group id. + * @param partId Partition number. + * @param fsPartFile New partition file on the same filesystem. + * @return Future that will be completed when partition will be fully re-initialized. The future result is the HWM + * value of update counter in read-only partition. + * @throws IgniteCheckedException If file store for specified partition doesn't exists or partition file cannot be + * moved. + */ + private IgniteInternalFuture> restorePartition( + int grpId, + int partId, + File fsPartFile, + IgniteInternalFuture destroyFut + ) throws IgniteCheckedException { + CacheGroupContext ctx = cctx.cache().cacheGroup(grpId); + + if (!destroyFut.isDone()) { + if (log.isDebugEnabled()) + log.debug("Await partition destroy [grp=" + grpId + ", partId=" + partId + "]"); + + destroyFut.get(); + } + + File dst = new File(getStorePath(grpId, partId)); + + if (log.isInfoEnabled()) + log.info("Moving downloaded partition file: " + fsPartFile + " --> " + dst); + + try { + Files.move(fsPartFile.toPath(), dst.toPath()); + } + catch (IOException e) { + // todo FileAlreadyExistsException -> retry ? + throw new IgniteCheckedException("Unable to move file from " + fsPartFile + " to " + dst, e); + } + + // Reinitialize file store afte rmoving partition file. + cctx.pageStore().ensure(grpId, partId); + + return cpLsnr.schedule(() -> { + // Save current update counter. + PartitionUpdateCounter maxCntr = ctx.topology().localPartition(partId).dataStore().partUpdateCounter(); + + // Replacing partition and cache data store with the new one. + // After this operation all on-heap cached entries should be cleaned. + // At this point all partition updates are queued. + // File page store should be reinitialized. + assert cctx.pageStore().exists(grpId, partId) : "File doesn't exist [grpId=" + grpId + ", p=" + partId + "]"; + + GridDhtLocalPartition part = ctx.topology().forceCreatePartition(partId, true); + + // Switching to new datastore. + part.readOnly(false); + + maxCntr.finalizeUpdateCounters(); + + return new T2<>(part.updateCounter(), maxCntr.get()); + }); + } + + /** + * Get partition file path. + * + * @param grpId Group ID. + * @param partId Partition ID. + * @return Absolute partition file path + * @throws IgniteCheckedException If cache or partition with the given ID was not created. + */ + private String getStorePath(int grpId, int partId) throws IgniteCheckedException { + return ((FilePageStore)((FilePageStoreManager)cctx.pageStore()).getStore(grpId, partId)).getFileAbsolutePath(); + } + + /** + * @param fut Future. + * @return {@code True} if rebalance topology version changed by exchange thread or force + * reassing exchange occurs, see {@link RebalanceReassignExchangeTask} for details. + */ + private boolean topologyChanged(FileRebalanceSingleNodeFuture fut) { + return !cctx.exchange().rebalanceTopologyVersion().equals(fut.topVer); + // todo || fut != rebalanceFut; // Same topology, but dummy exchange forced because of missing partitions. + } + + public void reserveHistoryForFilePreloading(GridDhtPartitionExchangeId exchId, + GridDhtPartitionsExchangeFuture exchangeFut) { + + } + + /** */ + private static class CheckpointListener implements DbCheckpointListener { + /** Queue. */ + private final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); + + /** {@inheritDoc} */ + @Override public void onMarkCheckpointBegin(Context ctx) { + Runnable r; + + while ((r = queue.poll()) != null) + r.run(); + } + + /** {@inheritDoc} */ + @Override public void onCheckpointBegin(Context ctx) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void beforeCheckpointBegin(Context ctx) { + // No-op. + } + + /** */ + public void cancelAll() { + queue.clear(); + } + + public IgniteInternalFuture schedule(final Runnable task) { + return schedule(() -> { + task.run(); + + return null; + }); + } + + public IgniteInternalFuture schedule(final Callable task) { + return schedule(new CheckpointTask<>(task)); + } + + private IgniteInternalFuture schedule(CheckpointTask task) { + queue.offer(task); + + return task.fut; + } + + /** */ + private static class CheckpointTask implements Runnable { + /** */ + final GridFutureAdapter fut = new GridFutureAdapter<>(); + + /** */ + final Callable task; + + /** */ + CheckpointTask(Callable task) { + this.task = task; + } + + /** {@inheritDoc} */ + @Override public void run() { + try { + fut.onDone(task.call()); + } + catch (Exception e) { + fut.onDone(e); + } + } + } + } + + /** */ + private class RebalanceDownloadHandler implements TransmissionHandler { + /** {@inheritDoc} */ + @Override public void onException(UUID nodeId, Throwable err) { + mainFut.onDone(err); + } + + /** {@inheritDoc} */ + @Override public String filePath(UUID nodeId, TransmissionMeta fileMeta) { + Integer grpId = (Integer)fileMeta.params().get("group"); + Integer partId = (Integer)fileMeta.params().get("part"); + + FileRebalanceSingleNodeFuture fut = mainFut.nodeRoutine(grpId, nodeId); + + try { + // todo how to abort receive? + if (staleFuture(fut)) { + log.warning("Rebalance routine for node \"" + nodeId + "\" was not found"); + + File file = File.createTempFile("ignite-stale-partition", ".$$$"); + + return file.toString(); + } + + assert grpId != null; + assert partId != null; + + return getStorePath(grpId, partId) + ".$$$"; + } catch (IgniteCheckedException | IOException e) { + fut.onDone(e); + + throw new IgniteException("File transfer exception.", e); + } + } + + /** {@inheritDoc} */ + @Override public Consumer chunkHandler(UUID nodeId, TransmissionMeta initMeta) { + assert false; + + return null; + } + + /** {@inheritDoc} */ + @Override public Consumer fileHandler(UUID nodeId, TransmissionMeta initMeta) { + return file -> { + Integer grpId = (Integer)initMeta.params().get("group"); + Integer partId = (Integer)initMeta.params().get("part"); + + FileRebalanceSingleNodeFuture fut = mainFut.nodeRoutine(grpId, nodeId); + + if (staleFuture(fut)) { + if (log.isInfoEnabled()) + log.info("Removing staled file [nodeId=" + nodeId + ", file=" + file + "]"); + + file.delete(); + + return; + } + + IgniteInternalFuture evictFut = fut.evictionFuture(grpId); + + try { + // todo should lock only on checkpoint + mainFut.lockMessaging(nodeId, grpId, partId); + + IgniteInternalFuture> switchFut = restorePartition(grpId, partId, file, evictFut); + + switchFut.listen( f -> { + try { + T2 cntrs = f.get(); + + assert cntrs != null; + + cctx.kernalContext().closure().runLocalSafe(() -> { + fut.onPartitionRestored(grpId, partId, cntrs.get1(), cntrs.get2()); + }); + } catch (IgniteCheckedException e) { + fut.onDone(e); + } + }); + } catch (IgniteCheckedException e) { + fut.onDone(e); + } + }; + } + } + + /** */ + private class FileRebalanceFuture extends GridFutureAdapter { + /** */ + private final Map, FileRebalanceSingleNodeFuture> futMap = new HashMap<>(); + + /** */ + private final CheckpointListener cpLsnr; + + /** */ + private final Map> allPartsMap = new HashMap<>(); + + /** */ + private final Map> allGroupsMap = new ConcurrentHashMap<>(); + + /** */ + private final AffinityTopologyVersion topVer; + + /** */ + private final AtomicReference switchFutRef = new AtomicReference<>(); + + /** */ + private final Map cleanupRegions = new HashMap<>(); + + public FileRebalanceFuture() { + this(null, null, null); + + onDone(true); + } + + /** + * @param lsnr Checkpoint listener. + */ + public FileRebalanceFuture(CheckpointListener lsnr, Map assignsMap, AffinityTopologyVersion startVer) { + cpLsnr = lsnr; + topVer = startVer; + + initialize(assignsMap); + } + + /** + * Initialize rebalancing mappings. + * + * @param assignments Assignments. + */ + private void initialize(Map assignments) { + if (assignments == null || assignments.isEmpty()) + return; + + Map> regionToParts = new HashMap<>(); + + for (Map.Entry entry : assignments.entrySet()) { + int grpId = entry.getKey(); + GridDhtPreloaderAssignments assigns = entry.getValue(); + + Set nodes = allGroupsMap.computeIfAbsent(grpId, v -> new GridConcurrentHashSet<>()); + + CacheGroupContext grp = cctx.cache().cacheGroup(grpId); + + if (!fileRebalanceRequired(grp, assigns.keySet())) + continue; + + String regName = cctx.cache().cacheGroup(grpId).dataRegion().config().getName(); + + Set regionParts = regionToParts.computeIfAbsent(regName, v -> new HashSet<>()); + + Set allPartitions = allPartsMap.computeIfAbsent(grpId, v -> new HashSet<>()); + + for (Map.Entry e : assigns.entrySet()) { + GridDhtPartitionDemandMessage msg = e.getValue(); + ClusterNode node = e.getKey(); + + nodes.add(node.id()); + + Set parttitions = msg.partitions().fullSet(); + + for (Integer partId : parttitions) { + regionParts.add(((long)grpId << 32) + partId); + + allPartitions.add(partId); + } + } + } + + for (Map.Entry> e : regionToParts.entrySet()) + cleanupRegions.put(e.getKey(), new PageMemCleanupTask(e.getKey(), e.getValue())); + } + + public synchronized void add(int order, FileRebalanceSingleNodeFuture fut) { + T2 k = new T2<>(order, fut.node.id()); + + futMap.put(k, fut); + } + + // todo add/get should be consistent (ORDER or GROUP_ID arg) + public synchronized FileRebalanceSingleNodeFuture nodeRoutine(int grpId, UUID nodeId) { + int order = cctx.cache().cacheGroup(grpId).config().getRebalanceOrder(); + + T2 k = new T2<>(order, nodeId); + + return futMap.get(k); + } + + /** {@inheritDoc} */ + @Override public synchronized boolean cancel() { + cpLsnr.cancelAll(); + + for (FileRebalanceSingleNodeFuture fut : futMap.values()) + fut.cancel(); + + futMap.clear(); + + return onDone(false, null, true); + } + + public IgniteInternalFuture lockMessagesFuture(UUID nodeId, int grpId, int partId) { + // todo we don't care from where request is coming - we should + // lock partition for all updates! nodeId is redundant + // FileRebalanceSingleNodeFuture currFut = futMap.get(nodeId); + + // todo how to get partition and group + // return staleFuture(currFut) ? null : currFut.switchFut(-1, -1); + + return switchFutRef.get(); + } + + public void lockMessaging(UUID nodeId, Integer grpId, Integer partId) { + switchFutRef.compareAndSet(null, new GridFutureAdapter()); + } + + public boolean unlockMessaging() { + GridFutureAdapter fut = switchFutRef.get(); + + if (fut != null && switchFutRef.compareAndSet(fut, null)) { + fut.onDone(); + + return true; + } + + return false; + } + + public void onNodeGroupDone(int grpId, UUID nodeId, boolean historical) { + Set remainingNodes = allGroupsMap.get(grpId); + + boolean rmvd = remainingNodes.remove(nodeId); + + assert rmvd : "Duplicate remove " + nodeId; + + if (remainingNodes.isEmpty() && allGroupsMap.remove(grpId) != null && !historical) { + CacheGroupContext gctx = cctx.cache().cacheGroup(grpId); + + log.info("Rebalancing complete [group=" + gctx.cacheOrGroupName() + "]"); + + if (gctx.localWalEnabled()) + cctx.exchange().scheduleResendPartitions(); + else + cctx.walState().onGroupRebalanceFinished(gctx.groupId(), mainFut.topVer); + } + } + + public synchronized void onNodeDone(FileRebalanceSingleNodeFuture fut, Boolean res, Throwable err, boolean cancel) { + if (err != null || cancel) { + onDone(res, err, cancel); + + return; + } + + GridFutureAdapter rmvdFut = futMap.remove(new T2<>(fut.order(), fut.nodeId())); + + assert rmvdFut != null && rmvdFut.isDone() : rmvdFut; + + if (futMap.isEmpty()) + onDone(true); + } + + /** + * Switch all rebalanced partitions to read-only mode. + */ + private void enableReadOnlyMode() { + IgniteInternalFuture switchFut = cpLsnr.schedule(() -> { + for (Map.Entry> e : allPartsMap.entrySet()) { + CacheGroupContext grp = cctx.cache().cacheGroup(e.getKey()); + + for (Integer partId : e.getValue()) { + GridDhtLocalPartition part = grp.topology().localPartition(partId); + + if (part.readOnly()) + continue; + + part.readOnly(true); + } + } + }); + + if (log.isDebugEnabled()) + log.debug("Await partition switch: " + allPartsMap); + + try { + if (!switchFut.isDone()) + cctx.database().wakeupForCheckpoint(String.format(REBALANCE_CP_REASON, allPartsMap.keySet())); + + switchFut.get(); + } + catch (IgniteCheckedException e) { + onDone(e); + + // todo throw exception? + return; + } + + for (Map.Entry> e : allPartsMap.entrySet()) { + int grpId = e.getKey(); + + CacheGroupContext gctx = cctx.cache().cacheGroup(grpId); + + for (Integer partId : e.getValue()) { + GridDhtLocalPartition part = gctx.topology().localPartition(partId); + + if (log.isDebugEnabled()) + log.debug("Add destroy future for partition " + part.id()); + + destroyPartitionAsync(part).listen(fut -> { + try { + if (!fut.get()) + throw new IgniteCheckedException("Partition was not destroyed " + + "properly [grp=" + gctx.cacheOrGroupName() + ", p=" + part.id() + "]"); + + boolean exists = gctx.shared().pageStore().exists(grpId, part.id()); + + assert !exists : "File exists [grp=" + gctx.cacheOrGroupName() + ", p=" + part.id() + "]"; + + onPartitionEvicted(grpId, partId); + } + catch (IgniteCheckedException ex) { + onDone(ex); + } + }); + } + } + } + + private void onPartitionEvicted(int grpId, int partId) throws IgniteCheckedException { + CacheGroupContext gctx = cctx.cache().cacheGroup(grpId); + + String regName = gctx.dataRegion().config().getName(); + + PageMemCleanupTask pageMemFut = cleanupRegions.get(regName); + + pageMemFut.cleanupMemory(); + } + + public IgniteInternalFuture evictionFuture(int grpId) { + String regName = cctx.cache().cacheGroup(grpId).dataRegion().config().getName(); + + return cleanupRegions.get(regName); + } + + private class PageMemCleanupTask extends GridFutureAdapter { + private final Set parts; + + private final AtomicInteger evictedCntr; + + private final String name; + + public PageMemCleanupTask(String regName, Set remainingParts) { + name = regName; + parts = remainingParts; + evictedCntr = new AtomicInteger(); + } + + public void cleanupMemory() throws IgniteCheckedException { + int evictedCnt = evictedCntr.incrementAndGet(); + + assert evictedCnt <= parts.size(); + + if (evictedCnt == parts.size()) { + ((PageMemoryEx)cctx.database().dataRegion(name).pageMemory()) + .clearAsync( + (grp, pageId) -> + parts.contains(((long)grp << 32) + PageIdUtils.partId(pageId)), true) + .listen(c1 -> { + if (log.isDebugEnabled()) + log.debug("Eviction is done [region=" + name + "]"); + + onDone(); + }); + } + } + } + } + + /** */ + private static class FileRebalanceSingleNodeFuture extends GridFutureAdapter { + /** Context. */ + protected GridCacheSharedContext cctx; + + /** Logger. */ + protected IgniteLogger log; + + /** */ + private long rebalanceId; + + /** */ + @GridToStringInclude + private Map> assigns; + + /** */ + private AffinityTopologyVersion topVer; + + /** */ + private Map> remaining; + + /** */ + private Map> remainingHist; + + /** {@code True} if the initial demand request has been sent. */ + private AtomicBoolean initReq = new AtomicBoolean(); + + /** */ + private final ClusterNode node; + + /** */ + private final FileRebalanceFuture mainFut; + + /** */ + private final int rebalanceOrder; + + /** + * Default constructor for the dummy future. + */ + public FileRebalanceSingleNodeFuture() { + this(null, null, null, null, 0, 0, Collections.emptyMap(), null); + + onDone(); + } + + /** + * @param node Supplier node. + * @param rebalanceId Rebalance id. + * @param assigns Map of assignments to request from remote. + * @param topVer Topology version. + */ + public FileRebalanceSingleNodeFuture( + GridCacheSharedContext cctx, + FileRebalanceFuture mainFut, + IgniteLogger log, + ClusterNode node, + int rebalanceOrder, + long rebalanceId, + Map> assigns, + AffinityTopologyVersion topVer + ) { + this.cctx = cctx; + this.mainFut = mainFut; + this.log = log; + this.node = node; + this.rebalanceOrder = rebalanceOrder; + this.rebalanceId = rebalanceId; + this.assigns = assigns; + this.topVer = topVer; + + remaining = new ConcurrentHashMap<>(assigns.size()); + remainingHist = new ConcurrentHashMap<>(assigns.size()); + + for (Map.Entry> entry : assigns.entrySet()) { + Set parts = entry.getValue(); + int grpId = entry.getKey(); + + assert !remaining.containsKey(grpId); + + remaining.put(grpId, new GridConcurrentHashSet<>(entry.getValue())); + } + } + + /** + * @return Rebalancing order. + */ + public int order() { + return rebalanceOrder; + } + + /** + * @return Supplier node ID. + */ + public UUID nodeId() { + return node.id(); + } + + /** {@inheritDoc} */ + @Override public boolean cancel() { + return onDone(false, null, true); + } + + /** + * @param grpId Cache group id to search. + * @param partId Cache partition to remove; + */ + public void onPartitionRestored(int grpId, int partId, long min, long max) { + Set parts = remaining.get(grpId); + + assert parts != null : "Invalid group identifier: " + grpId; + + boolean rmvd = parts.remove(partId); + + assert rmvd : "Partition not found: " + partId; + + remainingHist.computeIfAbsent(grpId, v -> new ConcurrentSkipListSet<>()) + .add(new HistoryDesc(partId, min, max)); + + if (log.isDebugEnabled()) { + log.debug("Partition done [grp=" + cctx.cache().cacheGroup(grpId).cacheOrGroupName() + + ", p=" + partId + ", remaining=" + parts.size() + "]"); + } + + if (parts.isEmpty()) { + mainFut.unlockMessaging(); + + onGroupRestored(grpId); + } + } + + private void onGroupRestored(int grpId) { + if (remaining.remove(grpId) == null) + return; + + Set parts0 = remainingHist.remove(grpId); + + CacheGroupContext grp = cctx.cache().cacheGroup(grpId); + + GridDhtPartitionDemandMessage msg = new GridDhtPartitionDemandMessage(rebalanceId, topVer, grpId); + + for (HistoryDesc desc : parts0) { + assert desc.toCntr >= desc.fromCntr : "from=" + desc.fromCntr + ", to=" + desc.toCntr; + + if (desc.fromCntr != desc.toCntr) { + msg.partitions().addHistorical(desc.partId, desc.fromCntr, desc.toCntr, parts0.size()); + + continue; + } + + if (log.isDebugEnabled()) { + log.debug("Prepare to request historical rebalancing [p=" + + desc.partId + ", from=" + desc.fromCntr + ", to=" + desc.toCntr + "]"); + } + + // No historical rebalancing required -can own partition. + if (grp.localWalEnabled()) { + boolean owned = grp.topology().own(grp.topology().localPartition(desc.partId)); + + assert owned : "part=" + desc.partId + ", grp=" + grp.cacheOrGroupName(); + } + } + + if (!msg.partitions().hasHistorical()) { + mainFut.onNodeGroupDone(grpId, nodeId(), false); + + if (remaining.isEmpty()) + onDone(true); + + return; + } + + GridDhtPartitionExchangeId exchId = cctx.exchange().lastFinishedFuture().exchangeId(); + + GridDhtPreloaderAssignments assigns = new GridDhtPreloaderAssignments(exchId, topVer); + + assigns.put(node, msg); + + GridCompoundFuture forceFut = new GridCompoundFuture<>(CU.boolReducer()); + + Runnable cur = grp.preloader().addAssignments(assigns, + true, + rebalanceId, + null, + forceFut); + + if (log.isDebugEnabled()) + log.debug("Triggering historical rebalancing [node=" + node.id() + ", group=" + grp.cacheOrGroupName() + "]"); + + cur.run(); + + forceFut.markInitialized(); + + forceFut.listen(c -> { + try { + mainFut.onNodeGroupDone(grpId, nodeId(), true); + + if (forceFut.get() && remaining.isEmpty()) + onDone(true); + else + cancel(); + } + catch (IgniteCheckedException e) { + onDone(e); + } + }); + } + + /** {@inheritDoc} */ + public boolean onDone(@Nullable Boolean res, @Nullable Throwable err, boolean cancel) { + boolean r = super.onDone(res, err, cancel); + + mainFut.onNodeDone(this, res, err, cancel); + + return r; + } + + public IgniteInternalFuture evictionFuture(int grpId) { + IgniteInternalFuture fut = mainFut.evictionFuture(grpId); + + assert fut != null; + + return fut; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(FileRebalanceSingleNodeFuture.class, this); + } + + private static class HistoryDesc implements Comparable { + /** Partition id. */ + final int partId; + + /** From counter. */ + final long fromCntr; + + /** To counter. */ + final long toCntr; + + public HistoryDesc(int partId, long fromCntr, long toCntr) { + this.partId = partId; + this.fromCntr = fromCntr; + this.toCntr = toCntr; + } + + @Override public int compareTo(@NotNull Object o) { + HistoryDesc otherDesc = (HistoryDesc)o; + + if (partId > otherDesc.partId) + return 1; + + if (partId < otherDesc.partId) + return -1; + + return 0; + } + } + } +} 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 9061de0b64c75..b3305249f4fa6 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 @@ -1276,6 +1276,8 @@ public boolean isInitial() { * @return {@code True}. */ @Override public boolean cancel() { + U.dumpStack("Rebalancing canceled [grp=" + grp.cacheOrGroupName() + "]"); + // Cancel lock is needed only for case when some message might be on the fly while rebalancing is // cancelled. cancelLock.writeLock().lock(); @@ -1379,8 +1381,11 @@ private void cleanupRemoteContexts(UUID nodeId) { */ private void partitionDone(UUID nodeId, int p, boolean updateState) { synchronized (this) { - if (updateState && grp.localWalEnabled()) - grp.topology().own(grp.topology().localPartition(p)); + if (updateState && grp.localWalEnabled()) { + boolean owned = grp.topology().own(grp.topology().localPartition(p)); + + System.out.println(grp.cacheOrGroupName() + " own " + p + (owned ? "OWNED" : "MOVED")); + } if (isDone()) return; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java index 5cbf61070d7f9..bd01cda8dfc75 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java @@ -222,8 +222,8 @@ public long updateSequence(long updateSeq, AffinityTopologyVersion topVer) { long old = this.updateSeq; // Overwrite update sequence without checking in case of greater topology version - if (topVer.compareTo(top) == 0) - assert updateSeq >= old : "Invalid update sequence [cur=" + old + ", new=" + updateSeq + ']'; +// if (topVer.compareTo(top) == 0) +// assert updateSeq >= old : "Invalid update sequence [cur=" + old + ", new=" + updateSeq + ']'; this.updateSeq = updateSeq; 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..32a856f797f4d 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 @@ -297,6 +297,11 @@ public void handleDemandMessage(int topicId, UUID nodeId, GridDhtPartitionDemand if (iter.isPartitionMissing(p)) continue; + assert grp.topology().localPartition(p).updateCounter() >= histMap.updateCounterAt(i) : "Invalid update counter [p=" + p + " curr=" + grp.topology().localPartition(p).updateCounter() + ", req=" + histMap.updateCounterAt(i) + "]"; + + if (log.isDebugEnabled()) + log.debug("Supply hist rebalancing p=" + p + " range [" + histMap.initialUpdateCounterAt(i) + " - " + histMap.updateCounterAt(i) + "]"); + supplyMsg.addEstimatedKeysCount(histMap.updateCounterAt(i) - histMap.initialUpdateCounterAt(i)); } } 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 a1fc523650766..f91c4cb492cac 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 @@ -2261,6 +2261,45 @@ private String exchangeTimingsLogMessage(String header, List timings) { } } + // todo reserve only moving partitions (not all) + // todo reserve only those partitions that will be supplied from current node + if (cctx.filePreloader() != null) { + for (CacheGroupContext ctx : cctx.cache().cacheGroups()) { + if (ctx.topology().hasMovingPartitions()) { + boolean reservedGrp = false; + + Set assigns = new HashSet<>(); + + for (GridDhtLocalPartition part : ctx.topology().localPartitions()) { + assigns.addAll(ctx.affinity().assignments(res).get(part.id())); + + if (reservedGrp = localReserved != null && localReserved.containsKey(new T2<>(ctx.groupId(), part.id()))) + break; + } + + if (reservedGrp || !assigns.contains(cctx.localNode()) || !cctx.filePreloader().fileRebalanceRequired(ctx, assigns)) + continue; + + for (GridDhtLocalPartition part : ctx.topology().localPartitions()) { + if (part.state() == GridDhtPartitionState.OWNING) { + if (localReserved != null && !localReserved.containsKey(new T2<>(ctx.groupId(), part.id()))) + continue; + + long cntr = part.updateCounter(); + + // todo debug + if (log.isInfoEnabled()) + log.info("Reserve WAL history for file preloading [cache=" + ctx.cacheOrGroupName() + ". p=" + part.id() + ", cntr=" + cntr); + + boolean reserved = cctx.database().reserveHistoryForPreloading(ctx.groupId(), part.id(), cntr); + + assert reserved : "Unable to reserve history [cache=" + ctx.cacheOrGroupName() + ". p=" + part.id() + ", cntr=" + cntr + "]"; + } + } + } + } + } + cctx.database().releaseHistoryForExchange(); if (err == null) { @@ -2274,6 +2313,8 @@ private String exchangeTimingsLogMessage(String header, List timings) { if (changedAffinity()) cctx.walState().changeLocalStatesOnExchangeDone(res, changedBaseline()); } + + cctx.filePreloader().onExchangeDone(this); } catch (Throwable t) { // In any case, this exchange future has to be completed. The original error should be preserved if exists. 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 dd4c25937dadb..8a9e6775d6ef6 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 @@ -42,6 +42,7 @@ import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.GridPlainRunnable; import org.apache.ignite.internal.util.typedef.CI1; +import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST; @@ -259,6 +260,8 @@ private IgniteCheckedException stopError() { GridDhtPartitionDemandMessage msg = assignments.get(histSupplier); if (msg == null) { + U.dumpStack(" >xxx> add assignments hist supplier " + histSupplier.id()); + assignments.put(histSupplier, msg = new GridDhtPartitionDemandMessage( top.updateSequence(), assignments.topologyVersion(), @@ -266,6 +269,8 @@ private IgniteCheckedException stopError() { ); } + System.out.println(">xxx> add hist cntrs p=" + p + " from=" + part.initialUpdateCounter() + ", to=" + countersMap.updateCounter(p)); + // TODO FIXME https://issues.apache.org/jira/browse/IGNITE-11790 msg.partitions().addHistorical(p, part.initialUpdateCounter(), countersMap.updateCounter(p), partitions); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java index c094b6fa7664a..450ce460f424f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridClientPartitionTopology.java @@ -459,7 +459,7 @@ else if (!node2part.nodeId().equals(loc.id())) { } /** {@inheritDoc} */ - @Override public GridDhtLocalPartition forceCreatePartition(int p) throws IgniteCheckedException { + @Override public GridDhtLocalPartition forceCreatePartition(int p, boolean replace) throws IgniteCheckedException { throw new UnsupportedOperationException(); } 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 65447c855f554..35f746ba7c476 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 @@ -38,6 +38,7 @@ import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.pagemem.wal.record.delta.PartitionMetaStateRecord; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheDataStoreEx; import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.cache.GridCacheConcurrentMapImpl; @@ -63,6 +64,7 @@ import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.GridIterator; import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; @@ -74,7 +76,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; @@ -158,7 +159,7 @@ public class GridDhtLocalPartition extends GridCacheConcurrentMapImpl implements /** */ @GridToStringExclude - private volatile CacheDataStore store; + private volatile CacheDataStoreEx store; /** Set if failed to move partition to RENTING state due to reservations, to be checked when * reservation is released. */ @@ -296,7 +297,7 @@ private CacheMapHolder cacheMapHolder(GridCacheContext cctx) { /** * @return Data store. */ - public CacheDataStore dataStore() { + public CacheDataStoreEx dataStore() { return store; } @@ -395,7 +396,7 @@ private void removeVersionedEntry(int cacheId, KeyCacheObject key, GridCacheVers public void cleanupRemoveQueue() { if (state() == MOVING) { if (rmvQueue.sizex() >= rmvQueueMaxSize) { - LT.warn(log, "Deletion queue cleanup for moving partition was delayed until rebalance is finished. " + + U.dumpStack(log, "Deletion queue cleanup for moving partition was delayed until rebalance is finished. " + "[grpId=" + this.grp.groupId() + ", partId=" + id() + ", grpParts=" + this.grp.affinity().partitions() + @@ -439,6 +440,42 @@ public void onDeferredDelete(int cacheId, KeyCacheObject key, GridCacheVersion v rmvQueue.add(new RemovedEntryHolder(cacheId, key, ver, rmvdEntryTtl)); } + /** + * Set {@link CacheDataStoreEx.StorageMode} to the corresponding local partition storage. + */ + public void readOnly(boolean readOnly) { + if (state() != MOVING) + throw new IgniteException("Expected MIVING partition, actual state is " + state()); + + store.readOnly(readOnly); + } + + /** + * @return The curretly active storage mode. + */ + public boolean readOnly() { + return store.readOnly(); + } + +// /** +// * @param mode The mode to associate with data storage instance. +// * @param storage The cache data storage instance to set to. +// */ +// public void dataStore(CacheDataStoreEx.StorageMode mode, IgniteCacheOffheapManager.CacheDataStore storage) { +// if (state() != MOVING) +// return; +// +// store.store(mode, storage); +// } + +// /** +// * @param mode The storage mode. +// * @return The storage intance for the given mode. +// */ +// public IgniteCacheOffheapManager.CacheDataStore dataStore(CacheDataStoreEx.StorageMode mode) { +// return store.store(mode); +// } + /** * Reserves a partition so it won't be cleared or evicted. * @@ -616,7 +653,7 @@ 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: " + partState; if (casState(state, MOVING)) break; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopology.java index 3416431d4a292..b47147f72e30b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopology.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopology.java @@ -168,10 +168,11 @@ public boolean initPartitionsWhenAffinityReady(AffinityTopologyVersion affVer, G * Unconditionally creates partition during restore of persisted partition state. * * @param p Partition ID. + * @param replace * @return Partition. * @throws IgniteCheckedException If failed. */ - public GridDhtLocalPartition forceCreatePartition(int p) throws IgniteCheckedException; + public GridDhtLocalPartition forceCreatePartition(int p, boolean replace) throws IgniteCheckedException; /** * @param topVer Topology version at the time of creation. 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 e5d5651064e8d..66155f4e139bf 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 @@ -907,13 +907,15 @@ public GridDhtLocalPartition getOrCreatePartition(int p) { } /** {@inheritDoc} */ - @Override public GridDhtLocalPartition forceCreatePartition(int p) throws IgniteCheckedException { + @Override public GridDhtLocalPartition forceCreatePartition(int p, boolean replace) throws IgniteCheckedException { lock.writeLock().lock(); try { GridDhtLocalPartition part = locParts.get(p); - if (part != null) { + assert !replace || part.state() == MOVING : part.state(); + + if (part != null && !replace) { if (part.state() != EVICTED) return part; else @@ -997,6 +999,8 @@ else if (loc != null && state == RENTING && !showRenting) { "[grp=" + grp.cacheOrGroupName() + ", part=" + p + ", topVer=" + topVer + ", this.topVer=" + this.readyTopVer + ']'); + System.out.println(">xxx> create partition"); + locParts.set(p, loc = partFactory.create(ctx, grp, p)); this.updateSeq.incrementAndGet(); @@ -1489,6 +1493,8 @@ private boolean shouldOverridePartitionMap(GridDhtPartitionMap currentMap, GridD boolean fullMapUpdated = (node2part == null); if (node2part != null) { + System.out.println(">xxx> node2part update "); + for (GridDhtPartitionMap part : node2part.values()) { GridDhtPartitionMap newPart = partMap.get(part.nodeId()); @@ -2377,8 +2383,12 @@ private GridDhtLocalPartition rebalancePartition(int p, boolean clear, GridDhtPa if (part.state() != MOVING) part.moving(); - if (clear) + if (clear) { + if (!grp.cacheOrGroupName().contains("sys-cache")) + U.dumpStack(ctx.localNodeId() + " >xxx> add historical part=" + p); + exchFut.addClearingPartition(grp, part.id()); + } assert part.state() == MOVING : part; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DbCheckpointListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DbCheckpointListener.java index 7c6938e0f4b21..d9a874c075d17 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DbCheckpointListener.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DbCheckpointListener.java @@ -17,6 +17,8 @@ package org.apache.ignite.internal.processors.cache.persistence; +import java.util.Map; +import java.util.Set; import java.util.concurrent.Executor; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.cache.persistence.partstate.PartitionAllocationMap; @@ -35,6 +37,16 @@ public interface Context { */ public boolean nextSnapshot(); + /** + * @return Collection partition which require meta to be collected. + */ + public Map> gatherPartStats(); + + /** + * @param parts Collection of partitions for which statistics should be gathered. + */ + public void gatherPartStats(Map> parts); + /** * @return Partition allocation statistic map */ 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 a738f2d724fef..e987fa173dd47 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 @@ -124,6 +124,8 @@ import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor; import org.apache.ignite.internal.processors.cache.ExchangeActions; import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; +import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; @@ -213,6 +215,8 @@ import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_FILE_MATCHER; import static org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO.getType; import static org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO.getVersion; +import static org.apache.ignite.internal.processors.dr.GridDrType.DR_NONE; +import static org.apache.ignite.internal.processors.dr.GridDrType.DR_PRELOAD; import static org.apache.ignite.internal.util.IgniteUtils.checkpointBufferSize; import static org.apache.ignite.internal.util.IgniteUtils.hexLong; @@ -1744,6 +1748,8 @@ private boolean safeToUpdatePageMemories() { /** {@inheritDoc} */ @Override public synchronized Map> reserveHistoryForExchange() { + log.info(cctx.localNodeId() + " >xxx> reserve history for exchange "); + assert reservedForExchange == null : reservedForExchange; reservedForExchange = new HashMap<>(); @@ -1861,6 +1867,8 @@ private Map> partitionsApplicableForWalRebalance() { /** {@inheritDoc} */ @Override public void releaseHistoryForPreloading() { +// U.dumpStack(cctx.localNodeId() + " >xxx> release history for preloading"); + for (Map.Entry, T2> e : reservedForPreloading.entrySet()) { try { cctx.wal().release(e.getValue().get2()); @@ -1875,6 +1883,28 @@ private Map> partitionsApplicableForWalRebalance() { reservedForPreloading.clear(); } + /** + * 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 reservedWALPointer(int grpId, int partId, long initCntr) { + assert reservedForPreloading != null; + + T2 reserved = reservedForPreloading.get(new T2<>(grpId, partId)); + + assert reserved != null : "History should be reserved"; + + long cntr = reserved.get1(); + + assert cntr <= initCntr : "reserved=" + cntr + ", init=" + initCntr; + + return (FileWALPointer)reserved.get2(); + } + /** * */ @@ -2799,7 +2829,7 @@ private RestoreLogicalState applyLogicalUpdates( CacheGroupContext ctx = cctx.cache().cacheGroup(rbRec.groupId()); if (ctx != null && !ctx.isLocal()) { - ctx.topology().forceCreatePartition(rbRec.partitionId()); + ctx.topology().forceCreatePartition(rbRec.partitionId(), false); ctx.offheap().onPartitionInitialCounterUpdated(rbRec.partitionId(), rbRec.start(), rbRec.range()); @@ -2959,7 +2989,7 @@ private void applyUpdate(GridCacheContext cacheCtx, DataEntry dataEntry) throws if (partId == -1) partId = cacheCtx.affinity().partition(dataEntry.key()); - GridDhtLocalPartition locPart = cacheCtx.isLocal() ? null : cacheCtx.topology().forceCreatePartition(partId); + GridDhtLocalPartition locPart = cacheCtx.isLocal() ? null : cacheCtx.topology().forceCreatePartition(partId, false); switch (dataEntry.op()) { case CREATE: @@ -3327,11 +3357,13 @@ public CheckpointEntry createCheckPointEntry( * @param grpId Group ID. * @param partId Partition ID. */ - public void schedulePartitionDestroy(int grpId, int partId) { + public IgniteInternalFuture schedulePartitionDestroy(int grpId, int partId) { Checkpointer cp = checkpointer; if (cp != null) - cp.schedulePartitionDestroy(cctx.cache().cacheGroup(grpId), grpId, partId); + return cp.schedulePartitionDestroy(cctx.cache().cacheGroup(grpId), grpId, partId); + + return null; } /** @@ -3366,6 +3398,118 @@ public void cancelOrWaitPartitionDestroy(int grpId, int partId) throws IgniteChe checkpointReadLockTimeout = val; } + public void applyFastUpdates( + WALIterator it, + IgniteBiPredicate recPredicate, + IgnitePredicate entryPredicate, + boolean restore + ) { + if (it == null) + return; + + while (it.hasNext()) { + IgniteBiTuple next = it.next(); + + WALRecord rec = next.get2(); + + System.out.println(">xxx> rec "); + + if (!recPredicate.apply(next.get1(), rec)) + break; + + applyFastWALRecord(rec, entryPredicate, restore); + } + } + + /** + * @param rec The WAL record to process. + * @param entryPredicate An entry filter to apply. + */ + private void applyFastWALRecord(WALRecord rec, IgnitePredicate entryPredicate, boolean restore) { + switch (rec.type()) { + case MVCC_DATA_RECORD: + case DATA_RECORD: + checkpointReadLock(); + + try { + DataRecord dataRec = (DataRecord)rec; + + for (DataEntry dataEntry : dataRec.writeEntries()) { + if (entryPredicate.apply(dataEntry)) { +// checkpointReadLock(); +// +// try { + int cacheId = dataEntry.cacheId(); + + GridCacheContext cacheCtx = cctx.cacheContext(cacheId); + + if (cacheCtx != null) + applyFastUpdate(cacheCtx, dataEntry); + else if (log != null) + log.warning("Cache is not started. Updates cannot be applied " + + "[cacheId=" + cacheId + ']'); +// } +// finally { +// checkpointReadUnlock(); +// } + } + } + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + finally { + checkpointReadUnlock(); + } + + break; + + case MVCC_TX_RECORD: + checkpointReadLock(); + + try { + MvccTxRecord txRecord = (MvccTxRecord)rec; + + byte txState = convertToTxState(txRecord.state()); + + cctx.coordinators().updateState(txRecord.mvccVersion(), txState, true); + } + finally { + checkpointReadUnlock(); + } + + break; + + default: + // Skip other records. + } + } + + private void applyFastUpdate(GridCacheContext cctx, DataEntry entry) throws IgniteCheckedException { + AffinityTopologyVersion topVer = cctx.topology().readyTopologyVersion(); + + GridCacheEntryEx cached = cctx.cache().entryEx(entry.key(), topVer); + + try { + boolean initVal = cached.initialValue(entry.value(), + entry.writeVersion(), + null, // cctx.mvccEnabled() ? ((MvccDataEntry)entry).mvccVersion() : null, + null, //cctx.mvccEnabled() ? ((MvccDataEntry)entry).newMvccVersion() : null, + (byte)0, // cctx.mvccEnabled() ? ((MvccDataEntry)entry).mvccTxState() : TxState.NA, + (byte)0, //cctx.mvccEnabled() ? ((MvccDataEntry)entry).newMvccTxState() : TxState.NA, + 0, + entry.expireTime(), + true, + topVer, + cctx.isDrEnabled() ? DR_PRELOAD : DR_NONE, + false); + + System.out.println(">xxx> applying fast update: " + entry.key().value(cctx.cacheObjectContext(), false) + ", init = " + initVal); + } catch (GridCacheEntryRemovedException ignore) { + log.info("Ignoring removed entry"); + } + } + /** * Partition destroy queue. */ @@ -3378,7 +3522,7 @@ private static class PartitionDestroyQueue { * @param grpCtx Group context. * @param partId Partition ID to destroy. */ - private void addDestroyRequest(@Nullable CacheGroupContext grpCtx, int grpId, int partId) { + private IgniteInternalFuture addDestroyRequest(@Nullable CacheGroupContext grpCtx, int grpId, int partId) { PartitionDestroyRequest req = new PartitionDestroyRequest(grpId, partId); PartitionDestroyRequest old = pendingReqs.putIfAbsent(new T2<>(grpId, partId), req); @@ -3387,6 +3531,8 @@ private void addDestroyRequest(@Nullable CacheGroupContext grpCtx, int grpId, in + "[grpId=" + grpId + ", grpName=" + grpCtx.cacheOrGroupName() + ", partId=" + partId + ']'; + + return old != null ? old.reqFut : req.reqFut; } /** @@ -3427,6 +3573,9 @@ private static class PartitionDestroyRequest { /** Destroy future. Not null if partition destroy has begun. */ private GridFutureAdapter destroyFut; + /** Destroy future. Not null if partition destroy has begun. */ + private GridFutureAdapter reqFut = new GridFutureAdapter<>(); + /** * @param grpId Group ID. * @param partId Partition ID. @@ -3450,6 +3599,8 @@ private synchronized boolean cancel() { cancelled = true; + reqFut.onDone(false); + return true; } @@ -3480,6 +3631,11 @@ private synchronized void onDone(Throwable err) { assert destroyFut != null; destroyFut.onDone(err); + + if (err == null) + reqFut.onDone(true); + else + reqFut.onDone(err); } /** @@ -3983,9 +4139,11 @@ private int destroyEvictedPartitions() throws IgniteCheckedException { * @param grpId Group ID. * @param partId Partition ID. */ - private void schedulePartitionDestroy(@Nullable CacheGroupContext grpCtx, int grpId, int partId) { + private IgniteInternalFuture schedulePartitionDestroy(@Nullable CacheGroupContext grpCtx, int grpId, int partId) { + IgniteInternalFuture resFut; + synchronized (this) { - scheduledCp.destroyQueue.addDestroyRequest(grpCtx, grpId, partId); + resFut = scheduledCp.destroyQueue.addDestroyRequest(grpCtx, grpId, partId); } if (log.isDebugEnabled()) @@ -3993,6 +4151,8 @@ private void schedulePartitionDestroy(@Nullable CacheGroupContext grpCtx, int gr if (grpCtx != null) wakeupForCheckpoint(PARTITION_DESTROY_CHECKPOINT_TIMEOUT, "partition destroy"); + + return resFut; } /** @@ -4385,6 +4545,16 @@ private DbCheckpointListener.Context createOnCheckpointBeginContext( return delegate.nextSnapshot(); } + /** {@inheritDoc} */ + @Override public Map> gatherPartStats() { + return delegate.gatherPartStats(); + } + + /** {@inheritDoc} */ + @Override public void gatherPartStats(Map> parts) { + delegate.gatherPartStats(parts); + } + /** {@inheritDoc} */ @Override public PartitionAllocationMap partitionStatMap() { return delegate.partitionStatMap(); @@ -4529,6 +4699,9 @@ private class DbCheckpointContextImpl implements DbCheckpointListener.Context { /** Partition map. */ private final PartitionAllocationMap map; + /** Collection of partitions to gather statistics. */ + private final Map> gatherParts = new HashMap<>(); + /** Pending tasks from executor. */ private GridCompoundFuture pendingTaskFuture; @@ -4547,6 +4720,19 @@ private DbCheckpointContextImpl(CheckpointProgress curr, PartitionAllocationMap return curr.nextSnapshot; } + /** {@inheritDoc} */ + @Override public Map> gatherPartStats() { + return gatherParts; + } + + /** {@inheritDoc} */ + @Override public void gatherPartStats(Map> parts) { + for (Map.Entry> e : parts.entrySet()) { + gatherParts.computeIfAbsent(e.getKey(), g -> new HashSet<>()) + .addAll(e.getValue()); + } + } + /** {@inheritDoc} */ @Override public PartitionAllocationMap partitionStatMap() { return map; 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 df847abb0e4ad..f1378d7e2f0e1 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 @@ -37,6 +37,7 @@ import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.failure.FailureContext; import org.apache.ignite.failure.FailureType; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.pagemem.FullPageId; import org.apache.ignite.internal.pagemem.PageIdAllocator; import org.apache.ignite.internal.pagemem.PageIdUtils; @@ -55,6 +56,8 @@ import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageUpdatePartitionDataRecordV2; import org.apache.ignite.internal.pagemem.wal.record.delta.PartitionDestroyRecord; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheDataStoreEx; +import org.apache.ignite.internal.processors.cache.CacheDataStoreExImpl; import org.apache.ignite.internal.processors.cache.CacheDiagnosticManager; import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; import org.apache.ignite.internal.processors.cache.CacheGroupContext; @@ -104,6 +107,7 @@ import org.apache.ignite.internal.util.lang.IgniteInClosure2X; import org.apache.ignite.internal.util.lang.IgnitePredicateX; import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; @@ -191,13 +195,16 @@ public IndexStorage getIndexStorage() { } /** {@inheritDoc} */ - @Override protected CacheDataStore createCacheDataStore0(int p) throws IgniteCheckedException { + @Override protected CacheDataStoreEx createCacheDataStore0(int p) throws IgniteCheckedException { if (ctx.database() instanceof GridCacheDatabaseSharedManager) ((GridCacheDatabaseSharedManager) ctx.database()).cancelOrWaitPartitionDestroy(grp.groupId(), p); boolean exists = ctx.pageStore() != null && ctx.pageStore().exists(grp.groupId(), p); - return new GridCacheDataStore(p, exists); + CacheDataStore store = new GridCacheDataStore(p, exists); + CacheDataStore readOnlyStore = new ReadOnlyGridCacheDataStore(grp, ctx, store, grp.groupId()); + + return new CacheDataStoreExImpl(grp.shared(), store, readOnlyStore, log); } /** {@inheritDoc} */ @@ -228,13 +235,14 @@ private void syncMetadata(Context ctx) throws IgniteCheckedException { boolean needSnapshot = ctx.nextSnapshot() && ctx.needToSnapshot(grp.cacheOrGroupName()); - if (needSnapshot) { + if (needSnapshot || + ctx.gatherPartStats().getOrDefault(grp.groupId(), new HashSet<>()).contains(PageIdAllocator.INDEX_PARTITION)) { if (execSvc == null) - addPartitions(ctx); + addIndexPartition(ctx); else { execSvc.execute(() -> { try { - addPartitions(ctx); + addIndexPartition(ctx); } catch (IgniteCheckedException e) { throw new IgniteException(e); @@ -253,11 +261,14 @@ private void syncMetadata(Context ctx) throws IgniteCheckedException { * @throws IgniteCheckedException If failed. */ private void syncMetadata(Context ctx, Executor execSvc, boolean needSnapshot) throws IgniteCheckedException { + final Set parts = ctx.gatherPartStats() + .getOrDefault(grp.groupId(), new HashSet<>()); + if (execSvc == null) { reuseList.saveMetadata(); for (CacheDataStore store : partDataStores.values()) - saveStoreMetadata(store, ctx, false, needSnapshot); + saveStoreMetadata(store, ctx, false, needSnapshot || parts.contains(store.partId())); } else { execSvc.execute(() -> { @@ -272,7 +283,7 @@ private void syncMetadata(Context ctx, Executor execSvc, boolean needSnapshot) t for (CacheDataStore store : partDataStores.values()) execSvc.execute(() -> { try { - saveStoreMetadata(store, ctx, false, needSnapshot); + saveStoreMetadata(store, ctx, false, needSnapshot || parts.contains(store.partId())); } catch (IgniteCheckedException e) { throw new IgniteException(e); @@ -289,8 +300,11 @@ private void saveStoreMetadata( CacheDataStore store, Context ctx, boolean beforeDestroy, - boolean needSnapshot + boolean gatherStats ) throws IgniteCheckedException { + if (store instanceof CacheDataStoreEx && ((CacheDataStoreEx)store).readOnly()) + return; + RowStore rowStore0 = store.rowStore(); if (rowStore0 != null) { @@ -422,7 +436,7 @@ else if (updCntrsBytes != null && link != 0) { int pageCnt; - if (needSnapshot) { + if (gatherStats) { pageCnt = this.ctx.pageStore().pages(grpId, store.partId()); io.setCandidatePageCount(partMetaPageAddr, size == 0 ? 0 : pageCnt); @@ -478,10 +492,10 @@ else if (state == MOVING || state == RENTING) { pageMem.releasePage(grpId, partMetaId, partMetaPage); } } - else if (needSnapshot) + else if (gatherStats) tryAddEmptyPartitionToSnapshot(store, ctx); } - else if (needSnapshot) + else if (gatherStats) tryAddEmptyPartitionToSnapshot(store, ctx); } @@ -517,7 +531,7 @@ else if (needSnapshot) processed++; - GridDhtLocalPartition part = grp.topology().forceCreatePartition(p); + GridDhtLocalPartition part = grp.topology().forceCreatePartition(p, false); // Triggers initialization of existing(having datafile) partition before acquiring cp read lock. part.dataStore().init(); @@ -570,7 +584,7 @@ else if (needSnapshot) } } else if (recoverState != null) { // Pre-create partition if having valid state. - GridDhtLocalPartition part = grp.topology().forceCreatePartition(p); + GridDhtLocalPartition part = grp.topology().forceCreatePartition(p, false); updateState(part, recoverState); @@ -756,7 +770,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(); @@ -829,7 +843,7 @@ private static boolean addPartition( } /** {@inheritDoc} */ - @Override protected void destroyCacheDataStore0(CacheDataStore store) throws IgniteCheckedException { + @Override protected IgniteInternalFuture destroyCacheDataStore0(CacheDataStore store) throws IgniteCheckedException { assert ctx.database() instanceof GridCacheDatabaseSharedManager : "Destroying cache data store when persistence is not enabled: " + ctx.database(); @@ -844,7 +858,7 @@ private static boolean addPartition( ctx.database().checkpointReadUnlock(); } - ((GridCacheDatabaseSharedManager)ctx.database()).schedulePartitionDestroy(grp.groupId(), partId); + return ((GridCacheDatabaseSharedManager)ctx.database()).schedulePartitionDestroy(grp.groupId(), partId); } /** @@ -1001,8 +1015,9 @@ 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); + // todo For file rebalancing we starting searching from reserved pointer. + // For regular historical rebalancing it may be more eefective to search pointer in checkpoint hostory + FileWALPointer startPtr = database.reservedWALPointer(grp.groupId(), p, initCntr); if (startPtr == null) throw new IgniteCheckedException("Could not find start pointer for partition [part=" + p + ", partCntrSince=" + initCntr + "]"); @@ -1359,6 +1374,8 @@ private void advance() { if (++rebalancedCntrs[idx] == to || entry.partitionCounter() == to && grp.hasAtomicCaches()) donePart = entry.partitionId(); +// System.out.println("p=" + entry.partitionId() + ", cntr=" + entry.partitionCounter() + " DONE e=" + entry.key()); +// System.out.println("p=" + entry.partitionId() + ", cntr=" + entry.partitionCounter() + " e=" + entry.key() + " rebCntr=" + rebalancedCntrs[idx]); next = entry; @@ -1404,12 +1421,26 @@ else if (rec.get2() instanceof RollbackRecord) { doneParts.add(rbRec.partitionId()); // Add to done set immediately. } +// else +// System.out.println("p=" + rbRec.partitionId() + ", overlap=" + rbRec.overlap(from, to)); } } } + if (doneParts.size() != partMap.size()) { + for (Map.Entry> e : CachePartitionPartialCountersMap.toCountersMap(partMap).entrySet()) { + int p = e.getKey(); + long from = e.getValue().get1(); + long to = e.getValue().get2(); + + if (!doneParts.contains(p)) + log.error("WAL iterator failed to restore history: [p=" + p + ", from=" + from + ", to=" + to + ", rebCtr=" + rebalancedCntrs[partMap.partitionIndex(p)]); + } + } + + //rebalancedCntrs[idx] assert entryIt != null || doneParts.size() == partMap.size() : - "Reached end of WAL but not all partitions are done"; + "Reached end of WAL but not all partitions are done ; done=" + doneParts + ", parts=" + partMap; } } } @@ -1571,7 +1602,7 @@ private static class Metas { */ public class GridCacheDataStore implements CacheDataStore { /** */ - private final int partId; + protected final int partId; /** */ private volatile AbstractFreeList freeList; @@ -1769,7 +1800,7 @@ private CacheDataStore init0(boolean checkExists) throws IgniteCheckedException } }; - PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory(); + PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();; delegate0 = new CacheDataStoreImpl(partId, rowStore, dataTree) { /** {@inheritDoc} */ 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 73459dd2bb7a7..58a291255d6e1 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 @@ -27,6 +27,8 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Predicate; +import javax.management.InstanceNotFoundException; import org.apache.ignite.DataRegionMetrics; import org.apache.ignite.DataRegionMetricsProvider; import org.apache.ignite.DataStorageMetrics; @@ -52,9 +54,11 @@ import org.apache.ignite.internal.pagemem.wal.WALPointer; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; 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.topology.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.persistence.evict.FairFifoPageEvictionTracker; import org.apache.ignite.internal.processors.cache.persistence.evict.NoOpPageEvictionTracker; import org.apache.ignite.internal.processors.cache.persistence.evict.PageEvictionTracker; @@ -66,6 +70,7 @@ import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList; import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage; import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetastorageLifecycleListener; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotOperation; import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList; import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageLockListener; import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport; @@ -894,6 +899,17 @@ public WALPointer lastCheckpointMarkWalPointer() { return null; } + /** + * Perform a snapshot operation on checkponter. + * + * @param op Snapshot operation. + * @param reason The text message reason. + * @return Checkpoint progress future. + */ + public CheckpointFuture wakeupForCheckpointOperation(SnapshotOperation op, String reason) { + return null; + } + /** * Waits until current state is checkpointed. * @@ -944,13 +960,26 @@ 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 cacheCtx Cache context to rebuild index at. + * @param partPred The partition filter predicate. + * @param restore true to rebuild indexes from the original store. + */ + public IgniteInternalFuture rebuildIndexesOnDemand( + GridCacheContext cacheCtx, + Predicate partPred, + boolean restore + ) { + return null; + } + /** * Needed action before any cache will stop */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/ReadOnlyGridCacheDataStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/ReadOnlyGridCacheDataStore.java new file mode 100644 index 0000000000000..5acf847dd8660 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/ReadOnlyGridCacheDataStore.java @@ -0,0 +1,491 @@ +/* + * 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.Collection; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import javax.cache.processor.EntryProcessor; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.metric.IoStatisticsHolder; +import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheMvccEntryInfo; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager; +import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.PartitionUpdateCounter; +import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; +import org.apache.ignite.internal.processors.cache.persistence.freelist.CacheFreeList; +import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList; +import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow; +import org.apache.ignite.internal.processors.cache.persistence.partstorage.PartitionMetaStorage; +import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler; +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.mvcc.data.MvccUpdateResult; +import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccLinkAwareSearchRow; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner; +import org.apache.ignite.internal.util.GridLongList; +import org.apache.ignite.internal.util.lang.GridCursor; +import org.apache.ignite.internal.util.lang.IgnitePredicateX; +import org.apache.ignite.lang.IgniteBiTuple; +import org.jetbrains.annotations.Nullable; + +/** + * todo CHECK with flag in gridcachedatastore + */ +public class ReadOnlyGridCacheDataStore implements CacheDataStore { + /** */ + private final IgniteLogger log; + + /** */ + private final CacheDataStore delegate; + + /** */ + private final NoopRowStore rowStore; + + /** */ + private final AtomicBoolean disableRemoves = new AtomicBoolean(); + + GridCacheSharedContext ctx; + + int grpId; + + /** + * todo + */ + public ReadOnlyGridCacheDataStore( + CacheGroupContext grp, + GridCacheSharedContext ctx, + CacheDataStore delegate, + int grpId + ) { + this.delegate = delegate; + this.ctx = ctx; + this.grpId = grpId; + + log = ctx.logger(getClass()); + + try { + rowStore = new NoopRowStore(grp, new NoopFreeList(grp.dataRegion())); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + + public void disableRemoves() { + if (disableRemoves.compareAndSet(false, true)) + log.info("Changing data store mode to READ [p=" + partId() + "]"); + } + + /** {@inheritDoc} */ + @Override public long nextUpdateCounter() { + return delegate.nextUpdateCounter(); + } + + /** {@inheritDoc} */ + @Override public long initialUpdateCounter() { + return delegate.initialUpdateCounter(); + } + + /** {@inheritDoc} */ + @Override public void resetUpdateCounter() { + delegate.resetUpdateCounter(); + } + + /** {@inheritDoc} */ + @Override public long getAndIncrementUpdateCounter(long delta) { + return delegate.getAndIncrementUpdateCounter(delta); + } + + /** {@inheritDoc} */ + @Override public long updateCounter() { + return delegate.updateCounter(); + } + + /** {@inheritDoc} */ + @Override public void updateCounter(long val) { + delegate.updateCounter(val); + } + + /** {@inheritDoc} */ + @Override public boolean updateCounter(long start, long delta) { + return delegate.updateCounter(start, delta); + } + + /** {@inheritDoc} */ + @Override public GridLongList finalizeUpdateCounters() { + return delegate.finalizeUpdateCounters(); + } + + /** {@inheritDoc} */ + @Override public int partId() { + return delegate.partId(); + } + + /** {@inheritDoc} */ + @Override public boolean isEmpty() { + return delegate.isEmpty(); + } + + /** {@inheritDoc} */ + @Override public long cacheSize(int cacheId) { + return delegate.cacheSize(cacheId); + } + + /** {@inheritDoc} */ + @Override public Map cacheSizes() { + return delegate.cacheSizes(); + } + + /** {@inheritDoc} */ + @Override public long fullSize() { + return delegate.fullSize(); + } + + /** {@inheritDoc} */ + @Override public void updateSize(int cacheId, long delta) { + delegate.updateSize(cacheId, delta); + } + + /** {@inheritDoc} */ + @Override public boolean init() { + return delegate.init(); + } + + /** {@inheritDoc} */ + @Override public long reservedCounter() { + return delegate.reservedCounter(); + } + + /** {@inheritDoc} */ + @Override public @Nullable PartitionUpdateCounter partUpdateCounter() { + return delegate.partUpdateCounter(); + } + + /** {@inheritDoc} */ + @Override public long reserve(long delta) { + return delegate.reserve(delta); + } + + /** {@inheritDoc} */ + @Override public void updateInitialCounter(long start, long delta) { + delegate.updateInitialCounter(start, delta); + } + + /** {@inheritDoc} */ + @Override public void setRowCacheCleaner(GridQueryRowCacheCleaner rowCacheCleaner) { + delegate.setRowCacheCleaner(rowCacheCleaner); + } + + /** {@inheritDoc} */ + @Override public PendingEntriesTree pendingTree() { + return delegate.pendingTree(); + } + + /** {@inheritDoc} */ + @Override public PartitionMetaStorage partStorage() { + return delegate.partStorage(); + } + + /** {@inheritDoc} */ + @Override public void preload() throws IgniteCheckedException { + delegate.preload(); + } + + /** {@inheritDoc} */ + @Override public void invoke( + GridCacheContext cctx, + KeyCacheObject key, + IgniteCacheOffheapManager.OffheapInvokeClosure clo + ) throws IgniteCheckedException { + // Assume we've performed an invoke operation on the B+ Tree and find nothing. + // Emulating that always inserting/removing a new value. + clo.call(null); + } + + /** {@inheritDoc} */ + @Override public void remove( + GridCacheContext cctx, + KeyCacheObject key, + int partId + ) throws IgniteCheckedException { + // todo think + if (!disableRemoves.get()) + delegate.remove(cctx, key, partId); + } + + /** {@inheritDoc} */ + @Override public CacheDataRow createRow( + GridCacheContext cctx, + KeyCacheObject key, + CacheObject val, + GridCacheVersion ver, + long expireTime, + @Nullable CacheDataRow oldRow + ) { + assert oldRow == null; + + if (key.partition() < 0) + key.partition(delegate.partId()); + + return new DataRow(key, val, ver, delegate.partId(), expireTime, cctx.cacheId()); + } + + /** {@inheritDoc} */ + @Override public void insertRows(Collection rows, IgnitePredicateX initPred){ + // No-op. + } + + /** {@inheritDoc} */ + @Override public GridCursor cursor() throws IgniteCheckedException { + return delegate.cursor(); + } + + /** {@inheritDoc} */ + @Override public GridCursor cursor(Object x) throws IgniteCheckedException { + return delegate.cursor(x); + } + + /** {@inheritDoc} */ + @Override public GridCursor cursor(MvccSnapshot mvccSnapshot) throws IgniteCheckedException { + return delegate.cursor(mvccSnapshot); + } + + /** {@inheritDoc} */ + @Override public GridCursor cursor(int cacheId) throws IgniteCheckedException { + return delegate.cursor(cacheId); + } + + /** {@inheritDoc} */ + @Override public GridCursor cursor(int cacheId, + MvccSnapshot mvccSnapshot) throws IgniteCheckedException { + return delegate.cursor(cacheId, mvccSnapshot); + } + + /** {@inheritDoc} */ + @Override public GridCursor cursor(int cacheId, KeyCacheObject lower, + KeyCacheObject upper) throws IgniteCheckedException { + return delegate.cursor(cacheId, lower, upper); + } + + /** {@inheritDoc} */ + @Override public GridCursor cursor(int cacheId, KeyCacheObject lower, KeyCacheObject upper, + Object x) throws IgniteCheckedException { + return delegate.cursor(cacheId, lower, upper, x); + } + + /** {@inheritDoc} */ + @Override public GridCursor cursor(int cacheId, KeyCacheObject lower, KeyCacheObject upper, + Object x, MvccSnapshot snapshot) throws IgniteCheckedException { + return delegate.cursor(cacheId, lower, upper, x, snapshot); + } + + /** {@inheritDoc} */ + @Override public void destroy() throws IgniteCheckedException { +// ((GridCacheOffheapManager)ctx.cache().cacheGroup(grpId).offheap()).destroyPartitionStore(grpId, partId()); + delegate.destroy(); + } + + /** {@inheritDoc} */ + @Override public void clear(int cacheId) throws IgniteCheckedException { + if (!disableRemoves.get()) + delegate.clear(cacheId); + } + + /** {@inheritDoc} */ + @Override public RowStore rowStore() { + return rowStore; + } + + /** {@inheritDoc} */ + @Override public void updateTxState(GridCacheContext cctx, CacheSearchRow row) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void update(GridCacheContext cctx, KeyCacheObject key, CacheObject val, GridCacheVersion ver, + long expireTime, @Nullable CacheDataRow oldRow) { + // No-op. + } + + @Override public int cleanup(GridCacheContext cctx, @Nullable List cleanupRows) { + // No-op. + return 0; + } + + @Override public boolean mvccInitialValue(GridCacheContext cctx, KeyCacheObject key, @Nullable CacheObject val, + GridCacheVersion ver, long expireTime, MvccVersion mvccVer, + MvccVersion newMvccVer) { + return false; + } + + @Override public boolean mvccApplyHistoryIfAbsent(GridCacheContext cctx, KeyCacheObject key, + List hist) { + return false; + } + + @Override public boolean mvccUpdateRowWithPreloadInfo(GridCacheContext cctx, KeyCacheObject key, + @Nullable CacheObject val, GridCacheVersion ver, long expireTime, MvccVersion mvccVer, + MvccVersion newMvccVer, byte mvccTxState, byte newMvccTxState) { + return false; + } + + @Override public MvccUpdateResult mvccUpdate(GridCacheContext cctx, KeyCacheObject key, CacheObject val, + GridCacheVersion ver, long expireTime, MvccSnapshot mvccSnapshot, @Nullable CacheEntryPredicate filter, + EntryProcessor entryProc, Object[] invokeArgs, boolean primary, boolean needHist, boolean noCreate, + boolean needOldVal, boolean retVal, boolean keepBinary) { + // todo empty result .. new MvccUpdateDataRow( PREV_NULL); + assert false; + + return null; + } + + @Override public MvccUpdateResult mvccRemove(GridCacheContext cctx, KeyCacheObject key, MvccSnapshot mvccSnapshot, + @Nullable CacheEntryPredicate filter, boolean primary, boolean needHistory, boolean needOldVal, + boolean retVal) throws IgniteCheckedException { + return delegate.mvccRemove(cctx, key, mvccSnapshot, filter, primary, needHistory, needOldVal, retVal); + } + + @Override public MvccUpdateResult mvccLock(GridCacheContext cctx, KeyCacheObject key, + MvccSnapshot mvccSnapshot) throws IgniteCheckedException { + return delegate.mvccLock(cctx, key, mvccSnapshot); + } + + @Override public void mvccRemoveAll(GridCacheContext cctx, KeyCacheObject key) throws IgniteCheckedException { + delegate.mvccRemoveAll(cctx, key); + } + + @Override public void mvccApplyUpdate(GridCacheContext cctx, KeyCacheObject key, CacheObject val, + GridCacheVersion ver, long expireTime, MvccVersion mvccVer) throws IgniteCheckedException { + + } + + /** {@inheritDoc} */ + @Override public CacheDataRow find(GridCacheContext cctx, KeyCacheObject key) throws IgniteCheckedException { + return null; + } + + @Override public GridCursor mvccAllVersionsCursor(GridCacheContext cctx, KeyCacheObject key, + Object x) throws IgniteCheckedException { + return delegate.mvccAllVersionsCursor(cctx, key, x); + } + + @Override public CacheDataRow mvccFind(GridCacheContext cctx, KeyCacheObject key, + MvccSnapshot snapshot) throws IgniteCheckedException { + return delegate.mvccFind(cctx, key, snapshot); + } + + @Override public List> mvccFindAllVersions(GridCacheContext cctx, + KeyCacheObject key) throws IgniteCheckedException { + return delegate.mvccFindAllVersions(cctx, key); + } + + /** */ + private static class NoopRowStore extends RowStore { + /** + * @param grp Cache group. + * @param freeList Free list. + */ + public NoopRowStore(CacheGroupContext grp, FreeList freeList) { + super(grp, freeList); + } + + /** {@inheritDoc} */ + @Override public void removeRow(long link, IoStatisticsHolder statHolder) { + // todo + } + + /** {@inheritDoc} */ + @Override public void addRow(CacheDataRow row, IoStatisticsHolder statHolder) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public boolean updateRow(long link, CacheDataRow row, IoStatisticsHolder statHolder) { + return true; + } + + /** {@inheritDoc} */ + @Override public void updateDataRow(long link, PageHandler pageHnd, S arg, + IoStatisticsHolder statHolder) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void setRowCacheCleaner(GridQueryRowCacheCleaner rowCacheCleaner) { + // No-op. + } + } + + /** */ + private static class NoopFreeList extends CacheFreeList { + /** */ + public NoopFreeList(DataRegion region) throws IgniteCheckedException { + super(0, null, null, region, null, 0, false, null); + } + + /** {@inheritDoc} */ + @Override public void insertDataRow(CacheDataRow row, IoStatisticsHolder statHolder) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void insertDataRows(Collection rows, IoStatisticsHolder statHolder) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public boolean updateDataRow(long link, CacheDataRow row, IoStatisticsHolder statHolder) { + // No-op. + + return true; + } + + /** {@inheritDoc} */ + @Override public void removeDataRowByLink(long link, IoStatisticsHolder statHolder) { + // todo + } + + /** {@inheritDoc} */ + @Override public void dumpStatistics(IgniteLogger log) { + + } + + /** {@inheritDoc} */ + @Override public Object updateDataRow(long link, PageHandler pageHnd, Object arg, + IoStatisticsHolder statHolder) { + return null; + } + + /** {@inheritDoc} */ + @Override public void saveMetadata() { + // No-op. + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/backup/IgniteBackupManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/backup/IgniteBackupManager.java new file mode 100644 index 0000000000000..94b4c1542acc5 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/backup/IgniteBackupManager.java @@ -0,0 +1,793 @@ +/* + * 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.backup; + +import java.io.Closeable; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicIntegerArray; +import java.util.function.BiConsumer; +import java.util.function.BooleanSupplier; +import java.util.function.Supplier; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.failure.FailureContext; +import org.apache.ignite.failure.FailureType; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.PageIdUtils; +import org.apache.ignite.internal.pagemem.store.PageStore; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.persistence.CheckpointFuture; +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.file.FileIOFactory; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStore; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; +import org.apache.ignite.internal.processors.cache.persistence.file.FileSerialPageStore; +import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; +import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId; +import org.apache.ignite.internal.processors.cache.persistence.partstate.PagesAllocationRange; +import org.apache.ignite.internal.processors.cache.persistence.partstate.PartitionAllocationMap; +import org.apache.ignite.internal.processors.metric.impl.LongAdderMetric; +import org.apache.ignite.internal.util.GridBusyLock; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.internal.A; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiClosure; +import org.apache.ignite.thread.IgniteThreadPoolExecutor; + +import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL; +import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA; +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.cacheDirName; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.cacheWorkDir; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.getPartitionFile; + +/** */ +public class IgniteBackupManager extends GridCacheSharedManagerAdapter { + /** */ + public static final String DELTA_SUFFIX = ".delta"; + + /** */ + public static final String PART_DELTA_TEMPLATE = PART_FILE_TEMPLATE + DELTA_SUFFIX; + + /** */ + public static final String BACKUP_CP_REASON = "Wakeup for checkpoint to take backup [name=%s]"; + + /** Prefix for backup threads. */ + private static final String BACKUP_RUNNER_THREAD_PREFIX = "backup-runner"; + + /** Total number of thread to perform local backup. */ + private static final int BACKUP_POOL_SIZE = 4; + + /** Factory to working with {@link FileSerialPageStore} as file storage. */ + private static final FileIOFactory ioFactory = new RandomAccessFileIOFactory(); + + /** Read-write lock to handle managers operations. */ + private final GridBusyLock busyLock = new GridBusyLock(); + + /** Map of registered cache backup processes and their corresponding contexts. */ + private final ConcurrentMap backupCtxs = new ConcurrentHashMap<>(); + + /** All registered page writers of all running backup processes. */ + private final ConcurrentMap> partWriters = new ConcurrentHashMap<>(); + + /** Backup thread pool. */ + private IgniteThreadPoolExecutor backupRunner; + + /** Checkpoint listener to handle scheduled backup requests. */ + private DbCheckpointListener cpLsnr; + + /** Database manager for enabled persistence. */ + private GridCacheDatabaseSharedManager dbMgr; + + /** Configured data storage page size. */ + private int pageSize; + + //// BELOW IS NOT USED + + /** Keep only the first page error. */ + private final ConcurrentMap pageTrackErrors = new ConcurrentHashMap<>(); + + /** */ + public IgniteBackupManager(GridKernalContext ctx) { + assert CU.isPersistenceEnabled(ctx.config()); + + } + + /** + * @param tmpDir Temporary directory to store files. + * @param partId Cache partition identifier. + * @return A file representation. + */ + private static File getPartionDeltaFile(File tmpDir, int partId) { + return new File(tmpDir, String.format(PART_DELTA_TEMPLATE, partId)); + } + + /** + * @param ccfg Cache configuration. + * @param partId Partiton identifier. + * @return The cache partiton file. + */ + private static File resolvePartitionFileCfg( + FilePageStoreManager storeMgr, + CacheConfiguration ccfg, + int partId + ) { + File cacheDir = storeMgr.cacheWorkDir(ccfg); + + return getPartitionFile(cacheDir, partId); + } + + /** {@inheritDoc} */ + @Override protected void start0() throws IgniteCheckedException { + super.start0(); + + pageSize = cctx.kernalContext() + .config() + .getDataStorageConfiguration() + .getPageSize(); + + assert pageSize > 0; + + if (!cctx.kernalContext().clientNode()) { + backupRunner = new IgniteThreadPoolExecutor( + BACKUP_RUNNER_THREAD_PREFIX, + cctx.igniteInstanceName(), + BACKUP_POOL_SIZE, + BACKUP_POOL_SIZE, + 30_000, + new LinkedBlockingQueue<>(), + SYSTEM_POOL, + (t, e) -> cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e))); + } + + dbMgr = (GridCacheDatabaseSharedManager)cctx.database(); + + dbMgr.addCheckpointListener(cpLsnr = new DbCheckpointListener() { + @Override public void beforeCheckpointBegin(Context ctx) { + for (BackupContext bctx0 : backupCtxs.values()) { + if (bctx0.started) + continue; + + // Gather partitions metainfo for thouse which will be copied. + ctx.gatherPartStats(bctx0.parts); + } + } + + @Override public void onMarkCheckpointBegin(Context ctx) { + // Under the write lock here. It's safe to add new stores + for (BackupContext bctx0 : backupCtxs.values()) { + if (bctx0.started) + continue; + + for (Map.Entry e : bctx0.partDeltaWriters.entrySet()) { + partWriters.computeIfAbsent(e.getKey(), p -> new LinkedList<>()) + .add(e.getValue()); + } + } + + // Remove not used delta stores. + for (List list0 : partWriters.values()) + list0.removeIf(PageStoreSerialWriter::stopped); + } + + @Override public void onCheckpointBegin(Context ctx) { + final FilePageStoreManager pageMgr = (FilePageStoreManager)cctx.pageStore(); + + // TODO move under the checkpoint write lock + for (BackupContext bctx0 : backupCtxs.values()) { + if (bctx0.started) + continue; + + try { + PartitionAllocationMap allocationMap = ctx.partitionStatMap(); + allocationMap.prepareForSnapshot(); + + assert !allocationMap.isEmpty() : "Partitions statistics has not been gathered: " + bctx0; + + for (GroupPartitionId pair : bctx0.partAllocLengths.keySet()) { + PagesAllocationRange allocRange = allocationMap.get(pair); + + assert allocRange != null : "Pages not allocated [pairId=" + pair + ", ctx=" + bctx0 + ']'; + + PageStore store = pageMgr.getStore(pair.getGroupId(), pair.getPartitionId()); + + bctx0.partAllocLengths.put(pair, + allocRange.getCurrAllocatedPageCnt() == 0 ? 0L : + (long)allocRange.getCurrAllocatedPageCnt() * pageSize + store.headerSize()); + + bctx0.partDeltaWriters.get(pair).pagesWrittenBits = + new AtomicIntegerArray(allocRange.getCurrAllocatedPageCnt()); + + GridDhtLocalPartition part = cctx.cache().cacheGroup(pair.getGroupId()).topology().localPartition(pair.getPartitionId()); + + System.out.println("before copy p="+part.id()+", reserved="+part.reservedCounter()+", cntr="+part.updateCounter()); + } + + // Submit all tasks for partitions and deltas processing. + submitTasks(bctx0, pageMgr.workDir()); + + bctx0.started = true; + } + catch (IgniteCheckedException e) { + bctx0.result.onDone(e); + } + } + } + }); + } + + /** {@inheritDoc} */ + @Override protected void stop0(boolean cancel) { + dbMgr.removeCheckpointListener(cpLsnr); + + for (BackupContext ctx : backupCtxs.values()) + closeBackupResources(ctx); + + partWriters.clear(); + backupRunner.shutdown(); + } + + /** + * @param name Unique backup name. + * @param parts Collection of pairs group and appropratate cache partition to be backuped. + * @param dir Local directory to save cache partition deltas to. + * @return Future which will be completed when backup is done. + * @throws IgniteCheckedException If initialiation fails. + */ + public IgniteInternalFuture createLocalBackup( + String name, + Map> parts, + File dir + ) throws IgniteCheckedException { + if (backupCtxs.containsKey(name)) + throw new IgniteCheckedException("Backup with requested name is already scheduled: " + name); + + BackupContext bctx = null; + File backupDir = new File(dir, name); + + try { + // Atomic operation, fails with exception if not. + Files.createDirectory(backupDir.toPath()); + + bctx = new BackupContext(name, + backupDir, + parts, + backupRunner, + (from, to, partSize) -> + new PartitionCopySupplier(log, from, to, partSize), + (from, delta) -> + new PartitionDeltaSupplier(log, + ((FilePageStoreManager)cctx.pageStore()) + .getFilePageStoreFactory(), + from, + delta)); + + for (Map.Entry> e : parts.entrySet()) { + final CacheGroupContext gctx = cctx.cache().cacheGroup(e.getKey()); + + // Create cache backup directory if not. + File grpDir = U.resolveWorkDirectory(bctx.backupDir.getAbsolutePath(), + cacheDirName(gctx.config()), false); + + U.ensureDirectory(grpDir, + "temporary directory for cache group: " + gctx.groupId(), + null); + + CompletableFuture cpEndFut0 = bctx.cpEndFut; + + for (int partId : e.getValue()) { + final GroupPartitionId pair = new GroupPartitionId(e.getKey(), partId); + + bctx.partAllocLengths.put(pair, 0L); + + final PageStore pageStore = dbMgr.getPageStore(e.getKey(), partId); + + bctx.partDeltaWriters.put(pair, + new PageStoreSerialWriter( + new FileSerialPageStore(log, + () -> getPartionDeltaFile(grpDir, partId) + .toPath(), + ioFactory, + pageStore.getPageSize()), + () -> cpEndFut0.isDone() && !cpEndFut0.isCompletedExceptionally(), + pageStore.getPageSize())); + } + } + + BackupContext ctx0 = backupCtxs.putIfAbsent(name, bctx); + + assert ctx0 == null : ctx0; + + CheckpointFuture cpFut = dbMgr.forceCheckpoint(String.format(BACKUP_CP_REASON, name)); + + BackupContext finalBctx = bctx; + + cpFut.finishFuture() + .listen(f -> { + if (f.error() == null) + finalBctx.cpEndFut.complete(true); + else + finalBctx.cpEndFut.completeExceptionally(f.error()); + }); + + cpFut.beginFuture() + .get(); + + U.log(log, "Backup operation scheduled with the following context: " + bctx); + } + catch (IOException e) { + closeBackupResources(bctx); + + try { + Files.delete(backupDir.toPath()); + } + catch (IOException ioe) { + throw new IgniteCheckedException("Error deleting backup directory during context initialization " + + "failed: " + name, e); + } + + throw new IgniteCheckedException(e); + } + + return bctx.result; + } + + /** + * @param bctx Context to clouse all resources. + */ + private static void closeBackupResources(BackupContext bctx) { + if (bctx == null) + return; + + for (PageStoreSerialWriter writer : bctx.partDeltaWriters.values()) + U.closeQuiet(writer); + } + + /** + * @param bctx Context to handle. + */ + private void submitTasks(BackupContext bctx, File cacheWorkDir) { + List> futs = new ArrayList<>(bctx.partAllocLengths.size()); + + U.log(log, "Partition allocated lengths: " + bctx.partAllocLengths); + + for (Map.Entry e : bctx.partAllocLengths.entrySet()) { + GroupPartitionId pair = e.getKey(); + + CacheConfiguration ccfg = cctx.cache().cacheGroup(pair.getGroupId()).config(); + + CompletableFuture fut0 = CompletableFuture.supplyAsync( + bctx.partSuppFactory + .apply( + getPartitionFile( + cacheWorkDir(cacheWorkDir, ccfg), + pair.getPartitionId()), + new File(bctx.backupDir, + cacheDirName(ccfg)), + bctx.partAllocLengths.get(pair)), + bctx.execSvc) + .thenApply(file -> { + bctx.partDeltaWriters.get(pair).partProcessed = true; + + return file; + }) + .thenCombineAsync(bctx.cpEndFut, + (from, res) -> { + assert res; + + // Call the factory which creates tasks for page delta processing. + return bctx.deltaTaskFactory.apply(from, + bctx.partDeltaWriters + .get(pair) + .serial) + .get(); + }, + bctx.execSvc); + + futs.add(fut0); + } + + CompletableFuture.allOf(futs.toArray(new CompletableFuture[bctx.partAllocLengths.size()])) + .whenComplete(new BiConsumer() { + @Override public void accept(Void res, Throwable t) { + if (t == null) + bctx.result.onDone(); + else + bctx.result.onDone(t); + } + }); + } + + /** + * @param backupName Unique backup name. + */ + public void stopCacheBackup(String backupName) { + + } + + /** + * @param pairId Cache group, partition identifiers pair. + * @param pageId Tracked page id. + * @param buf Buffer with page data. + */ + public void beforeStoreWrite(GroupPartitionId pairId, long pageId, ByteBuffer buf, PageStore store) { + assert buf.position() == 0 : buf.position(); + assert buf.order() == ByteOrder.nativeOrder() : buf.order(); + + try { + List writers = partWriters.get(pairId); + + if (writers == null || writers.isEmpty()) + return; + + for (PageStoreSerialWriter writer : writers) { + if (writer.stopped()) + continue; + + writer.write(pageId, buf, store); + } + } + catch (Exception e) { + U.error(log, "An error occured in the process of page backup " + + "[pairId=" + pairId + ", pageId=" + pageId + ']'); + + pageTrackErrors.putIfAbsent(pairId, + new IgniteCheckedException("Partition backup processing error [pageId=" + pageId + ']', e)); + } + } + + /** + * + */ + private static class PartitionDeltaSupplier implements Supplier { + /** Ignite logger to use. */ + private final IgniteLogger log; + + /** File page store factory */ + private final FilePageStoreFactory factory; + + /** Copied partition file to apply delta pages to. */ + private final File from; + + /** Delta pages storage for the given partition. */ + private final FileSerialPageStore serial; + + /** + * @param serial Storage with delta pages. + */ + public PartitionDeltaSupplier( + IgniteLogger log, + FilePageStoreFactory factory, + File from, + FileSerialPageStore serial + ) { + this.log = log.getLogger(PartitionDeltaSupplier.class); + this.factory = factory; + this.from = from; + this.serial = serial; + } + + /** {@inheritDoc} */ + @Override public File get() { + try { + FilePageStore store = (FilePageStore)factory.createPageStore(FLAG_DATA, + from::toPath, + new LongAdderMetric("NO_OP", null)); + + store.doRecover(serial); + + U.log(log, "Partition delta storage applied to: " + from.getName()); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + + return from; + } + } + + /** + * + */ + private static class PartitionCopySupplier implements Supplier { + /** Ignite logger to use. */ + private final IgniteLogger log; + + /** Partition file. */ + private final File from; + + /** Destination copy file to copy partition to. */ + private final File to; + + /** Size of partition. */ + private final long partSize; + + /** + * @param log Ignite logger to use. + * @param from Partition file. + * @param dir Destination copy file. + * @param partSize Size of partition. + */ + public PartitionCopySupplier( + IgniteLogger log, + File from, + File dir, + long partSize + ) { + A.ensure(dir.isDirectory(), "Destination path must be a directory"); + + this.log = log.getLogger(PartitionCopySupplier.class); + this.from = from; + this.partSize = partSize; + to = new File(dir, from.getName()); + } + + /** {@inheritDoc} */ + @Override public File get() { + try { + if (!to.exists() || to.delete()) + to.createNewFile(); + + if (partSize == 0) + return to; + + try (FileChannel src = new FileInputStream(from).getChannel(); + FileChannel dest = new FileOutputStream(to).getChannel()) { + src.position(0); + + long written = 0; + + while (written < partSize) + written += src.transferTo(written, partSize - written, dest); + } + + U.log(log, "Partition file has been copied [from=" + from.getAbsolutePath() + + ", fromSize=" + from.length() + ", to=" + to.getAbsolutePath() + ']'); + } + catch (IOException ex) { + throw new IgniteException(ex); + } + + return to; + } + } + + /** + * + */ + private static class PageStoreSerialWriter implements Closeable { + /** Storage to write pages to. */ + private final FileSerialPageStore serial; + + /** Local buffer to perpform copy-on-write operations. */ + private final ThreadLocal localBuff; + + /** {@code true} if need the original page from PageStore instead of given buffer. */ + private final BooleanSupplier checkpointComplete; + + /** {@code true} if current writer is stopped. */ + private volatile boolean partProcessed; + + /** + * Expected file length in bytes at the moment of checkpoind end. + * Size is collected under checkpoint write lock (#onMarkCheckpointBegin). + */ + private long expectedSize; + + /** Array of bits. 1 - means pages written, 0 - the otherwise. */ + private volatile AtomicIntegerArray pagesWrittenBits; + + /** + * @param serial Serial storage to write to. + * @param checkpointComplete Checkpoint finish flag. + * @param pageSize Size of page to use for local buffer. + */ + public PageStoreSerialWriter( + FileSerialPageStore serial, + BooleanSupplier checkpointComplete, + int pageSize + ) throws IOException { + this.serial = serial; + this.checkpointComplete = checkpointComplete; + + localBuff = ThreadLocal.withInitial(() -> + ByteBuffer.allocateDirect(pageSize).order(ByteOrder.nativeOrder())); + + serial.init(); + } + + /** + * @return {@code true} if writer is stopped and cannot write pages. + */ + public boolean stopped() { + return checkpointComplete.getAsBoolean() && partProcessed; + } + + /** + * @param pageId Page id to write. + * @param buf Page buffer. + * @param store Storage to write to. + */ + public void write(long pageId, ByteBuffer buf, PageStore store) throws IOException, IgniteCheckedException { + if (stopped()) + return; + + if (checkpointComplete.getAsBoolean()) { + assert expectedSize > 0; + + if (!pagesWrittenBits.compareAndSet(PageIdUtils.pageIndex(pageId), 0, 1)) + return; + + final ByteBuffer locBuf = localBuff.get(); + + assert locBuf.capacity() == store.getPageSize(); + + locBuf.clear(); + + if (store.readPage(pageId, locBuf, true) < 0) + return; + + locBuf.flip(); + + serial.writePage(pageId, locBuf); + } + else { + // Direct buffre is needs to be written, associated checkpoint not finished yet. + serial.writePage(pageId, buf); + + buf.rewind(); + } + } + + /** {@inheritDoc} */ + @Override public void close() { + U.closeQuiet(serial); + } + } + + /** + * + */ + private static class BackupContext { + /** Unique identifier of backup process. */ + private final String name; + + /** Absolute backup storage path. */ + private final File backupDir; + + /** Service to perform partitions copy. */ + private final ExecutorService execSvc; + + /** + * The length of file size per each cache partiton file. + * Partition has value greater than zero only for partitons in OWNING state. + * Information collected under checkpoint write lock. + */ + private final Map partAllocLengths = new HashMap<>(); + + /** Map of partitions to backup and theirs corresponding delta PageStores. */ + private final Map partDeltaWriters = new HashMap<>(); + + /** Future of result completion. */ + @GridToStringExclude + private final GridFutureAdapter result = new GridFutureAdapter<>(); + + /** Factory to create executable tasks for partition processing. */ + @GridToStringExclude + private final IgniteTriClosure> partSuppFactory; + + /** Factory to create executable tasks for partition delta pages processing. */ + @GridToStringExclude + private final IgniteBiClosure> deltaTaskFactory; + + /** Collection of partition to be backuped. */ + private final Map> parts; + + /** Checkpoint end future. */ + private final CompletableFuture cpEndFut = new CompletableFuture<>(); + + /** Flag idicates that this backup is start copying partitions. */ + private volatile boolean started; + + /** + * @param name Unique identifier of backup process. + * @param backupDir Backup storage directory. + * @param execSvc Service to perform partitions copy. + * @param partSuppFactory Factory to create executable tasks for partition processing. + */ + public BackupContext( + String name, + File backupDir, + Map> parts, + ExecutorService execSvc, + IgniteTriClosure> partSuppFactory, + IgniteBiClosure> deltaTaskFactory + ) { + A.notNull(name, "Backup name cannot be empty or null"); + A.notNull(backupDir, "You must secify correct backup directory"); + A.ensure(backupDir.isDirectory(), "Specified path is not a directory"); + A.notNull(execSvc, "Executor service must be not null"); + A.notNull(partSuppFactory, "Factory which procudes backup tasks to execute must be not null"); + A.notNull(deltaTaskFactory, "Factory which processes delta pages storage must be not null"); + + this.name = name; + this.backupDir = backupDir; + this.parts = parts; + this.execSvc = execSvc; + this.partSuppFactory = partSuppFactory; + this.deltaTaskFactory = deltaTaskFactory; + + result.listen(f -> { + if (f.error() != null) + closeBackupResources(this); + }); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + BackupContext ctx = (BackupContext)o; + + return name.equals(ctx.name); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return Objects.hash(name); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(BackupContext.class, this); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/backup/IgniteTriClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/backup/IgniteTriClosure.java new file mode 100644 index 0000000000000..1c032d11f1411 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/backup/IgniteTriClosure.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.backup; + +import java.io.Serializable; + +/** + * Defines generic closure with three parameters. Tri-Closure is a simple executable which + * accepts three parameters and returns a value. + * + * @param Type of the first parameter. + * @param Type of the second parameter. + * @param Type of the second parameter. + * @param Type of the closure's return value. + */ +@FunctionalInterface +public interface IgniteTriClosure extends Serializable { + /** + * Closure body. + * + * @param e1 First parameter. + * @param e2 Second parameter. + * @param e3 Third parameter. + * @return Closure return value. + */ + public R apply(E1 e1, E2 e2, E3 e3); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java index e1603ac4f8d37..476e6ca43e2d5 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java @@ -26,7 +26,6 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -34,6 +33,7 @@ import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.internal.pagemem.PageIdUtils; import org.apache.ignite.internal.pagemem.store.PageStore; +import org.apache.ignite.internal.pagemem.store.PageStoreListener; import org.apache.ignite.internal.processors.cache.persistence.StorageException; import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc; @@ -87,6 +87,9 @@ public class FilePageStore implements PageStore { /** Region metrics updater. */ private final LongAdderMetric allocatedTracker; + /** Page storage listener. */ + private volatile PageStoreListener lsnr = PageStoreListener.NO_OP; + /** */ protected final int pageSize; @@ -103,7 +106,7 @@ public class FilePageStore implements PageStore { private boolean skipCrc = IgniteSystemProperties.getBoolean(IGNITE_PDS_SKIP_CRC, false); /** */ - private final ReadWriteLock lock = new ReentrantReadWriteLock(); + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); /** */ public FilePageStore( @@ -122,6 +125,11 @@ public FilePageStore( this.allocatedTracker = allocatedTracker; } + /** {@inheritDoc} */ + @Override public void setListener(PageStoreListener lsnr) { + this.lsnr = lsnr; + } + /** {@inheritDoc} */ @Override public int getPageSize() { return pageSize; @@ -163,6 +171,8 @@ public FilePageStore( if (fileExists == null) { File file = pathProvider.apply().toFile(); +// System.out.println("file (exists="+file.exists()+"): " + file); + fileExists = file.exists() && file.length() > headerSize(); } } @@ -174,10 +184,8 @@ public FilePageStore( return fileExists; } - /** - * Size of page store header. - */ - public int headerSize() { + /** {@inheritDoc} */ + @Override public int headerSize() { return HEADER_SIZE; } @@ -343,6 +351,8 @@ private long checkFile(FileIO fileIO, File cfgFile) throws IOException { Path filePath = pathProvider.apply(); + U.dumpStack(">xxx> truncate " + filePath); + lock.writeLock().lock(); try { @@ -386,6 +396,22 @@ private long checkFile(FileIO fileIO, File cfgFile) throws IOException { @Override public void finishRecover() throws StorageException { lock.writeLock().lock(); + try { + updateAllocatedPages(); + + recover = false; + } + finally { + lock.writeLock().unlock(); + } + } + + /** + * @throws StorageException If fails. + */ + private void updateAllocatedPages() throws StorageException { + assert lock.isWriteLockedByCurrentThread(); + try { // Since we always have a meta-page in the store, never revert allocated counter to a value smaller than page. if (inited) { @@ -401,11 +427,38 @@ private long checkFile(FileIO fileIO, File cfgFile) throws IOException { allocatedTracker.add(delta / pageSize); } - - recover = false; } catch (IOException e) { - throw new StorageException("Failed to finish recover partition file [file=" + getFileAbsolutePath() + "]", e); + throw new StorageException("Failed to update partition file allocated pages " + + "[file=" + getFileAbsolutePath() + "]", e); + } + } + + /** + * @param serialStrg Serial page storage to reover current storage with. + * @throws IgniteCheckedException If fails. + */ + public void doRecover(FileSerialPageStore serialStrg) throws IgniteCheckedException { + lock.writeLock().lock(); + + try { + recover = true; + + ByteBuffer pageBuf = ByteBuffer.allocate(pageSize) + .order(ByteOrder.nativeOrder()); + long pages = serialStrg.pages(); + + for (int seq = 0; seq < pages; seq++) { + serialStrg.readPage(pageBuf, seq); + + write(PageIO.getPageId(pageBuf), pageBuf, 0, false); + + pageBuf.clear(); + } + + updateAllocatedPages(); + + recover = false; } finally { lock.writeLock().unlock(); @@ -433,7 +486,7 @@ private int getCrcSize(long pageId, ByteBuffer pageBuf) throws IOException { } /** {@inheritDoc} */ - @Override public void read(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteCheckedException { + @Override public int readPage(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteCheckedException { init(); try { @@ -453,7 +506,7 @@ private int getCrcSize(long pageId, ByteBuffer pageBuf) throws IOException { if (n < 0) { pageBuf.put(new byte[pageBuf.remaining()]); - return; + return n; } int savedCrc32 = PageIO.getCrc(pageBuf); @@ -478,6 +531,8 @@ private int getCrcSize(long pageId, ByteBuffer pageBuf) throws IOException { if (keepCrc) PageIO.setCrc(pageBuf, savedCrc32); + + return n; } catch (IOException e) { throw new StorageException("Failed to read page [file=" + getFileAbsolutePath() + ", pageId=" + pageId + "]", e); @@ -675,6 +730,10 @@ private void reinit(FileIO fileIO) throws IOException { assert pageBuf.position() == 0 : pageBuf.position(); + lsnr.onPageWrite(pageId, pageBuf); + + assert pageBuf.position() == 0 : pageBuf.position(); + fileIO.writeFully(pageBuf, off); PageIO.setCrc(pageBuf, 0); 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 83595ab7506f0..1e3462f011657 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 @@ -62,6 +62,7 @@ import org.apache.ignite.internal.pagemem.PageMemory; import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager; import org.apache.ignite.internal.pagemem.store.PageStore; +import org.apache.ignite.internal.pagemem.store.PageStoreListener; import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; import org.apache.ignite.internal.processors.cache.GridCacheContext; @@ -71,8 +72,10 @@ import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetricsImpl; import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cache.persistence.StorageException; +import org.apache.ignite.internal.processors.cache.persistence.backup.IgniteBackupManager; import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings; import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage; +import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager; import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; import org.apache.ignite.internal.processors.metric.impl.LongAdderMetric; @@ -159,6 +162,9 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen */ private FileIOFactory pageStoreFileIoFactory; + /** File page store factory. */ + private FileVersionCheckingFactory pageStoreFactory; + /** * File IO factory for page store V1 and for fast checking page store (non block read). * By default is taken from {@link #dsCfg}. @@ -692,7 +698,7 @@ private CacheStoreHolder initDir(File cacheWorkDir, cctx.gridConfig().getEncryptionSpi()); } - FileVersionCheckingFactory pageStoreFactory = new FileVersionCheckingFactory( + pageStoreFactory = new FileVersionCheckingFactory( pageStoreFileIoFactory, pageStoreV1FileIoFactory, igniteCfg.getDataStorageConfiguration() @@ -706,10 +712,13 @@ private CacheStoreHolder initDir(File cacheWorkDir, } PageStore idxStore = - pageStoreFactory.createPageStore( - PageMemory.FLAG_IDX, - idxFile, - allocatedTracker); + pageStoreFactory.createPageStore( + PageMemory.FLAG_IDX, + idxFile, + allocatedTracker); + + if (cctx.backup() != null) + idxStore.setListener(new BackupPageStoreListener(grpId, partitions, cctx.backup(), idxStore)); PageStore[] partStores = new PageStore[partitions]; @@ -722,8 +731,11 @@ private CacheStoreHolder initDir(File cacheWorkDir, () -> getPartitionFilePath(cacheWorkDir, p), allocatedTracker); - partStores[partId] = partStore; - } + if (cctx.backup() != null) + partStore.setListener(new BackupPageStoreListener(grpId, partId, cctx.backup(), partStore)); + + partStores[partId] = partStore; + } return new CacheStoreHolder(idxStore, partStores); } @@ -743,6 +755,14 @@ private CacheStoreHolder initDir(File cacheWorkDir, return new File(cacheWorkDir, String.format(PART_FILE_TEMPLATE, partId)).toPath(); } + /** + * @param cacheWorkDir Cache work directory. + * @param partId Partition id. + */ + @NotNull public static File getPartitionFile(File cacheWorkDir, int partId) { + return new File(cacheWorkDir, format(PART_FILE_TEMPLATE, partId)); + } + /** {@inheritDoc} */ @Override public boolean checkAndInitCacheWorkDir(CacheConfiguration cacheCfg) throws IgniteCheckedException { return checkAndInitCacheWorkDir(cacheWorkDir(cacheCfg)); @@ -999,23 +1019,52 @@ public File workDir() { * @return Store dir for given cache. */ public File cacheWorkDir(CacheConfiguration ccfg) { - boolean isSharedGrp = ccfg.getGroupName() != null; - - return cacheWorkDir(isSharedGrp, isSharedGrp ? ccfg.getGroupName() : ccfg.getName()); + return cacheWorkDir(storeWorkDir, ccfg); } /** * */ public File cacheWorkDir(boolean isSharedGroup, String cacheOrGroupName) { - String dirName; + return cacheWorkDir(storeWorkDir, isSharedGroup, cacheOrGroupName); + } - if (isSharedGroup) - dirName = CACHE_GRP_DIR_PREFIX + cacheOrGroupName; - else - dirName = CACHE_DIR_PREFIX + cacheOrGroupName; + /** + * @param storeWorkDir Configured file page store base directory. + * @param isSharedGroup {@code True} if cache is sharing the same `underlying` cache. + * @param cacheOrGroupName Cache name. + * @return Cache directory. + */ + public static File cacheWorkDir(File storeWorkDir, boolean isSharedGroup, String cacheOrGroupName) { + return new File(storeWorkDir, cacheDirName(isSharedGroup, cacheOrGroupName)); + } + + /** + * @param ccfg Cache configuration. + * @return Store directory for given cache. + */ + public static File cacheWorkDir(File storeWorkDir, CacheConfiguration ccfg) { + return new File(storeWorkDir, cacheDirName(ccfg)); + } + + /** + * @param isSharedGroup {@code True} if cache is sharing the same `underlying` cache. + * @param cacheOrGroupName Cache name. + * @return The full cache directory name. + */ + public static String cacheDirName(boolean isSharedGroup, String cacheOrGroupName) { + return isSharedGroup ? CACHE_GRP_DIR_PREFIX + cacheOrGroupName + : CACHE_DIR_PREFIX + cacheOrGroupName; + } + + /** + * @param ccfg Cache configuration. + * @return The full cache directory name. + */ + public static String cacheDirName(CacheConfiguration ccfg) { + boolean isSharedGrp = ccfg.getGroupName() != null; - return new File(storeWorkDir, dirName); + return cacheDirName(isSharedGrp, isSharedGrp ? ccfg.getGroupName() : ccfg.getName()); } /** @@ -1222,6 +1271,13 @@ public FileIOFactory getPageStoreFileIoFactory() { return pageStoreFileIoFactory; } + /** + * @return File page store factory currently used. + */ + public FilePageStoreFactory getFilePageStoreFactory() { + return pageStoreFactory; + } + /** * @return Durable memory page size in bytes. */ @@ -1430,4 +1486,40 @@ private static class IdxCacheStores extends ConcurrentHashMap { return longOperationAsyncExecutor.afterAsyncCompletion(() -> super.merge(key, val, remappingFunction)); } } + + /** */ + private static class BackupPageStoreListener implements PageStoreListener { + /** Pair of group id and its partiton id. */ + private final GroupPartitionId key; + + /** Backup manager. */ + private final IgniteBackupManager backup; + + /** Page store the listener associated with. */ + private final PageStore store; + + /** + * @param grpId Cache group id. + * @param partId Partition id. + * @param backup Backup manager. + * @param store Page store the listener associated with. + */ + public BackupPageStoreListener( + int grpId, + int partId, + IgniteBackupManager backup, + PageStore store + ) { + assert backup != null; + + key = new GroupPartitionId(grpId, partId); + this.backup = backup; + this.store = store; + } + + /** {@inheritDoc} */ + @Override public void onPageWrite(long pageId, ByteBuffer buf) { + backup.beforeStoreWrite(key, pageId, buf, store); + } + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileSerialPageStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileSerialPageStore.java new file mode 100644 index 0000000000000..3dd6c98fdb3a4 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileSerialPageStore.java @@ -0,0 +1,205 @@ +/* + * 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.file; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.file.Path; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Supplier; +import java.util.zip.CRC32; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.pagemem.PageIdUtils; +import org.apache.ignite.internal.pagemem.PageMemory; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; +import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.internal.A; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; + +/** + * + */ +public class FileSerialPageStore implements Closeable { + /** Ignite logger to use. */ + @GridToStringExclude + private final IgniteLogger log; + + /** Configuration file path provider. */ + private final Supplier cfgPath; + + /** Factory to produce an IO interface over underlying file. */ + @GridToStringExclude + private final FileIOFactory factory; + + /** Storage size. */ + private final AtomicLong pages = new AtomicLong(); + + /** Page size of stored pages. */ + private final int pageSize; + + /** Buse lock to perform write opertions. */ + private final ReadWriteLock lock = new ReentrantReadWriteLock(); + + /** IO over the underlying file */ + private volatile FileIO fileIo; + + /** + * @param log Ignite logger to use. + * @param cfgPath Configuration file path provider. + * @param factory Factory to produce an IO interface over underlying file. + * @param pageSize Page size of stored pages. + */ + public FileSerialPageStore(IgniteLogger log, Supplier cfgPath, FileIOFactory factory, int pageSize) { + A.notNull(cfgPath, "Configurations path cannot be empty"); + A.notNull(factory, "File configuration factory cannot be empty"); + + this.log = log.getLogger(FileSerialPageStore.class); + this.cfgPath = cfgPath; + this.factory = factory; + this.pageSize = pageSize; + } + + /** + * @throws IOException If failed to initialize store file. + */ + public FileSerialPageStore init() throws IOException { + if (fileIo == null) + fileIo = factory.create(cfgPath.get().toFile()); + + return this; + } + + /** + * @param pageId Page ID. + * @param pageBuf Page buffer to write. + * @throws IOException If page writing failed (IO error occurred). + */ + public void writePage(long pageId, ByteBuffer pageBuf) throws IOException { + assert fileIo != null : "Delta pages storage is not inited: " + this; + + if (!lock.readLock().tryLock()) + return; + + try { + assert pageBuf.position() == 0; + assert pageBuf.order() == ByteOrder.nativeOrder() : "Page buffer order " + pageBuf.order() + + " should be same with " + ByteOrder.nativeOrder(); + assert PageIdUtils.flag(pageId) == PageMemory.FLAG_DATA; + + int crc = PageIO.getCrc(pageBuf); + int crc32 = FastCrc.calcCrc(new CRC32(), pageBuf, pageBuf.limit()); + + // TODO remove debug + if (log.isTraceEnabled()) { + log.trace("onPageWrite [pageId=" + pageId + + ", pageIdBuff=" + PageIO.getPageId(pageBuf) + + ", part=" + cfgPath.get().toAbsolutePath() + + ", fileSize=" + fileIo.size() + + ", crcBuff=" + crc32 + + ", crcPage=" + crc + ']'); + } + + pageBuf.rewind(); + + // Write buffer to the end of the file. + fileIo.writeFully(pageBuf); + + pages.incrementAndGet(); + } + finally { + lock.readLock().unlock(); + } + } + + /** + * @param pageBuf Buffer to read page into. + * @param seq Page sequence in serial storage. + * @throws IgniteCheckedException If fails. + */ + public void readPage(ByteBuffer pageBuf, long seq) throws IgniteCheckedException { + assert fileIo != null : cfgPath.get(); + assert pageBuf.capacity() == pageSize : pageBuf.capacity(); + assert pageBuf.order() == ByteOrder.nativeOrder() : pageBuf.order(); + assert pageBuf.position() == 0 : pageBuf.position(); + + lock.readLock().lock(); + + try { + long readed = fileIo.readFully(pageBuf, seq * pageSize); + + assert readed == pageBuf.capacity(); + + pageBuf.flip(); + + long pageId = PageIO.getPageId(pageBuf); + int crc32 = FastCrc.calcCrc(new CRC32(), pageBuf, pageBuf.limit()); + int crc = PageIO.getCrc(pageBuf); + + if (log.isTraceEnabled()) { + log.trace("Read page from serial storage [path=" + cfgPath.get().toFile().getName() + + ", pageId=" + pageId + ", seq=" + seq + ", pages=" + pages.get() + ", crcBuff=" + crc32 + + ", crcPage=" + crc + ']'); + } + + pageBuf.rewind(); + } + catch (IOException e) { + throw new IgniteCheckedException("Error reading page from serial storage [seq=" + seq + ']'); + } + finally { + lock.readLock().unlock(); + } + } + + /** + * @return Total number of pages for this serial page storage. + */ + public long pages() { + return pages.get(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(FileSerialPageStore.class, this); + } + + /** {@inheritDoc} */ + @Override public void close() throws IOException { + if (fileIo == null) + return; + + lock.writeLock().lock(); + + try { + U.closeQuiet(fileIo); + } + finally { + fileIo = null; + + lock.writeLock().unlock(); + } + + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java index fcc6efec2088a..d82257a3cfb25 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java @@ -836,8 +836,11 @@ else if (relPtr == OUTDATED_REL_PTR) { rwLock.init(absPtr + PAGE_LOCK_OFFSET, PageIdUtils.tag(pageId)); } - else + else { + System.out.println("absolute ptr"); + absPtr = seg.absolute(relPtr); + } seg.acquirePage(absPtr); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CompoundSnapshotOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CompoundSnapshotOperation.java new file mode 100644 index 0000000000000..b1e86d8f81e78 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CompoundSnapshotOperation.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.ignite.internal.processors.cache.persistence.snapshot; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** */ +public class CompoundSnapshotOperation implements SnapshotOperation { + /** */ + private static final long serialVersionUID = 0L; + + /** + * The list of operations. Taking snapshot operation by {@link IgniteCacheSnapshotManager} + * will be always placed as the head of the list. + */ + private final List ops = new ArrayList<>(); + + /** {@inheritDoc} */ + @Override public Set cacheGroupIds() { + return ops.stream() + .map(SnapshotOperation::cacheGroupIds) + .collect(HashSet::new, Set::addAll, Set::addAll); + } + + /** {@inheritDoc} */ + @Override public Set cacheNames() { + return ops.stream() + .map(SnapshotOperation::cacheNames) + .collect(HashSet::new, Set::addAll, Set::addAll); + } + + /** {@inheritDoc} */ + @Override public Object extraParameter() { + return ops.get(0).extraParameter(); + } + + /** + * @param op Snapshot operation to add. + * @param top {@code True} to add operation to the head of the list. + */ + public void addSnapshotOperation(SnapshotOperation op, boolean top) { + if (top) + ops.add(0, op); // Other elements will be shifted to the right. + else + ops.add(op); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationAdapter.java new file mode 100644 index 0000000000000..26e54f0c5a1ce --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationAdapter.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.ignite.internal.processors.cache.persistence.snapshot; + +import java.util.HashSet; +import java.util.Set; + +/** + * A convinient adapter for default snapshot operation. + */ +public class SnapshotOperationAdapter implements SnapshotOperation { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public Set cacheGroupIds() { + return new HashSet<>(); + } + + /** {@inheritDoc} */ + @Override public Set cacheNames() { + return new HashSet<>(); + } + + /** {@inheritDoc} */ + @Override public Object extraParameter() { + return new Object(); + } +} 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 d3379446bc404..22428f392c926 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 @@ -2827,8 +2827,8 @@ private void init() throws IgniteCheckedException { try { fd = segmentRouter.findSegment(curWalSegmIdx); - if (log.isDebugEnabled()) - log.debug("Reading next file [absIdx=" + curWalSegmIdx + ", file=" + fd.file.getAbsolutePath() + ']'); + if (log.isInfoEnabled()) + log.info("Reading next file [absIdx=" + curWalSegmIdx + ", file=" + fd.file.getAbsolutePath() + ']'); nextHandle = initReadHandle(fd, start != null && curWalSegmIdx == start.index() ? start : null); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/crc/FastCrc.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/crc/FastCrc.java index 7cbaadb8ed773..44dfd9434e26f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/crc/FastCrc.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/crc/FastCrc.java @@ -112,7 +112,7 @@ public static int calcCrc(File file) throws IOException { * * @return Crc checksum. */ - private static int calcCrc(CRC32 crcAlgo, ByteBuffer buf, int len) { + public static int calcCrc(CRC32 crcAlgo, ByteBuffer buf, int len) { int initLimit = buf.limit(); buf.limit(buf.position() + len); 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 da276909be86a..ab5f26dd8b9e1 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 @@ -377,9 +377,9 @@ private FileDescriptor readFileDescriptor(File file, FileIOFactory ioFactory) { return new GridCacheSharedContext<>( kernalCtx, null, null, null, - null, null, null, dbMgr, 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/cache/preload/FileMetaInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/FileMetaInfo.java new file mode 100644 index 0000000000000..5f6291b93509d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/FileMetaInfo.java @@ -0,0 +1,19 @@ +package org.apache.ignite.internal.processors.cache.preload; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +public interface FileMetaInfo { + /** + * @param is The stream to read file meta info from. + * @throws IOException If fails. + */ + public void readMetaInfo(DataInputStream is) throws IOException; + + /** + * @param os The stream to write file meta info at. + * @throws IOException If fails. + */ + public void writeMetaInfo(DataOutputStream os) throws IOException; +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/GridPartitionBatchDemandMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/GridPartitionBatchDemandMessage.java new file mode 100644 index 0000000000000..acc815c945b57 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/GridPartitionBatchDemandMessage.java @@ -0,0 +1,178 @@ +package org.apache.ignite.internal.processors.cache.preload; + +import java.io.Externalizable; +import java.nio.ByteBuffer; +import java.util.Map; +import org.apache.ignite.internal.GridDirectMap; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.util.GridIntList; +import org.apache.ignite.internal.util.GridLongList; +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.MessageCollectionItemType; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; + +public class GridPartitionBatchDemandMessage implements Message { + /** */ + public static final short TYPE_CODE = 176; + + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private long rebId; + + /** */ + private AffinityTopologyVersion topVer; + + /** */ + @GridDirectMap(keyType = Integer.class, valueType = GridLongList.class) + private Map assigns; + + /** + * Empty constructor required for {@link Externalizable}. + */ + public GridPartitionBatchDemandMessage() { + // No-op. + } + + /** + * @param rebId Rebalance id for this node. + * @param topVer Topology version. + */ + public GridPartitionBatchDemandMessage( + long rebId, + AffinityTopologyVersion topVer, + Map assigns0 + ) { + assert assigns0 != null && !assigns0.isEmpty(); + + this.rebId = rebId; + this.topVer = topVer; + + assigns = U.newHashMap(assigns0.size()); + + for (Map.Entry e : assigns0.entrySet()) + assigns.put(e.getKey(), e.getValue().copy()); + } + + /** + * @param rebId Rebalance identifier to set. + */ + public void rebalanceId(long rebId) { + this.rebId = rebId; + } + + /** + * @return Unique rebalance session id. + */ + public long rebalanceId() { + return rebId; + } + + /** + * @return Topology version for which demand message is sent. + */ + public AffinityTopologyVersion topologyVersion() { + return topVer; + } + + /** + * @return The demanded cache group partions per each cache group. + */ + public Map assignments() { + return assigns; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeLong("rebId", rebId)) + return false; + + writer.incrementState(); + + case 1: + if (!writer.writeAffinityTopologyVersion("topVer", topVer)) + return false; + + writer.incrementState(); + + case 2: + if (!writer.writeMap("assigns", assigns, MessageCollectionItemType.INT, MessageCollectionItemType.MSG)) + return false; + + writer.incrementState(); + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + switch (reader.state()) { + case 0: + rebId = reader.readLong("rebId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 1: + topVer = reader.readAffinityTopologyVersion("topVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 2: + assigns = reader.readMap("assigns", MessageCollectionItemType.INT, MessageCollectionItemType.MSG, false); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + } + + return reader.afterMessageRead(GridPartitionBatchDemandMessage.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return TYPE_CODE; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 3; + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridPartitionBatchDemandMessage.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/IgniteBackupPageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/IgniteBackupPageStoreManager.java new file mode 100644 index 0000000000000..6378ce6c7ace6 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/IgniteBackupPageStoreManager.java @@ -0,0 +1,41 @@ +package org.apache.ignite.internal.processors.cache.preload; + +import java.util.Map; +import java.util.Set; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.store.PageStore; +import org.apache.ignite.internal.processors.cache.GridCacheSharedManager; +import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId; +import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport; + +public interface IgniteBackupPageStoreManager extends GridCacheSharedManager, IgniteChangeGlobalStateSupport { + /** + * Take backup of specified cache group partition files and syncronously wait to its completion. + * + * @param idx Unique process identifier. + * @param grpsBackup Backing up cache groups and corresponding partitions. + * @param hndlr Handler for processing partitions and corresponding partition deltas. + * @param fut A future of process flow control. + * @throws IgniteCheckedException If fails. + */ + public void backup( + long idx, + Map> grpsBackup, + //BackupProcessSupplier hndlr, + IgniteInternalFuture fut + ) throws IgniteCheckedException; + + /** + * @param pairId Cache group, partition identifiers pair. + * @param store Store to handle operatwion at. + * @param pageId Tracked page id. + */ + public void handleWritePageStore(GroupPartitionId pairId, PageStore store, long pageId); + + /** + * @param grpPartIdSet Collection of pairs cache group and partition ids. + * @throws IgniteCheckedException If fails. + */ + public void initTemporaryStores(Set grpPartIdSet) throws IgniteCheckedException; +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/PartitionFileMetaInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/PartitionFileMetaInfo.java new file mode 100644 index 0000000000000..d0c52f41db0f6 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/PartitionFileMetaInfo.java @@ -0,0 +1,106 @@ +package org.apache.ignite.internal.processors.cache.preload; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import org.apache.ignite.internal.util.typedef.internal.S; + +public class PartitionFileMetaInfo implements FileMetaInfo { + /** */ + private Integer grpId; + + /** */ + private Integer partId; + + /** */ + private String name; + + /** */ + private Long size; + + /** */ + private Integer type; + + /** */ + public PartitionFileMetaInfo() { + this(null, null, null, null, null); + } + + /** + * @param grpId Cache group identifier. + * @param name Cache partition file name. + * @param size Cache partition file size. + * @param type {@code 0} partition file, {@code 1} delta file. + */ + public PartitionFileMetaInfo(Integer grpId, Integer partId, String name, Long size, Integer type) { + this.grpId = grpId; + this.partId = partId; + this.name = name; + this.size = size; + this.type = type; + } + + /** + * @return Related cache group id. + */ + public Integer getGrpId() { + return grpId; + } + + /** + * @return Cache partition id. + */ + public Integer getPartId() { + return partId; + } + + /** + * @return Partition file name. + */ + public String getName() { + return name; + } + + /** + * @return Partition file size. + */ + public Long getSize() { + return size; + } + + /** + * @return {@code 0} partition file, {@code 1} delta file. + */ + public Integer getType() { + return type; + } + + /** {@inheritDoc} */ + @Override public void readMetaInfo(DataInputStream is) throws IOException { + grpId = is.readInt(); + partId = is.readInt(); + name = is.readUTF(); + size = is.readLong(); + type = is.readInt(); + + if (grpId == null || partId == null || name == null || size == null || type == null) + throw new IOException("File meta information incorrect: " + this); + } + + /** {@inheritDoc} */ + @Override public void writeMetaInfo(DataOutputStream os) throws IOException { + if (grpId == null || partId == null || name == null || size == null || type == null) + throw new IOException("File meta information incorrect: " + this); + + os.writeInt(grpId); + os.writeInt(partId); + os.writeUTF(name); + os.writeLong(size); + os.writeInt(type); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(PartitionFileMetaInfo.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/PartitionSwitchModeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/PartitionSwitchModeManager.java new file mode 100644 index 0000000000000..413c3249bf001 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/PartitionSwitchModeManager.java @@ -0,0 +1,132 @@ +package org.apache.ignite.internal.processors.cache.preload; + +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +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.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.typedef.internal.U; + +public class PartitionSwitchModeManager implements DbCheckpointListener { + /** */ + private final IgniteLogger log; + + /** */ + private final GridCacheSharedContext cctx; + + /** */ + private final ConcurrentLinkedQueue switchReqs = new ConcurrentLinkedQueue<>(); + + /** + * @param cctx Shared context. + */ + public PartitionSwitchModeManager(GridCacheSharedContext cctx) { + this.cctx = cctx; + this.log = cctx.logger(PartitionSwitchModeManager.class); + } + +// /** +// * @param p The condition to check. +// * @return The number of pending switch request satisfyed by given condition. +// */ +// public int pendingRequests(Predicate p) { +// int cnt = 0; +// +// for (SwitchModeRequest rq : switchReqs) { +// if (p.test(rq.nextMode)) +// cnt++; +// } +// +// return cnt; +// } + + /** {@inheritDoc} */ + @Override public void onMarkCheckpointBegin(Context ctx) { + SwitchModeRequest rq; + + while ((rq = switchReqs.poll()) != null) { + for (Map.Entry> e : rq.parts.entrySet()) { + CacheGroupContext grp = cctx.cache().cacheGroup(e.getKey()); + + for (Integer partId : e.getValue()) { + GridDhtLocalPartition locPart = grp.topology().localPartition(partId); + + if (locPart.readOnly() == rq.nextReadOnly) + continue; + + //TODO invalidate partition + +// IgniteCacheOffheapManager.CacheDataStore currStore = locPart.dataStore(locPart.readOnlyMode()); + + // Pre-init the new storage. +// locPart.dataStore(rq.nextReadOnly).init(currStore.updateCounter()); + + // Switching mode under the write lock. + locPart.readOnly(rq.nextReadOnly); + } + } + + rq.rqFut.onDone(); + } + } + + /** {@inheritDoc} */ + @Override public void onCheckpointBegin(Context ctx) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void beforeCheckpointBegin(Context ctx) { + // No-op. + } + + /** + * @param readOnly The storage mode to switch to. + * @param parts The set of partitions to change storage mode. + * @return The future which will be completed when request is done. + */ + public GridFutureAdapter offerSwitchRequest( + boolean readOnly, + Map> parts + ) { + SwitchModeRequest req = new SwitchModeRequest(readOnly, parts); + + boolean offered = switchReqs.offer(req); + + assert offered; + + U.log(log, "Change partition mode request registered [mode=" + readOnly + ", parts=" + parts + ']'); + + return req.rqFut; + } + + /** + * + */ + private static class SwitchModeRequest { + /** The storage mode to switch to. */ + private final boolean nextReadOnly; + + /** The map of cache groups and corresponding partition to switch mode to. */ + private final Map> parts; + + /** The future will be completed when the request has been processed. */ + private final GridFutureAdapter rqFut = new GridFutureAdapter<>(); + + /** + * @param nextReadOnly The mode to set to. + * @param parts The partitions to switch mode to. + */ + public SwitchModeRequest( + boolean nextReadOnly, + Map> parts + ) { + this.nextReadOnly = nextReadOnly; + this.parts = parts; + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/PartitionUploadManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/PartitionUploadManager.java new file mode 100644 index 0000000000000..b6cc8c71750e8 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/preload/PartitionUploadManager.java @@ -0,0 +1,427 @@ +package org.apache.ignite.internal.processors.cache.preload; + +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.managers.communication.GridIoManager; +import org.apache.ignite.internal.managers.communication.TransmissionPolicy; +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.distributed.dht.topology.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; +import org.apache.ignite.internal.util.GridIntIterator; +import org.apache.ignite.internal.util.GridIntList; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +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.S; +import org.apache.ignite.internal.util.typedef.internal.U; + +import static org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridCachePreloadSharedManager.rebalanceThreadTopic; + +/** + * + */ +public class PartitionUploadManager { + /** */ + private GridCacheSharedContext cctx; + + /** */ + private IgniteLogger log; + + /** */ + private final ConcurrentMap uploadFutMap = new ConcurrentHashMap<>(); + + /** */ + private final ReadWriteLock lock = new ReentrantReadWriteLock(); + + /** */ +// private IgniteBackupPageStoreManager backupMgr; + + /** + * @param ktx Kernal context to process. + */ + public PartitionUploadManager(GridKernalContext ktx) { + assert CU.isPersistenceEnabled(ktx.config()); + + cctx = ktx.cache().context(); + + log = ktx.log(getClass()); + } + +// /** +// * @return {@code True} if cluster rebalance via sending partition files can be applied. +// */ +// public static boolean persistenceRebalanceApplicable(GridCacheSharedContext cctx) { +// return !cctx.kernalContext().clientNode() && +// CU.isPersistenceEnabled(cctx.kernalContext().config()) && +// cctx.isRebalanceEnabled(); +// } + + /** + * @param cctx Cache shared context. + */ + public void start0(GridCacheSharedContext cctx) throws IgniteCheckedException { + this.cctx = cctx; + + //backupMgr = cctx.storeBackup(); + +// if (persistenceRebalanceApplicable(cctx)) { +// cctx.gridIO().addMessageListener(rebalanceThreadTopic(), new GridMessageListener() { +// @Override public void onMessage(UUID nodeId, Object msg, byte plc) { +// if (msg instanceof GridPartitionBatchDemandMessage) { +// // Start to checkpoint and upload process. +// lock.readLock().lock(); +// +// try { +// onDemandMessage0(nodeId, (GridPartitionBatchDemandMessage)msg, plc); +// } +// finally { +// lock.readLock().unlock(); +// } +// } +// } +// }); +// } + } + + /** + * @param cancel true to cancel all pending tasks. + */ + public void stop0(boolean cancel) { + lock.writeLock().lock(); + + try { + cctx.gridIO().removeMessageListener(rebalanceThreadTopic()); + + for (CachePartitionUploadFuture fut : uploadFutMap.values()) + fut.cancel(); + } + finally { + lock.writeLock().unlock(); + } + } + + /** + * This internal method will handle demand requests of copying cache group partitions to the remote node. + * It will perform checkpoint operation to take the latest partitions changes for list of demaned cache + * groups and upload them one by one. + * + * @param nodeId The nodeId request comes from. + * @param msg Message containing rebalance request params. + */ + public void onDemandMessage(UUID nodeId, GridPartitionBatchDemandMessage msg, byte plc) { +// todo +// IgniteSocketChannel ch = null; +// CachePartitionUploadFuture uploadFut = null; +// + CachePartitionUploadFuture uploadFut = null; + + log.info("Processing demand message from " + nodeId); + + try { + // todo compute if absent? + synchronized (uploadFutMap) { + // todo why we need this global mapping + uploadFut = uploadFutMap.getOrDefault(nodeId, + new CachePartitionUploadFuture(msg.rebalanceId(), msg.topologyVersion(), msg.assignments())); + + if (uploadFut.rebalanceId < msg.rebalanceId()) { + if (!uploadFut.isDone()) { + log.info("Restarting upload routine [node=" + nodeId + ", old=" + uploadFut.rebalanceId + ", new=" + msg.rebalanceId()); + + uploadFut.cancel(); + } + + uploadFutMap.put(nodeId, + uploadFut = new CachePartitionUploadFuture(msg.rebalanceId(), + msg.topologyVersion(), + msg.assignments())); + } + } + + // Need to start new partition upload routine. +// ch = cctx.gridIO().channelToTopic(nodeId, rebalanceThreadTopic(), plc); + + // History should be reserved on exchange done. + +// for (Map.Entry> e : uploadFut.getAssigns().entrySet()) { +// int grpId = e.getKey(); +// +// CacheGroupContext grp = cctx.cache().cacheGroup(grpId); +// +// // todo handle exceptions somehow +// // todo should we reserve partition when sending +//// for (int partId : e.getValue()) { +//// GridDhtLocalPartition part = grp.topology().localPartition(partId); +//// +//// boolean reserved = part.reserve(); +//// +//// assert reserved : part.id(); +//// +////// long updateCntr = part.updateCounter(); +//// +////// boolean histReserved = cctx.database().reserveHistoryForPreloading(grpId, partId, updateCntr); +////// +////// assert histReserved : part.id(); +////// +////// if (log.isDebugEnabled()) +////// log.debug("Reserved history for preloading [grp=" + grp.cacheOrGroupName() + ", part=" + partId + ", cntr=" + updateCntr); +//// } +// } + + // todo - exec trnasmission on supplier thread! + // History should be reserved on exchange done. + sendPartitions(uploadFut, nodeId).get(); + +// backupMgr.backup(uploadFut.rebalanceId, +// uploadFut.getAssigns(), +// new SocketBackupProcessSupplier( +// new FileTransferManager<>(cctx.kernalContext(), ch.channel(), uploadFut), +// log +// ), +// uploadFut); + } + catch (Exception e) { + U.error(log, "An error occured while processing initial demand request [" + + ", nodeId=" + nodeId + ", topVer=" + msg.topologyVersion() + ']', e); + + if (uploadFut != null) + uploadFut.onDone(e); + } +// finally { +// U.closeQuiet(ch); +// } + } + + /** + * @param fut Future. + * @param nodeId Node id. + */ + private IgniteInternalFuture sendPartitions(CachePartitionUploadFuture fut, UUID nodeId) throws IgniteCheckedException { + File tmpDir = new File(IgniteSystemProperties.getString("java.io.tmpdir")); + + assert tmpDir.exists() : tmpDir; + + if (log.isDebugEnabled()) + log.debug("Creating partitions snapshot for node=" + nodeId + " in " + tmpDir); + + String backupDir = "preload-" + fut.rebalanceId; + + cctx.backup().createLocalBackup(backupDir, fut.getAssigns(), tmpDir).get(); + +// cctx.preloader().offerCheckpointTask(() -> { +// try { +// Map> filesToSnd = new HashMap<>(); +// +// for (Map.Entry> e : fut.getAssigns().entrySet()) { +// +// int grpId = e.getKey(); +// +// Map partFiles = new HashMap<>(); +// +// for (int partId : e.getValue()) { +// String path = cctx.preloader().storePath(grpId, partId); +// +// File src = new File(path); +// File dest = new File(path + ".cpy"); +// +// log.info("Copying file \"" + src + "\" to \"" + dest + "\""); +// +// RandomAccessFileIOFactory ioFactory = new RandomAccessFileIOFactory(); +// +// GridFileUtils.copy(ioFactory, src, ioFactory, dest, Long.MAX_VALUE); +// +// partFiles.put(partId, dest); +// } +// +// filesToSnd.put(grpId, partFiles); +// } +// +// fut.partFiles(filesToSnd); +// } catch (IgniteCheckedException | IOException e) { +// fut.onDone(e); +// } +// }).listen( +// c -> { + // send files + GridIoManager io = cctx.kernalContext().io(); + + String dir = tmpDir + "/" + backupDir + "/"; + + try (GridIoManager.TransmissionSender snd = io.openTransmissionSender(nodeId, rebalanceThreadTopic())) { + try { + for (Map.Entry> e : fut.getAssigns().entrySet()) { + Integer grpId = e.getKey(); + + String grpDir = dir + FilePageStoreManager.cacheDirName(cctx.cache().cacheGroup(grpId).config()); + + for (Integer partId : e.getValue()) { + File file = new File(grpDir + "/" + "part-" + partId + ".bin"); + + assert file.exists() : file; + + snd.send(file, F.asMap("group", grpId, "part", partId), TransmissionPolicy.FILE); + + GridDhtLocalPartition part = cctx.cache().cacheGroup(grpId).topology().localPartition(partId); + + // todo release only once - after historical rebalancing + part.release(); + } + } + } finally { + U.delete(new File(dir)); + } + + fut.onDone(); + } + catch (IOException | IgniteCheckedException | InterruptedException e) { + fut.onDone(e); + } + //todo should we cleanup files on error? +// } +// ); + +// if (!fut.isDone()) +// cctx.database().wakeupForCheckpoint(String.format(REBALANCE_CP_REASON, fut.getAssigns().keySet())); + + // todo + return fut; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(PartitionUploadManager.class, this); + } + + /** */ +// private static class SocketBackupProcessSupplier implements BackupProcessSupplier { +// /** */ +// private final FileTransferManager ftMrg; +// +// /** */ +// private final IgniteLogger log; +// +// /** +// * @param ftMrg An upload helper class. +// */ +// public SocketBackupProcessSupplier(FileTransferManager ftMrg, IgniteLogger log) { +// this.ftMrg = ftMrg; +// this.log = log; +// } +// +// /** {@inheritDoc} */ +// @Override public void supplyPartition( +// GroupPartitionId grpPartId, +// File file, +// long size +// ) throws IgniteCheckedException { +// U.log(log, "Start partition meta info uploading: " + grpPartId); +// +// ftMrg.writeMetaFrom(new PartitionFileMetaInfo(grpPartId.getGroupId(), +// grpPartId.getPartitionId(), +// file.getName(), +// size, +// 0)); +// +// U.log(log, "Start partition uploading: " + file.getName()); +// +// ftMrg.writeFrom(file, 0, size); +// } +// +// /** {@inheritDoc} */ +// @Override public void supplyDelta( +// GroupPartitionId grpPartId, +// File file, +// long offset, +// long size +// ) throws IgniteCheckedException { +// U.log(log, "Start delta meta info uploading: " + grpPartId); +// +// ftMrg.writeMetaFrom(new PartitionFileMetaInfo(grpPartId.getGroupId(), +// grpPartId.getPartitionId(), +// file.getName(), +// size, +// 1)); +// +// U.log(log, "Start delta uploading: " + file.getName()); +// +// ftMrg.writeFrom(file, offset, size); +// } +// } + + /** */ + private static class CachePartitionUploadFuture extends GridFutureAdapter { + /** */ + private long rebalanceId; + + /** */ + private AffinityTopologyVersion topVer; + + /** */ + private Map assigns; + + private Map> filesToSend; + + /** */ + public CachePartitionUploadFuture( + long rebalanceId, + AffinityTopologyVersion topVer, + Map assigns + ) { + this.rebalanceId = rebalanceId; + this.topVer = topVer; + this.assigns = assigns; + } + + /** + * @return The map of assignments of each cache group. + */ + public Map> getAssigns() { + Map> result = new HashMap<>(); + + for (Map.Entry grpPartsEntry : assigns.entrySet()) { + GridIntIterator iterator = grpPartsEntry.getValue().iterator(); + + result.putIfAbsent(grpPartsEntry.getKey(), new HashSet<>()); + + while (iterator.hasNext()) + result.get(grpPartsEntry.getKey()).add(iterator.next()); + } + + return result; + } + + /** {@inheritDoc} */ + @Override public boolean cancel() { + return onCancelled(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(CachePartitionUploadFuture.class, this); + } + + public void partFiles(Map> send) { + filesToSend = send; + } + + public Map> partFiles() { + return filesToSend; + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index a146ad5a0be5a..360f929468147 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -95,6 +95,7 @@ import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.UPDATE; 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; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.OWNING; import static org.apache.ignite.internal.processors.dr.GridDrType.DR_NONE; import static org.apache.ignite.internal.processors.dr.GridDrType.DR_PRIMARY; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridIntList.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridIntList.java index c03cf58d6454c..2bb02b8290210 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridIntList.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridIntList.java @@ -26,6 +26,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.NoSuchElementException; +import java.util.Set; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.SB; import org.apache.ignite.plugin.extensions.communication.Message; @@ -137,6 +138,22 @@ public GridIntList copy() { return res; } + /** + * @param parts The set of partitions. + * @return Representation as primitive list. + */ + public static GridIntList valueOf(Set parts) { + if (parts == null) + return new GridIntList(0); + + GridIntList intParts = new GridIntList(parts.size()); + + for (Integer partId : parts) + intParts.add(partId); + + return intParts; + } + /** * @param l List to add all elements of. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/collection/IntHashMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/collection/IntHashMap.java index d6b57d8da8a8c..2ae58784e2a21 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/collection/IntHashMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/collection/IntHashMap.java @@ -180,6 +180,11 @@ public IntHashMap(int cap) { return size() == 0; } + @Override public void clear() { + for (int i = 0; i < entries.length; i++) + entries[i] = null; + } + /** {@inheritDoc} */ @Override public boolean containsKey(int key) { return find(key) >= 0; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/collection/IntMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/collection/IntMap.java index f1bbe51e55546..4a70801915bc4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/collection/IntMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/collection/IntMap.java @@ -21,6 +21,8 @@ * The map for integer keys. */ public interface IntMap { + void clear(); + /*** * The bridge for consuming all entries of the map. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/collection/IntRWHashMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/collection/IntRWHashMap.java index 8d379bb95eed2..28498d74fb31d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/collection/IntRWHashMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/collection/IntRWHashMap.java @@ -106,6 +106,16 @@ public IntRWHashMap() { return size() == 0; } + @Override public void clear() { + lock.readLock().lock(); + try { + delegate.clear(); + } + finally { + lock.readLock().unlock(); + } + } + /** {@inheritDoc} */ @Override public boolean containsKey(int key) { lock.readLock().lock(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java index 51cb5581d5d5a..c1da3c0f6bda7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java @@ -123,7 +123,11 @@ public GridNioSessionImpl( @Override public void sendNoFuture(Object msg, IgniteInClosure ackC) throws IgniteCheckedException { try { +// System.out.println(">xxx> before session write"); + chain().onSessionWrite(this, msg, false, ackC); + +// System.out.println(">xxx> send no future"); } catch (IgniteCheckedException e) { close(); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/FutureTask.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/FutureTask.java index 44f1b5c70f8dd..a94c978c0a286 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/FutureTask.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/FutureTask.java @@ -27,7 +27,7 @@ public abstract class FutureTask extends GridFutureAdapter implements Runn /** * */ - protected abstract T body(); + protected abstract T body() throws Exception; /** {@inheritDoc} */ @Override public void run() { diff --git a/modules/core/src/test/config/log4j-test.xml b/modules/core/src/test/config/log4j-test.xml index 7206b4e298959..af29e8ebbdd2f 100755 --- a/modules/core/src/test/config/log4j-test.xml +++ b/modules/core/src/test/config/log4j-test.xml @@ -100,11 +100,14 @@ - diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/GridCachePersistenceRebalanceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/GridCachePersistenceRebalanceSelfTest.java new file mode 100644 index 0000000000000..e083a26514c5c --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/GridCachePersistenceRebalanceSelfTest.java @@ -0,0 +1,488 @@ +/* + * 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.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicLong; +import javax.cache.Cache; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CachePeekMode; +import org.apache.ignite.cache.CacheRebalanceMode; +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.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.TestRecordingCommunicationSpi; +import org.apache.ignite.internal.processors.cache.IgniteInternalCache; +import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.preload.GridPartitionBatchDemandMessage; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiPredicate; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.WithSystemProperty; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.After; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_DUMP_THREADS_ON_FAILURE; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_JVM_PAUSE_DETECTOR_DISABLED; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_PERSISTENCE_REBALANCE_ENABLED; + +/** + * Test cases for checking cancellation rebalancing process if some events occurs. + */ +@RunWith(Parameterized.class) +public class GridCachePersistenceRebalanceSelfTest extends GridCommonAbstractTest { + /** */ + private static final int CACHE_PART_COUNT = 16; + + /** */ + private static final int TEST_SIZE = GridTestUtils.SF.applyLB(100_000, 10_000); + + /** */ + private static final String CACHE1 = "cache1"; + + /** */ + private static final String CACHE2 = "cache2"; + + @Parameterized.Parameter + public CacheAtomicityMode cacheAtomicityMode; + + /** */ + @Before + public void setup() throws Exception { + cleanPersistenceDir(); + } + + /** */ + @After + public void tearDown() { + stopAllGrids(); + } + + /** Parameters. */ + @Parameterized.Parameters(name = "{0}") + public static Iterable data() { + List params = new ArrayList<>(2); + + params.add(new CacheAtomicityMode[] {CacheAtomicityMode.TRANSACTIONAL}); +// params.add(new CacheAtomicityMode[] {CacheAtomicityMode.ATOMIC}); + + return params; + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + return super.getConfiguration(igniteInstanceName) + .setDataStorageConfiguration(new DataStorageConfiguration() + .setDefaultDataRegionConfiguration(new DataRegionConfiguration() + .setMaxSize(8 * 1024L * 1024 * 1024) + .setPersistenceEnabled(true)) + .setWalMode(WALMode.LOG_ONLY) + .setCheckpointFrequency(3_000)) // todo check with default timeout! +// .setWalSegmentSize(4 * 1024 * 1024) +// .setMaxWalArchiveSize(32 * 1024 * 1024 * 1024L)) + .setCacheConfiguration(cacheConfig(DEFAULT_CACHE_NAME), cacheConfig(CACHE1), cacheConfig(CACHE2)); + } + + private CacheConfiguration cacheConfig(String name) { + return new CacheConfiguration(name).setCacheMode(CacheMode.REPLICATED) + .setRebalanceMode(CacheRebalanceMode.ASYNC) + .setAtomicityMode(cacheAtomicityMode) + //.setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC) +// .setBackups(1) + .setAffinity(new RendezvousAffinityFunction(false, CACHE_PART_COUNT)); +// .setCommunicationSpi(new TestRecordingCommunicationSpi() + } + + /** */ + @Test + @WithSystemProperty(key = IGNITE_JVM_PAUSE_DETECTOR_DISABLED, value = "true") + @WithSystemProperty(key = IGNITE_DUMP_THREADS_ON_FAILURE, value = "false") + @WithSystemProperty(key = IGNITE_PERSISTENCE_REBALANCE_ENABLED, value = "true") + @WithSystemProperty(key = IGNITE_BASELINE_AUTO_ADJUST_ENABLED, value = "true") + public void testPersistenceRebalanceBase() throws Exception { + IgniteEx ignite0 = startGrid(0); + + ignite0.cluster().active(true); + ignite0.cluster().baselineAutoAdjustTimeout(0); + + loadData(ignite0, DEFAULT_CACHE_NAME, TEST_SIZE); + + forceCheckpoint(); + + IgniteEx ignite1 = startGrid(1); + + awaitPartitionMapExchange(); + + verifyLocalCache(ignite0.cachex(DEFAULT_CACHE_NAME), ignite1.cachex(DEFAULT_CACHE_NAME)); + } + + /** */ + @Test + @WithSystemProperty(key = IGNITE_JVM_PAUSE_DETECTOR_DISABLED, value = "true") + @WithSystemProperty(key = IGNITE_DUMP_THREADS_ON_FAILURE, value = "false") + @WithSystemProperty(key = IGNITE_PERSISTENCE_REBALANCE_ENABLED, value = "true") + @WithSystemProperty(key = IGNITE_BASELINE_AUTO_ADJUST_ENABLED, value = "true") + public void testPersistenceRebalanceUnderConstantLoad() throws Exception { + IgniteEx ignite0 = startGrid(0); + + ignite0.cluster().active(true); + ignite0.cluster().baselineAutoAdjustTimeout(0); + + loadData(ignite0, DEFAULT_CACHE_NAME, TEST_SIZE); + + AtomicLong cntr = new AtomicLong(TEST_SIZE); + + ConstantLoader ldr = new ConstantLoader(ignite0.cache(DEFAULT_CACHE_NAME), cntr); + + IgniteInternalFuture ldrFut = GridTestUtils.runMultiThreadedAsync(ldr, 8, "thread"); + + U.sleep(1_000); + + forceCheckpoint(ignite0); + + IgniteEx ignite1 = startGrid(1); + + U.sleep(1_000); + + awaitPartitionMapExchange(); + + U.sleep(1_000); + + ldr.stop(); + + ldrFut.get(); + + U.sleep(1_000); + + verifyLocalCache(ignite0.cachex(DEFAULT_CACHE_NAME), ignite1.cachex(DEFAULT_CACHE_NAME)); + } + + /** */ + @Test + @WithSystemProperty(key = IGNITE_JVM_PAUSE_DETECTOR_DISABLED, value = "true") + @WithSystemProperty(key = IGNITE_DUMP_THREADS_ON_FAILURE, value = "false") + @WithSystemProperty(key = IGNITE_PERSISTENCE_REBALANCE_ENABLED, value = "true") + @WithSystemProperty(key = IGNITE_BASELINE_AUTO_ADJUST_ENABLED, value = "true") + public void testPersistenceRebalanceMultipleCaches() throws Exception { + IgniteEx ignite0 = startGrid(0); + + ignite0.cluster().active(true); + ignite0.cluster().baselineAutoAdjustTimeout(0); + + loadData(ignite0, CACHE1, TEST_SIZE); + loadData(ignite0, CACHE2, TEST_SIZE); + +// AtomicLong cntr = new AtomicLong(TEST_SIZE); +// +// ConstantLoader ldr = new ConstantLoader(ignite0.cache(DEFAULT_CACHE_NAME), cntr); +// +// IgniteInternalFuture ldrFut = GridTestUtils.runMultiThreadedAsync(ldr, 8, "thread"); +// +// U.sleep(1_000); + + forceCheckpoint(ignite0); + + IgniteEx ignite1 = startGrid(1); + + U.sleep(1_000); + + awaitPartitionMapExchange(); + + U.sleep(1_000); + +// ldr.stop(); +// +// ldrFut.get(); + + U.sleep(1_000); + + verifyLocalCache(ignite0.cachex(CACHE1), ignite1.cachex(CACHE1)); + verifyLocalCache(ignite0.cachex(CACHE2), ignite1.cachex(CACHE2)); + } + + + /** */ + @Test + @Ignore + @WithSystemProperty(key = IGNITE_JVM_PAUSE_DETECTOR_DISABLED, value = "true") + @WithSystemProperty(key = IGNITE_DUMP_THREADS_ON_FAILURE, value = "false") + @WithSystemProperty(key = IGNITE_PERSISTENCE_REBALANCE_ENABLED, value = "true") + @WithSystemProperty(key = IGNITE_BASELINE_AUTO_ADJUST_ENABLED, value = "false") + public void testPersistenceRebalanceManualCache() throws Exception { + IgniteEx ignite0 = startGrid(0); + + ignite0.cluster().active(true); + + IgniteCache cache = ignite0.getOrCreateCache( + new CacheConfiguration("manual") + .setCacheMode(CacheMode.PARTITIONED) + .setRebalanceMode(CacheRebalanceMode.ASYNC) + .setAtomicityMode(CacheAtomicityMode.ATOMIC) + .setBackups(1) + .setRebalanceDelay(-1) + .setAffinity(new RendezvousAffinityFunction(false) + .setPartitions(8))); + + loadData(ignite0, "manual", TEST_SIZE); + + assertTrue(!ignite0.cluster().isBaselineAutoAdjustEnabled()); + + IgniteEx ignite1 = startGrid(1); + + ignite1.cluster().setBaselineTopology(ignite1.cluster().nodes()); + + printPartitionState("manual", 0); + + cache.put(TEST_SIZE, new byte[1000]); + + awaitPartitionMapExchange(true, true, Collections.singleton(ignite1.localNode()), true); + } + + /** */ + @Test + @Ignore + @WithSystemProperty(key = IGNITE_JVM_PAUSE_DETECTOR_DISABLED, value = "true") + @WithSystemProperty(key = IGNITE_DUMP_THREADS_ON_FAILURE, value = "false") + @WithSystemProperty(key = IGNITE_PERSISTENCE_REBALANCE_ENABLED, value = "true") + @WithSystemProperty(key = IGNITE_BASELINE_AUTO_ADJUST_ENABLED, value = "false") + public void testPersistenceRebalanceAsyncUpdates() throws Exception { + IgniteEx ignite0 = startGrid(0); + + ignite0.cluster().active(true); + + IgniteCache cache = ignite0.getOrCreateCache( + new CacheConfiguration(DEFAULT_CACHE_NAME) + .setCacheMode(CacheMode.PARTITIONED) + .setRebalanceMode(CacheRebalanceMode.ASYNC) + .setAtomicityMode(CacheAtomicityMode.ATOMIC) + .setBackups(1) + .setAffinity(new RendezvousAffinityFunction(false) + .setPartitions(8))); + + loadData(ignite0, DEFAULT_CACHE_NAME, TEST_SIZE); + + assertTrue(!ignite0.cluster().isBaselineAutoAdjustEnabled()); + + IgniteEx ignite1 = startGrid(1); + + TestRecordingCommunicationSpi.spi(ignite1) + .blockMessages(new IgniteBiPredicate() { + @Override public boolean apply(ClusterNode node, Message msg) { + return msg instanceof GridPartitionBatchDemandMessage; + } + }); + + ignite1.cluster().setBaselineTopology(ignite1.cluster().nodes()); + + TestRecordingCommunicationSpi.spi(ignite1).waitForBlocked(); + + cache.put(TEST_SIZE, new byte[1000]); + + awaitPartitionMapExchange(true, true, Collections.singleton(ignite1.localNode()), true); + } + + /** + * @param ignite Ignite instance to load. + * @param name The cache name to add random data to. + * @param size The total size of entries. + */ + private void loadData(Ignite ignite, String name, int size) { + try (IgniteDataStreamer streamer = ignite.dataStreamer(name)) { + streamer.allowOverwrite(true); + + for (int i = 0; i < size; i++) { + if ((i + 1) % (size / 10) == 0) + log.info("Prepared " + (i + 1) * 100 / (size) + "% entries."); + + streamer.addData(i, i + name.hashCode()); + } + } + } + + /** + * @param expCache Expected data cache. + * @param actCache Actual data cache. + + * @throws IgniteCheckedException If failed. + */ + private void verifyLocalCache(IgniteInternalCache expCache, + IgniteInternalCache actCache) throws IgniteCheckedException { + StringBuilder buf = new StringBuilder(); + + buf.append(verifyLocalCacheContent(expCache, actCache)); + buf.append(verifyLocalCacheContent(actCache, expCache)); + + for (GridDhtLocalPartition actPart : actCache.context().topology().localPartitions()) { + GridDhtLocalPartition expPart = expCache.context().topology().localPartition(actPart.id()); + + long expCntr = expPart.updateCounter(); + long actCntr = actPart.updateCounter(); + + if (expCntr != actCntr) + buf.append("\n").append("Counter not match p=").append(expPart.id()).append(", exp=").append(expCntr).append(", act=").append(actCntr); + + long expSize = expPart.fullSize(); + long actSize = actPart.fullSize(); + + if (expSize != actSize) + buf.append("\n").append("Size not match p=").append(expPart.id()).append(", exp=").append(expSize).append(", act=").append(actSize); + } + + assertTrue(buf.toString(), buf.length() == 0); + } + + /** + * @param cache1 Expected data cache. + * @param cache2 Actual data cache. + + * @throws IgniteCheckedException If failed. + * @return Buffer with descriptions of found problems during verification. + */ + private StringBuilder verifyLocalCacheContent(IgniteInternalCache cache1, + IgniteInternalCache cache2) throws IgniteCheckedException { + + CachePeekMode[] peekAll = new CachePeekMode[] {CachePeekMode.ALL}; + + UUID node1 = cache1.context().shared().localNodeId(); + UUID node2 = cache2.context().shared().localNodeId(); + + StringBuilder buf = new StringBuilder(); + + for (Cache.Entry entry : cache1.localEntries(peekAll)) { + Object expect = entry.getValue(); + Object actual = cache2.localPeek(entry.getKey(), peekAll); + + if (!Objects.equals(expect, actual)) + buf.append("\n").append(node1).append(" vs ").append(node2).append(", expected=").append(expect).append(", actual=").append(actual); + + if (buf.length() > 10 * 1024) { + buf.append("\n").append("... and so on"); + + break; + } + } + + return buf; + } + + /** */ + private static class ConstantLoader implements Runnable { + /** */ + private final AtomicLong cntr; + + /** */ + private volatile boolean pause; + + /** */ + private volatile boolean paused; + + /** */ + private volatile boolean stop; + + /** */ + private final IgniteCache cache; + + /** */ + public ConstantLoader(IgniteCache cache, AtomicLong cntr) { + this.cache = cache; + this.cntr = cntr; + } + + /** {@inheritDoc} */ + @Override public void run() { + while (!stop && !Thread.currentThread().isInterrupted()) { + if (pause) { + if (!paused) + paused = true; + + try { + U.sleep(100); + } + catch (IgniteInterruptedCheckedException e) { + break; + } + + continue; + } + + long from = cntr.getAndAdd(100); + + for (long i = from; i < from + 100; i++) + cache.put(i, i); + + for (long i = from; i < from + 100; i += 10) + cache.remove(i); + } + } + + /** + * Stop loader thread. + */ + public void stop() { + stop = true; + } + + /** + * Pause loading. + */ + public void pause() { + pause = true; + + while (!paused) { + try { + U.sleep(100); + } + catch (IgniteInterruptedCheckedException e) { + break; + } + } + } + + /** + * Resume loading. + */ + public void resume() { + paused = false; + pause = false; + + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/WalRebalanceBasicTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/WalRebalanceBasicTest.java new file mode 100644 index 0000000000000..e86ab56d60c71 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/WalRebalanceBasicTest.java @@ -0,0 +1,80 @@ +package org.apache.ignite.internal.processors.cache.persistence; + +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.testframework.junits.WithSystemProperty; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.Test; + +/** */ +public class WalRebalanceBasicTest extends GridCommonAbstractTest { + /** */ + private static final int PARTS = 8; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setDataStorageConfiguration(new DataStorageConfiguration().setDefaultDataRegionConfiguration(new DataRegionConfiguration().setPersistenceEnabled(true))); + + CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME); + + ccfg.setBackups(1); + ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + ccfg.setCacheMode(CacheMode.REPLICATED); + ccfg.setAffinity(new RendezvousAffinityFunction(false, PARTS)); + + cfg.setCacheConfiguration(ccfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + + super.afterTest(); + } + + @Test + @WithSystemProperty(key = IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD, value="1") + public void checkBasicWALRebalancing() throws Exception { + IgniteEx node1 = startGrid(0); + IgniteEx node2 = startGrid(1); + + node1.cluster().active(true); + + IgniteCache cache1 = node1.cache(DEFAULT_CACHE_NAME); + + for (int i = 0; i < PARTS * 2; i++) + cache1.put(i, i); + + forceCheckpoint(); + + stopGrid(1); + + awaitPartitionMapExchange(); + + for (int i = PARTS * 2; i < PARTS * 4; i++) + cache1.put(i, i); + + startGrid(1); + + awaitPartitionMapExchange(); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/backup/IgniteBackupManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/backup/IgniteBackupManagerSelfTest.java new file mode 100644 index 0000000000000..0d592d4c282e4 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/backup/IgniteBackupManagerSelfTest.java @@ -0,0 +1,291 @@ +/* + * 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.backup; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.FileChannel; +import java.nio.file.DirectoryStream; +import java.nio.file.Path; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CacheRebalanceMode; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.WALMode; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.pagemem.PageIdUtils; +import org.apache.ignite.internal.pagemem.store.PageStore; +import org.apache.ignite.internal.processors.cache.persistence.CheckpointFuture; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; +import org.apache.ignite.internal.processors.cache.persistence.file.FileVersionCheckingFactory; +import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; +import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import static java.nio.file.Files.newDirectoryStream; +import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.cacheDirName; + +/** */ +public class IgniteBackupManagerSelfTest extends GridCommonAbstractTest { + /** */ + private static final int CACHE_PARTS_COUNT = 8; + + /** */ + private static final int PAGE_SIZE = 1024; + + /** */ + private static final DataStorageConfiguration memCfg = new DataStorageConfiguration() + .setDefaultDataRegionConfiguration(new DataRegionConfiguration() + .setMaxSize(100L * 1024 * 1024) + .setPersistenceEnabled(true)) + .setPageSize(PAGE_SIZE) + .setWalMode(WALMode.LOG_ONLY); + + /** */ + private static final FilePageStoreFactory pageStoreFactory = + new FileVersionCheckingFactory(new RandomAccessFileIOFactory(), new RandomAccessFileIOFactory(), memCfg); + + /** */ + private static final CacheConfiguration defaultCacheCfg = + new CacheConfiguration(DEFAULT_CACHE_NAME) + .setCacheMode(CacheMode.PARTITIONED) + .setRebalanceMode(CacheRebalanceMode.ASYNC) + .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL) + .setAffinity(new RendezvousAffinityFunction(false) + .setPartitions(CACHE_PARTS_COUNT)); + + /** Directory to store temporary files on testing cache backup process. */ + private File backupDir; + + /** + * Calculate CRC for all partition files of specified cache. + * + * @param cacheDir Cache directory to iterate over partition files. + * @return The map of [fileName, checksum]. + * @throws IgniteCheckedException If fails. + */ + private static Map calculateCRC32Partitions(File cacheDir) throws IgniteCheckedException { + assert cacheDir.isDirectory() : cacheDir.getAbsolutePath(); + + Map result = new HashMap<>(); + + try { + try (DirectoryStream partFiles = newDirectoryStream(cacheDir.toPath(), + p -> p.toFile().getName().startsWith(PART_FILE_PREFIX) && p.toFile().getName().endsWith(FILE_SUFFIX)) + ) { + for (Path path : partFiles) + result.put(path.toFile().getName(), FastCrc.calcCrc(path.toFile())); + } + + return result; + } + catch (IOException e) { + throw new IgniteCheckedException(e); + } + } + + /** + * @param from File to copy from. + * @param offset Starting file position. + * @param count Bytes to copy to destination. + * @param to Output directory. + * @throws IgniteCheckedException If fails. + */ + private static File copy(File from, long offset, long count, File to) throws IgniteCheckedException { + assert to.isDirectory(); + + try { + File destFile = new File(to, from.getName()); + + if (!destFile.exists() || destFile.delete()) + destFile.createNewFile(); + + try (FileChannel src = new FileInputStream(from).getChannel(); + FileChannel dest = new FileOutputStream(destFile).getChannel()) { + src.position(offset); + + long written = 0; + + while (written < count) + written += src.transferTo(written, count - written, dest); + } + + return destFile; + } + catch (IOException e) { + throw new IgniteCheckedException(e); + } + } + + /** */ + @Before + public void beforeTestBackup() throws Exception { + cleanPersistenceDir(); + + backupDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), "backup", true); + } + + /** */ + @After + public void afterTestBackup() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + return super.getConfiguration(igniteInstanceName) + .setConsistentId(igniteInstanceName) + .setDataStorageConfiguration(memCfg) + .setCacheConfiguration(defaultCacheCfg); + } + + /** + * @param pageIdx Page index to track. + * @return {@code true} if + */ + private boolean track(AtomicLong pageTrackBits, int pageIdx) { + assert pageIdx >= 0; + + int mask = 1 << pageIdx; + + long next = pageTrackBits.getAndUpdate(b -> b |= mask); + + return (pageTrackBits.get() & mask) == mask; + } + + /** + * + */ + @Test + public void testShift() throws Exception { + final AtomicLong l = new AtomicLong(); + + for (int i = 5; i < 10; i ++) + track(l, i); + + System.out.println(String.format("%064d", new BigInteger(Long.toBinaryString(l.get())))); + } + + /** + * + */ + @Test + public void testBackupLocalPartitions() throws Exception { + final CountDownLatch slowCopy = new CountDownLatch(1); + + IgniteEx ig = startGrid(0); + + ig.cluster().active(true); + + for (int i = 0; i < 1024; i++) + ig.cache(DEFAULT_CACHE_NAME).put(i, i); + + CheckpointFuture cpFut = ig.context() + .cache() + .context() + .database() + .forceCheckpoint("the next one"); + + cpFut.finishFuture().get(); + + for (int i = 1024; i < 2048; i++) + ig.cache(DEFAULT_CACHE_NAME).put(i, i); + + Set parts = Stream.iterate(0, n -> n + 1) + .limit(CACHE_PARTS_COUNT) + .collect(Collectors.toSet()); + + Map> toBackup = new HashMap<>(); + toBackup.put(CU.cacheId(DEFAULT_CACHE_NAME), parts); + + IgniteInternalFuture backupFut = ig.context() + .cache() + .context() + .backup() + .createLocalBackup("testBackup", toBackup, backupDir); + + backupFut.get(); + + File cacheWorkDir = ((FilePageStoreManager)ig.context() + .cache() + .context() + .pageStore()) + .cacheWorkDir(defaultCacheCfg); + + // Calculate CRCs + final Map origParts = calculateCRC32Partitions(cacheWorkDir); + + final Map bakcupCRCs = calculateCRC32Partitions(new File(new File(backupDir.getAbsolutePath(), + "testBackup"), + cacheDirName(defaultCacheCfg))); + + assertEquals("Partitons the same after backup and after merge", origParts, bakcupCRCs); + } + + /** */ + private void partitionCRCs(PageStore pageStore, int partId) throws IgniteCheckedException { + long pageId = PageIdUtils.pageId(partId, FLAG_DATA, 0); + + ByteBuffer buf = ByteBuffer.allocate(pageStore.getPageSize()) + .order(ByteOrder.nativeOrder()); + + StringBuilder sb = new StringBuilder(); + + for (int pageNo = 0; pageNo < pageStore.pages(); pageId++, pageNo++) { + buf.clear(); + + pageStore.read(pageId, buf, true); + + sb.append("[pageId=") + .append(pageId) + .append(", crc=") + .append(PageIO.getCrc(buf)) + .append("]\n"); + } + + U.log(log, sb.append("[pages=").append(pageStore.pages()).append("]\n").toString()); + } +} 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 230cb5cfc92ec..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 @@ -150,6 +150,8 @@ private void checkInvariantSwitchSegmentSize(int serVer) throws Exception { null, null, null, + null, + null, null) ).createSerializer(serVer); @@ -470,6 +472,7 @@ private T2 initiate( null, null, null, + null, new GridCacheIoManager(), null, null, @@ -477,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 4af8c5f2e4e4c..b6d0e24115dd2 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 @@ -89,7 +89,9 @@ public class BPlusTreePageMemoryImplTest extends BPlusTreeSelfTest { null, null, null, - new CacheDiagnosticManager() + null, + new CacheDiagnosticManager(), + null ); PageMemory mem = new PageMemoryImpl( 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 d8951236a07d7..42817b18662de 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 @@ -88,6 +88,8 @@ public class BPlusTreeReuseListPageMemoryImplTest extends BPlusTreeReuseSelfTest null, 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 65ae95059ef87..90b5c6cc61d99 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 @@ -104,7 +104,9 @@ public class IndexStoragePageMemoryImplTest extends IndexStorageSelfTest { null, null, null, - new CacheDiagnosticManager() + null, + new CacheDiagnosticManager(), + null ); return new PageMemoryImpl( 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 5c74a8454ed70..0ce72f95d2310 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 @@ -94,6 +94,8 @@ public class PageMemoryImplNoLoadTest extends PageMemoryNoLoadSelfTest { null, 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 7c0a85ddd4b02..55a21ca34ae75 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 @@ -514,6 +514,8 @@ private PageMemoryImpl createPageMemory( null, null, null, + null, + null, null ); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/GridCacheReadOnlyStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/GridCacheReadOnlyStoreSelfTest.java new file mode 100644 index 0000000000000..405e5b461a956 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/GridCacheReadOnlyStoreSelfTest.java @@ -0,0 +1,447 @@ +/* + * 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.database; + +import java.util.HashSet; +import java.util.Random; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicBoolean; +import javax.cache.Cache; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CachePeekMode; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.IgniteInternalCache; +import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtInvalidPartitionException; +import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.MOVING; +import static org.apache.ignite.internal.util.IgniteUtils.GB; + +/** + * + */ +public class GridCacheReadOnlyStoreSelfTest extends GridCommonAbstractTest { + /** */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME); + + ccfg.setCacheMode(CacheMode.PARTITIONED); + ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + ccfg.setBackups(2); + ccfg.setAffinity(new RendezvousAffinityFunction(false, 16)); + //ccfg.setBackups(1); + // todo check different sync modes +// ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + + cfg.setCacheConfiguration(ccfg); + + DataStorageConfiguration dscfg = new DataStorageConfiguration(); + + DataRegionConfiguration reg = new DataRegionConfiguration(); + + reg.setMaxSize(2 * GB); + reg.setPersistenceEnabled(true); + + dscfg.setDefaultDataRegionConfiguration(reg); + dscfg.setCheckpointFrequency(3_000); + + cfg.setDataStorageConfiguration(dscfg); + + return cfg; + } + + /** */ + @Before + public void setup() throws Exception { + cleanPersistenceDir(); + } + + /** */ + @After + public void tearDown() throws Exception { + stopAllGrids(); + +// cleanPersistenceDir(); + } + + @Test + public void checkSwitchUnderConstantLoad() throws Exception { + doCheckReadonlyMode(4, 5, false, false); + } + + @Test + public void checkSwitchOnlyUnderConstantLoad() throws Exception { + doCheckReadonlyMode(4, 5, true, false); + } + + @Test + public void checkSwitchUnderConstantLoadUpdatesFromBackup() throws Exception { + doCheckReadonlyMode(4, 5, false, true); + } + + /** + * Ensure that partition counter doesn't change when evicting read-only partition. + * + * @throws Exception If failed. + */ + @Test + public void checkEvictions() throws Exception { + IgniteEx node = startGrid(0); + + node.cluster().active(true); + node.cluster().baselineAutoAdjustTimeout(0); + + IgniteInternalCache cache = node.cachex(DEFAULT_CACHE_NAME); + + for (int i = 0; i < 80_000; i++) + cache.put(i, i); + + int evictedId = 6; + + GridDhtLocalPartition part = cache.context().topology().localPartition(evictedId); + + part.moving(); + + long cntr = part.updateCounter(); + + assert cntr > 0 : cntr; + + log.info(">xxx> >> READ-ONLY"); + + node.context().cache().context().database().checkpointReadLock(); + + try { + part.readOnly(true); + } finally { + node.context().cache().context().database().checkpointReadUnlock(); + } + + assert cache.context().topology().localPartition(evictedId).dataStore().readOnly(); + + // generate keys + Set keys = new HashSet<>(); + + for (int i = 160_000; i < 160_300; i++) { + if (cache.affinity().partition(i) == evictedId) + keys.add(i); + } + + assert !keys.isEmpty(); + + CountDownLatch waitRent = new CountDownLatch(1); + + part.clearAsync(); + + for (Integer key : keys) + cache.put(key, key); + + part.onClearFinished(f -> waitRent.countDown()); + + waitRent.await(); + + forceCheckpoint(node); + + part = cache.context().topology().localPartition(evictedId); + + assertEquals(cntr + keys.size(), part.updateCounter()); + + assertEquals(0, part.fullSize()); + + assertEquals(0, part.entries(cache.context().cacheId()).size()); + } + + private void doCheckReadonlyMode(int grids, int partId, boolean validateCounters, boolean reqFromBackup) throws Exception { + Ignite node = startGrids(grids); + + node.cluster().active(true); + node.cluster().baselineAutoAdjustTimeout(0); + + awaitPartitionMapExchange(); + + AffinityTopologyVersion topVer = grid(0).context().cache().context().exchange().readyAffinityVersion(); + + AtomicBoolean stopper = new AtomicBoolean(); + AtomicBoolean rmv = new AtomicBoolean(); + + CountDownLatch startLatch = new CountDownLatch(1); + + T2 pair = detectPrimaryAndBackupNodes(grids, partId, topVer); + + int primaryIdx = pair.get1(); + int backupIdx = pair.get2(); + + IgniteCache reqCache = grid(reqFromBackup ? backupIdx : primaryIdx).cache(DEFAULT_CACHE_NAME); + + ConstantLoader ldr = new ConstantLoader(stopper, rmv, reqCache, startLatch); + + IgniteInternalFuture fut = GridTestUtils.runAsync(ldr); + + IgniteEx primaryNode = grid(primaryIdx); + IgniteEx backupNode = grid(backupIdx); + + log.info(">xxx> Partition: " + partId); + log.info(">xxx> Primary: " + primaryNode.localNode().id()); + log.info(">xxx> Backup: " + backupNode.localNode().id()); + + IgniteInternalCache backupCache = backupNode.cachex(DEFAULT_CACHE_NAME); + GridDhtLocalPartition backupPart = backupCache.context().topology().localPartition(partId); + + backupPart.moving(); + + assert backupPart.state() == MOVING : backupPart.state(); + + startLatch.await(); + + U.sleep(300); + + log.info(">xxx> >> READ-ONLY"); + + backupNode.context().cache().context().database().checkpointReadLock(); + + try { + // Switching mode under the write lock. + backupPart.readOnly(true); + + rmv.set(true); + } finally { + backupNode.context().cache().context().database().checkpointReadUnlock(); + } + + U.sleep(500); + + if (!validateCounters) { + log.info(">xxx> >> FULL"); + + backupNode.context().cache().context().database().checkpointReadLock(); + + try { + // Switching mode under the write lock. + backupPart.readOnly(false); + + rmv.set(false); + } + finally { + backupNode.context().cache().context().database().checkpointReadUnlock(); + } + } + + stopper.set(true); + + int lastKey = fut.get(); + + // validate + int rmvStop = ldr.rmvStopIdx(); + int rmvStart = ldr.rmvStartIdx(); + + CachePeekMode[] peekAll = new CachePeekMode[]{CachePeekMode.ALL}; + + Iterable> it = backupCache.localEntries(peekAll); + + log.info("Range [" + rmvStart + " - " + rmvStop + "]"); + + for (Cache.Entry e : it) { + // todo This check in unstable when raneg starts from zero. + if (backupCache.affinity().partition(e.getKey()) == partId && e.getKey() > rmvStart && e.getKey() < rmvStop) + fail("range [" + rmvStart + " - " + rmvStop + "] key=" + e.getKey()); + } + + // Ensure not readonly partitions consistency. + for (int i = 0; i < lastKey; i++) { + for (int n = 0; n < grids; n++) { + if (n == backupIdx) + continue; + + IgniteEx node0 = grid(n); + + IgniteInternalCache cache = node0.cachex(DEFAULT_CACHE_NAME); + + if (cache.affinity().isPrimaryOrBackup(node0.localNode(), i) && !ldr.rmvKeys().contains(i)) + assertNotNull("node=" + n + " " + i + " not found", cache.localPeek(i, peekAll)); + } + } + + // validate counters + if (validateCounters) { + long cntr = -1;// prevReserved = -1; + + String fail = null; + + for (int n = 0; n < grids; n++) { + GridDhtLocalPartition part = grid(n).cachex(DEFAULT_CACHE_NAME).context().topology().localPartition(partId); + + if (part == null) + continue; + + UUID nodeId = grid(n).localNode().id(); + + if (cntr >= 0 && cntr != part.updateCounter()) { + fail = "Incorrect update counter on node " + nodeId + ", expected=" + cntr + + ", actual=" + part.updateCounter(); + } + else + cntr = part.updateCounter(); + + log.info("Node " + nodeId + ", counter=" + part.updateCounter() + ", reserved=" + part.reservedCounter()); + } + + if (fail != null) + fail(fail); + } + } + + private T2 detectPrimaryAndBackupNodes(int cnt, int partId, AffinityTopologyVersion topVer) + throws GridDhtInvalidPartitionException { + Integer primary = null; + Integer backup = null; + + for (int n = 0; n < cnt; n++) { + try { + if (grid(n).cachex(DEFAULT_CACHE_NAME).context().topology().localPartition(partId).primary(topVer)) + primary = n; + else + backup = n; + } + catch (GridDhtInvalidPartitionException | NullPointerException ignore) { + continue; + } + + if (primary != null && backup != null) + return new T2<>(primary, backup); + } + + throw new IllegalStateException("primary=" + primary + ", backup=" + backup); + } + + /** */ + private class ConstantLoader implements Callable { + /** */ + private final AtomicBoolean stopper; + + /** */ + private final AtomicBoolean rmv; + + /** */ + private final IgniteCache cache; + + /** */ + private final CountDownLatch startLatch; + + /** */ + private int off = 0; + + /** */ + private int cnt = 10; + + /** */ + private volatile int rmvOffset = 0; + + /** */ + private volatile int rmvOffsetStop = -1; + + /** */ + private final Set rmvKeys = new HashSet<>(); + + /** */ + private final Random rnd = ThreadLocalRandom.current(); + + public ConstantLoader(AtomicBoolean stopper, AtomicBoolean rmv, IgniteCache cache, + CountDownLatch startLatch) { + this.stopper = stopper; + this.rmv = rmv; + this.cache = cache; + this.startLatch = startLatch; + } + + /** {@inheritDoc} */ + @Override public Integer call() throws Exception { + startLatch.countDown(); + + boolean rmvPrev = false; + + while (!stopper.get()) { + for (int i = off; i < off + cnt; i++) { + boolean rmv0 = rmv.get(); + + if (rmv0 != rmvPrev) { + if (rmv0) + rmvOffset = i; + else + rmvOffsetStop = i; + + rmvPrev = rmv0; + } + + cache.put(i, i); + + if (off > 0 && rmv0 && rnd.nextBoolean()) { + int rmvKey = i - off; + cache.remove(rmvKey); + + rmvKeys.add(rmvKey); + } + } + + U.sleep(rnd.nextInt(10)); + + off += cnt; + } + + int last = off - 1; + + if (rmvOffsetStop == -1) + rmvOffsetStop = last; + + return last; + } + + public int rmvStopIdx() { + return rmvOffsetStop; + } + + public int rmvStartIdx() { + return rmvOffset; + } + + public Set rmvKeys() { + return rmvKeys; + } + } +} 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 627c384b11ee6..6a5a6506b8cdb 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 @@ -44,6 +44,7 @@ import org.apache.ignite.internal.processors.cache.jta.CacheNoopJtaManager; import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridCachePreloadSharedManager; import org.apache.ignite.internal.processors.cache.query.GridCacheLocalQueryManager; import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager; import org.apache.ignite.internal.processors.cache.store.CacheOsStoreManager; @@ -75,6 +76,7 @@ public GridCacheTestContext(GridTestKernalContext ctx) throws Exception { null, new WalStateManager(null), new IgniteCacheDatabaseSharedManager(), + null, // TODO proper initialization new IgniteCacheSnapshotManager(), new GridCacheDeploymentManager(), new GridCachePartitionExchangeManager(), @@ -86,7 +88,8 @@ public GridCacheTestContext(GridTestKernalContext ctx) throws Exception { null, null, null, - new CacheDiagnosticManager() + new CacheDiagnosticManager(), + new GridCachePreloadSharedManager(ctx) ), defaultCacheConfiguration(), null,