diff --git a/modules/compress/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsCheckpointRecoveryWithCompressionTest.java b/modules/compress/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsCheckpointRecoveryWithCompressionTest.java new file mode 100644 index 0000000000000..173dc30555e25 --- /dev/null +++ b/modules/compress/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsCheckpointRecoveryWithCompressionTest.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.persistence.db; + +import org.apache.ignite.configuration.DiskPageCompression; + +/** + * Class containing tests for applying checkpoint recovery data with compression. + */ +public class IgnitePdsCheckpointRecoveryWithCompressionTest extends IgnitePdsCheckpointRecoveryTest { + /** */ + @Override protected DiskPageCompression getCompression() { + return DiskPageCompression.SNAPPY; + } +} diff --git a/modules/compress/src/test/java/org/apache/ignite/testsuites/IgnitePdsCompressionTestSuite.java b/modules/compress/src/test/java/org/apache/ignite/testsuites/IgnitePdsCompressionTestSuite.java index aae13a64ac0d4..d9fb92212150d 100644 --- a/modules/compress/src/test/java/org/apache/ignite/testsuites/IgnitePdsCompressionTestSuite.java +++ b/modules/compress/src/test/java/org/apache/ignite/testsuites/IgnitePdsCompressionTestSuite.java @@ -20,6 +20,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsCheckpointRecoveryWithCompressionTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgnitePdsCheckpointSimulationWithRealCpDisabledAndWalCompressionTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.WalCompactionAndPageCompressionTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.WalRecoveryWithPageCompressionAndTdeTest; @@ -64,6 +65,9 @@ public static List> suite() { suite.add(IgnitePdsCheckpointSimulationWithRealCpDisabledAndWalCompressionTest.class); suite.add(WalCompactionAndPageCompressionTest.class); + // Checkpoint recovery. + suite.add(IgnitePdsCheckpointRecoveryWithCompressionTest.class); + // Snapshots. suite.add(SnapshotCompressionBasicTest.class); 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 ea6eb863c4ab6..1b872552ba2cf 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 @@ -187,6 +187,12 @@ public class DataStorageConfiguration implements Serializable { /** Value used to indicate the use of half of the {@link #getMaxWalArchiveSize}. */ public static final long HALF_MAX_WAL_ARCHIVE_SIZE = -1; + /** Default value for {@link #writeRecoveryDataOnCheckpoint} property. */ + public static final boolean DFLT_WRITE_RECOVERY_DATA_ON_CP = false; + + /** Default compression algorithm for checkpoint recovery data. */ + public static final DiskPageCompression DFLT_CP_RECOVERY_DATA_COMRESSION = DiskPageCompression.SKIP_GARBAGE; + /** Memory page size. */ private int pageSize = IgniteSystemProperties.getInteger( IGNITE_DEFAULT_DATA_STORAGE_PAGE_SIZE, 0); @@ -339,6 +345,19 @@ public class DataStorageConfiguration implements Serializable { /** Default memory allocator for all data regions. */ @Nullable private MemoryAllocator memoryAllocator = null; + /** + * Mode for storing page recovery data. + * If {@code true}, page recovery data will be written during checkpoint. + * If {@code false}, WAL physical records will be used to store page recovery data. + */ + private boolean writeRecoveryDataOnCheckpoint = DFLT_WRITE_RECOVERY_DATA_ON_CP; + + /** Compression algorithm for checkpoint recovery data. */ + private DiskPageCompression cpRecoveryDataCompression = DFLT_CP_RECOVERY_DATA_COMRESSION; + + /** Compression level for checkpoint recovery data. */ + private Integer cpRecoveryDataCompressionLevel; + /** * Creates valid durable memory configuration with all default values. */ @@ -1394,6 +1413,72 @@ public DataStorageConfiguration setMemoryAllocator(MemoryAllocator allocator) { return this; } + /** + * @return Flag defining mode for storing page recovery data. If {@code true}, recovery data will be written + * during checkpoint, if {@code false}, WAL physical records will be used to store recovery data. + */ + public boolean isWriteRecoveryDataOnCheckpoint() { + return writeRecoveryDataOnCheckpoint; + } + + /** + * Sets mode for storing page recovery data. + * + * @param writeRecoveryDataOnCheckpoint If {@code true}, page recovery data will be written during checkpoint, + * if {@code false}, WAL physical records will be used to store page recovery data. + * Default is {@link #DFLT_WRITE_RECOVERY_DATA_ON_CP}. + * @return {@code this} for chaining. + */ + public DataStorageConfiguration setWriteRecoveryDataOnCheckpoint(boolean writeRecoveryDataOnCheckpoint) { + this.writeRecoveryDataOnCheckpoint = writeRecoveryDataOnCheckpoint; + + return this; + } + + /** + * Gets compression algorithm for checkpoint recovery data. + * + * @return Page compression algorithm. + */ + public DiskPageCompression getCheckpointRecoveryDataCompression() { + return cpRecoveryDataCompression == null ? DFLT_CP_RECOVERY_DATA_COMRESSION : cpRecoveryDataCompression; + } + + /** + * Sets compression algorithm for checkpoint recovery data. + * + * @param cpRecoveryDataCompression Compression algorithm. + * @return {@code this} for chaining. + */ + public DataStorageConfiguration setCheckpointRecoveryDataCompression(DiskPageCompression cpRecoveryDataCompression) { + this.cpRecoveryDataCompression = cpRecoveryDataCompression; + + return this; + } + + /** + * Gets {@link #getCheckpointRecoveryDataCompression()} algorithm specific compression level. + * + * @return Checkpoint recovery data compression level or {@code null} for default. + */ + public Integer getCheckpointRecoveryDataCompressionLevel() { + return cpRecoveryDataCompressionLevel; + } + + /** + * Sets {@link #setCheckpointRecoveryDataCompression(DiskPageCompression)} algorithm specific compression level. + * + * @param cpRecoveryDataCompressionLevel Checkpoint recovery data compression level or {@code null} to use default. + * {@link DiskPageCompression#ZSTD Zstd}: from {@code -131072} to {@code 22} (default {@code 3}). + * {@link DiskPageCompression#LZ4 LZ4}: from {@code 0} to {@code 17} (default {@code 0}). + * @return {@code this} for chaining. + */ + public DataStorageConfiguration setCheckpointRecoveryDataCompressionLevel(Integer cpRecoveryDataCompressionLevel) { + this.cpRecoveryDataCompressionLevel = cpRecoveryDataCompressionLevel; + + return this; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(DataStorageConfiguration.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java index d95f0121d7e26..faa9acdc69ec6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java @@ -1563,13 +1563,15 @@ private long requiredOffheap() { for (DataRegionConfiguration dataReg : dataRegions) { res += dataReg.getMaxSize(); - res += U.checkpointBufferSize(dataReg); + res += U.checkpointBufferSize(memCfg, dataReg); } } - res += memCfg.getDefaultDataRegionConfiguration().getMaxSize(); + DataRegionConfiguration dfltDataRegion = memCfg.getDefaultDataRegionConfiguration(); - res += U.checkpointBufferSize(memCfg.getDefaultDataRegionConfiguration()); + res += dfltDataRegion.getMaxSize(); + + res += U.checkpointBufferSize(memCfg, dfltDataRegion); return res; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/IgniteWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/IgniteWriteAheadLogManager.java index 2ff5f33f3332f..bb1cbd80fe420 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/IgniteWriteAheadLogManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/IgniteWriteAheadLogManager.java @@ -208,12 +208,12 @@ public WALIterator replay( public int reserved(WALPointer low, WALPointer high); /** - * Checks WAL disabled for cache group. + * Checks WAL page records disabled. * * @param grpId Group id. * @param pageId Page id. */ - public boolean disabled(int grpId, long pageId); + public boolean pageRecordsDisabled(int grpId, long pageId); /** * Getting local WAL segment size. 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 dc6af6fff6be4..510c376affc99 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 @@ -1012,13 +1012,16 @@ private WalStateResult awaitCheckpoint(CheckpointProgress cpFut, WalStatePropose } /** - * Checks WAL disabled for cache group. + * Checks WAL page records disabled. * * @param grpId Group id. * @param pageId Page id. * @return {@code True} if WAL disable for group. {@code False} If not. */ - public boolean isDisabled(int grpId, long pageId) { + public boolean isPageRecordsDisabled(int grpId, long pageId) { + if (cctx.kernalContext().config().getDataStorageConfiguration().isWriteRecoveryDataOnCheckpoint()) + return true; + CacheGroupContext ctx = cctx.cache().cacheGroup(grpId); return ctx != null && (!ctx.walEnabled() diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsImpl.java index 4d163a47c36fd..0ba6e04a99343 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsImpl.java @@ -89,6 +89,12 @@ public class DataStorageMetricsImpl { /** */ private final AtomicLongMetric lastCpSplitAndSortPagesDuration; + /** */ + private final AtomicLongMetric lastCpRecoveryDataWriteDuration; + + /** */ + private final AtomicLongMetric lastCpRecoveryDataSize; + /** */ private final AtomicLongMetric lastCpTotalPages; @@ -155,6 +161,9 @@ public class DataStorageMetricsImpl { /** */ private final HistogramMetricImpl cpSplitAndSortPagesHistogram; + /** */ + private final HistogramMetricImpl cpRecoveryDataWriteHistogram; + /** */ private final HistogramMetricImpl cpHistogram; @@ -246,6 +255,12 @@ public DataStorageMetricsImpl( lastCpSplitAndSortPagesDuration = mreg.longMetric("LastCheckpointSplitAndSortPagesDuration", "Duration of splitting and sorting checkpoint pages of the last checkpoint in milliseconds."); + lastCpRecoveryDataWriteDuration = mreg.longMetric("LastCheckpointRecoveryDataWriteDuration", + "Duration of checkpoint recovery data write in milliseconds."); + + lastCpRecoveryDataSize = mreg.longMetric("LastCheckpointRecoveryDataSize", + "Size of checkpoint recovery data in bytes."); + lastCpTotalPages = mreg.longMetric("LastCheckpointTotalPagesNumber", "Total number of pages written during the last checkpoint."); @@ -312,6 +327,9 @@ public DataStorageMetricsImpl( cpSplitAndSortPagesHistogram = mreg.histogram("CheckpointSplitAndSortPagesHistogram", cpBounds, "Histogram of splitting and sorting checkpoint pages duration in milliseconds."); + cpRecoveryDataWriteHistogram = mreg.histogram("CheckpointRecoveryDataWriteHistogram", cpBounds, + "Histogram of checkpoint recovery data write duration in milliseconds."); + cpHistogram = mreg.histogram("CheckpointHistogram", cpBounds, "Histogram of checkpoint duration in milliseconds."); @@ -672,11 +690,13 @@ public boolean metricsEnabled() { * @param walRecordFsyncDuration Duration of WAL fsync after logging {@link CheckpointRecord} on checkpoint begin. * @param writeEntryDuration Duration of checkpoint entry buffer writing to file. * @param splitAndSortPagesDuration Duration of splitting and sorting checkpoint pages. + * @param recoveryDataWriteDuration Recovery data write duration. * @param duration Total checkpoint duration. * @param start Checkpoint start time. * @param totalPages Total number of all pages in checkpoint. * @param dataPages Total number of data pages in checkpoint. * @param cowPages Total number of COW-ed pages in checkpoint. + * @param recoveryDataSize Recovery data size, in bytes. * @param storageSize Storage space allocated, in bytes. * @param sparseStorageSize Storage space allocated adjusted for possible sparsity, in bytes. */ @@ -691,11 +711,13 @@ public void onCheckpoint( long walRecordFsyncDuration, long writeEntryDuration, long splitAndSortPagesDuration, + long recoveryDataWriteDuration, long duration, long start, long totalPages, long dataPages, long cowPages, + long recoveryDataSize, long storageSize, long sparseStorageSize ) { @@ -712,11 +734,13 @@ public void onCheckpoint( lastCpWalRecordFsyncDuration.value(walRecordFsyncDuration); lastCpWriteEntryDuration.value(writeEntryDuration); lastCpSplitAndSortPagesDuration.value(splitAndSortPagesDuration); + lastCpRecoveryDataWriteDuration.value(recoveryDataWriteDuration); lastCpDuration.value(duration); lastCpStart.value(start); lastCpTotalPages.value(totalPages); lastCpDataPages.value(dataPages); lastCpCowPages.value(cowPages); + lastCpRecoveryDataSize.value(recoveryDataSize); this.storageSize.value(storageSize); this.sparseStorageSize.value(sparseStorageSize); @@ -732,6 +756,7 @@ public void onCheckpoint( cpWalRecordFsyncHistogram.value(walRecordFsyncDuration); cpWriteEntryHistogram.value(writeEntryDuration); cpSplitAndSortPagesHistogram.value(splitAndSortPagesDuration); + cpRecoveryDataWriteHistogram.value(recoveryDataWriteDuration); cpHistogram.value(duration); } 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 8b2c02a72dee5..9fb89cc5c2cfa 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java @@ -66,9 +66,9 @@ import org.apache.ignite.internal.mem.DirectMemoryProvider; import org.apache.ignite.internal.mem.DirectMemoryRegion; import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp; +import org.apache.ignite.internal.pagemem.FullPageId; import org.apache.ignite.internal.pagemem.PageIdAllocator; import org.apache.ignite.internal.pagemem.PageMemory; -import org.apache.ignite.internal.pagemem.PageUtils; import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager; import org.apache.ignite.internal.pagemem.store.PageStore; import org.apache.ignite.internal.pagemem.wal.WALIterator; @@ -106,6 +106,7 @@ import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointListener; import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager; import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointProgress; +import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointRecoveryFile; import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointStatus; import org.apache.ignite.internal.processors.cache.persistence.checkpoint.Checkpointer; import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager; @@ -125,6 +126,7 @@ import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId; import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO; +import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler; import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; import org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDataIntegrityViolationException; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; @@ -139,6 +141,7 @@ import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.StripedExecutor; import org.apache.ignite.internal.util.TimeBag; +import org.apache.ignite.internal.util.future.CountDownFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T2; @@ -1183,7 +1186,7 @@ private long[] calculateFragmentSizes(String regionName, int concLvl, long cache long cacheSize = regCfg.getMaxSize(); // Checkpoint buffer size can not be greater than cache size, it does not make sense. - long chpBufSize = checkpointBufferSize(regCfg); + long chpBufSize = checkpointBufferSize(dsCfg, regCfg); if (chpBufSize > cacheSize) { U.quietAndInfo(log, @@ -2126,9 +2129,98 @@ private RestoreBinaryState performBinaryMemoryRestore( ", walArchive=" + persistenceCfg.getWalArchivePath() + "]"); } + long start = U.currentTimeMillis(); + + AtomicLong applied = new AtomicLong(); + CacheStripedExecutor exec = new CacheStripedExecutor(cctx.kernalContext().pools().getStripedExecutorService()); - long start = U.currentTimeMillis(); + boolean restoredFromCheckpointRecoveryFiles = false; + + // Try to restore from checkpoint recovery files. + if (apply) { + List recoveryFiles = checkpointManager.checkpointRecoveryFiles(status.cpStartId); + + boolean useRecoveryFiles = cctx.kernalContext().config().getDataStorageConfiguration() + .isWriteRecoveryDataOnCheckpoint(); + + if (useRecoveryFiles && recoveryFiles.isEmpty()) { + throw new StorageException("Failed to restore memory state. Checkpoint recovery files are expected " + + "to exist, but not found (this can happen due to WriteRecoveryDataOnCheckpoint property change " + + "right after node crash or if files were manually deleted)"); + } + else if (!useRecoveryFiles && !recoveryFiles.isEmpty()) { + throw new StorageException("Failed to restore memory state. Checkpoint recovery files are not " + + "expected to exist, but found (this can happen due to WriteRecoveryDataOnCheckpoint property " + + "change right after node crash)"); + } + + if (!recoveryFiles.isEmpty() ) { + if (log.isInfoEnabled()) { + recoveryFiles.sort(Comparator.comparing(CheckpointRecoveryFile::checkpointerIndex)); + + String files = recoveryFiles.size() == 1 ? recoveryFiles.get(0).file().getName() : + recoveryFiles.get(0).file().getName() + " .. " + + recoveryFiles.get(recoveryFiles.size() - 1).file().getName(); + + log.info("Start physical recovery from checkpoint recovery files [" + files + ']'); + } + + CountDownFuture cpRecoveryFut = new CountDownFuture(recoveryFiles.size()); + + recoveryFiles.forEach(cpRecoveryFile -> + exec.submit(() -> { + Throwable err = null; + + try { + cpRecoveryFile.forAllPages(id -> cacheGroupsPredicate.apply(id.groupId()), + (fullPageId, buf) -> { + if (skipRemovedIndexUpdates(fullPageId.groupId(), partId(fullPageId.pageId()))) + return; + + try { + PageMemoryEx pageMem = getPageMemoryForCacheGroup(fullPageId.groupId()); + + if (pageMem == null) + return; + + applyPage(pageMem, fullPageId, buf); + + applied.incrementAndGet(); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + }); + } + catch (Throwable e) { + err = e; + } + finally { + try { + cpRecoveryFile.close(); + } + catch (Exception e) { + if (err == null) + err = e; + else + err.addSuppressed(e); + } + cpRecoveryFut.onDone(err); + } + // Use file index instead of grpId to define stripe of stripped executor. + }, cpRecoveryFile.checkpointerIndex(), 0) + ); + + cpRecoveryFut.get(); + + restoredFromCheckpointRecoveryFiles = true; + + // Fall through to restore from WAL after restoring from checkpoint recovery files. + // There will be no page snapshot records and page delta records in WAL in this case, but we still + // need to apply PART_META_UPDATE_STATE/PARTITION_DESTROY records. + } + } long lastArchivedSegment = cctx.wal().lastArchivedSegment(); @@ -2136,8 +2228,6 @@ private RestoreBinaryState performBinaryMemoryRestore( RestoreBinaryState restoreBinaryState = new RestoreBinaryState(status, it, lastArchivedSegment, cacheGroupsPredicate); - AtomicLong applied = new AtomicLong(); - try { while (restoreBinaryState.hasNext()) { if (exec.error()) @@ -2151,6 +2241,8 @@ private RestoreBinaryState performBinaryMemoryRestore( switch (rec.type()) { case PAGE_RECORD: if (restoreBinaryState.needApplyBinaryUpdate()) { + assert !restoredFromCheckpointRecoveryFiles; + PageSnapshot pageSnapshot = (PageSnapshot)rec; // Here we do not require tag check because we may be applying memory changes after @@ -2229,6 +2321,8 @@ private RestoreBinaryState performBinaryMemoryRestore( default: if (restoreBinaryState.needApplyBinaryUpdate() && rec instanceof PageDeltaRecord) { + assert !restoredFromCheckpointRecoveryFiles; + PageDeltaRecord pageDelta = (PageDeltaRecord)rec; int grpId = pageDelta.groupId(); @@ -2311,9 +2405,19 @@ public void stripedApplyPage( * @param pageSnapshotRecord Page snapshot record. * @throws IgniteCheckedException If failed. */ - public void applyPageSnapshot(PageMemoryEx pageMem, PageSnapshot pageSnapshotRecord) throws IgniteCheckedException { - int grpId = pageSnapshotRecord.fullPageId().groupId(); - long pageId = pageSnapshotRecord.fullPageId().pageId(); + private void applyPageSnapshot(PageMemoryEx pageMem, PageSnapshot pageSnapshotRecord) throws IgniteCheckedException { + applyPage(pageMem, pageSnapshotRecord.fullPageId(), pageSnapshotRecord.pageDataBuffer()); + } + + /** + * @param pageMem Page memory. + * @param fullPageId Full page ID. + * @param buf Page buffer to apply. + * @throws IgniteCheckedException If failed. + */ + private void applyPage(PageMemoryEx pageMem, FullPageId fullPageId, ByteBuffer buf) throws IgniteCheckedException { + int grpId = fullPageId.groupId(); + long pageId = fullPageId.pageId(); long page = pageMem.acquirePage(grpId, pageId, IoStatisticsHolderNoOp.INSTANCE, true); @@ -2321,14 +2425,14 @@ public void applyPageSnapshot(PageMemoryEx pageMem, PageSnapshot pageSnapshotRec long pageAddr = pageMem.writeLock(grpId, pageId, page, true); try { - PageUtils.putBytes(pageAddr, 0, pageSnapshotRecord.pageData()); + ByteBuffer pageMemBuf = pageMem.pageBuffer(pageAddr); - if (PageIO.getCompressionType(pageAddr) != CompressionProcessor.UNCOMPRESSED_PAGE) { - int realPageSize = pageMem.realPageSize(pageSnapshotRecord.groupId()); + PageHandler.copyMemory(buf, 0, pageMemBuf, 0, buf.remaining()); - assert pageSnapshotRecord.pageDataSize() <= realPageSize : pageSnapshotRecord.pageDataSize(); + if (PageIO.getCompressionType(pageAddr) != CompressionProcessor.UNCOMPRESSED_PAGE) { + int realPageSize = pageMem.realPageSize(grpId); - cctx.kernalContext().compress().decompressPage(pageMem.pageBuffer(pageAddr), realPageSize); + cctx.kernalContext().compress().decompressPage(pageMemBuf, realPageSize); } } finally { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/Checkpoint.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/Checkpoint.java index bc842dd98dcf6..1ae38f2ad817d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/Checkpoint.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/Checkpoint.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.cache.persistence.checkpoint; import org.apache.ignite.internal.pagemem.FullPageId; +import org.apache.ignite.internal.pagemem.wal.record.CheckpointRecord; import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx; import org.apache.ignite.internal.util.GridConcurrentMultiPairQueue; import org.apache.ignite.lang.IgniteBiTuple; @@ -36,6 +37,9 @@ class Checkpoint { /** Checkpoint progress status. */ final CheckpointProgressImpl progress; + /** Checkpoint WAL record. */ + final CheckpointRecord cpRecord; + /** WAL segments fully covered by this checkpoint. */ IgniteBiTuple walSegsCoveredRange; @@ -48,15 +52,18 @@ class Checkpoint { * @param cpEntry Checkpoint entry. * @param cpPages Pages to write to the page store. * @param progress Checkpoint progress status. + * @param cpRecord Checkpoint WAL record. */ Checkpoint( @Nullable CheckpointEntry cpEntry, GridConcurrentMultiPairQueue cpPages, - CheckpointProgressImpl progress + CheckpointProgressImpl progress, + CheckpointRecord cpRecord ) { this.cpEntry = cpEntry; this.cpPages = cpPages; this.progress = progress; + this.cpRecord = cpRecord; pagesSize = cpPages.initialSize(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointManager.java index 854d1a16a6102..11273a6d00220 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointManager.java @@ -20,6 +20,7 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.Collection; +import java.util.List; import java.util.UUID; import java.util.concurrent.Executor; import java.util.function.Function; @@ -35,6 +36,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheProcessor; import org.apache.ignite.internal.processors.cache.persistence.DataRegion; import org.apache.ignite.internal.processors.cache.persistence.DataStorageMetricsImpl; +import org.apache.ignite.internal.processors.cache.persistence.StorageException; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cache.persistence.filename.NodeFileTree; @@ -79,6 +81,9 @@ public class CheckpointManager { /** Checkpoint markers storage which mark the start and end of each checkpoint. */ private final CheckpointMarkersStorage checkpointMarkersStorage; + /** Storage for checkpoint recovery files. */ + private final CheckpointRecoveryFileStorage checkpointRecoveryFileStorage; + /** Timeout checkpoint lock which should be used while write to memory happened. */ final CheckpointTimeoutLock checkpointTimeoutLock; @@ -178,6 +183,7 @@ public CheckpointManager( }; checkpointPagesWriterFactory = new CheckpointPagesWriterFactory( + cacheProcessor.context().kernalContext(), logger, (pageMemEx, fullPage, buf, tag) -> pageStoreManager.write(fullPage.groupId(), fullPage.pageId(), buf, tag, true), persStoreMetrics, @@ -185,6 +191,9 @@ public CheckpointManager( pageMemoryGroupResolver ); + checkpointRecoveryFileStorage = new CheckpointRecoveryFileStorage(cacheProcessor.context().kernalContext(), + ft.checkpoint(), ioFactory); + checkpointerProvider = () -> new Checkpointer( igniteInstanceName, checkpointThreadName, @@ -196,6 +205,7 @@ public CheckpointManager( cacheProcessor, checkpointWorkflow, checkpointPagesWriterFactory, + checkpointRecoveryFileStorage, persistenceCfg.getCheckpointFrequency(), persistenceCfg.getCheckpointThreads(), cpFreqDeviation @@ -301,6 +311,13 @@ public CheckpointHistory checkpointHistory() { return checkpointMarkersStorage.history(); } + /** + * @return List of checkpoint recovery files. + */ + public List checkpointRecoveryFiles(@Nullable UUID cpId) throws StorageException { + return checkpointRecoveryFileStorage.list(cpId == null ? null : cpId::equals); + } + /** * Initialize checkpoint storage. */ @@ -326,7 +343,7 @@ public void cleanupTempCheckpointDirectory() throws IgniteCheckedException { } /** - * Clean checkpoint directory {@link CheckpointMarkersStorage#cpDir}. The operation is necessary when local node joined to + * Clean checkpoint directory {@link NodeFileTree#checkpoint()}. The operation is necessary when local node joined to * baseline topology with different consistentId. */ public void cleanupCheckpointDirectory() throws IgniteCheckedException { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointMarkersStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointMarkersStorage.java index 8926dd2eda5e7..8c33ac2834906 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointMarkersStorage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointMarkersStorage.java @@ -449,6 +449,8 @@ private void removeCheckpointFiles(CheckpointEntry cpEntry) throws IgniteChecked } /** + * Writes checkpoint entry buffer {@code entryBuf} to specified checkpoint file with 2-phase protocol. + * * @param entryBuf Buffer which would be written to disk. * @param cp Prepared checkpoint entry. * @param type Type of checkpoint marker. @@ -492,7 +494,7 @@ private void writeCheckpointEntry( } /** - * Writes checkpoint entry buffer {@code entryBuf} to specified checkpoint file with 2-phase protocol. + * Creates and writes checkpoint entry to checkpoint marker file. * * @param cpTs Checkpoint timestamp. * @param cpId Checkpoint id. @@ -510,14 +512,28 @@ public CheckpointEntry writeCheckpointEntry( CheckpointEntryType type, boolean skipSync ) throws StorageException { - CheckpointEntry entry = prepareCheckpointEntry( - tmpWriteBuf, - cpTs, - cpId, - ptr, - rec, - type - ); + CheckpointEntry entry = createCheckPointEntry(cpTs, ptr, cpId, rec, type); + + writeCheckpointEntry(entry, rec, type, skipSync); + + return entry; + } + + /** + * Writes checkpoint entry to checkpoint marker file. + * + * @param entry Checkpoint entry. + * @param rec Checkpoint WAL record. + * @param type Checkpoint type. + * @throws StorageException If failed to write checkpoint entry. + */ + public void writeCheckpointEntry( + CheckpointEntry entry, + @Nullable CheckpointRecord rec, + CheckpointEntryType type, + boolean skipSync + ) throws StorageException { + prepareCheckpointEntryBuf(tmpWriteBuf, entry.checkpointMark()); if (type == CheckpointEntryType.START) cpHistory.addCheckpoint(entry, rec.cacheGroupStates()); @@ -525,30 +541,15 @@ public CheckpointEntry writeCheckpointEntry( writeCheckpointEntry(tmpWriteBuf, entry, type, skipSync); onEarliestCheckpointMapChanged(); - - return entry; } /** - * Prepares checkpoint entry containing WAL pointer to checkpoint record. Writes into given {@code ptrBuf} WAL - * pointer content. + * Writes into given {@code entryBuf} WAL pointer content. * * @param entryBuf Buffer to fill - * @param cpTs Checkpoint timestamp. - * @param cpId Checkpoint id. * @param ptr WAL pointer containing record. - * @param rec Checkpoint WAL record. - * @param type Checkpoint type. - * @return Checkpoint entry. */ - private CheckpointEntry prepareCheckpointEntry( - ByteBuffer entryBuf, - long cpTs, - UUID cpId, - WALPointer ptr, - @Nullable CheckpointRecord rec, - CheckpointEntryType type - ) { + private void prepareCheckpointEntryBuf(ByteBuffer entryBuf, WALPointer ptr) { assert ptr != null; entryBuf.rewind(); @@ -560,8 +561,6 @@ private CheckpointEntry prepareCheckpointEntry( entryBuf.putInt(ptr.length()); entryBuf.flip(); - - return createCheckPointEntry(cpTs, ptr, cpId, rec, type); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointPagesWriter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointPagesWriter.java index 9349ee1b38f69..3a508ca679558 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointPagesWriter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointPagesWriter.java @@ -194,7 +194,7 @@ private GridConcurrentMultiPairQueue writePages( PageStoreWriter pageStoreWriter = pageStoreWriters.computeIfAbsent(pageMem, pageMemEx -> createPageStoreWriter(pageMemEx, pagesToRetry)); - pageMem.checkpointWritePage(fullId, tmpWriteBuf, pageStoreWriter, tracker); + pageMem.checkpointWritePage(fullId, tmpWriteBuf, pageStoreWriter, tracker, false); if (throttlingEnabled) { while (pageMem.isCpBufferOverflowThresholdExceeded()) { @@ -205,7 +205,7 @@ private GridConcurrentMultiPairQueue writePages( tmpWriteBuf.rewind(); - pageMem.checkpointWritePage(cpPageId, tmpWriteBuf, pageStoreWriter, tracker); + pageMem.checkpointWritePage(cpPageId, tmpWriteBuf, pageStoreWriter, tracker, false); } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointPagesWriterFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointPagesWriterFactory.java index 4713b83e0e3b2..a261823a69c82 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointPagesWriterFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointPagesWriterFactory.java @@ -21,12 +21,14 @@ import java.util.Collection; import java.util.HashMap; import java.util.Map; +import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.LongAdder; import java.util.function.BooleanSupplier; import java.util.function.Function; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.pagemem.FullPageId; import org.apache.ignite.internal.pagemem.store.PageStore; import org.apache.ignite.internal.processors.cache.persistence.DataStorageMetricsImpl; @@ -38,6 +40,7 @@ import org.apache.ignite.internal.util.future.CountDownFuture; import org.apache.ignite.internal.util.lang.IgniteThrowableFunction; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.util.worker.WorkProgressDispatcher; import org.jsr166.ConcurrentLinkedHashMap; /** @@ -46,6 +49,9 @@ * It holds all dependency which is needed for creation of checkpoint writer and recovery checkpoint writer. */ public class CheckpointPagesWriterFactory { + /** Context. */ + private final GridKernalContext ctx; + /** Logger. */ private final IgniteLogger log; @@ -65,6 +71,7 @@ public class CheckpointPagesWriterFactory { private final CheckpointPagesWriter.CheckpointPageWriter checkpointPageWriter; /** + * @param ctx Context. * @param logger Logger. * @param checkpointPageWriter Checkpoint page writer. * @param persStoreMetrics Persistence metrics. @@ -73,6 +80,7 @@ public class CheckpointPagesWriterFactory { * @param pageMemoryGroupResolver Page memory resolver. */ CheckpointPagesWriterFactory( + GridKernalContext ctx, Function, IgniteLogger> logger, CheckpointPagesWriter.CheckpointPageWriter checkpointPageWriter, DataStorageMetricsImpl persStoreMetrics, @@ -80,6 +88,7 @@ public class CheckpointPagesWriterFactory { ThreadLocal threadBuf, IgniteThrowableFunction pageMemoryGroupResolver ) { + this.ctx = ctx; this.log = logger.apply(getClass()); this.persStoreMetrics = persStoreMetrics; this.threadBuf = threadBuf; @@ -98,7 +107,7 @@ public class CheckpointPagesWriterFactory { * @param shutdownNow Checker of stop operation. * @return Instance of page checkpint writer. */ - CheckpointPagesWriter build( + Runnable buildCheckpointPagesWriter( CheckpointMetricsTracker tracker, GridConcurrentMultiPairQueue cpPages, ConcurrentLinkedHashMap updStores, @@ -124,6 +133,38 @@ CheckpointPagesWriter build( ); } + /** + * @param recoveryDataFile File to write recovery data. + * @param cpPages List of pages to write. + * @param cacheGrpIds Set of cache groups to process (cache groups with WAL enabled). + * @param doneWriteFut Write done future. + * @param workProgressDispatcher Work progress dispatcher. + * @param curCpProgress Current checkpoint data. + * @param shutdownNow Checker of stop operation. + * @return Instance of page checkpint writer. + */ + Runnable buildRecoveryDataWriter( + CheckpointRecoveryFile recoveryDataFile, + GridConcurrentMultiPairQueue cpPages, + Set cacheGrpIds, + CountDownFuture doneWriteFut, + WorkProgressDispatcher workProgressDispatcher, + CheckpointProgressImpl curCpProgress, + BooleanSupplier shutdownNow + ) { + return new RecoveryDataWriter( + ctx, + recoveryDataFile, + cpPages, + cacheGrpIds, + doneWriteFut, + workProgressDispatcher, + log, + curCpProgress, + shutdownNow + ); + } + /** * @param pages List of pages to write. * @param updStores Updated page store storage. @@ -131,7 +172,7 @@ CheckpointPagesWriter build( * @param cpPagesCnt Count of checkpointed pages. * @return Instance of page checkpint writer. */ - Runnable buildRecovery( + Runnable buildRecoveryFinalizer( GridConcurrentMultiPairQueue pages, Collection updStores, AtomicReference writePagesError, @@ -168,7 +209,7 @@ Runnable buildRecovery( // Write page content to page store via pageStoreWriter. // Tracker is null, because no need to track checkpoint metrics on recovery. - pageMem.checkpointWritePage(res.getValue(), tmpWriteBuf, pageStoreWriter, null); + pageMem.checkpointWritePage(res.getValue(), tmpWriteBuf, pageStoreWriter, null, false); // Add number of handled pages. pagesWritten++; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointProgress.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointProgress.java index 61bfe538275c3..18cb1716be5c0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointProgress.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointProgress.java @@ -78,6 +78,11 @@ public interface CheckpointProgress { */ public AtomicInteger evictedPagesCounter(); + /** + * @return Counter for written recovery pages during current checkpoint. Not null only if checkpoint is running. + */ + public AtomicInteger writtenRecoveryPagesCounter(); + /** * @return Number of pages in current checkpoint. If checkpoint is not running, returns 0. */ @@ -114,6 +119,13 @@ public interface CheckpointProgress { */ public void updateEvictedPages(int delta); + /** + * Update written recovery pages counter. + * + * @param delta Pages num to update. + */ + public void updateWrittenRecoveryPages(int delta); + /** Clear cp progress counters. */ public void clearCounters(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointProgressImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointProgressImpl.java index 196954b555808..a26aa397c09c7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointProgressImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointProgressImpl.java @@ -62,6 +62,9 @@ public class CheckpointProgressImpl implements CheckpointProgress { /** Counter for evicted checkpoint pages. Not null only if checkpoint is running. */ private volatile AtomicInteger evictedPagesCntr; + /** Counter for written recovery pages. Not null only if checkpoint is running. */ + private volatile AtomicInteger writtenRecoveryPagesCntr; + /** Number of pages in current checkpoint at the beginning of checkpoint. */ private volatile int currCheckpointPagesCnt; @@ -185,10 +188,10 @@ public void reason(String reason) { } /** {@inheritDoc} */ - @Override public void updateWrittenPages(int deltha) { - A.ensure(deltha > 0, "param must be positive"); + @Override public void updateWrittenPages(int delta) { + A.ensure(delta > 0, "param must be positive"); - writtenPagesCntr.addAndGet(deltha); + writtenPagesCntr.addAndGet(delta); } /** {@inheritDoc} */ @@ -197,10 +200,10 @@ public void reason(String reason) { } /** {@inheritDoc} */ - @Override public void updateSyncedPages(int deltha) { - A.ensure(deltha > 0, "param must be positive"); + @Override public void updateSyncedPages(int delta) { + A.ensure(delta > 0, "param must be positive"); - syncedPagesCntr.addAndGet(deltha); + syncedPagesCntr.addAndGet(delta); } /** {@inheritDoc} */ @@ -212,8 +215,25 @@ public void reason(String reason) { @Override public void updateEvictedPages(int delta) { A.ensure(delta > 0, "param must be positive"); - if (evictedPagesCounter() != null) - evictedPagesCounter().addAndGet(delta); + AtomicInteger cntr = evictedPagesCounter(); + + if (cntr != null) + cntr.addAndGet(delta); + } + + /** {@inheritDoc} */ + @Override public AtomicInteger writtenRecoveryPagesCounter() { + return writtenRecoveryPagesCntr; + } + + /** {@inheritDoc} */ + @Override public void updateWrittenRecoveryPages(int delta) { + A.ensure(delta > 0, "param must be positive"); + + AtomicInteger cntr = writtenRecoveryPagesCounter(); + + if (cntr != null) + cntr.addAndGet(delta); } /** {@inheritDoc} */ @@ -233,6 +253,7 @@ public void reason(String reason) { writtenPagesCntr = new AtomicInteger(); syncedPagesCntr = new AtomicInteger(); evictedPagesCntr = new AtomicInteger(); + writtenRecoveryPagesCntr = new AtomicInteger(); } /** {@inheritDoc} */ @@ -242,6 +263,7 @@ public void reason(String reason) { writtenPagesCntr = null; syncedPagesCntr = null; evictedPagesCntr = null; + writtenRecoveryPagesCntr = null; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointRecoveryFile.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointRecoveryFile.java new file mode 100644 index 0000000000000..0400c68d091cf --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointRecoveryFile.java @@ -0,0 +1,312 @@ +/* + * 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.checkpoint; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.UUID; +import java.util.function.BiConsumer; +import java.util.function.Predicate; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.managers.encryption.GroupKey; +import org.apache.ignite.internal.pagemem.FullPageId; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; +import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.spi.encryption.EncryptionSpi; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.configuration.DataStorageConfiguration.MAX_PAGE_SIZE; + +/** + * File for recovery on failure during checkpoint. + * Not thread safe. + */ +public class CheckpointRecoveryFile implements AutoCloseable { + /** Record header size. */ + private static final int HEADER_SIZE = + 8 /* pageId */ + + 4 /* grpId */ + + 4 /* pageSize */ + + 4 /* CRC */ + + 1 /* encrypted flag */ + + 1 /* encryption key id */; + + /** Record header buffer. */ + private final ByteBuffer hdrBuf = ByteBuffer.allocateDirect(HEADER_SIZE).order(ByteOrder.nativeOrder()); + + /** Buffer for encrypted data. */ + private final ByteBuffer encBuf; + + /** Buffer for file writes caching. */ + private final ByteBuffer writeBuf; + + /** Context. */ + private final GridKernalContext ctx; + + /** Encryption SPI */ + private final EncryptionSpi encSpi; + + /** File IO to read/write recovery data. */ + private final FileIO fileIo; + + /** Checkpoint timestamp. */ + private final long cpTs; + + /** Checkpoint ID. */ + private final UUID cpId; + + /** Checkpointer index. */ + private final int cpIdx; + + /** */ + private final File file; + + /** + * Ctor. + */ + CheckpointRecoveryFile(GridKernalContext ctx, long cpTs, UUID cpId, int cpIdx, File file, FileIO fileIo) { + this.ctx = ctx; + this.cpTs = cpTs; + this.cpId = cpId; + this.cpIdx = cpIdx; + this.file = file; + this.fileIo = fileIo; + + writeBuf = ByteBuffer.allocateDirect(ctx.cache().context().database().pageSize()).order(ByteOrder.nativeOrder()); + encSpi = ctx.encryption().enabled() ? ctx.config().getEncryptionSpi() : null; + encBuf = encSpi != null ? ByteBuffer.allocateDirect(encSpi.encryptedSize(MAX_PAGE_SIZE)) : null; + } + + /** Checkpoint timestamp. */ + public long checkpointTs() { + return cpTs; + } + + /** Checkpoint ID. */ + public UUID checkpointId() { + return cpId; + } + + /** Checkpointer index. */ + public int checkpointerIndex() { + return cpIdx; + } + + /** Recovery file. */ + public File file() { + return file; + } + + /** */ + public void fsync() throws IOException { + if (writeBuf.position() > 0) { + writeBuf.flip(); + fileIo.writeFully(writeBuf); + writeBuf.clear(); + } + + fileIo.force(); + } + + /** {@inheritDoc} */ + @Override public void close() throws Exception { + if (writeBuf.position() > 0) { + writeBuf.flip(); + fileIo.writeFully(writeBuf); + } + + fileIo.close(); + } + + /** */ + private void write(ByteBuffer buf) throws IOException { + do { + if (buf.remaining() > writeBuf.remaining()) { + int oldLimit = buf.limit(); + + buf.limit(buf.position() + writeBuf.remaining()); + + writeBuf.put(buf); + + buf.limit(oldLimit); + } + else + writeBuf.put(buf); + + if (writeBuf.remaining() == 0) { + writeBuf.rewind(); + + fileIo.writeFully(writeBuf); + + writeBuf.clear(); + } + } + while (buf.remaining() > 0); + } + + /** */ + public void writePage(FullPageId fullPageId, ByteBuffer buf) throws IOException { + // Encrypt if required. + byte encFlag = 0; + byte encKeyId = 0; + + if (encSpi != null) { + GroupKey grpKey = ctx.encryption().getActiveKey(fullPageId.groupId()); + + if (grpKey != null) { + encFlag = 1; + encKeyId = grpKey.id(); + encBuf.clear(); + encBuf.limit(encSpi.encryptedSize(buf.remaining())); + encSpi.encrypt(buf, grpKey.key(), encBuf); + encBuf.rewind(); + buf = encBuf; + } + } + + hdrBuf.clear(); + hdrBuf.putLong(fullPageId.pageId()); + hdrBuf.putInt(fullPageId.groupId()); + hdrBuf.putInt(buf.remaining()); + // We have dedicated CRC field in the page structure, but we intentionally store CRC to record header, since + // page buffer can be encrypted. + hdrBuf.putInt(FastCrc.calcCrc(buf, buf.remaining())); + hdrBuf.put(encFlag); + hdrBuf.put(encKeyId); + + hdrBuf.rewind(); + buf.rewind(); + + write(hdrBuf); + write(buf); + } + + /** */ + private @Nullable FullPageId readPage(Predicate pageIdPredicate, ByteBuffer buf) throws IOException { + // Read header. + hdrBuf.clear(); + long pos = fileIo.position(); // For error messages. + + int read = fileIo.readFully(hdrBuf); + + if (read <= 0) + return null; + + if (read < hdrBuf.capacity()) { + throw new IOException("Recovery file buffer underflow [file=" + file.getName() + + ", pos=" + pos + ", expSize=" + hdrBuf.capacity() + ", read=" + read + ']'); + } + + hdrBuf.rewind(); + FullPageId fullPageId = new FullPageId(hdrBuf.getLong(), hdrBuf.getInt()); + int pageSize = hdrBuf.getInt(); + int storedCrc = hdrBuf.getInt(); + byte encFlag = hdrBuf.get(); + byte encKeyId = hdrBuf.get(); + + if (pageSize <= 0 || pageSize + fileIo.position() > fileIo.size()) { + throw new IOException("Unexpected page size [file=" + file.getName() + + ", pos=" + pos + ", pageSize=" + pageSize + ']'); + } + + if (!pageIdPredicate.test(fullPageId)) { + fileIo.position(fileIo.position() + pageSize); + buf.clear(); + buf.limit(0); + return fullPageId; + } + + ByteBuffer decBuf = buf; // Buffer for decrypted data. + + if (encFlag != 0) + buf = encBuf; + + if (pageSize > buf.capacity()) { + throw new IOException("Unexpected page size [file=" + file.getName() + + ", pos=" + pos + ", pageSize=" + pageSize + ']'); + } + + // Read page data. + buf.clear(); + buf.limit(pageSize); + read = fileIo.readFully(buf); + + if (read < pageSize) { + throw new IOException("Recovery file buffer underflow [file=" + file.getName() + + ", pos=" + pos + ", expSize=" + pageSize + ", read=" + read + ']'); + } + + buf.rewind(); + + int calcCrc = FastCrc.calcCrc(buf, buf.remaining()); + + if (storedCrc != calcCrc) { + throw new IOException("CRC validation failed [file=" + file.getName() + ", pos=" + pos + + ", storedCrc=" + U.hexInt(storedCrc) + ", calcCrc=" + U.hexInt(calcCrc) + "]"); + } + + buf.rewind(); + + if (encFlag != 0) { + if (encSpi == null) { + throw new IOException("Found encrypted record, but encryption is disabled [file=" + + file.getName() + ", pos=" + pos + ']'); + } + + GroupKey grpKey = ctx.encryption().groupKey(fullPageId.groupId(), encKeyId); + + if (grpKey == null) { + throw new IOException("Not found encryption key id [file=" + + file.getName() + ", pos=" + pos + ", grpId=" + fullPageId.groupId() + ", keyId=" + encKeyId + ']'); + } + + decBuf.clear(); + encSpi.decrypt(buf, grpKey.key(), decBuf); + decBuf.limit(decBuf.position()); + decBuf.rewind(); + } + + return fullPageId; + } + + /** + * Preforms action on all pages stored in recovery file. + */ + public void forAllPages( + Predicate pageIdPredicate, + BiConsumer action + ) throws IOException { + fileIo.position(0); + + ByteBuffer buf = ByteBuffer.allocateDirect(MAX_PAGE_SIZE).order(ByteOrder.nativeOrder()); + + FullPageId fullPageId = readPage(pageIdPredicate, buf); + + while (fullPageId != null) { + // Correct fullPageId (but with empty buffer) still will be returned, even when fullPageId + // doesn't satisfy pageIdPredicate. + if (pageIdPredicate.test(fullPageId)) + action.accept(fullPageId, buf); + + fullPageId = readPage(pageIdPredicate, buf); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointRecoveryFileStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointRecoveryFileStorage.java new file mode 100644 index 0000000000000..37a676ed47e3f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointRecoveryFileStorage.java @@ -0,0 +1,135 @@ +/* + * 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.checkpoint; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.function.Predicate; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.processors.cache.persistence.StorageException; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; +import org.jetbrains.annotations.Nullable; + +import static java.nio.file.StandardOpenOption.CREATE; +import static java.nio.file.StandardOpenOption.READ; +import static java.nio.file.StandardOpenOption.TRUNCATE_EXISTING; +import static java.nio.file.StandardOpenOption.WRITE; + +/** + * + */ +public class CheckpointRecoveryFileStorage { + /** */ + private static final String FILE_NAME_TAG = "-RECOVERY-"; + + /** Checkpoint recovery file name pattern. */ + public static final Pattern FILE_NAME_PATTERN = Pattern.compile("(\\d+)-(.*)" + FILE_NAME_TAG + "(\\d+)\\.bin"); + + /** Context. */ + private final GridKernalContext ctx; + + /** + * File IO factory. + */ + private final FileIOFactory fileIoFactory; + + /** + * Recovery file location. + */ + private final File dir; + + /** + * Ctor. + */ + public CheckpointRecoveryFileStorage(GridKernalContext ctx, File dir, FileIOFactory fileIoFactory) { + this.ctx = ctx; + this.dir = dir; + this.fileIoFactory = fileIoFactory; + } + + /** */ + private static String fileName(long cpTs, UUID cpId, int idx) { + return cpTs + "-" + cpId + FILE_NAME_TAG + idx + ".bin"; + } + + /** + * Factory method. + */ + public CheckpointRecoveryFile create(long cpTs, UUID cpId, int idx) throws StorageException { + File file = new File(dir, fileName(cpTs, cpId, idx)); + + try { + FileIO fileIO = fileIoFactory.create(file, CREATE, TRUNCATE_EXISTING, WRITE); + + return new CheckpointRecoveryFile(ctx, cpTs, cpId, idx, file, fileIO); + } + catch (IOException e) { + throw new StorageException("Failed to create checkpoint recovery file [file=" + file + ']', e); + } + } + + /** + * Gets list of recovery files, satisfying given predicate (or all files in storage if predicate is null). + * + * @return List of recovery files. + */ + public List list(@Nullable Predicate predicate) throws StorageException { + File[] files = dir.listFiles(f -> f.isFile() && f.getName().contains(FILE_NAME_TAG)); + List fileList = new ArrayList<>(); + + for (File file : files) { + Matcher matcher = FILE_NAME_PATTERN.matcher(file.getName()); + if (matcher.matches()) { + long ts = Long.parseLong(matcher.group(1)); + UUID id = UUID.fromString(matcher.group(2)); + int idx = Integer.parseInt(matcher.group(3)); + + if (predicate == null || predicate.test(id)) { + try { + fileList.add(new CheckpointRecoveryFile(ctx, ts, id, idx, file, fileIoFactory.create(file, READ))); + } + catch (IOException e) { + throw new StorageException("Failed to open checkpoint recovery file [file=" + file + ']', e); + } + } + } + } + + return fileList; + } + + /** + * Deletes all recovery files in storage. + */ + public void clear() throws StorageException { + File[] files = dir.listFiles(f -> f.isFile() + && f.getName().contains(FILE_NAME_TAG) + && FILE_NAME_PATTERN.matcher(f.getName()).matches()); + + for (File file : files) { + if (!file.delete()) + throw new StorageException("Failed to delete checkpoint recovery file [file=" + file + ']'); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointWorkflow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointWorkflow.java index 733f60a9fd448..9862fa0db0364 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointWorkflow.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointWorkflow.java @@ -57,6 +57,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState; import org.apache.ignite.internal.processors.cache.persistence.DataRegion; +import org.apache.ignite.internal.processors.cache.persistence.StorageException; import org.apache.ignite.internal.processors.cache.persistence.pagemem.CheckpointMetricsTracker; import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx; import org.apache.ignite.internal.processors.cache.persistence.partstate.PartitionAllocationMap; @@ -65,8 +66,8 @@ import org.apache.ignite.internal.util.GridConcurrentMultiPairQueue; import org.apache.ignite.internal.util.GridMultiCollectionWrapper; import org.apache.ignite.internal.util.StripedExecutor; +import org.apache.ignite.internal.util.function.ThrowableSupplier; import org.apache.ignite.internal.util.future.GridCompoundFuture; -import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.U; @@ -215,6 +216,7 @@ private IgniteThreadPoolExecutor initializeCheckpointPool() { * @param curr Current checkpoint event info. * @param tracker Checkpoint metrics tracker. * @param workProgressDispatcher Work progress dispatcher. + * @param writeRecoveryData Write recovery data on checkpoint. * @return Checkpoint collected info. * @throws IgniteCheckedException if fail. */ @@ -222,7 +224,8 @@ public Checkpoint markCheckpointBegin( long cpTs, CheckpointProgressImpl curr, CheckpointMetricsTracker tracker, - WorkProgressDispatcher workProgressDispatcher + WorkProgressDispatcher workProgressDispatcher, + boolean writeRecoveryData ) throws IgniteCheckedException { Collection checkpointedRegions = dataRegions.get(); @@ -275,8 +278,24 @@ curr, new PartitionAllocationMap(), checkpointCollectPagesInfoPool, workProgress fillCacheGroupState(cpRec); - //There are allowable to replace pages only after checkpoint entry was stored to disk. - cpPagesHolder = beginAllCheckpoints(checkpointedRegions, curr.futureFor(MARKER_STORED_TO_DISK)); + IgniteInternalFuture markerStoredToDiskFut = curr.futureFor(MARKER_STORED_TO_DISK); + + // There are allowable to replace pages only after checkpoint entry was stored to disk. + ThrowableSupplier allowToReplace = writeRecoveryData + // If we write recovery data on checkpoint it's not safe to wait for MARKER_STORED_TO_DISK future, + // recovery data writers acquire page memory segments locks to write the pages, in the same time + // another thread can lock page memory segment for writing during page replacement and wait for + // marker stored to disk, so deadlock is possible. + ? () -> curr.greaterOrEqualTo(MARKER_STORED_TO_DISK) + : () -> { + // Uninterruptibly is important because otherwise in case of interrupt of client thread node + // would be stopped. + markerStoredToDiskFut.getUninterruptibly(); + + return true; + }; + + cpPagesHolder = beginAllCheckpoints(checkpointedRegions, allowToReplace); curr.currentCheckpointPagesCount(cpPagesHolder.pagesNum()); @@ -315,35 +334,35 @@ curr, new PartitionAllocationMap(), checkpointCollectPagesInfoPool, workProgress tracker.onWalCpRecordFsyncEnd(); - CheckpointEntry checkpointEntry = null; - - if (checkpointMarkersStorage != null) - checkpointEntry = checkpointMarkersStorage.writeCheckpointEntry( - cpTs, - cpRec.checkpointId(), - cpPtr, - cpRec, - CheckpointEntryType.START, - skipSync - ); - - curr.transitTo(MARKER_STORED_TO_DISK); - - tracker.onSplitAndSortCpPagesStart(); - GridConcurrentMultiPairQueue cpPages = splitAndSortCpPagesIfNeeded(cpPagesHolder); tracker.onSplitAndSortCpPagesEnd(); - return new Checkpoint(checkpointEntry, cpPages, curr); + CheckpointEntry cpEntry = new CheckpointEntry(cpTs, cpPtr, cpRec.checkpointId(), cpRec.cacheGroupStates()); + + return new Checkpoint(cpEntry, cpPages, curr, cpRec); } else { if (ctx0.walFlush() && wal != null) wal.flush(null, true); - return new Checkpoint(null, GridConcurrentMultiPairQueue.EMPTY, curr); + return new Checkpoint(null, GridConcurrentMultiPairQueue.EMPTY, curr, cpRec); + } + } + + /** Stores begin checkpoint marker to disk. */ + public void storeBeginMarker(Checkpoint cp) throws StorageException { + if (checkpointMarkersStorage != null && cp.cpEntry != null) { + checkpointMarkersStorage.writeCheckpointEntry( + cp.cpEntry, + cp.cpRecord, + CheckpointEntryType.START, + skipSync + ); } + + cp.progress.transitTo(MARKER_STORED_TO_DISK); } /** @@ -415,8 +434,10 @@ private void fillCacheGroupState(CheckpointRecord cpRec) throws IgniteCheckedExc * @return holder of FullPageIds obtained from each PageMemory, overall number of dirty pages, and flag defines at * least one user page became a dirty since last checkpoint. */ - private CheckpointPagesInfoHolder beginAllCheckpoints(Collection regions, - IgniteInternalFuture allowToReplace) { + private CheckpointPagesInfoHolder beginAllCheckpoints( + Collection regions, + ThrowableSupplier allowToReplace + ) { Collection>> res = new ArrayList<>(regions.size()); @@ -610,7 +631,7 @@ public void finalizeCheckpointOnRecovery( Collection regions = dataRegions.get(); - CheckpointPagesInfoHolder cpPagesHolder = beginAllCheckpoints(regions, new GridFinishedFuture<>()); + CheckpointPagesInfoHolder cpPagesHolder = beginAllCheckpoints(regions, () -> Boolean.TRUE); // Sort and split all dirty pages set to several stripes. GridConcurrentMultiPairQueue pages = @@ -627,7 +648,7 @@ public void finalizeCheckpointOnRecovery( for (int stripeIdx = 0; stripeIdx < exec.stripesCount(); stripeIdx++) exec.execute( stripeIdx, - checkpointPagesWriterFactory.buildRecovery(pages, updStores, writePagesError, cpPagesCnt) + checkpointPagesWriterFactory.buildRecoveryFinalizer(pages, updStores, writePagesError, cpPagesCnt) ); // Await completion all write tasks. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/Checkpointer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/Checkpointer.java index e09d91451a027..46f3e9814072d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/Checkpointer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/Checkpointer.java @@ -17,9 +17,12 @@ package org.apache.ignite.internal.processors.cache.persistence.checkpoint; +import java.io.File; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.UUID; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.RejectedExecutionException; @@ -34,6 +37,7 @@ import org.apache.ignite.IgniteLogger; 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.IgniteInterruptedCheckedException; import org.apache.ignite.internal.LongJVMPauseDetector; @@ -46,6 +50,7 @@ import org.apache.ignite.internal.processors.cache.persistence.DataStorageMetricsImpl; import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager; +import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage; import org.apache.ignite.internal.processors.cache.persistence.pagemem.CheckpointMetricsTracker; import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx; import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; @@ -105,8 +110,9 @@ public class Checkpointer extends GridWorker { "checkpointListenersExecuteTime=%dms, " + "checkpointLockHoldTime=%dms, " + "walCpRecordFsyncDuration=%dms, " + - "writeCheckpointEntryDuration=%dms, " + "splitAndSortCpPagesDuration=%dms, " + + "writeRecoveryDataDuration=%dms, " + + "writeCheckpointEntryDuration=%dms, " + "%s" + "pages=%d, " + "reason='%s']"; @@ -149,6 +155,9 @@ public class Checkpointer extends GridWorker { /** Factory for the creation of page-write workers. */ private final CheckpointPagesWriterFactory checkpointPagesWriterFactory; + /** Storage for checkpoint recovery files. */ + @Nullable private final CheckpointRecoveryFileStorage checkpointRecoveryFileStorage; + /** The number of IO-bound threads which will write pages to disk. */ private final int checkpointWritePageThreads; @@ -173,6 +182,9 @@ public class Checkpointer extends GridWorker { /** Performance statistics processor. */ private final PerformanceStatisticsProcessor psproc; + /** Write recovery data during checkpoint. */ + private final boolean writeRecoveryData; + /** For testing only. */ private GridFutureAdapter enableChangeApplied; @@ -205,6 +217,7 @@ public class Checkpointer extends GridWorker { GridCacheProcessor cacheProcessor, CheckpointWorkflow checkpoint, CheckpointPagesWriterFactory factory, + @Nullable CheckpointRecoveryFileStorage checkpointRecoveryFileStorage, long checkpointFrequency, int checkpointWritePageThreads, Supplier cpFreqDeviation @@ -220,7 +233,10 @@ public class Checkpointer extends GridWorker { this.checkpointWritePageThreads = Math.max(checkpointWritePageThreads, 1); this.checkpointWritePagesPool = initializeCheckpointPool(); this.cpFreqDeviation = cpFreqDeviation; - this.psproc = cacheProcessor.context().kernalContext().performanceStatistics(); + GridKernalContext ctx = cacheProcessor.context().kernalContext(); + this.psproc = ctx.performanceStatistics(); + this.writeRecoveryData = ctx.config().getDataStorageConfiguration().isWriteRecoveryDataOnCheckpoint(); + this.checkpointRecoveryFileStorage = checkpointRecoveryFileStorage; scheduledCp = new CheckpointProgressImpl(nextCheckpointInterval()); } @@ -385,7 +401,56 @@ private void doCheckpoint() { startCheckpointProgress(); try { - chp = checkpointWorkflow.markCheckpointBegin(lastCpTs, curCpProgress, tracker, this); + chp = checkpointWorkflow.markCheckpointBegin(lastCpTs, curCpProgress, tracker, this, + writeRecoveryData); + + tracker.onMarkEnd(); + + currentProgress().initCounters(chp.pagesSize); + + long recoveryDataSize = 0; + + try { + if (checkpointRecoveryFileStorage != null && chp.hasDelta()) { + checkpointRecoveryFileStorage.clear(); + + if (writeRecoveryData) { + if (log.isInfoEnabled()) { + log.info(String.format("Checkpoint recovery data write started [" + + "checkpointId=%s, " + + "startPtr=%s, " + + "pages=%d, " + + "checkpointBeforeLockTime=%dms, " + + "checkpointLockWait=%dms, " + + "checkpointListenersExecuteTime=%dms, " + + "checkpointLockHoldTime=%dms, " + + "walCpRecordFsyncDuration=%dms, " + + "splitAndSortCpPagesDuration=%dms, " + + "reason='%s']", + chp.cpEntry == null ? "" : chp.cpEntry.checkpointId(), + chp.cpEntry == null ? "" : chp.cpEntry.checkpointMark(), + chp.pagesSize, + tracker.beforeLockDuration(), + tracker.lockWaitDuration(), + tracker.listenersExecuteDuration(), + tracker.lockHoldDuration(), + tracker.walCpRecordFsyncDuration(), + tracker.splitAndSortCpPagesDuration(), + chp.progress.reason() + )); + } + + recoveryDataSize = writeRecoveryData(chp); + } + } + } + finally { + tracker.onWriteRecoveryDataEnd(recoveryDataSize); + } + + checkpointWorkflow.storeBeginMarker(chp); + + tracker.onCpMarkerStoreEnd(); } catch (Exception e) { if (curCpProgress != null) @@ -399,8 +464,6 @@ private void doCheckpoint() { updateHeartbeat(); - currentProgress().initCounters(chp.pagesSize); - if (chp.hasDelta()) { if (log.isInfoEnabled()) { long possibleJvmPauseDur = possibleLongJvmPauseDuration(tracker); @@ -415,8 +478,9 @@ private void doCheckpoint() { tracker.listenersExecuteDuration(), tracker.lockHoldDuration(), tracker.walCpRecordFsyncDuration(), - tracker.writeCheckpointEntryDuration(), tracker.splitAndSortCpPagesDuration(), + tracker.recoveryDataWriteDuration(), + tracker.writeCheckpointEntryDuration(), possibleJvmPauseDur > 0 ? "possibleJvmPauseDuration=" + possibleJvmPauseDur + "ms, " : "", chp.pagesSize, chp.progress.reason() @@ -454,12 +518,14 @@ private void doCheckpoint() { if (chp.hasDelta() || destroyedPartitionsCnt > 0) { if (log.isInfoEnabled()) { log.info(String.format("Checkpoint finished [cpId=%s, pages=%d, markPos=%s, " + - "walSegmentsCovered=%s, markDuration=%dms, pagesWrite=%dms, fsync=%dms, total=%dms]", + "walSegmentsCovered=%s, markDuration=%dms, recoveryWrite=%dms, pagesWrite=%dms, " + + "fsync=%dms, total=%dms]", chp.cpEntry != null ? chp.cpEntry.checkpointId() : "", chp.pagesSize, chp.cpEntry != null ? chp.cpEntry.checkpointMark() : "", walRangeStr(chp.walSegsCoveredRange), tracker.markDuration(), + tracker.recoveryDataWriteDuration(), tracker.pagesWriteDuration(), tracker.fsyncDuration(), tracker.totalDuration())); @@ -475,6 +541,62 @@ private void doCheckpoint() { } } + /** + * Writes data required for storage recovery in case of crash during write pages phase. + * + * @param cp Current checkpoint. + * @return Size of written recovery data. + */ + long writeRecoveryData(Checkpoint cp) throws IgniteCheckedException { + IgniteThreadPoolExecutor pageWritePool = checkpointWritePagesPool; + + int threads = pageWritePool == null ? 1 : pageWritePool.getMaximumPoolSize(); + + // Cache group to process: cache groups with enabled WAL and reserved system group placeholders (without context). + Set cacheGrpIds = new HashSet<>(cp.cpRecord.cacheGroupStates().keySet()); + cacheGrpIds.add(MetaStorage.METASTORAGE_CACHE_ID); + + CountDownFuture doneFut = new CountDownFuture(threads); + List files = new ArrayList<>(threads); + + for (int i = 0; i < threads; i++) { + CheckpointRecoveryFile file = checkpointRecoveryFileStorage.create(cp.cpEntry.timestamp(), cp.cpEntry.checkpointId(), i); + files.add(file.file()); + + Runnable write = checkpointPagesWriterFactory.buildRecoveryDataWriter( + file, + cp.cpPages, + cacheGrpIds, + doneFut, + this, + cp.progress, + this::isShutdownNow + ); + + if (pageWritePool == null) + write.run(); + else { + try { + pageWritePool.execute(write); + } + catch (RejectedExecutionException ignore) { + // Run the task synchronously. + write.run(); + } + } + } + + // Wait and check for errors. + doneFut.get(); + + long recoveryDataSize = 0; + + for (File file : files) + recoveryDataSize += file.length(); + + return recoveryDataSize; + } + /** * @param workProgressDispatcher Work progress dispatcher. * @param tracker Checkpoint metrics tracker. @@ -489,6 +611,8 @@ boolean writePages( WorkProgressDispatcher workProgressDispatcher, BooleanSupplier shutdownNow ) throws IgniteCheckedException { + cpPages.rewind(); + IgniteThreadPoolExecutor pageWritePool = checkpointWritePagesPool; int checkpointWritePageThreads = pageWritePool == null ? 1 : pageWritePool.getMaximumPoolSize(); @@ -501,7 +625,7 @@ boolean writePages( tracker.onPagesWriteStart(); for (int i = 0; i < checkpointWritePageThreads; i++) { - Runnable write = checkpointPagesWriterFactory.build( + Runnable write = checkpointPagesWriterFactory.buildCheckpointPagesWriter( tracker, cpPages, updStores, @@ -580,6 +704,7 @@ private void updateMetrics(Checkpoint chp, CheckpointMetricsTracker tracker) { tracker.walCpRecordFsyncDuration(), tracker.writeCheckpointEntryDuration(), tracker.splitAndSortCpPagesDuration(), + tracker.recoveryDataWriteDuration(), tracker.totalDuration(), tracker.checkpointStartTime(), chp.pagesSize, @@ -601,11 +726,13 @@ private void updateMetrics(Checkpoint chp, CheckpointMetricsTracker tracker) { tracker.walCpRecordFsyncDuration(), tracker.writeCheckpointEntryDuration(), tracker.splitAndSortCpPagesDuration(), + tracker.recoveryDataWriteDuration(), tracker.totalDuration(), tracker.checkpointStartTime(), chp.pagesSize, tracker.dataPagesWritten(), tracker.cowPagesWritten(), + tracker.recoveryDataSize(), dbMgr.forAllPageStores(PageStore::size), dbMgr.forAllPageStores(PageStore::getSparseSize) ); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/LightweightCheckpointManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/LightweightCheckpointManager.java index 8a82c6e626f42..6f427b716c64c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/LightweightCheckpointManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/LightweightCheckpointManager.java @@ -129,6 +129,7 @@ public LightweightCheckpointManager( }; checkpointPagesWriterFactory = new CheckpointPagesWriterFactory( + cacheProcessor.context().kernalContext(), logger, (pageMemEx, fullPage, buf, tag) -> pageMemEx.pageManager().write(fullPage.groupId(), fullPage.pageId(), buf, tag, true), @@ -149,6 +150,7 @@ public LightweightCheckpointManager( cacheProcessor, checkpointWorkflow, checkpointPagesWriterFactory, + null, persistenceCfg.getCheckpointFrequency(), persistenceCfg.getCheckpointThreads(), () -> 0 diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/RecoveryDataWriter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/RecoveryDataWriter.java new file mode 100644 index 0000000000000..7a642b4064acd --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/RecoveryDataWriter.java @@ -0,0 +1,264 @@ +/* + * 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.checkpoint; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.BooleanSupplier; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.DiskPageCompression; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.NodeStoppingException; +import org.apache.ignite.internal.pagemem.FullPageId; +import org.apache.ignite.internal.processors.cache.persistence.PageStoreWriter; +import org.apache.ignite.internal.processors.cache.persistence.StorageException; +import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx; +import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImpl; +import org.apache.ignite.internal.processors.compress.CompressionProcessor; +import org.apache.ignite.internal.util.GridConcurrentMultiPairQueue; +import org.apache.ignite.internal.util.future.CountDownFuture; +import org.apache.ignite.internal.util.worker.WorkProgressDispatcher; + +import static org.apache.ignite.configuration.DataStorageConfiguration.MAX_PAGE_SIZE; +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.util.IgniteUtils.hexLong; + +/** + * Implementation of writer which able to store recovery data to disk during checkpoint. + */ +public class RecoveryDataWriter implements Runnable { + /** File to write recovery data. */ + private final CheckpointRecoveryFile file; + + /** Context. */ + private final GridKernalContext ctx; + + /** Logger. */ + private final IgniteLogger log; + + /** Collection of page IDs to write under this task. Overall pages to write may be greater than this collection. */ + private final GridConcurrentMultiPairQueue writePageIds; + + /** Set of cache groups to process. */ + private final Set cacheGrpIds; + + /** Future which should be finished when all pages would be written. */ + private final CountDownFuture doneFut; + + /** Work progress dispatcher. */ + private final WorkProgressDispatcher workProgressDispatcher; + + /** Current checkpoint. This field is updated only by checkpoint thread. */ + private final CheckpointProgressImpl curCpProgress; + + /** Shutdown now. */ + private final BooleanSupplier shutdownNow; + + /** Page data buffer. */ + private final ByteBuffer pageBuf = ByteBuffer.allocateDirect(MAX_PAGE_SIZE).order(ByteOrder.nativeOrder()); + + /** */ + private final DiskPageCompression compressionType; + + /** */ + private final int compressionLevel; + + /** + * Creates task for write recovery data. + * + * @param ctx Context. + * @param file Checkpoint recovery file. + * @param writePageIds Collection of page IDs to write. + * @param doneFut Done future. + * @param workProgressDispatcher Work progress dispatcher. + * @param log Logger. + * @param curCpProgress Checkpoint progress. + * @param shutdownNow Shutdown supplier. + */ + RecoveryDataWriter( + GridKernalContext ctx, + CheckpointRecoveryFile file, + GridConcurrentMultiPairQueue writePageIds, + Set cacheGrpIds, + CountDownFuture doneFut, + WorkProgressDispatcher workProgressDispatcher, + IgniteLogger log, + CheckpointProgressImpl curCpProgress, + BooleanSupplier shutdownNow + ) { + this.ctx = ctx; + this.file = file; + this.writePageIds = writePageIds; + this.cacheGrpIds = cacheGrpIds; + this.doneFut = doneFut; + this.workProgressDispatcher = workProgressDispatcher; + this.log = log; + this.curCpProgress = curCpProgress; + this.shutdownNow = shutdownNow; + + DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration(); + compressionType = dsCfg.getCheckpointRecoveryDataCompression(); + compressionLevel = CompressionProcessor.getCompressionLevel(dsCfg.getCheckpointRecoveryDataCompressionLevel(), + compressionType); + } + + /** {@inheritDoc} */ + @Override public void run() { + GridConcurrentMultiPairQueue writePageIds = this.writePageIds; + + Throwable err = null; + + try { + GridConcurrentMultiPairQueue pagesToRetry = writePages(writePageIds); + + if (!pagesToRetry.isEmpty()) { + if (log.isInfoEnabled()) { + log.info(pagesToRetry.initialSize() + " recovery pages were not written yet due to " + + "unsuccessful page write lock acquisition and will be retried"); + } + + while (!pagesToRetry.isEmpty()) + pagesToRetry = writePages(pagesToRetry); + } + + if (shutdownNow.getAsBoolean()) { + doneFut.onDone(new NodeStoppingException("Node is stopping.")); + + return; + } + + workProgressDispatcher.blockingSectionBegin(); + + try { + file.fsync(); + } + finally { + workProgressDispatcher.blockingSectionEnd(); + } + } + catch (Throwable e) { + err = e; + } + finally { + try { + file.close(); + } + catch (Exception e) { + if (err == null) + err = e; + else + err.addSuppressed(e); + } + doneFut.onDone(err); + } + } + + /** + * @param writePageIds Collections of pages to write. + * @return pagesToRetry Pages which should be retried. + */ + private GridConcurrentMultiPairQueue writePages( + GridConcurrentMultiPairQueue writePageIds + ) throws IgniteCheckedException { + Map> pagesToRetry = new HashMap<>(); + + Map pageStoreWriters = new HashMap<>(); + + GridConcurrentMultiPairQueue.Result res = + new GridConcurrentMultiPairQueue.Result<>(); + + while (writePageIds.next(res)) { + if (shutdownNow.getAsBoolean()) + break; + + workProgressDispatcher.updateHeartbeat(); + + FullPageId fullId = res.getValue(); + + PageMemoryEx pageMem = res.getKey(); + + if (!cacheGrpIds.contains(fullId.groupId())) + continue; + + pageBuf.rewind(); + pageBuf.limit(pageMem.pageSize()); + + PageStoreWriter pageStoreWriter = + pageStoreWriters.computeIfAbsent(pageMem, pageMemEx -> createPageStoreWriter(pageMemEx, pagesToRetry)); + + pageMem.checkpointWritePage(fullId, pageBuf, pageStoreWriter, null, true); + } + + return pagesToRetry.isEmpty() ? + GridConcurrentMultiPairQueue.EMPTY : + new GridConcurrentMultiPairQueue<>(pagesToRetry); + } + + /** + * Factory method for create {@link PageStoreWriter}. + * + * @param pageMemEx Page memory. + * @param pagesToRetry List pages for retry. + * @return Checkpoint page write context. + */ + private PageStoreWriter createPageStoreWriter( + PageMemoryEx pageMemEx, + Map> pagesToRetry + ) { + return new PageStoreWriter() { + /** {@inheritDoc} */ + @Override public void writePage(FullPageId fullPageId, ByteBuffer buf, int tag) throws IgniteCheckedException { + if (tag == PageMemoryImpl.TRY_AGAIN_TAG) { + pagesToRetry.computeIfAbsent(pageMemEx, k -> new ArrayList<>()).add(fullPageId); + + return; + } + + long pageId = fullPageId.pageId(); + + assert getType(buf) != 0 : "Invalid state. Type is 0! pageId = " + hexLong(pageId); + assert getVersion(buf) != 0 : "Invalid state. Version is 0! pageId = " + hexLong(pageId); + + buf.limit(pageMemEx.realPageSize(fullPageId.groupId())); + + if (compressionType != DiskPageCompression.DISABLED) { + buf = ctx.compress().compressPage(buf, buf.remaining(), 1, + compressionType, compressionLevel); + } + + try { + file.writePage(fullPageId, buf); + } + catch (IOException e) { + throw new StorageException("Failed to write page to recovery file [file=" + file.file() + ']', e); + } + + curCpProgress.updateWrittenRecoveryPages(1); + } + }; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java index 3ecdd22f962c9..aa925dfe8acad 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java @@ -66,7 +66,7 @@ public interface FileIO extends AutoCloseable { * * @param destBuf Destination byte buffer. * - * @return Number of written bytes. + * @return Number of read bytes. * * @throws IOException If some I/O error occurs. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/AbstractPagesWriteThrottle.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/AbstractPagesWriteThrottle.java new file mode 100644 index 0000000000000..2dcf522320c48 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/AbstractPagesWriteThrottle.java @@ -0,0 +1,87 @@ +/* + * 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.pagemem; + +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.processors.cache.persistence.CheckpointLockStateChecker; +import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointProgress; +import org.apache.ignite.lang.IgniteOutClosure; + +/** + * Abstract throttling policy + */ +public abstract class AbstractPagesWriteThrottle implements PagesWriteThrottlePolicy { + /** Page memory. */ + protected final PageMemoryImpl pageMemory; + + /** Checkpoint progress provider. */ + protected final IgniteOutClosure cpProgress; + + /** Checkpoint lock state checker. */ + protected final CheckpointLockStateChecker cpLockStateChecker; + + /** Checkpoint buffer protection logic. */ + protected final ThrottlingStrategy cpBufProtector; + + /** Checkpoint Buffer-related logic used to keep it safe. */ + protected final CheckpointBufferOverflowWatchdog cpBufWatchdog; + + /** Logger. */ + protected final IgniteLogger log; + + /** + * @param pageMemory Page memory. + * @param cpProgress Checkpoint progress provider. + * @param cpLockStateChecker checkpoint lock state checker. + * @param fillRateBasedCpBufProtection If true, fill rate based throttling will be used to protect from + * checkpoint buffer overflow. + * @param log Logger. + */ + protected AbstractPagesWriteThrottle( + PageMemoryImpl pageMemory, + IgniteOutClosure cpProgress, + CheckpointLockStateChecker cpLockStateChecker, + boolean fillRateBasedCpBufProtection, + IgniteLogger log + ) { + this.pageMemory = pageMemory; + this.cpProgress = cpProgress; + this.cpLockStateChecker = cpLockStateChecker; + this.log = log; + + if (fillRateBasedCpBufProtection) { + cpBufWatchdog = new CheckpointBufferOverflowWatchdog(pageMemory, CP_BUF_THROTTLING_THRESHOLD_FILL_RATE, + CP_BUF_DANGER_THRESHOLD, CP_BUF_WAKEUP_THRESHOLD_FILL_RATE); + cpBufProtector = new FillRateBasedThrottlingStrategy(cpBufWatchdog, cpProgress); + } + else { + cpBufWatchdog = new CheckpointBufferOverflowWatchdog(pageMemory, CP_BUF_DANGER_THRESHOLD, + CP_BUF_DANGER_THRESHOLD, CP_BUF_WAKEUP_THRESHOLD_EXP_BACKOFF); + cpBufProtector = new ExponentialBackoffThrottlingStrategy(); + } + } + + /** */ + @Override public int checkpointBufferThrottledThreadsWakeupThreshold() { + return cpBufWatchdog.checkpointBufferThrottledThreadsWakeupThreshold(); + } + + /** {@inheritDoc} */ + @Override public boolean isCpBufferOverflowThresholdExceeded() { + return cpBufWatchdog.isInDangerZone(); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/CheckpointBufferOverflowWatchdog.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/CheckpointBufferOverflowWatchdog.java index e3c1d294e9286..6d523ce8c4350 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/CheckpointBufferOverflowWatchdog.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/CheckpointBufferOverflowWatchdog.java @@ -17,8 +17,6 @@ package org.apache.ignite.internal.processors.cache.persistence.pagemem; -import static org.apache.ignite.internal.processors.cache.persistence.pagemem.PagesWriteThrottlePolicy.CP_BUF_FILL_THRESHOLD; - /** * Logic used to determine whether Checkpoint Buffer is in danger zone and writer threads should be throttled. */ @@ -26,25 +24,66 @@ class CheckpointBufferOverflowWatchdog { /** Page memory. */ private final PageMemoryImpl pageMemory; + /** Checkpoint buffer pages throttling threshold. */ + private final int cpBufPagesThrottlingThreshold; + + /** Checkpoint buffer pages danger bound. */ + private final int cpBufPagesDangerThreshold; + + /** Checkpoint buffer pages bound to wake up throttled threads. */ + private final int cpBufPagesThreadsWakeupThreshold; + /** * Creates a new instance. * * @param pageMemory page memory to use + * @param cpBufThrottlingThreshold Checkpoint buffer fulfill bound to start throttling. + * @param cpBufDangerThreshold Checkpoint buffer danger fulfill bound. + * @param cpBufThreadsWakeupThreshold Checkpoint buffer fulfill bound to wake up throttled threads. */ - CheckpointBufferOverflowWatchdog(PageMemoryImpl pageMemory) { + CheckpointBufferOverflowWatchdog( + PageMemoryImpl pageMemory, + double cpBufThrottlingThreshold, + double cpBufDangerThreshold, + double cpBufThreadsWakeupThreshold + ) { this.pageMemory = pageMemory; + cpBufPagesThrottlingThreshold = (int)(pageMemory.checkpointBufferPagesSize() * cpBufThrottlingThreshold); + cpBufPagesDangerThreshold = (int)(pageMemory.checkpointBufferPagesSize() * cpBufDangerThreshold); + cpBufPagesThreadsWakeupThreshold = (int)(pageMemory.checkpointBufferPagesSize() * cpBufThreadsWakeupThreshold); } /** - * Returns true if Checkpoint Buffer is in danger zone (more than - * {@link PagesWriteThrottlePolicy#CP_BUF_FILL_THRESHOLD} of the buffer is filled) and, hence, writer threads need - * to be throttled. + * Returns true if Checkpoint Buffer is in danger zone (more than danger bound of the buffer is filled) + * and, hence, checkpoint pages writer should prioritize the pages in checkpoint buffer. * - * @return {@code true} iff Checkpoint Buffer is in danger zone + * @return {@code true} if Checkpoint Buffer is in danger zone */ boolean isInDangerZone() { - int checkpointBufLimit = (int)(pageMemory.checkpointBufferPagesSize() * CP_BUF_FILL_THRESHOLD); + return pageMemory.checkpointBufferPagesCount() > cpBufPagesDangerThreshold; + } + + /** + * Returns true if Checkpoint Buffer is in throttling zone (more than throttling bound of the buffer is filled) + * and, hence, writer threads need to be throttled. + * + * @return {@code true} if Checkpoint Buffer is in danger zone + */ + boolean isInThrottlingZone() { + return pageMemory.checkpointBufferPagesCount() > cpBufPagesThrottlingThreshold; + } + + /** + * @return Checkpoint buffer pages bound to wake up throttled threads. + */ + int checkpointBufferThrottledThreadsWakeupThreshold() { + return cpBufPagesThreadsWakeupThreshold; + } - return pageMemory.checkpointBufferPagesCount() > checkpointBufLimit; + /** + * @return Checkpoint Buffer fill rate. + */ + double fillRate() { + return (double)pageMemory.checkpointBufferPagesCount() / pageMemory.checkpointBufferPagesSize(); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/CheckpointMetricsTracker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/CheckpointMetricsTracker.java index 474f837b67485..f8ba2d1adcbcf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/CheckpointMetricsTracker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/CheckpointMetricsTracker.java @@ -54,7 +54,7 @@ public class CheckpointMetricsTracker { private volatile int cowPages; /** */ - private long cpStart = System.currentTimeMillis(); + private final long cpStart = System.currentTimeMillis(); /** */ private long cpLockWaitStart; @@ -62,6 +62,9 @@ public class CheckpointMetricsTracker { /** */ private long cpMarkStart; + /** */ + private long cpMarkEnd; + /** */ private long cpLockRelease; @@ -81,11 +84,17 @@ public class CheckpointMetricsTracker { private long walCpRecordFsyncEnd; /** */ - private long splitAndSortCpPagesStart; + private long cpMarkerStoreEnd; /** */ private long splitAndSortCpPagesEnd; + /** */ + private long cpRecoveryDataWriteEnd; + + /** */ + private long cpRecoveryDataSize; + /** */ private long listenersExecEnd; @@ -125,6 +134,11 @@ public void onMarkStart() { cpMarkStart = System.currentTimeMillis(); } + /** */ + public void onMarkEnd() { + cpMarkEnd = System.currentTimeMillis(); + } + /** */ public void onLockRelease() { cpLockRelease = System.currentTimeMillis(); @@ -156,8 +170,8 @@ public void onWalCpRecordFsyncStart() { } /** */ - public void onSplitAndSortCpPagesStart() { - splitAndSortCpPagesStart = System.currentTimeMillis(); + public void onCpMarkerStoreEnd() { + cpMarkerStoreEnd = System.currentTimeMillis(); } /** */ @@ -170,6 +184,12 @@ public void onWalCpRecordFsyncEnd() { walCpRecordFsyncEnd = System.currentTimeMillis(); } + /** */ + public void onWriteRecoveryDataEnd(long recoveryDataSize) { + cpRecoveryDataSize = recoveryDataSize; + cpRecoveryDataWriteEnd = System.currentTimeMillis(); + } + /** * @return Total checkpoint duration. */ @@ -202,7 +222,7 @@ public long listenersExecuteDuration() { * @return Checkpoint mark duration. */ public long markDuration() { - return cpPagesWriteStart - cpMarkStart; + return cpMarkEnd - cpMarkStart; } /** @@ -234,19 +254,33 @@ public long walCpRecordFsyncDuration() { } /** - * @return Duration of checkpoint entry buffer writing to file. - * - * @see CheckpointMarkersStorage#writeCheckpointEntry(long, UUID, WALPointer, CheckpointRecord, CheckpointEntryType, boolean) + * @return Duration of splitting and sorting checkpoint pages. */ - public long writeCheckpointEntryDuration() { - return splitAndSortCpPagesStart - walCpRecordFsyncEnd; + public long splitAndSortCpPagesDuration() { + return splitAndSortCpPagesEnd - walCpRecordFsyncEnd; } /** - * @return Duration of splitting and sorting checkpoint pages. + * @return Duration of writing recovery data. */ - public long splitAndSortCpPagesDuration() { - return splitAndSortCpPagesEnd - splitAndSortCpPagesStart; + public long recoveryDataWriteDuration() { + return cpRecoveryDataWriteEnd - cpMarkEnd; + } + + /** + * @return Size of writing recovery data. + */ + public long recoveryDataSize() { + return cpRecoveryDataSize; + } + + /** + * @return Duration of checkpoint entry buffer writing to file. + * + * @see CheckpointMarkersStorage#writeCheckpointEntry(long, UUID, WALPointer, CheckpointRecord, CheckpointEntryType, boolean) + */ + public long writeCheckpointEntryDuration() { + return cpMarkerStoreEnd - cpRecoveryDataWriteEnd; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/CheckpointPages.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/CheckpointPages.java index 46540f92c0224..d962991b6a832 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/CheckpointPages.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/CheckpointPages.java @@ -19,8 +19,8 @@ import java.util.Collection; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.pagemem.FullPageId; +import org.apache.ignite.internal.util.function.ThrowableSupplier; /** * View of pages which should be stored during current checkpoint. @@ -30,15 +30,15 @@ class CheckpointPages { private final Collection segCheckpointPages; /** The sign which allows to replace pages from a checkpoint by page replacer. */ - private final IgniteInternalFuture allowToReplace; + private final ThrowableSupplier allowToReplace; /** * @param pages Pages which would be stored to disk in current checkpoint. - * @param replaceFuture The sign which allows to replace pages from a checkpoint by page replacer. + * @param allowToReplace The sign which allows to replace pages from a checkpoint by page replacer. */ - CheckpointPages(Collection pages, IgniteInternalFuture replaceFuture) { + CheckpointPages(Collection pages, ThrowableSupplier allowToReplace) { segCheckpointPages = pages; - allowToReplace = replaceFuture; + this.allowToReplace = allowToReplace; } /** @@ -51,8 +51,9 @@ public boolean allowToSave(FullPageId fullPageId) throws IgniteCheckedException if (checkpointPages == null || allowToReplace == null) return false; - //Uninterruptibly is important because otherwise in case of interrupt of client thread node would be stopped. - allowToReplace.getUninterruptibly(); + // Here, in allowToReplace we can wait on future for some time (until checkpoint marker is stored to disk). + if (!allowToReplace.get()) + return false; return checkpointPages.contains(fullPageId); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/ExponentialBackoff.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/ExponentialBackoff.java index 1d00179e2bac7..eee17d66ed0f3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/ExponentialBackoff.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/ExponentialBackoff.java @@ -65,7 +65,7 @@ public long nextDuration() { * Resets the exponential backoff counter so that next call to {@link #nextDuration()} * will return {@link #startingBackoffNanos}. * - * @return {@code true} iff this backoff was not already in a reset state + * @return {@code true} if this backoff was not already in a reset state */ public boolean reset() { int oldVal = exponentialBackoffCounter.getAndSet(0); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/ExponentialBackoffThrottlingStrategy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/ExponentialBackoffThrottlingStrategy.java index 9cf6c5d8a08cc..33183f6152524 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/ExponentialBackoffThrottlingStrategy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/ExponentialBackoffThrottlingStrategy.java @@ -20,7 +20,7 @@ /** * Logic used to protect memory (mainly, Checkpoint Buffer) from exhaustion using exponential backoff. */ -class ExponentialBackoffThrottlingStrategy { +class ExponentialBackoffThrottlingStrategy implements ThrottlingStrategy { /** * Starting throttle time. Limits write speed to 1000 MB/s. */ @@ -36,21 +36,13 @@ class ExponentialBackoffThrottlingStrategy { */ private final ExponentialBackoff backoff = new ExponentialBackoff(STARTING_THROTTLE_NANOS, BACKOFF_RATIO); - /** - * Computes next duration (in nanos) to throttle a thread to protect Checkpoint Buffer. - * - * @return park time in nanos - */ - long protectionParkTime() { + /** {@inheritDoc} */ + @Override public long protectionParkTime() { return backoff.nextDuration(); } - /** - * Resets the backoff counter. Invoked when no throttling is needed anymore. - * - * @return {@code true} iff the backoff was not already in a reset state - */ - boolean resetBackoff() { + /** {@inheritDoc} */ + @Override public boolean reset() { return backoff.reset(); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/FillRateBasedThrottlingStrategy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/FillRateBasedThrottlingStrategy.java new file mode 100644 index 0000000000000..7d5cf83b3bce8 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/FillRateBasedThrottlingStrategy.java @@ -0,0 +1,98 @@ +/* + * 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.pagemem; + +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointProgress; +import org.apache.ignite.lang.IgniteOutClosure; + +/** + * Logic used to protect memory (Checkpoint Buffer) from exhaustion using throttling duration based on storage fill rate. + */ +class FillRateBasedThrottlingStrategy implements ThrottlingStrategy { + /** + * Minimum throttle time. 10 microseconds. + */ + private static final long MIN_THROTTLE_NANOS = 10_000L; + + /** + * Maximum throttle time. 1 second. + */ + private static final long MAX_THROTTLE_NANOS = 1_000_000_000L; + + /** + * The exponent to calculate park time. + */ + private static final double POW = Math.log((double)MAX_THROTTLE_NANOS / MIN_THROTTLE_NANOS); + + /** */ + private final CheckpointBufferOverflowWatchdog cpBufOverflowWatchdog; + + /** Checkpoint progress provider. */ + private final IgniteOutClosure cpProgress; + + /** */ + private final AtomicBoolean throttlingStarted = new AtomicBoolean(); + + /** */ + FillRateBasedThrottlingStrategy(CheckpointBufferOverflowWatchdog watchdog, IgniteOutClosure cpProgress) { + cpBufOverflowWatchdog = watchdog; + this.cpProgress = cpProgress; + } + + /** {@inheritDoc} */ + @Override public long protectionParkTime() { + CheckpointProgress cp = cpProgress.apply(); + + // Checkpoint has not been started. + if (cp == null) + return 0; + + AtomicInteger cpWrittenRecoveryPagesCounter = cp.writtenRecoveryPagesCounter(); + AtomicInteger cpWrittenPagesCounter = cp.writtenPagesCounter(); + int cpTotalPages = cp.currentCheckpointPagesCount(); + + // Checkpoint has been finished. + if (cpTotalPages == 0 || cpWrittenPagesCounter == null || cpWrittenRecoveryPagesCounter == null) + return 0; + + // Time to write and fsync all recovery data on checkpoint is close to time of write and fsync all pages to + // page store, but we don't need to take into account fsync time for data store, since up to this phase + // checkpoint buffer should be free. So, about 2/3 of time, when checkpoint buffer is widely used, takes + // recovery data writing, and 1/3 takes writing pages to page store (without fsync). + double cpProgressRate = (2d * cpWrittenRecoveryPagesCounter.get() + cpWrittenPagesCounter.get()) / 3d / cpTotalPages; + double cpBufFillRate = cpBufOverflowWatchdog.fillRate(); + + if (cpBufFillRate > cpProgressRate && cpProgressRate < 1d) { + throttlingStarted.set(true); + + // Normalized checkpoint buffer fill rate on range [cpProgressRate .. 1]. Result value in range [0 .. 1]. + double cpBufFillRateNorm = ((cpBufFillRate - cpProgressRate) / (1d - cpProgressRate)); + + return (long)(Math.exp(POW * cpBufFillRateNorm) * MIN_THROTTLE_NANOS); + } + else + return 0; + } + + /** {@inheritDoc} */ + @Override public boolean reset() { + return throttlingStarted.getAndSet(false); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryEx.java index a592b835c5b0f..4fb148cd4d2dc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryEx.java @@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.cache.persistence.PageStoreWriter; import org.apache.ignite.internal.processors.cache.persistence.StorageException; import org.apache.ignite.internal.util.GridMultiCollectionWrapper; +import org.apache.ignite.internal.util.function.ThrowableSupplier; /** * Page memory with some persistence related additions. @@ -119,7 +120,7 @@ public long acquirePage(int grpId, long pageId, IoStatisticsHolder statHldr, * @throws IgniteException If checkpoint has been already started and was not finished. * @param allowToReplace The sign which allows to replace pages from a checkpoint by page replacer. */ - public GridMultiCollectionWrapper beginCheckpoint(IgniteInternalFuture allowToReplace) + public GridMultiCollectionWrapper beginCheckpoint(ThrowableSupplier allowToReplace) throws IgniteException; /** @@ -132,17 +133,19 @@ public GridMultiCollectionWrapper beginCheckpoint(IgniteInternalFutu *{@link PageStoreWriter} will be called when the page will be ready to write. * * @param pageId Page ID to get byte buffer for. The page ID must be present in the collection returned by - * the {@link #beginCheckpoint(IgniteInternalFuture)} method call. + * the {@link #beginCheckpoint(ThrowableSupplier)} method call. * @param buf Temporary buffer to write changes into. * @param pageWriter Checkpoint page write context. * @param tracker Checkpoint metrics tracker. + * @param keepDirty Don't reset dirty flag on page. * @throws IgniteCheckedException If failed to obtain page data. */ public void checkpointWritePage( FullPageId pageId, ByteBuffer buf, PageStoreWriter pageWriter, - CheckpointMetricsTracker tracker + CheckpointMetricsTracker tracker, + boolean keepDirty ) throws IgniteCheckedException; /** */ 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 4d554a894085a..4a196cdc9927d 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 @@ -81,6 +81,7 @@ import org.apache.ignite.internal.util.GridMultiCollectionWrapper; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.OffheapReadWriteLock; +import org.apache.ignite.internal.util.function.ThrowableSupplier; import org.apache.ignite.internal.util.future.CountDownFuture; import org.apache.ignite.internal.util.lang.GridPlainRunnable; import org.apache.ignite.internal.util.typedef.internal.CU; @@ -426,12 +427,27 @@ public PageMemoryImpl( * Resolves instance of {@link PagesWriteThrottlePolicy} according to chosen throttle policy. */ private void initWriteThrottle() { - if (throttlingPlc == ThrottlingPolicy.SPEED_BASED) - writeThrottle = new PagesWriteSpeedBasedThrottle(this, cpProgressProvider, stateChecker, log); - else if (throttlingPlc == ThrottlingPolicy.TARGET_RATIO_BASED) - writeThrottle = new PagesWriteThrottle(this, cpProgressProvider, stateChecker, false, log); - else if (throttlingPlc == ThrottlingPolicy.CHECKPOINT_BUFFER_ONLY) - writeThrottle = new PagesWriteThrottle(this, null, stateChecker, true, log); + // Exponential backoff throttling strategy is not suitable for mode with writing recovery data on checkpoint. + // In this mode used checkpoint buffer can't be reduced during recovery data write phase. After starting, + // throttling never reset until the next phase of the checkpoint. Throttling time grows very fast + // (for example, it reaches one day after about 500 invocations). This completely blocks pages changes + // until the next checkpoint phase. So, use another throttling strategy to protect checkpoint buffer in + // this mode. + boolean fillRateBasedCpBufProtection = ctx.gridConfig().getDataStorageConfiguration() + .isWriteRecoveryDataOnCheckpoint(); + + if (throttlingPlc == ThrottlingPolicy.SPEED_BASED) { + writeThrottle = new PagesWriteSpeedBasedThrottle(this, cpProgressProvider, stateChecker, + fillRateBasedCpBufProtection, log); + } + else if (throttlingPlc == ThrottlingPolicy.TARGET_RATIO_BASED) { + writeThrottle = new PagesWriteThrottle(this, cpProgressProvider, stateChecker, + false, fillRateBasedCpBufProtection, log); + } + else if (throttlingPlc == ThrottlingPolicy.CHECKPOINT_BUFFER_ONLY) { + writeThrottle = new PagesWriteThrottle(this, cpProgressProvider, stateChecker, + true, fillRateBasedCpBufProtection, log); + } } /** {@inheritDoc} */ @@ -617,7 +633,7 @@ else if (throttlingPlc == ThrottlingPolicy.CHECKPOINT_BUFFER_ONLY) trackingIO.initNewPage(pageAddr, pageId, realPageSize(grpId), metrics); - if (!ctx.wal().disabled(fullId.groupId(), fullId.pageId())) { + if (!ctx.wal().pageRecordsDisabled(fullId.groupId(), fullId.pageId())) { if (!ctx.wal().isAlwaysWriteFullPages()) ctx.wal().log( new InitNewPageRecord( @@ -939,7 +955,7 @@ else if (relPtr == OUTDATED_REL_PTR) { private void releaseCheckpointBufferPage(long tmpBufPtr) { int resCntr = checkpointPool.releaseFreePage(tmpBufPtr); - if (resCntr == checkpointBufferPagesSize() / 2 && writeThrottle != null) + if (writeThrottle != null && resCntr == writeThrottle.checkpointBufferThrottledThreadsWakeupThreshold()) writeThrottle.wakeupThrottledThreads(); } @@ -1110,7 +1126,7 @@ boolean shouldThrottle(double dirtyRatioThreshold) { /** {@inheritDoc} */ @Override public GridMultiCollectionWrapper beginCheckpoint( - IgniteInternalFuture allowToReplace + ThrowableSupplier allowToReplace ) throws IgniteException { if (segments == null) return new GridMultiCollectionWrapper<>(Collections.emptyList()); @@ -1172,7 +1188,8 @@ private boolean isThrottlingEnabled() { FullPageId fullId, ByteBuffer buf, PageStoreWriter pageStoreWriter, - CheckpointMetricsTracker metricsTracker + CheckpointMetricsTracker metricsTracker, + boolean keepDirty ) throws IgniteCheckedException { assert buf.remaining() == pageSize(); @@ -1243,7 +1260,7 @@ private boolean isThrottlingEnabled() { } } - copyPageForCheckpoint(absPtr, fullId, buf, tag, pageSingleAcquire, pageStoreWriter, metricsTracker); + copyPageForCheckpoint(absPtr, fullId, buf, tag, pageSingleAcquire, pageStoreWriter, metricsTracker, keepDirty); } /** @@ -1253,6 +1270,7 @@ private boolean isThrottlingEnabled() { * @param pageSingleAcquire Page is acquired only once. We don't pin the page second time (until page will not be * copied) in case checkpoint temporary buffer is used. * @param pageStoreWriter Checkpoint page write context. + * @param keepDirty Keep page in checkpoint buffer and don't reset dirty flag. */ private void copyPageForCheckpoint( long absPtr, @@ -1261,7 +1279,8 @@ private void copyPageForCheckpoint( Integer tag, boolean pageSingleAcquire, PageStoreWriter pageStoreWriter, - CheckpointMetricsTracker tracker + CheckpointMetricsTracker tracker, + boolean keepDirty ) throws IgniteCheckedException { assert absPtr != 0; assert PageHeader.isAcquired(absPtr) || !isInCheckpoint(fullId); @@ -1286,7 +1305,9 @@ private void copyPageForCheckpoint( return; } - if (!clearCheckpoint(fullId)) { + boolean inCheckpoint = keepDirty ? isInCheckpoint(fullId) : clearCheckpoint(fullId); + + if (!inCheckpoint) { rwLock.writeUnlock(absPtr + PAGE_LOCK_OFFSET, OffheapReadWriteLock.TAG_LOCK_ALWAYS); if (!pageSingleAcquire) @@ -1299,30 +1320,33 @@ private void copyPageForCheckpoint( long tmpRelPtr = PageHeader.tempBufferPointer(absPtr); if (tmpRelPtr != INVALID_REL_PTR) { - PageHeader.tempBufferPointer(absPtr, INVALID_REL_PTR); - long tmpAbsPtr = checkpointPool.absolute(tmpRelPtr); - copyInBuffer(tmpAbsPtr, buf); + copyToBuffer(tmpAbsPtr, buf); + + if (!keepDirty) { + PageHeader.tempBufferPointer(absPtr, INVALID_REL_PTR); - PageHeader.fullPageId(tmpAbsPtr, NULL_PAGE); + PageHeader.fullPageId(tmpAbsPtr, NULL_PAGE); - GridUnsafe.zeroMemory(tmpAbsPtr + PAGE_OVERHEAD, pageSize()); + GridUnsafe.zeroMemory(tmpAbsPtr + PAGE_OVERHEAD, pageSize()); - if (tracker != null) - tracker.onCowPageWritten(); + if (tracker != null) + tracker.onCowPageWritten(); - releaseCheckpointBufferPage(tmpRelPtr); + releaseCheckpointBufferPage(tmpRelPtr); - // Need release again because we pin page when resolve abs pointer, - // and page did not have tmp buffer page. - if (!pageSingleAcquire) - PageHeader.releasePage(absPtr); + // Need release again because we pin page when resolve abs pointer, + // and page did not have tmp buffer page. + if (!pageSingleAcquire) + PageHeader.releasePage(absPtr); + } } else { - copyInBuffer(absPtr, buf); + copyToBuffer(absPtr, buf); - PageHeader.dirty(absPtr, false); + if (!keepDirty) + PageHeader.dirty(absPtr, false); } assert PageIO.getType(buf) != 0 : "Invalid state. Type is 0! pageId = " + U.hexLong(fullId.pageId()); @@ -1345,17 +1369,18 @@ private void copyPageForCheckpoint( // We pinned the page either when allocated the temp buffer, or when resolved abs pointer. // Must release the page only after write unlock. - PageHeader.releasePage(absPtr); + if (!keepDirty || !pageSingleAcquire) + PageHeader.releasePage(absPtr); } } /** * @param absPtr Absolute ptr. - * @param buf Tmp buffer. + * @param out Output buffer. */ - private void copyInBuffer(long absPtr, ByteBuffer buf) { - if (buf.isDirect()) { - long tmpPtr = GridUnsafe.bufferAddress(buf); + private void copyToBuffer(long absPtr, ByteBuffer out) { + if (out.isDirect()) { + long tmpPtr = GridUnsafe.bufferAddress(out); GridUnsafe.copyMemory(absPtr + PAGE_OVERHEAD, tmpPtr, pageSize()); @@ -1363,7 +1388,7 @@ private void copyInBuffer(long absPtr, ByteBuffer buf) { assert PageIO.getCrc(tmpPtr) == 0; //TODO GG-11480 } else { - byte[] arr = buf.array(); + byte[] arr = out.array(); assert arr != null; assert arr.length == pageSize(); @@ -1860,7 +1885,7 @@ private void setDirty(FullPageId pageId, long absPtr, boolean dirty, boolean for * */ void beforeReleaseWrite(FullPageId pageId, long ptr, boolean pageWalRec) throws IgniteCheckedException { - boolean walIsNotDisabled = walMgr != null && !walMgr.disabled(pageId.groupId(), pageId.pageId()); + boolean walIsNotDisabled = walMgr != null && !walMgr.pageRecordsDisabled(pageId.groupId(), pageId.pageId()); boolean pageRecOrAlwaysWriteFullPage = walMgr != null && (pageWalRec || walMgr.isAlwaysWriteFullPages()); if (pageRecOrAlwaysWriteFullPage && walIsNotDisabled) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteSpeedBasedThrottle.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteSpeedBasedThrottle.java index f9dae4a585485..b73f469a01466 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteSpeedBasedThrottle.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteSpeedBasedThrottle.java @@ -20,7 +20,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.LockSupport; - import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.processors.cache.persistence.CheckpointLockStateChecker; import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointProgress; @@ -38,19 +37,13 @@ * * @see Speed-based throttling description */ -public class PagesWriteSpeedBasedThrottle implements PagesWriteThrottlePolicy { +public class PagesWriteSpeedBasedThrottle extends AbstractPagesWriteThrottle { /** * Throttling 'duration' used to signal that no throttling is needed, and no certain side-effects are allowed * (like stats collection). */ static final long NO_THROTTLING_MARKER = Long.MIN_VALUE; - /** Page memory. */ - private final PageMemoryImpl pageMemory; - - /** Checkpoint progress provider. */ - private final IgniteOutClosure cpProgress; - /** Threads set. Contains threads which are currently parked because of throttling. */ private final GridConcurrentHashSet parkedThreads = new GridConcurrentHashSet<>(); @@ -62,12 +55,6 @@ public class PagesWriteSpeedBasedThrottle implements PagesWriteThrottlePolicy { * */ private final IntervalBasedMeasurement markSpeedAndAvgParkTime = new IntervalBasedMeasurement(250, 3); - /** Checkpoint lock state provider. */ - private final CheckpointLockStateChecker cpLockStateChecker; - - /** Logger. */ - private final IgniteLogger log; - /** Previous warning time, nanos. */ private final AtomicLong prevWarnTime = new AtomicLong(); @@ -77,36 +64,28 @@ public class PagesWriteSpeedBasedThrottle implements PagesWriteThrottlePolicy { /** Warning threshold: minimal level of pressure that causes warning messages to log. */ static final double WARN_THRESHOLD = 0.2; - /** Checkpoint buffer protection logic. */ - private final ExponentialBackoffThrottlingStrategy cpBufferProtector - = new ExponentialBackoffThrottlingStrategy(); - /** Clean pages protection logic. */ private final SpeedBasedMemoryConsumptionThrottlingStrategy cleanPagesProtector; - /** Checkpoint Buffer-related logic used to keep it safe. */ - private final CheckpointBufferOverflowWatchdog cpBufferWatchdog; - /** * @param pageMemory Page memory. * @param cpProgress Database manager. - * @param stateChecker Checkpoint lock state provider. + * @param cpLockStateChecker Checkpoint lock state provider. + * @param fillRateBasedCpBufProtection If true, fill rate based throttling will be used to protect from + * checkpoint buffer overflow. * @param log Logger. */ public PagesWriteSpeedBasedThrottle( PageMemoryImpl pageMemory, IgniteOutClosure cpProgress, - CheckpointLockStateChecker stateChecker, + CheckpointLockStateChecker cpLockStateChecker, + boolean fillRateBasedCpBufProtection, IgniteLogger log ) { - this.pageMemory = pageMemory; - this.cpProgress = cpProgress; - cpLockStateChecker = stateChecker; - this.log = log; + super(pageMemory, cpProgress, cpLockStateChecker, fillRateBasedCpBufProtection, log); cleanPagesProtector = new SpeedBasedMemoryConsumptionThrottlingStrategy(pageMemory, cpProgress, markSpeedAndAvgParkTime); - cpBufferWatchdog = new CheckpointBufferOverflowWatchdog(pageMemory); } /** {@inheritDoc} */ @@ -129,13 +108,13 @@ else if (throttleParkTimeNs > 0) { /***/ private long computeThrottlingParkTime(boolean isPageInCheckpoint, long curNanoTime) { - if (isPageInCheckpoint && isCpBufferOverflowThresholdExceeded()) - return cpBufferProtector.protectionParkTime(); + if (isPageInCheckpoint && cpBufWatchdog.isInThrottlingZone()) + return cpBufProtector.protectionParkTime(); else { if (isPageInCheckpoint) { // The fact that we are here means that we checked whether CP Buffer is in danger zone and found that // it is ok, so its protector may relax, hence we reset it. - cpBufferProtector.resetBackoff(); + cpBufProtector.reset(); } return cleanPagesProtector.protectionParkTime(curNanoTime); } @@ -230,7 +209,7 @@ long getCleanPagesProtectionParkTime( /** {@inheritDoc} */ @Override public void onFinishCheckpoint() { - cpBufferProtector.resetBackoff(); + cpBufProtector.reset(); cleanPagesProtector.finish(); markSpeedAndAvgParkTime.finishInterval(); @@ -305,15 +284,10 @@ public double throttleWeight() { /** {@inheritDoc} */ @Override public void wakeupThrottledThreads() { - if (!isCpBufferOverflowThresholdExceeded()) { - cpBufferProtector.resetBackoff(); + if (!cpBufWatchdog.isInThrottlingZone()) { + cpBufProtector.reset(); unparkParkedThreads(); } } - - /** {@inheritDoc} */ - @Override public boolean isCpBufferOverflowThresholdExceeded() { - return cpBufferWatchdog.isInDangerZone(); - } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottle.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottle.java index 3683369761c0f..d816e4ee0bc87 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottle.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottle.java @@ -29,77 +29,59 @@ * Throttles threads that generate dirty pages during ongoing checkpoint. * Designed to avoid zero dropdowns that can happen if checkpoint buffer is overflowed. */ -public class PagesWriteThrottle implements PagesWriteThrottlePolicy { - /** Page memory. */ - private final PageMemoryImpl pageMemory; - - /** Database manager. */ - private final IgniteOutClosure cpProgress; - +public class PagesWriteThrottle extends AbstractPagesWriteThrottle { /** If true, throttle will only protect from checkpoint buffer overflow, not from dirty pages ratio cap excess. */ private final boolean throttleOnlyPagesInCheckpoint; - /** Checkpoint lock state checker. */ - private final CheckpointLockStateChecker stateChecker; - - /** In-checkpoint protection logic. */ - private final ExponentialBackoffThrottlingStrategy inCheckpointProtection - = new ExponentialBackoffThrottlingStrategy(); - /** Not-in-checkpoint protection logic. */ private final ExponentialBackoffThrottlingStrategy notInCheckpointProtection = new ExponentialBackoffThrottlingStrategy(); - /** Checkpoint Buffer-related logic used to keep it safe. */ - private final CheckpointBufferOverflowWatchdog cpBufferWatchdog; - - /** Logger. */ - private final IgniteLogger log; - /** Threads that are throttled due to checkpoint buffer overflow. */ private final ConcurrentHashMap cpBufThrottledThreads = new ConcurrentHashMap<>(); /** * @param pageMemory Page memory. * @param cpProgress Database manager. - * @param stateChecker checkpoint lock state checker. + * @param cpLockStateChecker Checkpoint lock state checker. * @param throttleOnlyPagesInCheckpoint If true, throttle will only protect from checkpoint buffer overflow. + * @param fillRateBasedCpBufProtection If true, fill rate based throttling will be used to protect from + * checkpoint buffer overflow. * @param log Logger. */ public PagesWriteThrottle(PageMemoryImpl pageMemory, IgniteOutClosure cpProgress, - CheckpointLockStateChecker stateChecker, + CheckpointLockStateChecker cpLockStateChecker, boolean throttleOnlyPagesInCheckpoint, + boolean fillRateBasedCpBufProtection, IgniteLogger log ) { - this.pageMemory = pageMemory; - this.cpProgress = cpProgress; - this.stateChecker = stateChecker; + super(pageMemory, cpProgress, cpLockStateChecker, fillRateBasedCpBufProtection, log); this.throttleOnlyPagesInCheckpoint = throttleOnlyPagesInCheckpoint; - cpBufferWatchdog = new CheckpointBufferOverflowWatchdog(pageMemory); - this.log = log; - assert throttleOnlyPagesInCheckpoint || cpProgress != null + assert (throttleOnlyPagesInCheckpoint && !fillRateBasedCpBufProtection) || cpProgress != null : "cpProgress must be not null if ratio based throttling mode is used"; } /** {@inheritDoc} */ @Override public void onMarkDirty(boolean isPageInCheckpoint) { - assert stateChecker.checkpointLockIsHeldByThread(); + assert cpLockStateChecker.checkpointLockIsHeldByThread(); boolean shouldThrottle = false; if (isPageInCheckpoint) - shouldThrottle = isCpBufferOverflowThresholdExceeded(); + shouldThrottle = cpBufWatchdog.isInThrottlingZone(); if (!shouldThrottle && !throttleOnlyPagesInCheckpoint) { CheckpointProgress progress = cpProgress.apply(); - AtomicInteger writtenPagesCntr = progress == null ? null : cpProgress.apply().writtenPagesCounter(); + AtomicInteger writtenPagesCntr = progress == null ? null : progress.writtenPagesCounter(); + AtomicInteger writtenRecoveryPagesCntr = progress == null ? null : progress.writtenRecoveryPagesCounter(); - if (progress == null || writtenPagesCntr == null) + if (progress == null || writtenPagesCntr == null || writtenRecoveryPagesCntr == null) return; // Don't throttle if checkpoint is not running. + int cpWrittenRecoveryPages = writtenRecoveryPagesCntr.get(); int cpWrittenPages = writtenPagesCntr.get(); int cpTotalPages = progress.currentCheckpointPagesCount(); @@ -109,7 +91,8 @@ public PagesWriteThrottle(PageMemoryImpl pageMemory, shouldThrottle = pageMemory.shouldThrottle(3.0 / 4); } else { - double dirtyRatioThreshold = ((double)cpWrittenPages) / cpTotalPages; + double dirtyRatioThreshold = cpWrittenRecoveryPages == 0 ? ((double)cpWrittenPages) / cpTotalPages : + (cpWrittenRecoveryPages + cpWrittenPages) / 2d / cpTotalPages; // Starting with 0.05 to avoid throttle right after checkpoint start // 7/12 is maximum ratio of dirty pages @@ -119,12 +102,14 @@ public PagesWriteThrottle(PageMemoryImpl pageMemory, } } - ExponentialBackoffThrottlingStrategy exponentialThrottle = isPageInCheckpoint - ? inCheckpointProtection : notInCheckpointProtection; + ThrottlingStrategy exponentialThrottle = isPageInCheckpoint ? cpBufProtector : notInCheckpointProtection; if (shouldThrottle) { long throttleParkTimeNs = exponentialThrottle.protectionParkTime(); + if (throttleParkTimeNs == 0) + return; + Thread curThread = Thread.currentThread(); if (throttleParkTimeNs > LOGGING_THRESHOLD) { @@ -155,7 +140,7 @@ public PagesWriteThrottle(PageMemoryImpl pageMemory, pageMemory.metrics().addThrottlingTime(U.currentTimeMillis() - startTime); } else { - boolean backoffWasAlreadyStarted = exponentialThrottle.resetBackoff(); + boolean backoffWasAlreadyStarted = exponentialThrottle.reset(); if (isPageInCheckpoint && backoffWasAlreadyStarted) unparkParkedThreads(); @@ -164,8 +149,8 @@ public PagesWriteThrottle(PageMemoryImpl pageMemory, /** {@inheritDoc} */ @Override public void wakeupThrottledThreads() { - if (!isCpBufferOverflowThresholdExceeded()) { - inCheckpointProtection.resetBackoff(); + if (!cpBufWatchdog.isInThrottlingZone()) { + cpBufProtector.reset(); unparkParkedThreads(); } @@ -184,12 +169,7 @@ private void unparkParkedThreads() { /** {@inheritDoc} */ @Override public void onFinishCheckpoint() { - inCheckpointProtection.resetBackoff(); - notInCheckpointProtection.resetBackoff(); - } - - /** {@inheritDoc} */ - @Override public boolean isCpBufferOverflowThresholdExceeded() { - return cpBufferWatchdog.isInDangerZone(); + cpBufProtector.reset(); + notInCheckpointProtection.reset(); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottlePolicy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottlePolicy.java index 43616b28cbc92..a1a0b5a85351f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottlePolicy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottlePolicy.java @@ -55,26 +55,51 @@ *

* There are two additional methods for interfacing with other parts of the system: *

    + *
  • {@link #checkpointBufferThrottledThreadsWakeupThreshold()} which is called to check if it's a time to wake up + * throttled threads.
  • *
  • {@link #wakeupThrottledThreads()} which wakes up the threads currently being throttled; in the current - * implementation, it is called when Checkpoint Buffer utilization falls below 1/2.
  • + * implementation, it is called when Checkpoint Buffer utilization falls + * below {@link #checkpointBufferThrottledThreadsWakeupThreshold()}. *
  • {@link #isCpBufferOverflowThresholdExceeded()} which is called by a checkpointer to see whether the Checkpoint Buffer is * in a danger zone and, if yes, it starts to prioritize writing pages from the Checkpoint Buffer over * pages from the normal checkpoint sequence.
  • *
*/ public interface PagesWriteThrottlePolicy { - /** @see IgniteSystemProperties#IGNITE_THROTTLE_LOG_THRESHOLD */ + /** + * @see IgniteSystemProperties#IGNITE_THROTTLE_LOG_THRESHOLD + */ static int DFLT_THROTTLE_LOG_THRESHOLD = 10; - /** Max park time. */ + /** + * Max park time. + */ long LOGGING_THRESHOLD = TimeUnit.SECONDS.toNanos( IgniteSystemProperties.getInteger(IGNITE_THROTTLE_LOG_THRESHOLD, DFLT_THROTTLE_LOG_THRESHOLD)); - /** Checkpoint buffer fullfill upper bound. */ - float CP_BUF_FILL_THRESHOLD = 2f / 3; + /** + * Checkpoint buffer danger fulfill bound. + */ + float CP_BUF_DANGER_THRESHOLD = 2f / 3; + + /** + * Checkpoint buffer fulfill bound to start throttling (fill rate based implementation). + */ + float CP_BUF_THROTTLING_THRESHOLD_FILL_RATE = 1f / 4; + + /** + * Checkpoint buffer fulfill bound to wake up throttled threads (exponential backoff implemetation). + */ + float CP_BUF_WAKEUP_THRESHOLD_EXP_BACKOFF = 1f / 2; + + /** + * Checkpoint buffer fulfill bound to wake up throttled threads (fill rate based implemetation). + */ + float CP_BUF_WAKEUP_THRESHOLD_FILL_RATE = 1f / 5; /** * Callback to apply throttling delay. + * * @param isPageInCheckpoint flag indicating if current page is in scope of current checkpoint. */ void onMarkDirty(boolean isPageInCheckpoint); @@ -102,4 +127,11 @@ public interface PagesWriteThrottlePolicy { * and {@code false} otherwise. */ boolean isCpBufferOverflowThresholdExceeded(); + + /** + * Checkpoint buffer threshold (pages count) to wake up throttled threads. + * + * @return Checkpoint buffer throttled threads wakeup threshold. + */ + int checkpointBufferThrottledThreadsWakeupThreshold(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/SpeedBasedMemoryConsumptionThrottlingStrategy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/SpeedBasedMemoryConsumptionThrottlingStrategy.java index 92d796a4ff33d..149af8eb14733 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/SpeedBasedMemoryConsumptionThrottlingStrategy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/SpeedBasedMemoryConsumptionThrottlingStrategy.java @@ -145,7 +145,7 @@ private void resetStatistics() { /***/ private long computeParkTime(@NotNull AtomicInteger writtenPagesCounter, long curNanoTime) { final int cpWrittenPages = writtenPagesCounter.get(); - final long donePages = cpDonePagesEstimation(cpWrittenPages); + final int donePages = cpDonePagesEstimation(cpWrittenPages); final long instantaneousMarkDirtySpeed = markSpeedAndAvgParkTime.getSpeedOpsPerSec(curNanoTime); // NB: we update progress for speed calculation only in this (clean pages protection) scenario, because @@ -168,8 +168,7 @@ private long computeParkTime(@NotNull AtomicInteger writtenPagesCounter, long cu return parkTimeToThrottleByJustCPSpeed(instantaneousMarkDirtySpeed, avgCpWriteSpeed); } else { - return speedBasedParkTime(cpWrittenPages, donePages, cpTotalPages, instantaneousMarkDirtySpeed, - avgCpWriteSpeed); + return speedBasedParkTime(donePages, cpTotalPages, instantaneousMarkDirtySpeed, avgCpWriteSpeed); } } @@ -185,6 +184,13 @@ private int cpDonePagesEstimation(int cpWrittenPages) { // TODO: IGNITE-16879 - this only works correctly if time-to-write a page is close to time-to-sync a page. // In reality, this does not seem to hold, which produces wrong estimations. We could measure the real times // in Checkpointer and make this estimation a lot more precise. + // Here we also assume that time-to-write + time-to-sync checkpoint recovery data (if this phase is used by + // checkpointer) is close to time-to-write + time-to-sync of pages to page store. + int cpWrittenRecoveryPages = cpWrittenRecoveryPages(); + + if (cpWrittenRecoveryPages > 0) + return (cpWrittenRecoveryPages * 2 + cpWrittenPages + cpSyncedPages()) / 4; + return (cpWrittenPages + cpSyncedPages()) / 2; } @@ -207,13 +213,17 @@ private long parkTimeToThrottleByJustCPSpeed(long markDirtySpeed, long curCpWrit } /***/ - private long speedBasedParkTime(int cpWrittenPages, long donePages, int cpTotalPages, - long instantaneousMarkDirtySpeed, long avgCpWriteSpeed) { + private long speedBasedParkTime( + int donePages, + int cpTotalPages, + long instantaneousMarkDirtySpeed, + long avgCpWriteSpeed + ) { final double dirtyPagesRatio = pageMemory.getDirtyPagesRatio(); currDirtyRatio = dirtyPagesRatio; - detectCpPagesWriteStart(cpWrittenPages, dirtyPagesRatio); + detectCpPagesWriteStart(donePages, dirtyPagesRatio); if (dirtyPagesRatio >= MAX_DIRTY_PAGES) return 0; // too late to throttle, will wait on safe to update instead. @@ -414,6 +424,15 @@ int cpSyncedPages() { return syncedPagesCntr == null ? 0 : syncedPagesCntr.get(); } + /** + * @return Counter for written recovery pages on checkpoint. + */ + int cpWrittenRecoveryPages() { + AtomicInteger writtenRecoveryPagesCounter = cpProgress.apply().writtenRecoveryPagesCounter(); + + return writtenRecoveryPagesCounter == null ? 0 : writtenRecoveryPagesCounter.get(); + } + /** * @return Number of pages in current checkpoint. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/ThrottlingStrategy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/ThrottlingStrategy.java new file mode 100644 index 0000000000000..29cd27c2d22fa --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/ThrottlingStrategy.java @@ -0,0 +1,36 @@ +/* + * 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.pagemem; + +/** + * Strategy used to protect memory from exhaustion. + */ +public interface ThrottlingStrategy { + /** + * Computes next duration (in nanos) to throttle a thread. + * + * @return park time in nanos. + */ + public long protectionParkTime(); + + /** + * Resets the state. Invoked when no throttling is needed anymore. + * + * @return {@code true} if the instance was not already in a reset state + */ + public boolean reset(); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/util/PageHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/util/PageHandler.java index 39ce3b4873f1c..b62b7d4154caa 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/util/PageHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/util/PageHandler.java @@ -490,7 +490,7 @@ public static boolean isWalDeltaRecordNeeded( @Nullable Boolean walPlc) { // If the page is clean, then it is either newly allocated or just after checkpoint. // In both cases we have to write full page contents to WAL. - return wal != null && !wal.isAlwaysWriteFullPages() && walPlc != TRUE && !wal.disabled(cacheId, pageId) && + return wal != null && !wal.isAlwaysWriteFullPages() && walPlc != TRUE && !wal.pageRecordsDisabled(cacheId, pageId) && (walPlc == FALSE || pageMem.isDirty(cacheId, pageId, page)); } 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 bb75a96019f4a..dec6ee92bd34e 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 @@ -1239,8 +1239,8 @@ private boolean segmentReservedOrLocked(long absIdx) { } /** {@inheritDoc} */ - @Override public boolean disabled(int grpId, long pageId) { - return cctx.walState().isDisabled(grpId, pageId); + @Override public boolean pageRecordsDisabled(int grpId, long pageId) { + return cctx.walState().isPageRecordsDisabled(grpId, pageId); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsReader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsReader.java index 50da2b765d56f..e902dfc7db330 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsReader.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsReader.java @@ -57,6 +57,7 @@ import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_ROWS; import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK; import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT; +import static org.apache.ignite.internal.processors.performancestatistics.OperationType.VERSION; import static org.apache.ignite.internal.processors.performancestatistics.OperationType.cacheOperation; import static org.apache.ignite.internal.processors.performancestatistics.OperationType.cacheRecordSize; import static org.apache.ignite.internal.processors.performancestatistics.OperationType.cacheStartRecordSize; @@ -145,6 +146,7 @@ public void read(List filesOrDirs) throws IOException { try (FileIO io = ioFactory.create(file)) { fileIo = io; + boolean first = true; while (true) { if (io.read(buf) <= 0) { @@ -166,7 +168,8 @@ public void read(List filesOrDirs) throws IOException { buf.mark(); - while (deserialize(buf, nodeId)) { + while (deserialize(buf, nodeId, first)) { + first = false; if (forwardRead != null && forwardRead.found) { if (forwardRead.resetBuf) { buf.limit(0); @@ -201,9 +204,10 @@ public void read(List filesOrDirs) throws IOException { /** * @param buf Buffer. * @param nodeId Node id. + * @param firstRecord Is it first record in the file. * @return {@code True} if operation deserialized. {@code False} if not enough bytes. */ - private boolean deserialize(ByteBuffer buf, UUID nodeId) throws IOException { + private boolean deserialize(ByteBuffer buf, UUID nodeId, boolean firstRecord) throws IOException { if (buf.remaining() < 1) return false; @@ -211,7 +215,23 @@ private boolean deserialize(ByteBuffer buf, UUID nodeId) throws IOException { OperationType opType = OperationType.of(opTypeByte); - if (cacheOperation(opType)) { + if (firstRecord && opType != VERSION) + throw new IgniteException("Unsupported file format"); + + if (opType == VERSION) { + if (buf.remaining() < OperationType.versionRecordSize()) + return false; + + short ver = buf.getShort(); + + if (ver != FilePerformanceStatisticsWriter.FILE_FORMAT_VERSION) { + throw new IgniteException("Unsupported file format version [fileVer=" + ver + ", supportedVer=" + + FilePerformanceStatisticsWriter.FILE_FORMAT_VERSION + ']'); + } + + return true; + } + else if (cacheOperation(opType)) { if (buf.remaining() < cacheRecordSize()) return false; @@ -459,6 +479,7 @@ else if (opType == CHECKPOINT) { long walCpRecordFsyncDuration = buf.getLong(); long writeCheckpointEntryDuration = buf.getLong(); long splitAndSortCpPagesDuration = buf.getLong(); + long recoveryDataWriteDuration = buf.getLong(); long totalDuration = buf.getLong(); long cpStartTime = buf.getLong(); int pagesSize = buf.getInt(); @@ -477,6 +498,7 @@ else if (opType == CHECKPOINT) { walCpRecordFsyncDuration, writeCheckpointEntryDuration, splitAndSortCpPagesDuration, + recoveryDataWriteDuration, totalDuration, cpStartTime, pagesSize, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java index f1ae19b96846d..9907dce102533 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java @@ -96,6 +96,12 @@ public class FilePerformanceStatisticsWriter { /** Default maximum cached strings threshold. String caching will stop on threshold excess. */ public static final int DFLT_CACHED_STRINGS_THRESHOLD = 10 * 1024; + /** + * File format version. This version should be incremented each time when format of existing events are + * changed (fields added/removed) to avoid unexpected non-informative errors on deserialization. + */ + public static final short FILE_FORMAT_VERSION = 1; + /** File writer thread name. */ static final String WRITER_THREAD_NAME = "performance-statistics-writer"; @@ -159,6 +165,8 @@ public FilePerformanceStatisticsWriter(GridKernalContext ctx) throws IgniteCheck ringByteBuf = new SegmentedRingByteBuffer(bufSize, fileMaxSize, SegmentedRingByteBuffer.BufferMode.DIRECT); fileWriter = new FileWriter(ctx, log); + + doWrite(OperationType.VERSION, OperationType.versionRecordSize(), buf -> buf.putShort(FILE_FORMAT_VERSION)); } /** Starts collecting performance statistics. */ @@ -380,6 +388,7 @@ File file() { * @param walCpRecordFsyncDuration Wal cp record fsync duration. * @param writeCpEntryDuration Write checkpoint entry duration. * @param splitAndSortCpPagesDuration Split and sort cp pages duration. + * @param recoveryDataWriteDuration Recovery data write duration in milliseconds. * @param totalDuration Total duration in milliseconds. * @param cpStartTime Checkpoint start time in milliseconds. * @param pagesSize Pages size. @@ -397,6 +406,7 @@ public void checkpoint( long walCpRecordFsyncDuration, long writeCpEntryDuration, long splitAndSortCpPagesDuration, + long recoveryDataWriteDuration, long totalDuration, long cpStartTime, int pagesSize, @@ -414,6 +424,7 @@ public void checkpoint( buf.putLong(walCpRecordFsyncDuration); buf.putLong(writeCpEntryDuration); buf.putLong(splitAndSortCpPagesDuration); + buf.putLong(recoveryDataWriteDuration); buf.putLong(totalDuration); buf.putLong(cpStartTime); buf.putInt(pagesSize); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/OperationType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/OperationType.java index df18195b6538d..17de3905a488b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/OperationType.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/OperationType.java @@ -91,7 +91,10 @@ public enum OperationType { QUERY_ROWS(20), /** Custom query property. */ - QUERY_PROPERTY(21); + QUERY_PROPERTY(21), + + /** Version. */ + VERSION(255); /** Cache operations. */ public static final EnumSet CACHE_OPS = EnumSet.of(CACHE_GET, CACHE_PUT, CACHE_REMOVE, @@ -217,7 +220,12 @@ public static int jobRecordSize() { /** @return Checkpoint record size. */ public static int checkpointRecordSize() { - return 8 * 12 + 4 * 3; + return 8 * 13 + 4 * 3; + } + + /** @return Version record size. */ + public static int versionRecordSize() { + return Short.BYTES; } /** @return Pages write throttle record size. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsHandler.java index cc1a203a03faf..f8c9b58a056b1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsHandler.java @@ -131,6 +131,7 @@ void queryReads(UUID nodeId, GridCacheQueryType type, UUID queryNodeId, long id, * @param walCpRecordFsyncDuration Wal cp record fsync duration. * @param writeCpEntryDuration Write checkpoint entry duration. * @param splitAndSortCpPagesDuration Split and sort cp pages duration. + * @param recoveryDataWriteDuration Recovery data write duration. * @param totalDuration Total duration in milliseconds. * @param cpStartTime Checkpoint start time in milliseconds. * @param pagesSize Pages size. @@ -149,6 +150,7 @@ void checkpoint( long walCpRecordFsyncDuration, long writeCpEntryDuration, long splitAndSortCpPagesDuration, + long recoveryDataWriteDuration, long totalDuration, long cpStartTime, int pagesSize, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsProcessor.java index 72584be9db894..8ec6e43350a41 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsProcessor.java @@ -255,6 +255,7 @@ public void checkpoint( long walCpRecordFsyncDuration, long writeCpEntryDuration, long splitAndSortCpPagesDuration, + long recoveryDataWriteDuration, long totalDuration, long cpStartTime, int pagesSize, @@ -270,6 +271,7 @@ public void checkpoint( walCpRecordFsyncDuration, writeCpEntryDuration, splitAndSortCpPagesDuration, + recoveryDataWriteDuration, totalDuration, cpStartTime, pagesSize, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridConcurrentMultiPairQueue.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridConcurrentMultiPairQueue.java index 858d9332111df..dcc637efeeaf8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridConcurrentMultiPairQueue.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridConcurrentMultiPairQueue.java @@ -146,6 +146,13 @@ public boolean next(Result res) { return true; } + /** + * Rewind the queue to start iterating from the beginning. + */ + public void rewind() { + pos.set(0); + } + /** * @return {@code true} if empty. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index 11739560d218b..08c4267303707 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -337,8 +337,15 @@ public abstract class IgniteUtils { /** Minimum checkpointing page buffer size (may be adjusted by Ignite). */ public static final Long DFLT_MIN_CHECKPOINTING_PAGE_BUFFER_SIZE = GB / 4; - /** Default minimum checkpointing page buffer size (may be adjusted by Ignite). */ - public static final Long DFLT_MAX_CHECKPOINTING_PAGE_BUFFER_SIZE = 2 * GB; + /** Default maximum checkpointing page buffer size (when recovery data stored in WAL). */ + public static final Long DFLT_MAX_CHECKPOINTING_PAGE_BUFFER_SIZE_WAL_RECOVERY = 2 * GB; + + /** + * Default maximum checkpointing page buffer size (when recovery data stored on checkpoint). + * In this mode checkpoint duration can be twice as long as for mode with storing recovery data to WAL. + * Also, checkpoint buffer pages can't be released during write recovery data phase, so we need larger buffer size. + */ + public static final Long DFLT_MAX_CHECKPOINTING_PAGE_BUFFER_SIZE_CP_RECOVERY = 5 * GB; /** @see IgniteSystemProperties#IGNITE_MBEAN_APPEND_CLASS_LOADER_ID */ public static final boolean DFLT_MBEAN_APPEND_CLASS_LOADER_ID = true; @@ -11063,19 +11070,21 @@ public static T fromBytes(byte[] data) { * @param regCfg Configuration. * @return Checkpoint buffer size. */ - public static long checkpointBufferSize(DataRegionConfiguration regCfg) { + public static long checkpointBufferSize(DataStorageConfiguration dsCfg, DataRegionConfiguration regCfg) { if (!regCfg.isPersistenceEnabled()) return 0L; long res = regCfg.getCheckpointPageBufferSize(); if (res == 0L) { + long maxCpPageBufSize = dsCfg.isWriteRecoveryDataOnCheckpoint() ? + DFLT_MAX_CHECKPOINTING_PAGE_BUFFER_SIZE_CP_RECOVERY : + DFLT_MAX_CHECKPOINTING_PAGE_BUFFER_SIZE_WAL_RECOVERY; + if (regCfg.getMaxSize() < GB) res = Math.min(DFLT_MIN_CHECKPOINTING_PAGE_BUFFER_SIZE, regCfg.getMaxSize()); - else if (regCfg.getMaxSize() < 8 * GB) - res = regCfg.getMaxSize() / 4; else - res = DFLT_MAX_CHECKPOINTING_PAGE_BUFFER_SIZE; + res = Math.min(regCfg.getMaxSize() / 4, maxCpPageBufSize); } return res; @@ -11097,7 +11106,7 @@ public static long adjustedWalHistorySize(DataStorageConfiguration dsCfg, @Nulla if (dsCfg.getDataRegionConfigurations() != null) { for (DataRegionConfiguration regCfg : dsCfg.getDataRegionConfigurations()) { - long cpBufSize = checkpointBufferSize(regCfg); + long cpBufSize = checkpointBufferSize(dsCfg, regCfg); if (cpBufSize > regCfg.getMaxSize()) cpBufSize = regCfg.getMaxSize(); @@ -11110,7 +11119,7 @@ public static long adjustedWalHistorySize(DataStorageConfiguration dsCfg, @Nulla { DataRegionConfiguration regCfg = dsCfg.getDefaultDataRegionConfiguration(); - long cpBufSize = checkpointBufferSize(regCfg); + long cpBufSize = checkpointBufferSize(dsCfg, regCfg); if (cpBufSize > regCfg.getMaxSize()) cpBufSize = regCfg.getMaxSize(); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/encryption/EncryptionSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/encryption/EncryptionSpi.java index 31f1b8416cfad..b7d7d5412aeef 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/encryption/EncryptionSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/encryption/EncryptionSpi.java @@ -81,6 +81,28 @@ public interface EncryptionSpi extends IgniteSpi { */ byte[] decrypt(byte[] data, Serializable key); + /** + * Decrypts data encrypted with {@link #encrypt(ByteBuffer, Serializable, ByteBuffer)}. + * Note: Default method implementation was introduced for compatibility. This implementation is not effective + * for direct byte buffers, since it requires additional array creation and copy. + * It's better to have own implementation of this method in SPI. + * + * @param data Data to decrypt. + * @param key Encryption key. + * @param res Destination of the decrypted data. + */ + default void decrypt(ByteBuffer data, Serializable key, ByteBuffer res) { + byte[] arr; + + if (data.hasArray()) + arr = data.array(); + else { + arr = new byte[data.remaining()]; + data.get(arr); + } + res.put(decrypt(arr, key)); + } + /** * Decrypts data encrypted with {@link #encryptNoPadding(ByteBuffer, Serializable, ByteBuffer)} * diff --git a/modules/core/src/main/java/org/apache/ignite/spi/encryption/keystore/KeystoreEncryptionSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/encryption/keystore/KeystoreEncryptionSpi.java index effe8bf3439d1..7613c5bccc664 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/encryption/keystore/KeystoreEncryptionSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/encryption/keystore/KeystoreEncryptionSpi.java @@ -217,8 +217,22 @@ public class KeystoreEncryptionSpi extends IgniteSpiAdapter implements Encryptio } } + /** {@inheritDoc} */ + @Override public void decrypt(ByteBuffer data, Serializable key, ByteBuffer res) { + doDecryption(data, aesWithPadding.get(), key, res); + } + /** {@inheritDoc} */ @Override public void decryptNoPadding(ByteBuffer data, Serializable key, ByteBuffer res) { + doDecryption(data, aesWithoutPadding.get(), key, res); + } + + /** + * @param data Plain data. + * @param cipher Cipher. + * @param key Encryption key. + */ + private void doEncryption(ByteBuffer data, Cipher cipher, Serializable key, ByteBuffer res) { assert key instanceof KeystoreEncryptionKey; ensureStarted(); @@ -226,28 +240,27 @@ public class KeystoreEncryptionSpi extends IgniteSpiAdapter implements Encryptio try { SecretKeySpec keySpec = new SecretKeySpec(((KeystoreEncryptionKey)key).key().getEncoded(), CIPHER_ALGO); - Cipher cipher = aesWithoutPadding.get(); - - byte[] iv = new byte[cipher.getBlockSize()]; + byte[] iv = initVector(cipher); - data.get(iv); + res.put(iv); - cipher.init(DECRYPT_MODE, keySpec, new IvParameterSpec(iv)); + cipher.init(ENCRYPT_MODE, keySpec, new IvParameterSpec(iv)); cipher.doFinal(data, res); } - catch (InvalidAlgorithmParameterException | InvalidKeyException | IllegalBlockSizeException | - ShortBufferException | BadPaddingException e) { + catch (ShortBufferException | InvalidAlgorithmParameterException | InvalidKeyException | + IllegalBlockSizeException | BadPaddingException e) { throw new IgniteSpiException(e); } } /** - * @param data Plain data. + * @param data Data to decrypt. * @param cipher Cipher. * @param key Encryption key. + * @param res Destination of the decrypted data. */ - private void doEncryption(ByteBuffer data, Cipher cipher, Serializable key, ByteBuffer res) { + private void doDecryption(ByteBuffer data, Cipher cipher, Serializable key, ByteBuffer res) { assert key instanceof KeystoreEncryptionKey; ensureStarted(); @@ -255,16 +268,16 @@ private void doEncryption(ByteBuffer data, Cipher cipher, Serializable key, Byte try { SecretKeySpec keySpec = new SecretKeySpec(((KeystoreEncryptionKey)key).key().getEncoded(), CIPHER_ALGO); - byte[] iv = initVector(cipher); + byte[] iv = new byte[cipher.getBlockSize()]; - res.put(iv); + data.get(iv); - cipher.init(ENCRYPT_MODE, keySpec, new IvParameterSpec(iv)); + cipher.init(DECRYPT_MODE, keySpec, new IvParameterSpec(iv)); cipher.doFinal(data, res); } - catch (ShortBufferException | InvalidAlgorithmParameterException | InvalidKeyException | - IllegalBlockSizeException | BadPaddingException e) { + catch (InvalidAlgorithmParameterException | InvalidKeyException | IllegalBlockSizeException | + ShortBufferException | BadPaddingException e) { throw new IgniteSpiException(e); } } @@ -346,10 +359,17 @@ private int encryptedSize(int dataSize, String algo) { switch (algo) { case AES_WITH_PADDING: + // One extra block is added to the start of the message to store initial vector. + // If original data is not aligned to block size we pad bytes to align to block size. + // If original data is aligned to block size we pad 1 additional block. + // In general, we pad `blockSize - (dataSize % blockSize)` bytes. + // See RFC2315 for more information about padding. cntBlocks = 2; break; case AES_WITHOUT_PADDING: + // We use AES_WITHOUT_PADDING only for data aligned to block size. In this case only one extra block + // is added to store initial vector. cntBlocks = 1; break; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteDataStorageMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteDataStorageMetricsSelfTest.java index 4a04259afda5f..8b92363822d04 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteDataStorageMetricsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteDataStorageMetricsSelfTest.java @@ -272,16 +272,18 @@ public void testCheckpointMetrics() throws Exception { "checkpointListenersExecuteTime=(\\d+)ms, " + "checkpointLockHoldTime=(\\d+)ms, " + "walCpRecordFsyncDuration=(\\d+)ms, " + - "writeCheckpointEntryDuration=(\\d+)ms, " + - "splitAndSortCpPagesDuration=(\\d+)ms"); + "splitAndSortCpPagesDuration=(\\d+)ms, " + + "writeRecoveryDataDuration=(\\d+)ms, " + + "writeCheckpointEntryDuration=(\\d+)ms, "); AtomicLong expLastCpBeforeLockDuration = new AtomicLong(); AtomicLong expLastCpLockWaitDuration = new AtomicLong(); AtomicLong expLastCpListenersExecDuration = new AtomicLong(); AtomicLong expLastCpLockHoldDuration = new AtomicLong(); AtomicLong expLastCpWalRecordFsyncDuration = new AtomicLong(); - AtomicLong expLastCpWriteEntryDuration = new AtomicLong(); AtomicLong expLastCpSplitAndSortPagesDuration = new AtomicLong(); + AtomicLong expLastCpRecoveryDataWriteDuration = new AtomicLong(); + AtomicLong expLastCpWriteEntryDuration = new AtomicLong(); AtomicInteger cpCnt = new AtomicInteger(); listeningLog.registerListener(s -> { @@ -295,8 +297,9 @@ public void testCheckpointMetrics() throws Exception { expLastCpListenersExecDuration.set(Long.parseLong(matcher.group(3))); expLastCpLockHoldDuration.set(Long.parseLong(matcher.group(4))); expLastCpWalRecordFsyncDuration.set(Long.parseLong(matcher.group(5))); - expLastCpWriteEntryDuration.set(Long.parseLong(matcher.group(6))); - expLastCpSplitAndSortPagesDuration.set(Long.parseLong(matcher.group(7))); + expLastCpSplitAndSortPagesDuration.set(Long.parseLong(matcher.group(6))); + expLastCpRecoveryDataWriteDuration.set(Long.parseLong(matcher.group(7))); + expLastCpWriteEntryDuration.set(Long.parseLong(matcher.group(8))); cpCnt.incrementAndGet(); }); @@ -309,7 +312,7 @@ public void testCheckpointMetrics() throws Exception { db.checkpointReadLock(); try { - waitForCondition(() -> cpCnt.get() > 0, getTestTimeout()); + assertTrue(waitForCondition(() -> cpCnt.get() > 0, 5_000L)); MetricRegistry mreg = node.context().metric().registry(DATASTORAGE_METRIC_PREFIX); @@ -321,6 +324,8 @@ public void testCheckpointMetrics() throws Exception { AtomicLongMetric lastCpWriteEntryDuration = mreg.findMetric("LastCheckpointWriteEntryDuration"); AtomicLongMetric lastCpSplitAndSortPagesDuration = mreg.findMetric("LastCheckpointSplitAndSortPagesDuration"); + AtomicLongMetric lastCpRecoveryDataWriteDuration = + mreg.findMetric("LastCheckpointRecoveryDataWriteDuration"); HistogramMetric cpBeforeLockHistogram = mreg.findMetric("CheckpointBeforeLockHistogram"); HistogramMetric cpLockWaitHistogram = mreg.findMetric("CheckpointLockWaitHistogram"); @@ -334,7 +339,7 @@ public void testCheckpointMetrics() throws Exception { HistogramMetric cpSplitAndSortPagesHistogram = mreg.findMetric("CheckpointSplitAndSortPagesHistogram"); HistogramMetric cpHistogram = mreg.findMetric("CheckpointHistogram"); - waitForCondition(() -> cpCnt.get() == Arrays.stream(cpHistogram.value()).sum(), getTestTimeout()); + assertTrue(waitForCondition(() -> cpCnt.get() == Arrays.stream(cpHistogram.value()).sum(), 5_000)); assertEquals(cpCnt.get(), Arrays.stream(cpBeforeLockHistogram.value()).sum()); assertEquals(cpCnt.get(), Arrays.stream(cpLockWaitHistogram.value()).sum()); @@ -352,8 +357,9 @@ public void testCheckpointMetrics() throws Exception { assertEquals(expLastCpListenersExecDuration.get(), lastCpListenersExecDuration.value()); assertEquals(expLastCpLockHoldDuration.get(), lastCpLockHoldDuration.value()); assertEquals(expLastCpWalRecordFsyncDuration.get(), lastCpWalRecordFsyncDuration.value()); - assertEquals(expLastCpWriteEntryDuration.get(), lastCpWriteEntryDuration.value()); assertEquals(expLastCpSplitAndSortPagesDuration.get(), lastCpSplitAndSortPagesDuration.value()); + assertEquals(expLastCpRecoveryDataWriteDuration.get(), lastCpRecoveryDataWriteDuration.value()); + assertEquals(expLastCpWriteEntryDuration.get(), lastCpWriteEntryDuration.value()); } finally { db.checkpointReadUnlock(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java index 358a4a4987140..dca17340c396b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java @@ -52,7 +52,6 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager; import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; -import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Test; @@ -347,7 +346,7 @@ private void generateWal( } } - Collection pageIds = mem.beginCheckpoint(new GridFinishedFuture()); + Collection pageIds = mem.beginCheckpoint(() -> Boolean.TRUE); info("Acquired pages for checkpoint: " + pageIds.size()); @@ -384,7 +383,7 @@ private void generateWal( if (pageIds.contains(fullId)) { long cpStart = System.nanoTime(); - mem.checkpointWritePage(fullId, tmpBuf, pageStoreWriter, null); + mem.checkpointWritePage(fullId, tmpBuf, pageStoreWriter, null, false); long cpEnd = System.nanoTime(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsCheckpointRecoveryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsCheckpointRecoveryTest.java new file mode 100644 index 0000000000000..f6f269cdd4b76 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsCheckpointRecoveryTest.java @@ -0,0 +1,371 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.persistence.db; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.file.OpenOption; +import java.util.Collection; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.regex.Pattern; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteState; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cluster.ClusterState; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.DiskPageCompression; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.failure.StopNodeFailureHandler; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.encryption.AbstractEncryptionTest; +import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointRecoveryFileStorage; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIODecorator; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; +import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.spi.encryption.keystore.KeystoreEncryptionSpi; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.GridAbstractTest; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_CP_RECOVERY_DATA_COMRESSION; +import static org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointRecoveryFileStorage.FILE_NAME_PATTERN; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX; + +/** + * Class containing tests for applying checkpoint recovery data. + */ +@RunWith(Parameterized.class) +public class IgnitePdsCheckpointRecoveryTest extends GridCommonAbstractTest { + /** */ + private static final int KEYS_CNT = 20_000; + + /** */ + private static final int PARTS = 10; + + /** */ + private final AtomicBoolean fail = new AtomicBoolean(); + + /** */ + private final AtomicInteger spoiledPageLimit = new AtomicInteger(); + + /** */ + private Pattern spoilFilePattern; + + /** */ + @Parameterized.Parameter(0) + public boolean encrypt; + + /** */ + @Parameterized.Parameters(name = "encrypt={0}") + public static Collection parameters() { + return F.asList(new Object[] {false}, new Object[] {true}); + } + + /** */ + protected DiskPageCompression getCompression() { + return DFLT_CP_RECOVERY_DATA_COMRESSION; + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + KeystoreEncryptionSpi encSpi = new KeystoreEncryptionSpi(); + + encSpi.setKeyStorePath(AbstractEncryptionTest.KEYSTORE_PATH); + encSpi.setKeyStorePassword(AbstractEncryptionTest.KEYSTORE_PASSWORD.toCharArray()); + + return super.getConfiguration(igniteInstanceName) + .setFailureHandler(new StopNodeFailureHandler()) + .setEncryptionSpi(encSpi) + .setDataStorageConfiguration(new DataStorageConfiguration() + .setFileIOFactory(new PageStoreSpoilingFileIOFactory(fail, spoiledPageLimit, spoilFilePattern)) + .setWriteRecoveryDataOnCheckpoint(true) + .setCheckpointRecoveryDataCompression(getCompression()) + .setDefaultDataRegionConfiguration(new DataRegionConfiguration() + .setPersistenceEnabled(true) + )); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** */ + @Test + public void testRecoverFromCheckpointRecoveryFiles() throws Exception { + spoilFilePattern = Pattern.compile('^' + Pattern.quote(PART_FILE_PREFIX) + ".*"); + + IgniteEx ignite = initIgnite(); + IgniteCache cache = ignite.cache(DEFAULT_CACHE_NAME); + + AtomicInteger val = new AtomicInteger(KEYS_CNT); + + IgniteInternalFuture fut = GridTestUtils.runAsync(() -> { + while (true) + cache.put(ThreadLocalRandom.current().nextInt(KEYS_CNT), val.incrementAndGet()); + }); + + File cpDir = ignite.context().pdsFolderResolver().fileTree().checkpoint(); + + spoiledPageLimit.set(10); + fail.set(true); + + try { + forceCheckpoint(); + } + catch (Throwable ignore) { + // Expected. + } + + try { + fut.get(10_000); + } + catch (Throwable ignore) { + // Expected. + } + + assertTrue(GridTestUtils.waitForCondition( + () -> Ignition.state(getTestIgniteInstanceName(0)) == IgniteState.STOPPED_ON_FAILURE, + 10_000 + )); + + fail.set(false); + + assertTrue(cpDir.listFiles(((dir, name) -> FILE_NAME_PATTERN.matcher(name).matches())).length > 0); + + ignite = startGrid(0); + IgniteCache cache0 = ignite.cache(DEFAULT_CACHE_NAME); + + int max = 0; + for (int i = 0; i < KEYS_CNT; i++) + max = Math.max(max, cache0.get(i)); + + // There are two cases possible: + // 1. Failure during put before writting cache entry to WAL, in this case, after restore we will get last value + // in cache: val.get() - 1 + // 2. Failure during put after writting cache entry to WAL, in this case, after restore we will get last value + // in cache: val.get() + assertTrue("Expected value between " + (val.get() - 1) + " and " + val.get() + ", actual value: " + max, + max >= val.get() - 1 && max <= val.get()); + } + + /** */ + @Test + public void testFailToRecoverFromSpoiledCheckpointRecoveryFiles() throws Exception { + spoilFilePattern = Pattern.compile('^' + Pattern.quote(PART_FILE_PREFIX) + ".*"); + + IgniteEx ignite = initIgnite(); + + File cpDir = ignite.context().pdsFolderResolver().fileTree().checkpoint(); + + spoiledPageLimit.set(10); + fail.set(true); + + try { + forceCheckpoint(); + } + catch (Throwable ignore) { + // Expected. + } + + assertTrue(GridTestUtils.waitForCondition( + () -> Ignition.state(getTestIgniteInstanceName(0)) == IgniteState.STOPPED_ON_FAILURE, + 10_000 + )); + + fail.set(false); + + assertTrue(cpDir.listFiles(((dir, name) -> FILE_NAME_PATTERN.matcher(name).matches())).length > 0); + + // Spoil recovery files. + for (File file : cpDir.listFiles(((dir, name) -> FILE_NAME_PATTERN.matcher(name).matches()))) { + if (file.length() == 0) + continue; + + FileIOFactory fileIoFactory = new RandomAccessFileIOFactory(); + + FileIO fileIO = fileIoFactory.create(file); + + for (int i = 0; i < 100; i++) { + // Spoil random bytes. + fileIO.position(ThreadLocalRandom.current().nextLong(file.length() - 1)); + fileIO.write(new byte[] {(byte)ThreadLocalRandom.current().nextInt(256)}, 0, 1); + } + } + + try { + startGrid(0); + + fail(); + } + catch (Exception ignore) { + // Recovery files inconsistency should be detected by CRC or fields check (depending on bytes spoiled). + } + } + + /** */ + @Test + public void testFailureOnCheckpointRecoveryFilesWrite() throws Exception { + spoilFilePattern = CheckpointRecoveryFileStorage.FILE_NAME_PATTERN; + + IgniteEx ignite = initIgnite(); + + spoiledPageLimit.set(10); + fail.set(true); + + try { + forceCheckpoint(); + } + catch (Throwable ignore) { + // Expected. + } + + assertTrue(GridTestUtils.waitForCondition( + () -> Ignition.state(getTestIgniteInstanceName(0)) == IgniteState.STOPPED_ON_FAILURE, + 10_000 + )); + + fail.set(false); + + File cpDir = ignite.context().pdsFolderResolver().fileTree().checkpoint(); + + assertTrue(cpDir.listFiles(((dir, name) -> FILE_NAME_PATTERN.matcher(name).matches())).length > 0); + + ignite = startGrid(0); + + IgniteCache cache0 = ignite.cache(DEFAULT_CACHE_NAME); + + for (int i = 0; i < KEYS_CNT; i++) + assertEquals((Integer)i, cache0.get(i)); + } + + /** */ + private IgniteEx initIgnite() throws Exception { + IgniteEx ignite = startGrid(0); + ignite.cluster().state(ClusterState.ACTIVE); + + CacheConfiguration cacheCfg = GridAbstractTest.defaultCacheConfiguration() + .setAffinity(new RendezvousAffinityFunction(false, PARTS)) + .setEncryptionEnabled(encrypt); + + if (encrypt) + cacheCfg.setDiskPageCompression(DiskPageCompression.DISABLED); + + IgniteCache cache = ignite.createCache(cacheCfg); + + for (int i = 0; i < KEYS_CNT; i++) + cache.put(i, i); + + return ignite; + } + + /** */ + private static final class PageStoreSpoilingFileIOFactory implements FileIOFactory { + /** */ + private final FileIOFactory delegateFactory; + + /** */ + private final AtomicBoolean failFlag; + + /** */ + private final AtomicInteger spoiledPageLimit; + + /** */ + private final Pattern filePattern; + + /** */ + PageStoreSpoilingFileIOFactory(AtomicBoolean failFlag, AtomicInteger spoiledPageLimit, Pattern filePattern) { + delegateFactory = new RandomAccessFileIOFactory(); + + this.failFlag = failFlag; + this.spoiledPageLimit = spoiledPageLimit; + this.filePattern = filePattern; + } + + /** {@inheritDoc}*/ + @Override public FileIO create(File file, OpenOption... modes) throws IOException { + FileIO delegate = delegateFactory.create(file, modes); + + return filePattern.matcher(file.getName()).matches() + ? new PageStoreSpoiling(delegate) + : delegate; + } + + /** */ + final class PageStoreSpoiling extends FileIODecorator { + /** */ + private final AtomicInteger spoiledPages = new AtomicInteger(); + + /** + * @param delegate File I/O delegate + */ + public PageStoreSpoiling(FileIO delegate) { + super(delegate); + } + + /** {@inheritDoc} */ + @Override public int writeFully(ByteBuffer srcBuf) throws IOException { + spoilBufferIfNeeded(srcBuf); + + return delegate.writeFully(srcBuf); + } + + /** {@inheritDoc} */ + @Override public int writeFully(ByteBuffer srcBuf, long position) throws IOException { + spoilBufferIfNeeded(srcBuf); + + return delegate.writeFully(srcBuf, position); + } + + /** */ + private void spoilBufferIfNeeded(ByteBuffer srcBuf) throws IOException { + if (failFlag.get()) { + // Spoil specified pages amount and after that throw an exception. + if (spoiledPages.getAndIncrement() > spoiledPageLimit.get()) + throw new IOException("Test exception."); + else { + srcBuf = ByteBuffer.allocate(srcBuf.remaining()).order(ByteOrder.nativeOrder()); + ThreadLocalRandom.current().nextBytes(srcBuf.array()); + } + } + } + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/CheckpointStartLoggingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/CheckpointStartLoggingTest.java index 0a6f8c731688b..c9f4d9b6cb792 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/CheckpointStartLoggingTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/CheckpointStartLoggingTest.java @@ -43,8 +43,9 @@ public class CheckpointStartLoggingTest extends GridCommonAbstractTest { "checkpointListenersExecuteTime=" + VALID_MS_PATTERN + ", " + "checkpointLockHoldTime=" + VALID_MS_PATTERN + ", " + "walCpRecordFsyncDuration=" + VALID_MS_PATTERN + ", " + - "writeCheckpointEntryDuration=" + VALID_MS_PATTERN + ", " + "splitAndSortCpPagesDuration=" + VALID_MS_PATTERN + ", " + + "writeRecoveryDataDuration=" + VALID_MS_PATTERN + ", " + + "writeCheckpointEntryDuration=" + VALID_MS_PATTERN + ", " + ".*pages=[1-9][0-9]*, " + "reason=.*"; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java index 7793c994f3822..7c29e30bf5694 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java @@ -74,7 +74,6 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.TrackingPageIO; import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; -import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.lang.GridCloseableIterator; import org.apache.ignite.internal.util.lang.GridFilteredClosableIterator; import org.apache.ignite.internal.util.typedef.F; @@ -571,7 +570,7 @@ public void testDirtyFlag() throws Exception { ig.context().cache().context().database().checkpointReadUnlock(); } - Collection cpPages = mem.beginCheckpoint(new GridFinishedFuture()); + Collection cpPages = mem.beginCheckpoint(() -> Boolean.TRUE); ig.context().cache().context().database().checkpointReadLock(); @@ -603,7 +602,7 @@ public void testDirtyFlag() throws Exception { buf.rewind(); mem.checkpointWritePage(fullId, buf, (fullPageId, buffer, tag) -> { - }, null); + }, null, false); buf.position(PageIO.COMMON_HEADER_END); @@ -901,7 +900,7 @@ private IgniteBiTuple, WALPointer> runCheckpointing( try { snapshot = new HashMap<>(resMap); - pageIds = mem.beginCheckpoint(new GridFinishedFuture()); + pageIds = mem.beginCheckpoint(() -> Boolean.TRUE); checkpoints--; @@ -940,7 +939,7 @@ private IgniteBiTuple, WALPointer> runCheckpointing( }; while (true) { - mem.checkpointWritePage(fullId, tmpBuf, pageStoreWriter, null); + mem.checkpointWritePage(fullId, tmpBuf, pageStoreWriter, null, false); tag = tag0.get(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java index a1879db9c080b..b66fcd5f9e9b9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java @@ -48,7 +48,6 @@ import org.apache.ignite.internal.processors.performancestatistics.PerformanceStatisticsProcessor; import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor; import org.apache.ignite.internal.util.GridMultiCollectionWrapper; -import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteOutClosure; import org.apache.ignite.logger.NullLogger; @@ -120,7 +119,7 @@ public void testReplacementWithDelayCausesLockForRead() throws IgniteCheckedExce memory.releasePage(1, pageId, ptr); } - GridMultiCollectionWrapper ids = memory.beginCheckpoint(new GridFinishedFuture()); + GridMultiCollectionWrapper ids = memory.beginCheckpoint(() -> Boolean.TRUE); int cpPages = ids.size(); log.info("Started CP with [" + cpPages + "] pages in it, created [" + markDirty + "] pages"); @@ -183,7 +182,7 @@ public void testBackwardCompatibilityMode() throws IgniteCheckedException { memory.releasePage(1, pageId, ptr); } - GridMultiCollectionWrapper ids = memory.beginCheckpoint(new GridFinishedFuture()); + GridMultiCollectionWrapper ids = memory.beginCheckpoint(() -> Boolean.TRUE); int cpPages = ids.size(); log.info("Started CP with [" + cpPages + "] pages in it, created [" + markDirty + "] pages"); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgniteThrottlingUnitTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgniteThrottlingUnitTest.java index cef358e6c5f6a..d59011189590b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgniteThrottlingUnitTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgniteThrottlingUnitTest.java @@ -48,6 +48,7 @@ import org.mockito.Mockito; import static java.lang.Thread.State.TIMED_WAITING; +import static org.apache.ignite.internal.processors.cache.persistence.pagemem.PagesWriteThrottlePolicy.CP_BUF_THROTTLING_THRESHOLD_FILL_RATE; import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThan; @@ -102,7 +103,8 @@ public class IgniteThrottlingUnitTest extends GridCommonAbstractTest { */ @Test public void shouldThrottleWhenWritingTooFast() { - PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, null, stateChecker, log); + PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, null, + stateChecker, false, log); long parkTime = throttle.getCleanPagesProtectionParkTime(0.67, (362584 + 67064) / 2, @@ -119,7 +121,8 @@ public void shouldThrottleWhenWritingTooFast() { */ @Test public void shouldNotThrottleWhenWritingSlowly() { - PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, null, stateChecker, log); + PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, null, + stateChecker, false, log); long parkTime = throttle.getCleanPagesProtectionParkTime(0.47, ((362584 + 67064) / 2), @@ -137,7 +140,8 @@ public void shouldNotThrottleWhenWritingSlowly() { */ @Test public void shouldNotThrottleWhenThereArePlentyCleanPages() { - PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, null, stateChecker, log); + PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, null, + stateChecker, false, log); long parkTime = throttle.getCleanPagesProtectionParkTime(0.0, (362584 + 67064) / 2, @@ -155,7 +159,8 @@ public void shouldNotThrottleWhenThereArePlentyCleanPages() { */ @Test public void testCorrectTimeToPark() { - PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, null, stateChecker, log); + PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, null, + stateChecker, false, log); int markDirtySpeed = 34422; int cpWriteSpeed = 19416; @@ -239,7 +244,8 @@ public void speedWithDelayCalculation() throws InterruptedException { */ @Test public void beginOfCp() { - PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, null, stateChecker, log); + PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, null, + stateChecker, false, log); assertEquals(0, throttle.getCleanPagesProtectionParkTime(0.01, 100, 400000, 1, @@ -266,7 +272,8 @@ public void beginOfCp() { */ @Test public void enforceThrottleAtTheEndOfCp() { - PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, null, stateChecker, log); + PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, null, + stateChecker, false, log); long time1 = throttle.getCleanPagesProtectionParkTime(0.70, 300000, 400000, 1, 20200, 23000); @@ -289,7 +296,8 @@ public void enforceThrottleAtTheEndOfCp() { */ @Test public void doNotThrottleWhenDirtyPagesRatioIsTooHigh() { - PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, null, stateChecker, log); + PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, null, + stateChecker, false, log); // 363308 350004 348976 10604 long time = throttle.getCleanPagesProtectionParkTime(0.75, @@ -314,7 +322,8 @@ public void wakeupSpeedBaseThrottledThreadOnCheckpointFinish() throws IgniteInte IgniteOutClosure cpProgress = mock(IgniteOutClosure.class); when(cpProgress.apply()).thenReturn(cl0); - PagesWriteThrottlePolicy plc = new PagesWriteSpeedBasedThrottle(pageMemory2g, cpProgress, stateChecker, log) { + PagesWriteThrottlePolicy plc = new PagesWriteSpeedBasedThrottle(pageMemory2g, cpProgress, stateChecker, + false, log) { @Override protected void doPark(long throttleParkTimeNs) { //Force parking to long time. super.doPark(TimeUnit.SECONDS.toNanos(1)); @@ -372,7 +381,10 @@ private void stopReportingCheckpointProgress(IgniteOutClosure loadThreads = new ArrayList<>(); @@ -387,8 +399,6 @@ public void wakeupThrottledThread() throws IgniteInterruptedCheckedException { )); } - when(pageMemory2g.checkpointBufferPagesSize()).thenReturn(100); - AtomicInteger checkpointBufPagesCnt = new AtomicInteger(70); when(pageMemory2g.checkpointBufferPagesCount()).thenAnswer(mock -> checkpointBufPagesCnt.get()); @@ -442,7 +452,8 @@ public void warningInCaseTooMuchThrottling() { Mockito.when(progress.writtenPagesCounter()).thenReturn(written); - PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, cpProvider, stateChecker, log) { + PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, cpProvider, + stateChecker, false, log) { @Override protected void doPark(long throttleParkTimeNs) { //do nothing } @@ -476,7 +487,7 @@ public void speedBasedThrottleShouldThrottleWhenCheckpointBufferIsInDangerZone() simulateCheckpointProgressIsStarted(); simulateCheckpointBufferInDangerZoneSituation(); PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, cpProvider, - stateChecker, log); + stateChecker, false, log); throttle.onMarkDirty(true); @@ -500,7 +511,7 @@ public void speedBasedThrottleShouldThrottleWhenCheckpointCountersAreNotReadyYet simulateCheckpointProgressNotYetStarted(); simulateCheckpointBufferInDangerZoneSituation(); PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, cpProvider, - stateChecker, log); + stateChecker, false, log); throttle.onMarkDirty(true); @@ -518,7 +529,7 @@ public void speedBasedThrottleShouldNotLeaveTracesInStatisticsWhenCPBufferIsInSa simulateCheckpointProgressNotYetStarted(); simulateCheckpointBufferInSafeZoneSituation(); PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, cpProvider, - stateChecker, log); + stateChecker, false, log); throttle.onMarkDirty(true); @@ -538,7 +549,7 @@ public void speedBasedThrottleShouldResetCPBufferProtectionParkTimeWhenItSeesTha simulateCheckpointProgressIsStarted(); AtomicLong parkTimeNanos = new AtomicLong(); PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, cpProvider, - stateChecker, log) { + stateChecker, false, log) { @Override protected void doPark(long throttleParkTimeNs) { super.doPark(1); parkTimeNanos.set(throttleParkTimeNs); @@ -575,7 +586,7 @@ public void speedBasedThrottleShouldNotResetCPBufferProtectionParkTimeWhenItSees simulateCheckpointProgressIsStarted(); AtomicLong parkTimeNanos = new AtomicLong(); PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, cpProvider, - stateChecker, log) { + stateChecker, false, log) { @Override protected void doPark(long throttleParkTimeNs) { super.doPark(1); parkTimeNanos.set(throttleParkTimeNs); @@ -605,7 +616,7 @@ public void speedBasedThrottleShouldNotResetCPBufferProtectionParkTimeWhenItSees public void speedBasedThrottleShouldReportCpWriteSpeedWhenThePageIsNotInCheckpointAndProgressIsReported() throws InterruptedException { PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, cpProvider, - stateChecker, log); + stateChecker, false, log); simulateCheckpointProgressIsStarted(); allowSomeTimeToPass(); throttle.onMarkDirty(false); @@ -622,7 +633,7 @@ private void allowSomeTimeToPass() throws InterruptedException { @Test public void speedBasedThrottleShouldResetCPProgressToZeroOnCheckpointStart() throws InterruptedException { PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, cpProvider, - stateChecker, log); + stateChecker, false, log); simulateCheckpointProgressIsStarted(); allowSomeTimeToPass(); throttle.onMarkDirty(false); @@ -632,4 +643,68 @@ public void speedBasedThrottleShouldResetCPProgressToZeroOnCheckpointStart() thr // verify progress speed to make a conclusion about progress itself assertThat(throttle.getCpWriteSpeed(), is(0L)); } + + /** */ + private void simulateCheckpointProgress( + double cpBufFillRate, + double cpRecoveryDataWriteProgress, + double cpPageWriteProgress + ) { + int cpBufPagesCnt = (int)(pageMemory2g.checkpointBufferPagesSize() * cpBufFillRate); + when(pageMemory2g.checkpointBufferPagesCount()).thenReturn(cpBufPagesCnt); + + int cpWrittenRecoveryPagesCnt = (int)(progress.currentCheckpointPagesCount() * cpRecoveryDataWriteProgress); + when(progress.writtenRecoveryPagesCounter()).thenReturn(new AtomicInteger(cpWrittenRecoveryPagesCnt)); + + int cpWrittenDataPagesCnt = (int)(progress.currentCheckpointPagesCount() * cpPageWriteProgress); + when(progress.writtenPagesCounter()).thenReturn(new AtomicInteger(cpWrittenDataPagesCnt)); + } + + /** */ + @Test + public void testFillRateBasedCheckpointBufferProtection() { + int cpBufSize = 1_000; + int cpPagesCnt = 1_000_000; + + when(progress.currentCheckpointPagesCount()).thenReturn(cpPagesCnt); + when(pageMemory2g.checkpointBufferPagesSize()).thenReturn(cpBufSize); + + AtomicLong parkTimeNanos = new AtomicLong(); + PagesWriteSpeedBasedThrottle throttle = new PagesWriteSpeedBasedThrottle(pageMemory2g, cpProvider, + stateChecker, true, log) { + @Override protected void doPark(long throttleParkTimeNs) { + super.doPark(1); + parkTimeNanos.set(throttleParkTimeNs); + } + }; + + parkTimeNanos.set(0); + simulateCheckpointProgress(CP_BUF_THROTTLING_THRESHOLD_FILL_RATE - 0.01, 0.01, 0); + throttle.onMarkDirty(true); + assertEquals(0, parkTimeNanos.get()); + + parkTimeNanos.set(0); + simulateCheckpointProgress(CP_BUF_THROTTLING_THRESHOLD_FILL_RATE + 0.01, 0.01, 0); + throttle.onMarkDirty(true); + long parkTime0 = parkTimeNanos.get(); + assertTrue(parkTime0 > 0); + + parkTimeNanos.set(0); + simulateCheckpointProgress(CP_BUF_THROTTLING_THRESHOLD_FILL_RATE + 0.1, 0.01, 0); + throttle.onMarkDirty(true); + long parkTime1 = parkTimeNanos.get(); + assertTrue(parkTime1 > parkTime0); + + parkTimeNanos.set(0); + simulateCheckpointProgress(0.99, 0.5, 0); + throttle.onMarkDirty(true); + long parkTime2 = parkTimeNanos.get(); + assertTrue(parkTime2 > parkTime1); + + // Checkpoint progress ahead of checkpoint buffer fill rate. + parkTimeNanos.set(0); + simulateCheckpointProgress(0.9, 1, 0.9); + throttle.onMarkDirty(true); + assertEquals(0, parkTimeNanos.get()); + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java index 67e66077887d9..104ad12800028 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java @@ -119,7 +119,7 @@ public class NoOpWALManager implements IgniteWriteAheadLogManager { } /** {@inheritDoc} */ - @Override public boolean disabled(int grpId, long pageId) { + @Override public boolean pageRecordsDisabled(int grpId, long pageId) { return false; } 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 7c91311d2c4cd..ea72d1b9c1864 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 @@ -60,7 +60,6 @@ import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor; import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor; import org.apache.ignite.internal.util.GridMultiCollectionWrapper; -import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgniteOutClosure; @@ -135,7 +134,7 @@ public void testCheckpointBufferOverusageDontCauseWriteLockLeak() throws Excepti //Success } - memory.beginCheckpoint(new GridFinishedFuture()); + memory.beginCheckpoint(() -> Boolean.TRUE); final AtomicReference lastPage = new AtomicReference<>(); @@ -246,14 +245,14 @@ public void testCheckpointProtocolWriteDirtyPageAfterWriteUnlock() throws Except writePage(memory, fullId, (byte)1); } - doCheckpoint(memory.beginCheckpoint(new GridFinishedFuture()), memory, pageStoreMgr); + doCheckpoint(memory.beginCheckpoint(() -> Boolean.TRUE), memory, pageStoreMgr); FullPageId cowPageId = allocated.get(0); // Mark some pages as dirty. writePage(memory, cowPageId, (byte)2); - GridMultiCollectionWrapper cpPages = memory.beginCheckpoint(new GridFinishedFuture()); + GridMultiCollectionWrapper cpPages = memory.beginCheckpoint(() -> Boolean.TRUE); assertEquals(1, cpPages.size()); @@ -306,7 +305,7 @@ public void runThrottlingEmptifyCpBufFirst(PageMemoryImpl.ThrottlingPolicy plc) writePage(memory, fullId, (byte)1); } - GridMultiCollectionWrapper markedPages = memory.beginCheckpoint(new GridFinishedFuture()); + GridMultiCollectionWrapper markedPages = memory.beginCheckpoint(() -> Boolean.TRUE); for (int i = 0; i < pagesForStartThrottling + (memory.checkpointBufferPagesSize() * 2 / 3); i++) writePage(memory, allocated.get(i), (byte)1); @@ -342,7 +341,7 @@ private void doCheckpoint( ByteBuffer buf = ByteBuffer.wrap(data); - memory.checkpointWritePage(cpPage, buf, pageStoreWriter, null); + memory.checkpointWritePage(cpPage, buf, pageStoreWriter, null, false); while (memory.isCpBufferOverflowThresholdExceeded()) { FullPageId cpPageId = memory.pullPageFromCpBuffer(); @@ -356,7 +355,7 @@ private void doCheckpoint( tmpWriteBuf.rewind(); - memory.checkpointWritePage(cpPageId, tmpWriteBuf, pageStoreWriter, null); + memory.checkpointWritePage(cpPageId, tmpWriteBuf, pageStoreWriter, null, false); } } @@ -392,7 +391,7 @@ public void testCheckpointProtocolCannotReplaceUnwrittenPage() throws Exception } // CP Write lock. - memory.beginCheckpoint(new GridFinishedFuture()); + memory.beginCheckpoint(() -> Boolean.TRUE); // CP Write unlock. byte[] buf = new byte[PAGE_SIZE]; @@ -416,7 +415,7 @@ public void testCheckpointProtocolCannotReplaceUnwrittenPage() throws Exception assertTrue("Should oom before check replaced page.", oom); assertTrue("Missing page: " + fullPageId, memory.hasLoadedPage(fullPageId)); - }, null); + }, null, false); } /** @@ -466,7 +465,7 @@ private void testCheckpointBufferCantOverflowWithThrottlingMixedLoad(PageMemoryI acquireAndReleaseWriteLock(memory, fullPageId); } - memory.beginCheckpoint(new GridFinishedFuture()); + memory.beginCheckpoint(() -> Boolean.TRUE); CheckpointMetricsTracker mockTracker = Mockito.mock(CheckpointMetricsTracker.class); @@ -474,7 +473,7 @@ private void testCheckpointBufferCantOverflowWithThrottlingMixedLoad(PageMemoryI memory.checkpointWritePage(checkpointPage, ByteBuffer.allocate(PAGE_SIZE), (fullPageId, buffer, tag) -> { // No-op. - }, mockTracker); + }, mockTracker, false); memory.finishCheckpoint(); @@ -488,7 +487,7 @@ private void testCheckpointBufferCantOverflowWithThrottlingMixedLoad(PageMemoryI acquireAndReleaseWriteLock(memory, fullPageId); } - memory.beginCheckpoint(new GridFinishedFuture()); + memory.beginCheckpoint(() -> Boolean.TRUE); Collections.shuffle(pages); // Mix pages in checkpoint with clean pages diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/AbstractPerformanceStatisticsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/AbstractPerformanceStatisticsTest.java index 4384d5471bd8e..cc0d4ad41003c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/AbstractPerformanceStatisticsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/AbstractPerformanceStatisticsTest.java @@ -223,6 +223,7 @@ public static class TestHandler implements PerformanceStatisticsHandler { long walCpRecordFsyncDuration, long writeCpEntryDuration, long splitAndSortCpPagesDuration, + long recoveryDataWriteDuration, long totalDuration, long cpStartTime, int pagesSize, diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/CheckpointTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/CheckpointTest.java index eaf53f385b8ae..2604b6e25e98c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/CheckpointTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/CheckpointTest.java @@ -125,6 +125,7 @@ public void testCheckpoint() throws Exception { long walCpRecordFsyncDuration, long writeCpEntryDuration, long splitAndSortCpPagesDuration, + long recoveryDataWriteDuration, long totalDuration, long cpStartTime, int pagesSize, @@ -150,6 +151,8 @@ public void testCheckpoint() throws Exception { writeCpEntryDuration); assertEquals(mreg.findMetric("LastCheckpointSplitAndSortPagesDuration").value(), splitAndSortCpPagesDuration); + assertEquals(mreg.findMetric("LastCheckpointRecoveryDataWriteDuration").value(), + recoveryDataWriteDuration); assertEquals(mreg.findMetric("LastCheckpointDuration").value(), totalDuration); assertEquals(lastStart.value(), cpStartTime); assertEquals(mreg.findMetric("LastCheckpointTotalPagesNumber").value(), pagesSize); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/ForwardReadTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/ForwardReadTest.java index 2719bc76d927e..c5c2238e13489 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/ForwardReadTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/ForwardReadTest.java @@ -91,6 +91,9 @@ private Map createStatistics(File dir) throws Exception { try (FileIO fileIo = new RandomAccessFileIOFactory().create(file)) { ByteBuffer buf = ByteBuffer.allocate(10 * 1024).order(ByteOrder.nativeOrder()); + buf.put(OperationType.VERSION.id()); + buf.putShort(FilePerformanceStatisticsWriter.FILE_FORMAT_VERSION); + expTasks = writeData(buf); buf.flip(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsPropertiesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsPropertiesTest.java index 1b7a35b253558..576a422bbf889 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsPropertiesTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsPropertiesTest.java @@ -73,7 +73,9 @@ public class PerformanceStatisticsPropertiesTest extends AbstractPerformanceStat @Test @WithSystemProperty(key = IGNITE_PERF_STAT_FILE_MAX_SIZE, value = "" + TEST_FILE_MAX_SIZE) public void testFileMaxSize() throws Exception { - long initLen = srv.context().cache().cacheDescriptors().values().stream().mapToInt( + long initLen = 1 + OperationType.versionRecordSize(); + + initLen += srv.context().cache().cacheDescriptors().values().stream().mapToInt( desc -> 1 + cacheStartRecordSize(desc.cacheName().getBytes().length, false)).sum(); long expOpsCnt = (TEST_FILE_MAX_SIZE - initLen) / (/*typeOp*/1 + OperationType.cacheRecordSize()); @@ -155,7 +157,7 @@ public void testCachedStringsThreshold() throws Exception { startCollectStatistics(); - int expLen = 0; + int expLen = 1 + OperationType.versionRecordSize(); for (int i = 0; i < tasksCnt; i++) { String taskName = "TestTask-" + i; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/StringCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/StringCacheTest.java index d423bf207f5ce..db8239286417f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/StringCacheTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/StringCacheTest.java @@ -64,7 +64,8 @@ public void testCacheTaskName() throws Exception { assertEquals(executions, tasks.get()); - long expLen = taskRecordSize(testTaskName.getBytes().length, false) + + long expLen = 1 + OperationType.versionRecordSize(); + expLen += taskRecordSize(testTaskName.getBytes().length, false) + taskRecordSize(0, true) * (executions - 1) + jobRecordSize() * executions + /*opType*/ 2 * executions; diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite8.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite8.java index 87743fe7c3793..f38ef204f360f 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite8.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite8.java @@ -34,6 +34,7 @@ import org.apache.ignite.internal.processors.cache.persistence.NoUnnecessaryRebalanceTest; import org.apache.ignite.internal.processors.cache.persistence.PagesPossibleCorruptionDiagnosticTest; import org.apache.ignite.internal.processors.cache.persistence.PendingTreeCorruptionTest; +import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsCheckpointRecoveryTest; import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsPageReplacementDuringPartitionClearTest; import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsTransactionsHangTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.HistoricalReservationTest; @@ -107,6 +108,7 @@ public static List> suite(Collection ignoredTests) { GridTestUtils.addTestIfNeeded(suite, MaintenancePersistenceTaskTest.class, ignoredTests); GridTestUtils.addTestIfNeeded(suite, ConnectionEnabledPropertyTest.class, ignoredTests); + GridTestUtils.addTestIfNeeded(suite, IgnitePdsCheckpointRecoveryTest.class, ignoredTests); return suite; } diff --git a/modules/ducktests/tests/ignitetest/services/utils/ignite_configuration/data_storage.py b/modules/ducktests/tests/ignitetest/services/utils/ignite_configuration/data_storage.py index 7b642109999ce..dbc0674468f05 100644 --- a/modules/ducktests/tests/ignitetest/services/utils/ignite_configuration/data_storage.py +++ b/modules/ducktests/tests/ignitetest/services/utils/ignite_configuration/data_storage.py @@ -32,6 +32,7 @@ class DataRegionConfiguration(NamedTuple): max_size: int = 512 * 1024 * 1024 metrics_enabled: bool = True metrics_rate_time_interval: int = None + checkpoint_page_buffer_size: int = None class DataStorageConfiguration(NamedTuple): @@ -56,3 +57,6 @@ class DataStorageConfiguration(NamedTuple): wal_page_compression: str = None wal_page_compression_level: int = None wal_path: str = None + write_recovery_data_on_checkpoint: bool = None + checkpoint_recovery_data_compression: str = None + checkpoint_recovery_data_compression_level: int = None