From d5976e5483f17ad8610381752b5d739dbab318d8 Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Wed, 9 Dec 2020 21:00:26 +0300 Subject: [PATCH 01/21] IGNITE-13831 wip --- .../persistence/checkpoint/Checkpoint.java | 19 +++++-------------- .../checkpoint/CheckpointHistory.java | 6 +----- .../persistence/checkpoint/Checkpointer.java | 16 +++++++++------- 3 files changed, 15 insertions(+), 26 deletions(-) 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 e6a378b18de77..bc842dd98dcf6 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 @@ -21,7 +21,6 @@ import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx; import org.apache.ignite.internal.util.GridConcurrentMultiPairQueue; import org.apache.ignite.lang.IgniteBiTuple; -import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; /** @@ -34,26 +33,25 @@ class Checkpoint { /** Checkpoint pages. */ final GridConcurrentMultiPairQueue cpPages; - /** */ + /** Checkpoint progress status. */ final CheckpointProgressImpl progress; - /** Number of deleted WAL files. */ - int walFilesDeleted; - /** WAL segments fully covered by this checkpoint. */ IgniteBiTuple walSegsCoveredRange; - /** */ + /** Number of dirty pages. */ final int pagesSize; /** + * Constructor. + * * @param cpEntry Checkpoint entry. * @param cpPages Pages to write to the page store. * @param progress Checkpoint progress status. */ Checkpoint( @Nullable CheckpointEntry cpEntry, - @NotNull GridConcurrentMultiPairQueue cpPages, + GridConcurrentMultiPairQueue cpPages, CheckpointProgressImpl progress ) { this.cpEntry = cpEntry; @@ -70,13 +68,6 @@ public boolean hasDelta() { return pagesSize != 0; } - /** - * @param walFilesDeleted Wal files deleted. - */ - public void walFilesDeleted(int walFilesDeleted) { - this.walFilesDeleted = walFilesDeleted; - } - /** * @param walSegsCoveredRange WAL segments fully covered by this checkpoint. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java index 11125792f0c1b..b34bbfcee49ba 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java @@ -375,12 +375,8 @@ public List onCheckpointFinished(Checkpoint chp) { List deletedCheckpoints = onWalTruncated(checkpointMarkUntilDel); - int deleted = 0; - if (truncateWalOnCpFinish) - deleted += wal.truncate(null, firstCheckpointPointer()); - - chp.walFilesDeleted(deleted); + wal.truncate(null, firstCheckpointPointer()); return deletedCheckpoints; } 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 6747e583a8ccc..63440b2dbca4c 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 @@ -456,16 +456,12 @@ private void doCheckpoint() { if (chp.hasDelta() || destroyedPartitionsCnt > 0) { if (log.isInfoEnabled()) { - String walSegsCoveredMsg = chp.walSegsCoveredRange == null ? "" : prepareWalSegsCoveredMsg(chp.walSegsCoveredRange); - log.info(String.format("Checkpoint finished [cpId=%s, pages=%d, markPos=%s, " + - "walSegmentsCleared=%d, walSegmentsCovered=%s, markDuration=%dms, pagesWrite=%dms, fsync=%dms, " + - "total=%dms]", + "walSegmentsCovered=%s, markDuration=%dms, pagesWrite=%dms, fsync=%dms, total=%dms]", chp.cpEntry != null ? chp.cpEntry.checkpointId() : "", chp.pagesSize, chp.cpEntry != null ? chp.cpEntry.checkpointMark() : "", - chp.walFilesDeleted, - walSegsCoveredMsg, + walRangeStr(chp.walSegsCoveredRange), tracker.markDuration(), tracker.pagesWriteDuration(), tracker.fsyncDuration(), @@ -591,9 +587,15 @@ private void updateMetrics(Checkpoint chp, CheckpointMetricsTracker tracker) { } /** + * Creates a string of a range WAL segments. + * + * @param walRange Range of WAL segments. * @return The message about how many WAL segments was between previous checkpoint and current one. */ - private String prepareWalSegsCoveredMsg(IgniteBiTuple walRange) { + private String walRangeStr(@Nullable IgniteBiTuple walRange) { + if (walRange == null) + return ""; + String res; long startIdx = walRange.get1(); From 892def921b535b8eba1d7736c1743ecf534e2710 Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Thu, 10 Dec 2020 10:18:32 +0300 Subject: [PATCH 02/21] IGNITE-13831 wip add archiveSize --- .../wal/FileWriteAheadLogManager.java | 171 +++++++++++------- 1 file changed, 107 insertions(+), 64 deletions(-) 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 ff64f5b034156..dbc18917de2b3 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 @@ -400,6 +400,9 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl */ private final Map segmentSize = new ConcurrentHashMap<>(); + /** Current size of WAL archive in bytes. */ + private final AtomicLong archiveSize = new AtomicLong(); + /** * @param ctx Kernal context. */ @@ -1061,6 +1064,8 @@ private boolean hasIndex(long absIdx) { long lastArchived = archivedAbsIdx >= 0 ? archivedAbsIdx : lastArchivedIndex(); + long len = desc.file.length(); + // We need to leave at least one archived segment to correctly determine the archive index. if (desc.idx < high.index() && desc.idx < lastArchived) { if (!desc.file.delete()) { @@ -1071,6 +1076,7 @@ private boolean hasIndex(long absIdx) { deleted++; segmentSize.remove(desc.idx()); + archiveSize.addAndGet(-len); } // Bump up the oldest archive segment index. @@ -1277,6 +1283,8 @@ private FileWriteHandle rollOver(FileWriteHandle cur, @Nullable WALRecord rec) t switchSegmentRecordOffset.set(idx, hnd.getSwitchSegmentRecordOffset()); } + // TODO: 09.12.2020 где-тут или ниже + FileWriteHandle next; try { next = initNextWriteHandle(cur); @@ -1294,6 +1302,11 @@ private FileWriteHandle rollOver(FileWriteHandle cur, @Nullable WALRecord rec) t assert ptr != null; } + segmentSize.put(next.getSegmentId(), maxWalSegmentSize); + + if (archiver == null) + archiveSize.addAndGet(maxWalSegmentSize); + if (next.getSegmentId() - lashCheckpointFileIdx() >= maxSegCountWithoutCheckpoint) cctx.database().forceCheckpoint("too big size of WAL without checkpoint"); @@ -2023,10 +2036,10 @@ public SegmentArchiveResult archiveSegment(long absIdx) throws StorageException } try { - Files.deleteIfExists(dstTmpFile.toPath()); - boolean copied = false; + assert switchSegmentRecordOffset != null; + long offs = switchSegmentRecordOffset.get((int)segIdx); if (offs > 0) { @@ -2051,8 +2064,11 @@ public SegmentArchiveResult archiveSegment(long absIdx) throws StorageException } segmentSize.put(absIdx, dstFile.length()); + archiveSize.addAndGet(dstFile.length()); } catch (IOException e) { + deleteArchiveFiles(false, dstFile, dstTmpFile); + throw new StorageException("Failed to archive WAL segment [" + "srcFile=" + origFile.getAbsolutePath() + ", dstFile=" + dstTmpFile.getAbsolutePath() + ']', e); @@ -2257,8 +2273,6 @@ private void body0() { if ((segIdx = tryReserveNextSegmentOrWait()) == -1) continue; - deleteObsoleteRawSegments(); - String segmentFileName = fileName(segIdx); File tmpZip = new File(walArchiveDir, segmentFileName + ZIP_SUFFIX + TMP_SUFFIX); @@ -2267,33 +2281,42 @@ private void body0() { File raw = new File(walArchiveDir, segmentFileName); - if (!Files.exists(raw.toPath())) - throw new IgniteCheckedException("WAL archive segment is missing: " + raw); + try { + deleteObsoleteRawSegments(); + + if (!Files.exists(raw.toPath())) + throw new IgniteCheckedException("WAL archive segment is missing: " + raw); - compressSegmentToFile(segIdx, raw, tmpZip); + compressSegmentToFile(segIdx, raw, tmpZip); - Files.move(tmpZip.toPath(), zip.toPath()); + Files.move(tmpZip.toPath(), zip.toPath()); - try (FileIO f0 = ioFactory.create(zip, CREATE, READ, WRITE)) { - f0.force(); + try (FileIO f0 = ioFactory.create(zip, CREATE, READ, WRITE)) { + f0.force(); + } + + segmentSize.put(segIdx, zip.length()); + archiveSize.addAndGet(zip.length()); + + segmentAware.onSegmentCompressed(segIdx); + + if (evt.isRecordable(EVT_WAL_SEGMENT_COMPACTED) && !cctx.kernalContext().recoveryMode()) + evt.record(new WalSegmentCompactedEvent(cctx.localNode(), segIdx, zip.getAbsoluteFile())); } + catch (IgniteCheckedException | IOException e) { + deleteArchiveFiles(false, zip, tmpZip); + + lastCompressionError = e; - segmentAware.onSegmentCompressed(segIdx); + U.error(log, "Compression of WAL segment [idx=" + segIdx + + "] was skipped due to unexpected error", lastCompressionError); - if (evt.isRecordable(EVT_WAL_SEGMENT_COMPACTED) && !cctx.kernalContext().recoveryMode()) - evt.record(new WalSegmentCompactedEvent(cctx.localNode(), segIdx, zip.getAbsoluteFile())); + segmentAware.onSegmentCompressed(segIdx); + } } catch (IgniteInterruptedCheckedException ignore) { Thread.currentThread().interrupt(); } - catch (IgniteCheckedException | IOException e) { - lastCompressionError = e; - - U.error(log, "Compression of WAL segment [idx=" + segIdx + - "] was skipped due to unexpected error", lastCompressionError); - - segmentAware.onSegmentCompressed(segIdx); - } finally { if (segIdx != -1L) release(new WALPointer(segIdx, 0, 0)); @@ -2353,8 +2376,6 @@ private void compressSegmentToFile(long idx, File raw, File zip) throws IOExcept zos.write(heapBuf.array()); } - - segmentSize.put(idx, zip.length()); } /** @@ -2382,7 +2403,7 @@ private ByteBuffer prepareSwitchSegmentRecordBuffer( * Deletes raw WAL segments if they aren't locked and already have compressed copies of themselves. */ private void deleteObsoleteRawSegments() { - FileDescriptor[] descs = scan(walArchiveDir.listFiles(WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER)); + FileDescriptor[] descs = walArchiveFiles(); Set indices = new HashSet<>(); Set duplicateIndices = new HashSet<>(); @@ -2400,13 +2421,8 @@ private void deleteObsoleteRawSegments() { if (segmentReservedOrLocked(desc.idx)) return; - if (desc.idx < segmentAware.keepUncompressedIdxFrom() && duplicateIndices.contains(desc.idx)) { - if (desc.file.exists() && !desc.file.delete()) { - U.warn(log, "Failed to remove obsolete WAL segment " + - "(make sure the process has enough rights): " + desc.file.getAbsolutePath() + - ", exists: " + desc.file.exists()); - } - } + if (desc.idx < segmentAware.keepUncompressedIdxFrom() && duplicateIndices.contains(desc.idx)) + deleteArchiveFiles(true, desc.file); } } } @@ -2440,24 +2456,27 @@ private class FileDecompressor extends GridWorker { while (!isCancelled()) { long segmentToDecompress = -1L; + blockingSectionBegin(); + try { - blockingSectionBegin(); + segmentToDecompress = segmentsQueue.take(); + } + finally { + blockingSectionEnd(); + } - try { - segmentToDecompress = segmentsQueue.take(); - } - finally { - blockingSectionEnd(); - } + if (isCancelled()) + break; - if (isCancelled()) - break; + String segmentFileName = fileName(segmentToDecompress); - String segmentFileName = fileName(segmentToDecompress); + File zip = new File(walArchiveDir, segmentFileName + ZIP_SUFFIX); + File unzipTmp = new File(walArchiveDir, segmentFileName + TMP_SUFFIX); + File unzip = new File(walArchiveDir, segmentFileName); - File zip = new File(walArchiveDir, segmentFileName + ZIP_SUFFIX); - File unzipTmp = new File(walArchiveDir, segmentFileName + TMP_SUFFIX); - File unzip = new File(walArchiveDir, segmentFileName); + try { + if (unzip.exists()) + throw new FileAlreadyExistsException(unzip.getAbsolutePath()); try (ZipInputStream zis = new ZipInputStream(new BufferedInputStream(new FileInputStream(zip))); FileIO io = ioFactory.create(unzipTmp)) { @@ -2467,32 +2486,35 @@ private class FileDecompressor extends GridWorker { updateHeartbeat(); } - try { - Files.move(unzipTmp.toPath(), unzip.toPath()); - } - catch (FileAlreadyExistsException e) { + Files.move(unzipTmp.toPath(), unzip.toPath()); + + archiveSize.addAndGet(unzip.length()); + } + catch (IOException e) { + deleteArchiveFiles(false, unzipTmp); + + if (e instanceof FileAlreadyExistsException) { U.error(log, "Can't rename temporary unzipped segment: raw segment is already present " + "[tmp=" + unzipTmp + ", raw=" + unzip + "]", e); - - if (!unzipTmp.delete()) - U.error(log, "Can't delete temporary unzipped segment [tmp=" + unzipTmp + "]"); } + else { + if (!isCancelled && segmentToDecompress != -1L) { + IgniteCheckedException ex = new IgniteCheckedException("Error during WAL segment " + + "decompression [segmentIdx=" + segmentToDecompress + "]", e); - updateHeartbeat(); + synchronized (this) { + decompressionFutures.remove(segmentToDecompress).onDone(ex); + } - synchronized (this) { - decompressionFutures.remove(segmentToDecompress).onDone(); + continue; + } } } - catch (IOException ex) { - if (!isCancelled && segmentToDecompress != -1L) { - IgniteCheckedException e = new IgniteCheckedException("Error during WAL segment " + - "decompression [segmentIdx=" + segmentToDecompress + "]", ex); - synchronized (this) { - decompressionFutures.remove(segmentToDecompress).onDone(e); - } - } + updateHeartbeat(); + + synchronized (this) { + decompressionFutures.remove(segmentToDecompress).onDone(); } } } @@ -3105,8 +3127,6 @@ private boolean updateCurrentHandle(FileWriteHandle n, @Nullable FileWriteHandle else res = CURR_HND_UPD.compareAndSet(this, c, n); - segmentSize.put(n.getSegmentId(), maxWalSegmentSize); - return res; } @@ -3120,4 +3140,27 @@ public static boolean isSegmentFileName(@Nullable String name) { return name != null && (WAL_NAME_PATTERN.matcher(name).matches() || WAL_SEGMENT_FILE_COMPACTED_PATTERN.matcher(name).matches()); } + + /** + * Removing files from {@link #walArchiveDir} with updating {@link #archiveSize}. + * + * @param updateArchiveSize Flag to update {@link #archiveSize}. + * @param files Files from {@link #walArchiveDir}. + */ + private void deleteArchiveFiles(boolean updateArchiveSize, File... files) { + for (File file : files) { + if (file.exists()) { + long len = file.length(); + + if (file.delete()) { + if (updateArchiveSize) + archiveSize.addAndGet(-len); + } + else if (file.exists()) { + U.warn(log, "Unable to delete file from WAL archive" + + " (make sure the process has enough rights): " + file.getAbsolutePath()); + } + } + } + } } From cc1564fb09e5ef47a33a21e239d0e7f3d8da5c7f Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Thu, 10 Dec 2020 16:27:00 +0300 Subject: [PATCH 03/21] IGNITE-13831 wip move clean wal archive to rollOver --- .../apache/ignite/IgniteSystemProperties.java | 6 +- .../wal/IgniteWriteAheadLogManager.java | 5 - .../GridCacheDatabaseSharedManager.java | 7 +- .../checkpoint/CheckpointHistory.java | 81 ++--------- .../checkpoint/CheckpointManager.java | 7 +- .../checkpoint/CheckpointMarkersStorage.java | 18 ++- .../wal/FileWriteAheadLogManager.java | 133 +++++++++--------- .../persistence/pagemem/NoOpWALManager.java | 5 - 8 files changed, 103 insertions(+), 159 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index aa12e541b06e2..6dd359241c600 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -1356,11 +1356,11 @@ public final class IgniteSystemProperties { public static final String IGNITE_CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE = "IGNITE_CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE"; /** - * Property for setup percentage of WAL archive size to calculate threshold since which removing of old archive should be started. - * Default value is 0.5 + * Property for setting percentage of WAL archive size for calculating threshold + * to which old segments will be deleted when maximum is reached. Default value is 0.5 */ @SystemProperty(value = "Percentage of WAL archive size to calculate threshold " + - "since which removing of old archive should be started", type = Double.class, + "to which old segments will be deleted when maximum is reached", type = Double.class, defaults = "" + DFLT_THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE) public static final String IGNITE_THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE = "IGNITE_THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE"; 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 cb4fc306cdb49..2dcbc7e523f3e 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 @@ -186,11 +186,6 @@ public WALIterator replay( */ public long lastCompactedSegment(); - /** - * @return Max allowed index of archived segment to delete or -1 if it does not exist. - */ - public long maxArchivedSegmentToDelete(); - /** * Checks if WAL segment is under lock or reserved * 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 4429ec76da02e..d42cc68e8d96b 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 @@ -2743,11 +2743,12 @@ private byte convertToTxState(TransactionState state) { } /** - * Wal truncate callBack. + * Wal truncate callback. * - * @param highBound WALPointer. + * @param highBound Upper bound. + * @throws IgniteCheckedException If failed. */ - public void onWalTruncated(WALPointer highBound) throws IgniteCheckedException { + public void onWalTruncated(@Nullable WALPointer highBound) throws IgniteCheckedException { checkpointManager.removeCheckpointsUntil(highBound); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java index b34bbfcee49ba..f1642f2d8807e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java @@ -323,10 +323,14 @@ public boolean hasSpace() { /** * Clears checkpoint history after WAL truncation. * + * @param highBound Upper bound. * @return List of checkpoint entries removed from history. */ - public List onWalTruncated(WALPointer highBound) { - List removed = new ArrayList<>(); + public List onWalTruncated(@Nullable WALPointer highBound) { + if (highBound == null) + return Collections.emptyList(); + + List rmv = new ArrayList<>(); for (CheckpointEntry cpEntry : histMap.values()) { WALPointer cpPnt = cpEntry.checkpointMark(); @@ -344,56 +348,30 @@ public List onWalTruncated(WALPointer highBound) { synchronized (earliestCp) { CheckpointEntry deletedCpEntry = histMap.remove(cpEntry.timestamp()); - CheckpointEntry oldestCpInHistory = firstCheckpoint(); + CheckpointEntry oldestCpInHist = firstCheckpoint(); for (Map.Entry grpPartPerCp : earliestCp.entrySet()) { if (grpPartPerCp.getValue() == deletedCpEntry) - grpPartPerCp.setValue(oldestCpInHistory); + grpPartPerCp.setValue(oldestCpInHist); } } - removed.add(cpEntry); + rmv.add(cpEntry); } - return removed; + return rmv; } /** * Logs and clears checkpoint history after checkpoint finish. * + * @param chp Finished checkpoint. * @return List of checkpoints removed from history. */ public List onCheckpointFinished(Checkpoint chp) { chp.walSegsCoveredRange(calculateWalSegmentsCovered()); - WALPointer checkpointMarkUntilDel = isWalHistorySizeParameterEnabled //check for compatibility mode. - ? checkpointMarkUntilDeleteByMemorySize() - : newerPointer(checkpointMarkUntilDeleteByMemorySize(), checkpointMarkUntilDeleteByArchiveSize()); - - if (checkpointMarkUntilDel == null) - return Collections.emptyList(); - - List deletedCheckpoints = onWalTruncated(checkpointMarkUntilDel); - - if (truncateWalOnCpFinish) - wal.truncate(null, firstCheckpointPointer()); - - return deletedCheckpoints; - } - - /** - * @param first One of pointers to choose the newest. - * @param second One of pointers to choose the newest. - * @return The newest pointer from input ones. - */ - private WALPointer newerPointer(WALPointer first, WALPointer second) { - if (first == null) - return second; - - if (second == null) - return first; - - return first.index() > second.index() ? first : second; + return onWalTruncated(truncateWalOnCpFinish ? null : checkpointMarkUntilDeleteByMemorySize()); } /** @@ -415,41 +393,6 @@ private WALPointer checkpointMarkUntilDeleteByMemorySize() { return lastCheckpoint().checkpointMark(); } - /** - * Calculate mark until delete by maximum allowed archive size. - * - * @return Checkpoint mark until which checkpoints can be deleted(not including this pointer). - */ - @Nullable private WALPointer checkpointMarkUntilDeleteByArchiveSize() { - long absFileIdxToDel = wal.maxArchivedSegmentToDelete(); - - if (absFileIdxToDel < 0) - return null; - - long fileUntilDel = absFileIdxToDel + 1; - - long checkpointFileIdx = absFileIdx(lastCheckpoint()); - - for (CheckpointEntry cpEntry : histMap.values()) { - long currFileIdx = absFileIdx(cpEntry); - - if (checkpointFileIdx <= currFileIdx || fileUntilDel <= currFileIdx) - return cpEntry.checkpointMark(); - } - - return lastCheckpoint().checkpointMark(); - } - - /** - * Retrieve absolute file index by checkpoint entry. - * - * @param pointer checkpoint entry for which need to calculate absolute file index. - * @return absolute file index for given checkpoint entry. - */ - private long absFileIdx(CheckpointEntry pointer) { - return pointer.checkpointMark().index(); - } - /** * Calculates indexes of WAL segments covered by last checkpoint. * 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 7fe7002db5f04..072d3bfe7f3eb 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 @@ -300,11 +300,12 @@ public void initializeStorage() throws IgniteCheckedException { } /** - * Wal truncate callBack. + * Wal truncate callback. * - * @param highBound WALPointer. + * @param highBound Upper bound. + * @throws IgniteCheckedException If failed. */ - public void removeCheckpointsUntil(WALPointer highBound) throws IgniteCheckedException { + public void removeCheckpointsUntil(@Nullable WALPointer highBound) throws IgniteCheckedException { checkpointMarkersStorage.removeCheckpointsUntil(highBound); } 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 b421a33fc6cec..d601d53fe65c1 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 @@ -141,24 +141,28 @@ public void initialize() throws IgniteCheckedException { } /** - * Wal truncate callBack. + * Wal truncate callback. * - * @param highBound WALPointer. + * @param highBound Upper bound. + * @throws IgniteCheckedException If failed. */ - public void removeCheckpointsUntil(WALPointer highBound) throws IgniteCheckedException { - List removedFromHistory = history().onWalTruncated(highBound); + public void removeCheckpointsUntil(@Nullable WALPointer highBound) throws IgniteCheckedException { + List rmvFromHist = history().onWalTruncated(highBound); - for (CheckpointEntry cp : removedFromHistory) + for (CheckpointEntry cp : rmvFromHist) removeCheckpointFiles(cp); } /** * Logs and clears checkpoint history after checkpoint finish. + * + * @param chp Finished checkpoint. + * @throws IgniteCheckedException If failed. */ public void onCheckpointFinished(Checkpoint chp) throws IgniteCheckedException { - List removedFromHistory = history().onCheckpointFinished(chp); + List rmvFromHist = history().onCheckpointFinished(chp); - for (CheckpointEntry cp : removedFromHistory) + for (CheckpointEntry cp : rmvFromHist) removeCheckpointFiles(cp); } 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 dbc18917de2b3..1960acd153e0f 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 @@ -169,46 +169,31 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl public static final Pattern WAL_TEMP_NAME_PATTERN = Pattern.compile("\\d{16}\\.wal\\.tmp"); /** WAL segment file filter, see {@link #WAL_NAME_PATTERN} */ - public static final FileFilter WAL_SEGMENT_FILE_FILTER = new FileFilter() { - @Override public boolean accept(File file) { - return !file.isDirectory() && WAL_NAME_PATTERN.matcher(file.getName()).matches(); - } - }; + public static final FileFilter WAL_SEGMENT_FILE_FILTER = file -> !file.isDirectory() && + WAL_NAME_PATTERN.matcher(file.getName()).matches(); /** WAL segment temporary file filter, see {@link #WAL_TEMP_NAME_PATTERN} */ - private static final FileFilter WAL_SEGMENT_TEMP_FILE_FILTER = new FileFilter() { - @Override public boolean accept(File file) { - return !file.isDirectory() && WAL_TEMP_NAME_PATTERN.matcher(file.getName()).matches(); - } - }; + private static final FileFilter WAL_SEGMENT_TEMP_FILE_FILTER = file -> !file.isDirectory() && + WAL_TEMP_NAME_PATTERN.matcher(file.getName()).matches(); /** */ public static final Pattern WAL_SEGMENT_FILE_COMPACTED_PATTERN = Pattern.compile("\\d{16}\\.wal\\.zip"); /** WAL segment file filter, see {@link #WAL_NAME_PATTERN} */ - public static final FileFilter WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER = new FileFilter() { - @Override public boolean accept(File file) { - return !file.isDirectory() && (WAL_NAME_PATTERN.matcher(file.getName()).matches() || - WAL_SEGMENT_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches()); - } - }; + public static final FileFilter WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER = file -> !file.isDirectory() && + (WAL_NAME_PATTERN.matcher(file.getName()).matches() || + WAL_SEGMENT_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches()); /** */ private static final Pattern WAL_SEGMENT_TEMP_FILE_COMPACTED_PATTERN = Pattern.compile("\\d{16}\\.wal\\.zip\\.tmp"); /** */ - private static final FileFilter WAL_SEGMENT_FILE_COMPACTED_FILTER = new FileFilter() { - @Override public boolean accept(File file) { - return !file.isDirectory() && WAL_SEGMENT_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches(); - } - }; + private static final FileFilter WAL_SEGMENT_FILE_COMPACTED_FILTER = file -> !file.isDirectory() && + WAL_SEGMENT_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches(); /** */ - private static final FileFilter WAL_SEGMENT_TEMP_FILE_COMPACTED_FILTER = new FileFilter() { - @Override public boolean accept(File file) { - return !file.isDirectory() && WAL_SEGMENT_TEMP_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches(); - } - }; + private static final FileFilter WAL_SEGMENT_TEMP_FILE_COMPACTED_FILTER = file -> !file.isDirectory() && + WAL_SEGMENT_TEMP_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches(); /** Buffer size. */ private static final int BUF_SIZE = 1024 * 1024; @@ -240,7 +225,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl DFLT_CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE); /** - * Percentage of WAL archive size to calculate threshold since which removing of old archive should be started. + * Percentage of WAL archive size to calculate threshold to which old segments will be deleted when maximum is reached. */ private static final double THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE = IgniteSystemProperties.getDouble(IGNITE_THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE, @@ -400,8 +385,11 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl */ private final Map segmentSize = new ConcurrentHashMap<>(); + /** Maximum WAL archive size in bytes. */ + private final long maxWalArchiveSize; + /** Current size of WAL archive in bytes. */ - private final AtomicLong archiveSize = new AtomicLong(); + private final AtomicLong walArchiveSize = new AtomicLong(); /** * @param ctx Kernal context. @@ -435,6 +423,8 @@ public FileWriteAheadLogManager(final GridKernalContext ctx) { / dsCfg.getWalSegmentSize()); switchSegmentRecordOffset = isArchiverEnabled() ? new AtomicLongArray(dsCfg.getWalSegments()) : null; + + maxWalArchiveSize = dsCfg.getMaxWalArchiveSize(); } /** @@ -716,6 +706,9 @@ private void checkWalConfiguration() throws IgniteCheckedException { assert currHnd == null; + // It needs to be calculated before #startArchiverAndCompressor, to avoid possible race. + walArchiveSize.set(Stream.of(walArchiveFiles()).mapToLong(fd -> fd.file.length()).sum()); + startArchiverAndCompressor(); assert (isArchiverEnabled() && archiver != null) || (!isArchiverEnabled() && archiver == null) : @@ -1076,7 +1069,7 @@ private boolean hasIndex(long absIdx) { deleted++; segmentSize.remove(desc.idx()); - archiveSize.addAndGet(-len); + walArchiveSize.addAndGet(-len); } // Bump up the oldest archive segment index. @@ -1283,7 +1276,7 @@ private FileWriteHandle rollOver(FileWriteHandle cur, @Nullable WALRecord rec) t switchSegmentRecordOffset.set(idx, hnd.getSwitchSegmentRecordOffset()); } - // TODO: 09.12.2020 где-тут или ниже + cleanWalArchive(); FileWriteHandle next; try { @@ -1305,7 +1298,7 @@ private FileWriteHandle rollOver(FileWriteHandle cur, @Nullable WALRecord rec) t segmentSize.put(next.getSegmentId(), maxWalSegmentSize); if (archiver == null) - archiveSize.addAndGet(maxWalSegmentSize); + walArchiveSize.addAndGet(maxWalSegmentSize); if (next.getSegmentId() - lashCheckpointFileIdx() >= maxSegCountWithoutCheckpoint) cctx.database().forceCheckpoint("too big size of WAL without checkpoint"); @@ -1653,33 +1646,6 @@ private FileDescriptor[] walArchiveFiles() { return scan(walArchiveDir.listFiles(WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER)); } - /** {@inheritDoc} */ - @Override public long maxArchivedSegmentToDelete() { - //When maxWalArchiveSize==MAX_VALUE deleting files is not permit. - if (dsCfg.getMaxWalArchiveSize() == Long.MAX_VALUE) - return -1; - - FileDescriptor[] archivedFiles = walArchiveFiles(); - - Long totalArchiveSize = Stream.of(archivedFiles) - .map(desc -> desc.file().length()) - .reduce(0L, Long::sum); - - if (archivedFiles.length == 0 || totalArchiveSize < allowedThresholdWalArchiveSize) - return -1; - - long sizeOfOldestArchivedFiles = 0; - - for (FileDescriptor desc : archivedFiles) { - sizeOfOldestArchivedFiles += desc.file().length(); - - if (totalArchiveSize - sizeOfOldestArchivedFiles < allowedThresholdWalArchiveSize) - return desc.getIdx(); - } - - return archivedFiles[archivedFiles.length - 1].getIdx(); - } - /** * @return Sorted WAL files descriptors. */ @@ -2064,7 +2030,7 @@ public SegmentArchiveResult archiveSegment(long absIdx) throws StorageException } segmentSize.put(absIdx, dstFile.length()); - archiveSize.addAndGet(dstFile.length()); + walArchiveSize.addAndGet(dstFile.length()); } catch (IOException e) { deleteArchiveFiles(false, dstFile, dstTmpFile); @@ -2296,7 +2262,7 @@ private void body0() { } segmentSize.put(segIdx, zip.length()); - archiveSize.addAndGet(zip.length()); + walArchiveSize.addAndGet(zip.length()); segmentAware.onSegmentCompressed(segIdx); @@ -2488,7 +2454,7 @@ private class FileDecompressor extends GridWorker { Files.move(unzipTmp.toPath(), unzip.toPath()); - archiveSize.addAndGet(unzip.length()); + walArchiveSize.addAndGet(unzip.length()); } catch (IOException e) { deleteArchiveFiles(false, unzipTmp); @@ -3142,9 +3108,9 @@ public static boolean isSegmentFileName(@Nullable String name) { } /** - * Removing files from {@link #walArchiveDir} with updating {@link #archiveSize}. + * Removing files from {@link #walArchiveDir} with updating {@link #walArchiveSize}. * - * @param updateArchiveSize Flag to update {@link #archiveSize}. + * @param updateArchiveSize Flag to update {@link #walArchiveSize}. * @param files Files from {@link #walArchiveDir}. */ private void deleteArchiveFiles(boolean updateArchiveSize, File... files) { @@ -3154,7 +3120,7 @@ private void deleteArchiveFiles(boolean updateArchiveSize, File... files) { if (file.delete()) { if (updateArchiveSize) - archiveSize.addAndGet(-len); + walArchiveSize.addAndGet(-len); } else if (file.exists()) { U.warn(log, "Unable to delete file from WAL archive" + @@ -3163,4 +3129,43 @@ else if (file.exists()) { } } } + + /** + * Clearing WAL archive when reaching the maximum. + * + * @throws IgniteCheckedException If failed. + */ + private void cleanWalArchive() throws IgniteCheckedException { + if (maxWalArchiveSize != Long.MAX_VALUE && walArchiveSize.get() < maxWalArchiveSize) + return; + + FileDescriptor[] walArchiveFiles = walArchiveFiles(); + + FileDescriptor high = null; + + long size = 0; + + for (FileDescriptor fileDesc : walArchiveFiles) { + if (fileDesc.idx >= lashCheckpointFileIdx() || + (size += fileDesc.file.length()) > allowedThresholdWalArchiveSize) + break; + else + high = fileDesc; + } + + if (high != null) { + WALPointer highPtr = new WALPointer(high.idx, 0, 0); + + ((GridCacheDatabaseSharedManager)cctx.database()).onWalTruncated(highPtr); + + int truncated = truncate(null, highPtr); + + if (log.isInfoEnabled()) { + log.info("Cleaning WAL archive completed [highIdx=" + high.idx + + ", cleanCnt=" + truncated + + ", currSize=" + U.humanReadableByteCount(walArchiveSize.get()) + + ", maxSize=" + U.humanReadableByteCount(maxWalArchiveSize) + ']'); + } + } + } } 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 2e78ad03c4b4a..4d91cdda67e2e 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 @@ -184,11 +184,6 @@ public class NoOpWALManager implements IgniteWriteAheadLogManager { return -1L; } - /** {@inheritDoc} */ - @Override public long maxArchivedSegmentToDelete() { - return -1; - } - /** {@inheritDoc} */ @Override public long segmentSize(long idx) { return -1; From 380b672b216fbc19d79ec96536edeb532d0c35d9 Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Thu, 10 Dec 2020 18:40:23 +0300 Subject: [PATCH 04/21] IGNITE-13831 wip add reserved wal archive size and some test --- .../wal/FileWriteAheadLogManager.java | 46 ++++++++++++------- .../ignite/internal/util/IgniteUtils.java | 20 ++++++++ .../internal/util/IgniteUtilsSelfTest.java | 32 +++++++++++++ 3 files changed, 82 insertions(+), 16 deletions(-) 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 1960acd153e0f..e95a9c70de923 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 @@ -391,6 +391,9 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl /** Current size of WAL archive in bytes. */ private final AtomicLong walArchiveSize = new AtomicLong(); + /** Reserved size of WAL archive in bytes. */ + private final AtomicLong reservedWalArchiveSize = new AtomicLong(); + /** * @param ctx Kernal context. */ @@ -708,6 +711,7 @@ private void checkWalConfiguration() throws IgniteCheckedException { // It needs to be calculated before #startArchiverAndCompressor, to avoid possible race. walArchiveSize.set(Stream.of(walArchiveFiles()).mapToLong(fd -> fd.file.length()).sum()); + reservedWalArchiveSize.set(0); startArchiverAndCompressor(); @@ -2001,24 +2005,18 @@ public SegmentArchiveResult archiveSegment(long absIdx) throws StorageException ", origFile=" + origFile.getAbsolutePath() + ", dstFile=" + dstFile.getAbsolutePath() + ']'); } - try { - boolean copied = false; - - assert switchSegmentRecordOffset != null; + assert switchSegmentRecordOffset != null; - long offs = switchSegmentRecordOffset.get((int)segIdx); + long offs = switchSegmentRecordOffset.getAndSet((int)segIdx, 0); + long origLen = origFile.length(); - if (offs > 0) { - switchSegmentRecordOffset.set((int)segIdx, 0); + long reservedSize = offs > 0 && offs < origLen ? offs : origLen; + reservedWalArchiveSize.addAndGet(reservedSize); - if (offs < origFile.length()) { - GridFileUtils.copy(ioFactory, origFile, ioFactory, dstTmpFile, offs); - - copied = true; - } - } - - if (!copied) + try { + if (offs > 0 && offs < origLen) + GridFileUtils.copy(ioFactory, origFile, ioFactory, dstTmpFile, offs); + else Files.copy(origFile.toPath(), dstTmpFile.toPath()); Files.move(dstTmpFile.toPath(), dstFile.toPath()); @@ -2039,6 +2037,9 @@ public SegmentArchiveResult archiveSegment(long absIdx) throws StorageException "srcFile=" + origFile.getAbsolutePath() + ", dstFile=" + dstTmpFile.getAbsolutePath() + ']', e); } + finally { + reservedWalArchiveSize.addAndGet(-reservedSize); + } if (log.isInfoEnabled()) { log.info("Copied file [src=" + origFile.getAbsolutePath() + @@ -2247,6 +2248,9 @@ private void body0() { File raw = new File(walArchiveDir, segmentFileName); + long reservedSize = raw.length(); + reservedWalArchiveSize.addAndGet(reservedSize); + try { deleteObsoleteRawSegments(); @@ -2279,6 +2283,9 @@ private void body0() { segmentAware.onSegmentCompressed(segIdx); } + finally { + reservedWalArchiveSize.addAndGet(-reservedSize); + } } catch (IgniteInterruptedCheckedException ignore) { Thread.currentThread().interrupt(); @@ -2440,6 +2447,9 @@ private class FileDecompressor extends GridWorker { File unzipTmp = new File(walArchiveDir, segmentFileName + TMP_SUFFIX); File unzip = new File(walArchiveDir, segmentFileName); + long reservedSize = U.uncompressedSize(zip); + reservedWalArchiveSize.addAndGet(reservedSize); + try { if (unzip.exists()) throw new FileAlreadyExistsException(unzip.getAbsolutePath()); @@ -2476,6 +2486,9 @@ private class FileDecompressor extends GridWorker { } } } + finally { + reservedWalArchiveSize.addAndGet(-reservedSize); + } updateHeartbeat(); @@ -3136,7 +3149,8 @@ else if (file.exists()) { * @throws IgniteCheckedException If failed. */ private void cleanWalArchive() throws IgniteCheckedException { - if (maxWalArchiveSize != Long.MAX_VALUE && walArchiveSize.get() < maxWalArchiveSize) + if (maxWalArchiveSize != Long.MAX_VALUE && + (walArchiveSize.get() + reservedWalArchiveSize.get()) < maxWalArchiveSize) return; FileDescriptor[] walArchiveFiles = walArchiveFiles(); 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 2dc9910ab5709..8469f60a44579 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 @@ -12027,4 +12027,24 @@ public static String humanReadableDuration(long millis) { return sb.toString(); } + + /** + * Getting the total size of uncompressed data in zip. + * + * @param zip Zip file. + * @return Total uncompressed size. + * @throws IOException If failed. + */ + public static long uncompressedSize(File zip) throws IOException { + try (ZipFile zipFile = new ZipFile(zip)) { + long size = 0; + + Enumeration entries = zipFile.entries(); + + while (entries.hasMoreElements()) + size += entries.nextElement().getSize(); + + return size; + } + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java index 2d3843981d437..3f1cf07733f17 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java @@ -25,6 +25,7 @@ import java.io.DataOutput; import java.io.DataOutputStream; import java.io.File; +import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStreamReader; import java.io.ObjectInputStream; @@ -39,6 +40,7 @@ import java.net.InetSocketAddress; import java.net.URL; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -55,6 +57,7 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; +import java.util.stream.IntStream; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -90,6 +93,7 @@ import static java.util.concurrent.TimeUnit.MINUTES; import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.stream.Collectors.joining; +import static org.apache.ignite.testframework.GridTestUtils.assertThrows; import static org.apache.ignite.testframework.GridTestUtils.readResource; import static org.junit.Assert.assertArrayEquals; @@ -1423,6 +1427,34 @@ public void testHumanReadableByteCount() { assertEquals("6.1 MB", U.humanReadableByteCount(6 * 1024 * 1024 + 130 * 1024)); } + /** + * Test to verify the {@link U#uncompressedSize}. + * + * @throws Exception If failed. + */ + @Test + public void testUncompressedSize() throws Exception { + File zipFile = new File(System.getProperty("java.io.tmpdir"), "test.zip"); + + try { + assertThrows(log, () -> U.uncompressedSize(zipFile), IOException.class, null); + + byte[] raw = IntStream.range(0, 10).mapToObj(i -> zipFile.getAbsolutePath() + i) + .collect(joining()).getBytes(StandardCharsets.UTF_8); + + try (FileOutputStream fos = new FileOutputStream(zipFile)) { + fos.write(U.zip(raw)); + + fos.flush(); + } + + assertEquals(raw.length, U.uncompressedSize(zipFile)); + } + finally { + assertTrue(U.delete(zipFile)); + } + } + /** * Reading lines from a resource file and passing them to consumer. * If read string is {@code "null"}, it is converted to {@code null}. From 4aaef364dbd98ad89e612090e119c1ea27759b9f Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Fri, 11 Dec 2020 07:22:29 +0300 Subject: [PATCH 05/21] IGNITE-13831 wip fix FileDecomressor --- .../wal/FileWriteAheadLogManager.java | 21 ++++++++----------- 1 file changed, 9 insertions(+), 12 deletions(-) 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 e95a9c70de923..054316a83129c 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 @@ -2441,6 +2441,9 @@ private class FileDecompressor extends GridWorker { if (isCancelled()) break; + if (segmentToDecompress == -1) + continue; + String segmentFileName = fileName(segmentToDecompress); File zip = new File(walArchiveDir, segmentFileName + ZIP_SUFFIX); @@ -2450,6 +2453,8 @@ private class FileDecompressor extends GridWorker { long reservedSize = U.uncompressedSize(zip); reservedWalArchiveSize.addAndGet(reservedSize); + IgniteCheckedException ex = null; + try { if (unzip.exists()) throw new FileAlreadyExistsException(unzip.getAbsolutePath()); @@ -2473,17 +2478,9 @@ private class FileDecompressor extends GridWorker { U.error(log, "Can't rename temporary unzipped segment: raw segment is already present " + "[tmp=" + unzipTmp + ", raw=" + unzip + "]", e); } - else { - if (!isCancelled && segmentToDecompress != -1L) { - IgniteCheckedException ex = new IgniteCheckedException("Error during WAL segment " + - "decompression [segmentIdx=" + segmentToDecompress + "]", e); - - synchronized (this) { - decompressionFutures.remove(segmentToDecompress).onDone(ex); - } - - continue; - } + else if (!isCancelled) { + ex = new IgniteCheckedException("Error during WAL segment decompression [segmentIdx=" + + segmentToDecompress + "]", e); } } finally { @@ -2493,7 +2490,7 @@ private class FileDecompressor extends GridWorker { updateHeartbeat(); synchronized (this) { - decompressionFutures.remove(segmentToDecompress).onDone(); + decompressionFutures.remove(segmentToDecompress).onDone(ex); } } } From 968b4a498fd57bd0995fad5310aae11313816d70 Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Fri, 11 Dec 2020 11:09:43 +0300 Subject: [PATCH 06/21] IGNITE-13831 wip fix clean wal archive and tests --- .../wal/FileWriteAheadLogManager.java | 16 ++-- .../wal/WalDeletionArchiveAbstractTest.java | 82 ++++++++++--------- 2 files changed, 52 insertions(+), 46 deletions(-) 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 054316a83129c..ef89bb595a511 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 @@ -1274,13 +1274,16 @@ private FileWriteHandle rollOver(FileWriteHandle cur, @Nullable WALRecord rec) t if (metrics.metricsEnabled()) metrics.onWallRollOver(); + long nextArchiveSegmentSize = maxWalSegmentSize; + if (switchSegmentRecordOffset != null) { int idx = (int)(cur.getSegmentId() % dsCfg.getWalSegments()); switchSegmentRecordOffset.set(idx, hnd.getSwitchSegmentRecordOffset()); + nextArchiveSegmentSize = hnd.getSwitchSegmentRecordOffset(); } - cleanWalArchive(); + cleanWalArchive(nextArchiveSegmentSize); FileWriteHandle next; try { @@ -1646,7 +1649,7 @@ private File pollNextFile(long curIdx) throws StorageException, IgniteInterrupte /** * Files from archive WAL directory. */ - private FileDescriptor[] walArchiveFiles() { + public FileDescriptor[] walArchiveFiles() { return scan(walArchiveDir.listFiles(WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER)); } @@ -3143,11 +3146,12 @@ else if (file.exists()) { /** * Clearing WAL archive when reaching the maximum. * + * @param nextArchiveSegmentSize Size of next segment to be archived in bytes. * @throws IgniteCheckedException If failed. */ - private void cleanWalArchive() throws IgniteCheckedException { - if (maxWalArchiveSize != Long.MAX_VALUE && - (walArchiveSize.get() + reservedWalArchiveSize.get()) < maxWalArchiveSize) + private void cleanWalArchive(long nextArchiveSegmentSize) throws IgniteCheckedException { + if (maxWalArchiveSize == Long.MAX_VALUE || + walArchiveSize.get() + reservedWalArchiveSize.get() + nextArchiveSegmentSize < maxWalArchiveSize) return; FileDescriptor[] walArchiveFiles = walArchiveFiles(); @@ -3165,7 +3169,7 @@ private void cleanWalArchive() throws IgniteCheckedException { } if (high != null) { - WALPointer highPtr = new WALPointer(high.idx, 0, 0); + WALPointer highPtr = new WALPointer(high.idx + 1, 0, 0); ((GridCacheDatabaseSharedManager)cctx.database()).onWalTruncated(highPtr); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java index 044c1732682b6..9d098cfe411e5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java @@ -24,17 +24,18 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteException; import org.apache.ignite.cache.CacheAtomicityMode; +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.IgniteConfiguration; import org.apache.ignite.configuration.WALMode; import org.apache.ignite.internal.IgniteEx; -import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointHistory; import org.apache.ignite.internal.processors.cache.persistence.checkpoint.Checkpointer; import org.apache.ignite.internal.processors.cache.persistence.wal.FileDescriptor; +import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.junits.WithSystemProperty; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -67,14 +68,14 @@ private Ignite startGrid(Consumer customConfigurator) Ignite ignite = startGrid(configuration); - ignite.active(true); + ignite.cluster().state(ClusterState.ACTIVE); return ignite; } /** */ - private CacheConfiguration cacheConfiguration() { - CacheConfiguration ccfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME); + private CacheConfiguration cacheConfiguration() { + CacheConfiguration ccfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME); return ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); } @@ -136,15 +137,16 @@ private String findSourceMessage(Throwable ex) { public void testCorrectDeletedArchivedWalFiles() throws Exception { //given: configured grid with setted max wal archive size long maxWalArchiveSize = 2 * 1024 * 1024; - Ignite ignite = startGrid(dbCfg -> { - dbCfg.setMaxWalArchiveSize(maxWalArchiveSize); - }); + Ignite ignite = startGrid(dbCfg -> dbCfg.setMaxWalArchiveSize(maxWalArchiveSize)); GridCacheDatabaseSharedManager dbMgr = gridDatabase(ignite); + CheckpointHistory hist = dbMgr.checkpointHistory(); + assertNotNull(hist); + long allowedThresholdWalArchiveSize = maxWalArchiveSize / 2; - IgniteCache cache = ignite.getOrCreateCache(cacheConfiguration()); + IgniteCache cache = ignite.getOrCreateCache(cacheConfiguration()); //when: put to cache more than 2 MB for (int i = 0; i < 500; i++) @@ -153,20 +155,18 @@ public void testCorrectDeletedArchivedWalFiles() throws Exception { forceCheckpoint(); //then: total archive size less than half of maxWalArchiveSize(by current logic) - IgniteWriteAheadLogManager wal = wal(ignite); + FileWriteAheadLogManager wal = wal(ignite); - FileDescriptor[] files = (FileDescriptor[])U.findNonPublicMethod(wal.getClass(), "walArchiveFiles").invoke(wal); + FileDescriptor[] files = wal.walArchiveFiles(); Long totalSize = Stream.of(files) .map(desc -> desc.file().length()) .reduce(0L, Long::sum); assertTrue(files.length >= 1); - assertTrue(totalSize <= allowedThresholdWalArchiveSize); + assertTrue(totalSize <= maxWalArchiveSize && totalSize >= allowedThresholdWalArchiveSize); assertFalse(Stream.of(files).anyMatch(desc -> desc.file().getName().endsWith("00001.wal"))); - CheckpointHistory hist = dbMgr.checkpointHistory(); - assertTrue(!hist.checkpoints().isEmpty()); } @@ -177,12 +177,12 @@ public void testCorrectDeletedArchivedWalFiles() throws Exception { public void testCheckpointStarted_WhenWalHasTooBigSizeWithoutCheckpoint() throws Exception { //given: configured grid with max wal archive size = 1MB, wal segment size = 512KB Ignite ignite = startGrid(dbCfg -> { - dbCfg.setMaxWalArchiveSize(1 * 1024 * 1024);// 1 Mbytes + dbCfg.setMaxWalArchiveSize(1024 * 1024);// 1 Mbytes }); GridCacheDatabaseSharedManager dbMgr = gridDatabase(ignite); - IgniteCache cache = ignite.getOrCreateCache(cacheConfiguration()); + IgniteCache cache = ignite.getOrCreateCache(cacheConfiguration()); for (int i = 0; i < 500; i++) cache.put(i, i); @@ -201,34 +201,37 @@ public void testCheckpointStarted_WhenWalHasTooBigSizeWithoutCheckpoint() throws * @deprecated Test old removing process depends on WalHistorySize. */ @Test - public void testCheckpointHistoryRemovingByWalHistorySize() throws Exception { - //given: configured grid with wal history size = 10 - int walHistorySize = 10; - - Ignite ignite = startGrid(dbCfg -> { - dbCfg.setWalHistorySize(walHistorySize); - }); + public void testCheckpointHistoryRemovingByTruncate() throws Exception { + Ignite ignite = startGrid(dbCfg -> dbCfg.setMaxWalArchiveSize(2 * 1024 * 1024)); GridCacheDatabaseSharedManager dbMgr = gridDatabase(ignite); - IgniteCache cache = ignite.getOrCreateCache(cacheConfiguration()); + IgniteCache cache = ignite.getOrCreateCache(cacheConfiguration()); + + CheckpointHistory hist = dbMgr.checkpointHistory(); + assertNotNull(hist); + + int startHistSize = hist.checkpoints().size(); - //when: put to cache and do checkpoint - int testNumberOfCheckpoint = walHistorySize * 2; + int checkpointCnt = 10; - for (int i = 0; i < testNumberOfCheckpoint; i++) { + for (int i = 0; i < checkpointCnt; i++) { cache.put(i, i); //and: wait for checkpoint finished forceCheckpoint(); + // Check that the history is growing. + assertEquals(startHistSize + (i + 1), hist.checkpoints().size()); } - //then: number of checkpoints less or equal than walHistorySize - CheckpointHistory hist = dbMgr.checkpointHistory(); - assertTrue(hist.checkpoints().size() == walHistorySize); + // Ensure rollover and wal archive cleaning. + for (int i = 0; i < 6; i++) + cache.put(i, new byte[ignite.configuration().getDataStorageConfiguration().getWalSegmentSize() / 2]); + + assertTrue(hist.checkpoints().size() < checkpointCnt + startHistSize); File[] cpFiles = dbMgr.checkpointDirectory().listFiles(); - assertTrue(cpFiles.length <= (walHistorySize * 2 + 1));// starts & ends + node_start + assertTrue(cpFiles.length <= (checkpointCnt * 2 + 1));// starts & ends + node_start } /** @@ -239,13 +242,14 @@ public void testCheckpointHistoryRemovingByWalHistorySize() throws Exception { @WithSystemProperty(key = IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE, value = "2") public void testCorrectDeletedCheckpointHistoryButKeepWalFiles() throws Exception { //given: configured grid with disabled WAL removing. - Ignite ignite = startGrid(dbCfg -> { - dbCfg.setMaxWalArchiveSize(Long.MAX_VALUE); - }); + Ignite ignite = startGrid(dbCfg -> dbCfg.setMaxWalArchiveSize(Long.MAX_VALUE)); GridCacheDatabaseSharedManager dbMgr = gridDatabase(ignite); - IgniteCache cache = ignite.getOrCreateCache(cacheConfiguration()); + CheckpointHistory hist = dbMgr.checkpointHistory(); + assertNotNull(hist); + + IgniteCache cache = ignite.getOrCreateCache(cacheConfiguration()); //when: put to cache for (int i = 0; i < 500; i++) { @@ -258,17 +262,15 @@ public void testCorrectDeletedCheckpointHistoryButKeepWalFiles() throws Exceptio forceCheckpoint(); //then: WAL files was not deleted but some of checkpoint history was deleted. - IgniteWriteAheadLogManager wal = wal(ignite); + FileWriteAheadLogManager wal = wal(ignite); - FileDescriptor[] files = (FileDescriptor[])U.findNonPublicMethod(wal.getClass(), "walArchiveFiles").invoke(wal); + FileDescriptor[] files = wal.walArchiveFiles(); boolean hasFirstSegment = Stream.of(files) .anyMatch(desc -> desc.file().getName().endsWith("0001.wal")); assertTrue(hasFirstSegment); - CheckpointHistory hist = dbMgr.checkpointHistory(); - assertTrue(hist.checkpoints().size() == 2); } @@ -282,7 +284,7 @@ private GridCacheDatabaseSharedManager gridDatabase(Ignite ignite) { /** * Extract IgniteWriteAheadLogManager. */ - private IgniteWriteAheadLogManager wal(Ignite ignite) { - return ((IgniteEx)ignite).context().cache().context().wal(); + private FileWriteAheadLogManager wal(Ignite ignite) { + return (FileWriteAheadLogManager)((IgniteEx)ignite).context().cache().context().wal(); } } From 1cae4186eb3fcb30b3e22e853431c3ca10f2cac6 Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Fri, 11 Dec 2020 11:43:29 +0300 Subject: [PATCH 07/21] IGNITE-13831 wip fix error tmp file for archiver --- .../cache/persistence/wal/FileWriteAheadLogManager.java | 3 +++ 1 file changed, 3 insertions(+) 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 ef89bb595a511..6f3f4c8ade681 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 @@ -2017,6 +2017,9 @@ public SegmentArchiveResult archiveSegment(long absIdx) throws StorageException reservedWalArchiveSize.addAndGet(reservedSize); try { + // TODO: 11.12.2020 Can be remove after IGNITE-13815 + deleteArchiveFiles(false, dstTmpFile); + if (offs > 0 && offs < origLen) GridFileUtils.copy(ioFactory, origFile, ioFactory, dstTmpFile, offs); else From 9279cf7d73f45a542419c4dee3b848665659aa25 Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Fri, 11 Dec 2020 13:26:27 +0300 Subject: [PATCH 08/21] IGNITE-13831 wip --- .../cache/persistence/wal/FileWriteAheadLogManager.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 6f3f4c8ade681..f1718aa2acc6e 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 @@ -1283,10 +1283,10 @@ private FileWriteHandle rollOver(FileWriteHandle cur, @Nullable WALRecord rec) t nextArchiveSegmentSize = hnd.getSwitchSegmentRecordOffset(); } - cleanWalArchive(nextArchiveSegmentSize); - FileWriteHandle next; try { + cleanWalArchive(nextArchiveSegmentSize); + next = initNextWriteHandle(cur); } catch (IgniteCheckedException e) { @@ -2017,7 +2017,7 @@ public SegmentArchiveResult archiveSegment(long absIdx) throws StorageException reservedWalArchiveSize.addAndGet(reservedSize); try { - // TODO: 11.12.2020 Can be remove after IGNITE-13815 + // TODO: 11.12.2020 Can be remove after IGNITE-13815 deleteArchiveFiles(false, dstTmpFile); if (offs > 0 && offs < origLen) From b057c1526474ee07d2bccc931f532aecb9e9f82d Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Fri, 11 Dec 2020 14:04:53 +0300 Subject: [PATCH 09/21] IGNITE-13831 wip fix test --- .../db/wal/WalDeletionArchiveAbstractTest.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java index 9d098cfe411e5..e9019ee030451 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java @@ -149,10 +149,12 @@ public void testCorrectDeletedArchivedWalFiles() throws Exception { IgniteCache cache = ignite.getOrCreateCache(cacheConfiguration()); //when: put to cache more than 2 MB - for (int i = 0; i < 500; i++) - cache.put(i, i); + for (int i = 0; i < 500; i++) { + if (i % 100 == 0) + forceCheckpoint(); - forceCheckpoint(); + cache.put(i, i); + } //then: total archive size less than half of maxWalArchiveSize(by current logic) FileWriteAheadLogManager wal = wal(ignite); From bc51b9d25918eb2f1ab82f6cf1c657b8a10db078 Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Mon, 14 Dec 2020 09:34:58 +0300 Subject: [PATCH 10/21] IGNITE-13831 fix after merge master --- .../checkpoint/CheckpointHistory.java | 137 ++++++------------ .../wal/FileWriteAheadLogManager.java | 4 +- 2 files changed, 49 insertions(+), 92 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java index 1b188869c6be5..f6f1ec89a7548 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java @@ -87,22 +87,26 @@ public class CheckpointHistory { * Constructor. * * @param dsCfg Data storage configuration. - * @param wal Write ahead log. + * @param logFun Function for getting a logger. + * @param wal WAL manager. * @param inapplicable Checkpoint inapplicable filter. */ CheckpointHistory( DataStorageConfiguration dsCfg, - Function, IgniteLogger> logger, + Function, IgniteLogger> logFun, IgniteWriteAheadLogManager wal, IgniteThrowableBiPredicate inapplicable ) { - this.log = logger.apply(getClass()); + this.log = logFun.apply(getClass()); this.wal = wal; this.checkpointInapplicable = inapplicable; isWalTruncationEnabled = dsCfg.getMaxWalArchiveSize() != DataStorageConfiguration.UNLIMITED_WAL_ARCHIVE; - maxCpHistMemSize = IgniteSystemProperties.getInteger(IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE, DFLT_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE); + maxCpHistMemSize = IgniteSystemProperties.getInteger( + IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE, + DFLT_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE + ); reservationDisabled = dsCfg.getWalMode() == WALMode.NONE; } @@ -228,7 +232,7 @@ private void updateEarliestCpMap(CheckpointEntry entry) { addCpGroupStatesToEarliestCpMap(entry, states); } catch (IgniteCheckedException ex) { - U.warn(log, "Failed to process checkpoint: " + (entry != null ? entry : "none"), ex); + U.warn(log, "Failed to process checkpoint: " + entry, ex); earliestCp.clear(); } @@ -318,7 +322,7 @@ public boolean hasSpace() { * @return List of checkpoint entries removed from history. */ public List onWalTruncated(WALPointer highBound) { - List removed = new ArrayList<>(); + List rmv = new ArrayList<>(); for (CheckpointEntry cpEntry : histMap.values()) { WALPointer cpPnt = cpEntry.checkpointMark(); @@ -329,10 +333,10 @@ public List onWalTruncated(WALPointer highBound) { if (!removeCheckpoint(cpEntry)) break; - removed.add(cpEntry); + rmv.add(cpEntry); } - return removed; + return rmv; } /** @@ -340,32 +344,32 @@ public List onWalTruncated(WALPointer highBound) { * * @return List of checkpoint entries removed from history. */ - public List removeCheckpoints(int countToRemove) { - if (countToRemove == 0) + public List removeCheckpoints(int cntToRmv) { + if (cntToRmv == 0) return Collections.emptyList(); - List removed = new ArrayList<>(); + List rmv = new ArrayList<>(); - for (Iterator> iterator = histMap.entrySet().iterator(); - iterator.hasNext() && removed.size() < countToRemove; ) { - Map.Entry entry = iterator.next(); + for (Iterator> iter = histMap.entrySet().iterator(); + iter.hasNext() && rmv.size() < cntToRmv; ) { + Map.Entry entry = iter.next(); CheckpointEntry checkpoint = entry.getValue(); if (!removeCheckpoint(checkpoint)) break; - removed.add(checkpoint); + rmv.add(checkpoint); } - return removed; + return rmv; } /** - * Remove checkpoint from history + * Remove checkpoint from history. * - * @param checkpoint Checkpoint to be removed - * @return Whether checkpoint was removed from history + * @param checkpoint Checkpoint to be removed. + * @return Whether checkpoint was removed from history. */ private boolean removeCheckpoint(CheckpointEntry checkpoint) { if (wal.reserved(checkpoint.checkpointMark())) { @@ -378,11 +382,11 @@ private boolean removeCheckpoint(CheckpointEntry checkpoint) { synchronized (earliestCp) { CheckpointEntry deletedCpEntry = histMap.remove(checkpoint.timestamp()); - CheckpointEntry oldestCpInHistory = firstCheckpoint(); + CheckpointEntry oldestCpInHist = firstCheckpoint(); for (Map.Entry grpPartPerCp : earliestCp.entrySet()) { if (grpPartPerCp.getValue() == deletedCpEntry) - grpPartPerCp.setValue(oldestCpInHistory); + grpPartPerCp.setValue(oldestCpInHist); } } @@ -398,56 +402,7 @@ private boolean removeCheckpoint(CheckpointEntry checkpoint) { public List onCheckpointFinished(Checkpoint chp) { chp.walSegsCoveredRange(calculateWalSegmentsCovered()); - int removeCount = isWalTruncationEnabled - ? checkpointCountUntilDeleteByArchiveSize() - : (histMap.size() - maxCpHistMemSize); - - if (removeCount <= 0) - return Collections.emptyList(); - - List deletedCheckpoints = removeCheckpoints(removeCount); - - if (isWalTruncationEnabled) { - int deleted = wal.truncate(firstCheckpointPointer()); - - chp.walFilesDeleted(deleted); - } - - return deletedCheckpoints; - } - - /** - * @param first One of pointers to choose the newest. - * @param second One of pointers to choose the newest. - * @return The newest pointer from input ones. - */ - private WALPointer newerPointer(WALPointer first, WALPointer second) { - if (first == null) - return second; - - if (second == null) - return first; - - return first.index() > second.index() ? first : second; - } - - /** - * Calculate mark until delete by maximum checkpoint history memory size. - * - * @return Checkpoint mark until which checkpoints can be deleted(not including this pointer). - */ - private WALPointer checkpointMarkUntilDeleteByMemorySize() { - if (histMap.size() <= maxCpHistMemSize) - return null; - - int calculatedCpHistSize = maxCpHistMemSize; - - for (Map.Entry entry : histMap.entrySet()) { - if (histMap.size() <= calculatedCpHistSize++) - return entry.getValue().checkpointMark(); - } - - return lastCheckpoint().checkpointMark(); + return removeCheckpoints(isWalTruncationEnabled ? 0 : histMap.size() - maxCpHistMemSize); } /** @@ -483,32 +438,32 @@ private IgniteBiTuple calculateWalSegmentsCovered() { * Search the earliest WAL pointer for particular group, matching by counter for partitions. * * @param grpId Group id. - * @param partsCounter Partition mapped to update counter. + * @param partsCntr Partition mapped to update counter. * @param latestReservedPointer Latest reserved WAL pointer. * @param margin Margin pointer. * @return Earliest WAL pointer for group specified. */ @Nullable public WALPointer searchEarliestWalPointer( int grpId, - Map partsCounter, + Map partsCntr, WALPointer latestReservedPointer, long margin ) throws IgniteCheckedException { - if (F.isEmpty(partsCounter)) + if (F.isEmpty(partsCntr)) return null; - Map modifiedPartsCounter = new HashMap<>(partsCounter); + Map modifiedPartsCntr = new HashMap<>(partsCntr); WALPointer minPtr = null; - LinkedList historyPointerCandidate = new LinkedList<>(); + LinkedList histPointerCandidate = new LinkedList<>(); for (Long cpTs : checkpoints(true)) { CheckpointEntry cpEntry = entry(cpTs); - minPtr = getMinimalPointer(partsCounter, margin, minPtr, historyPointerCandidate, cpEntry); + minPtr = getMinimalPointer(partsCntr, margin, minPtr, histPointerCandidate, cpEntry); - Iterator> iter = modifiedPartsCounter.entrySet().iterator(); + Iterator> iter = modifiedPartsCntr.entrySet().iterator(); WALPointer ptr = cpEntry.checkpointMark(); @@ -526,31 +481,31 @@ private IgniteBiTuple calculateWalSegmentsCovered() { } if (foundCntr + margin > entry.getValue()) { - historyPointerCandidate.add(new WalPointerCandidate(grpId, entry.getKey(), entry.getValue(), ptr, + histPointerCandidate.add(new WalPointerCandidate(grpId, entry.getKey(), entry.getValue(), ptr, foundCntr)); continue; } - partsCounter.put(entry.getKey(), entry.getValue() - margin); + partsCntr.put(entry.getKey(), entry.getValue() - margin); if (minPtr == null || ptr.compareTo(minPtr) < 0) minPtr = ptr; } } - if ((F.isEmpty(modifiedPartsCounter) && F.isEmpty(historyPointerCandidate)) || ptr.compareTo(latestReservedPointer) == 0) + if ((F.isEmpty(modifiedPartsCntr) && F.isEmpty(histPointerCandidate)) || ptr.compareTo(latestReservedPointer) == 0) break; } - if (!F.isEmpty(modifiedPartsCounter)) { - Map.Entry entry = modifiedPartsCounter.entrySet().iterator().next(); + if (!F.isEmpty(modifiedPartsCntr)) { + Map.Entry entry = modifiedPartsCntr.entrySet().iterator().next(); throw new IgniteCheckedException("Could not find start pointer for partition [part=" + entry.getKey() + ", partCntrSince=" + entry.getValue() + "]"); } - minPtr = getMinimalPointer(partsCounter, margin, minPtr, historyPointerCandidate, null); + minPtr = getMinimalPointer(partsCntr, margin, minPtr, histPointerCandidate, null); return minPtr; } @@ -558,23 +513,23 @@ private IgniteBiTuple calculateWalSegmentsCovered() { /** * Finds a minimal WAL pointer. * - * @param partsCounter Partition mapped to update counter. + * @param partsCntr Partition mapped to update counter. * @param margin Margin pointer. * @param minPtr Minimal WAL pointer which was determined before. - * @param historyPointerCandidate Collection of candidates for a historical WAL pointer. + * @param histPointerCandidate Collection of candidates for a historical WAL pointer. * @param cpEntry Checkpoint entry. * @return Minimal WAL pointer. */ private WALPointer getMinimalPointer( - Map partsCounter, + Map partsCntr, long margin, WALPointer minPtr, - LinkedList historyPointerCandidate, + LinkedList histPointerCandidate, CheckpointEntry cpEntry ) { - while (!F.isEmpty(historyPointerCandidate)) { - WALPointer ptr = historyPointerCandidate.poll() - .choose(cpEntry, margin, partsCounter); + while (!F.isEmpty(histPointerCandidate)) { + WALPointer ptr = histPointerCandidate.poll() + .choose(cpEntry, margin, partsCntr); if (minPtr == null || ptr.compareTo(minPtr) < 0) minPtr = ptr; 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 c6cde41333ade..d297f05af0969 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 @@ -1066,6 +1066,8 @@ private boolean hasIndex(long absIdx) { return deleted; if (desc.idx < high.index()) { + long len = desc.file.length(); + if (!desc.file.delete()) { U.warn(log, "Failed to remove obsolete WAL segment (make sure the process has enough rights): " + desc.file.getAbsolutePath()); @@ -3147,7 +3149,7 @@ private void cleanWalArchive(long nextArchiveSegmentSize) throws IgniteCheckedEx ((GridCacheDatabaseSharedManager)cctx.database()).onWalTruncated(highPtr); - int truncated = truncate(null, highPtr); + int truncated = truncate(highPtr); if (log.isInfoEnabled()) { log.info("Cleaning WAL archive completed [highIdx=" + high.idx From 670ad936192612449c7e747ecde94f564130832f Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Mon, 14 Dec 2020 10:17:20 +0300 Subject: [PATCH 11/21] IGNITE-13831 wip some minor fix --- .../checkpoint/CheckpointHistory.java | 72 +++++++++---------- .../wal/FileWriteAheadLogManager.java | 3 - 2 files changed, 36 insertions(+), 39 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java index f6f1ec89a7548..87be2ebc6cfc7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java @@ -322,7 +322,7 @@ public boolean hasSpace() { * @return List of checkpoint entries removed from history. */ public List onWalTruncated(WALPointer highBound) { - List rmv = new ArrayList<>(); + List removed = new ArrayList<>(); for (CheckpointEntry cpEntry : histMap.values()) { WALPointer cpPnt = cpEntry.checkpointMark(); @@ -333,10 +333,10 @@ public List onWalTruncated(WALPointer highBound) { if (!removeCheckpoint(cpEntry)) break; - rmv.add(cpEntry); + removed.add(cpEntry); } - return rmv; + return removed; } /** @@ -344,32 +344,32 @@ public List onWalTruncated(WALPointer highBound) { * * @return List of checkpoint entries removed from history. */ - public List removeCheckpoints(int cntToRmv) { - if (cntToRmv == 0) + public List removeCheckpoints(int countToRemove) { + if (countToRemove == 0) return Collections.emptyList(); - List rmv = new ArrayList<>(); + List removed = new ArrayList<>(); - for (Iterator> iter = histMap.entrySet().iterator(); - iter.hasNext() && rmv.size() < cntToRmv; ) { - Map.Entry entry = iter.next(); + for (Iterator> iterator = histMap.entrySet().iterator(); + iterator.hasNext() && removed.size() < countToRemove; ) { + Map.Entry entry = iterator.next(); CheckpointEntry checkpoint = entry.getValue(); if (!removeCheckpoint(checkpoint)) break; - rmv.add(checkpoint); + removed.add(checkpoint); } - return rmv; + return removed; } /** - * Remove checkpoint from history. + * Remove checkpoint from history * - * @param checkpoint Checkpoint to be removed. - * @return Whether checkpoint was removed from history. + * @param checkpoint Checkpoint to be removed + * @return Whether checkpoint was removed from history */ private boolean removeCheckpoint(CheckpointEntry checkpoint) { if (wal.reserved(checkpoint.checkpointMark())) { @@ -382,11 +382,11 @@ private boolean removeCheckpoint(CheckpointEntry checkpoint) { synchronized (earliestCp) { CheckpointEntry deletedCpEntry = histMap.remove(checkpoint.timestamp()); - CheckpointEntry oldestCpInHist = firstCheckpoint(); + CheckpointEntry oldestCpInHistory = firstCheckpoint(); for (Map.Entry grpPartPerCp : earliestCp.entrySet()) { if (grpPartPerCp.getValue() == deletedCpEntry) - grpPartPerCp.setValue(oldestCpInHist); + grpPartPerCp.setValue(oldestCpInHistory); } } @@ -438,32 +438,32 @@ private IgniteBiTuple calculateWalSegmentsCovered() { * Search the earliest WAL pointer for particular group, matching by counter for partitions. * * @param grpId Group id. - * @param partsCntr Partition mapped to update counter. + * @param partsCounter Partition mapped to update counter. * @param latestReservedPointer Latest reserved WAL pointer. * @param margin Margin pointer. * @return Earliest WAL pointer for group specified. */ @Nullable public WALPointer searchEarliestWalPointer( int grpId, - Map partsCntr, + Map partsCounter, WALPointer latestReservedPointer, long margin ) throws IgniteCheckedException { - if (F.isEmpty(partsCntr)) + if (F.isEmpty(partsCounter)) return null; - Map modifiedPartsCntr = new HashMap<>(partsCntr); + Map modifiedPartsCounter = new HashMap<>(partsCounter); WALPointer minPtr = null; - LinkedList histPointerCandidate = new LinkedList<>(); + LinkedList historyPointerCandidate = new LinkedList<>(); for (Long cpTs : checkpoints(true)) { CheckpointEntry cpEntry = entry(cpTs); - minPtr = getMinimalPointer(partsCntr, margin, minPtr, histPointerCandidate, cpEntry); + minPtr = getMinimalPointer(partsCounter, margin, minPtr, historyPointerCandidate, cpEntry); - Iterator> iter = modifiedPartsCntr.entrySet().iterator(); + Iterator> iter = modifiedPartsCounter.entrySet().iterator(); WALPointer ptr = cpEntry.checkpointMark(); @@ -481,31 +481,31 @@ private IgniteBiTuple calculateWalSegmentsCovered() { } if (foundCntr + margin > entry.getValue()) { - histPointerCandidate.add(new WalPointerCandidate(grpId, entry.getKey(), entry.getValue(), ptr, + historyPointerCandidate.add(new WalPointerCandidate(grpId, entry.getKey(), entry.getValue(), ptr, foundCntr)); continue; } - partsCntr.put(entry.getKey(), entry.getValue() - margin); + partsCounter.put(entry.getKey(), entry.getValue() - margin); if (minPtr == null || ptr.compareTo(minPtr) < 0) minPtr = ptr; } } - if ((F.isEmpty(modifiedPartsCntr) && F.isEmpty(histPointerCandidate)) || ptr.compareTo(latestReservedPointer) == 0) + if ((F.isEmpty(modifiedPartsCounter) && F.isEmpty(historyPointerCandidate)) || ptr.compareTo(latestReservedPointer) == 0) break; } - if (!F.isEmpty(modifiedPartsCntr)) { - Map.Entry entry = modifiedPartsCntr.entrySet().iterator().next(); + if (!F.isEmpty(modifiedPartsCounter)) { + Map.Entry entry = modifiedPartsCounter.entrySet().iterator().next(); throw new IgniteCheckedException("Could not find start pointer for partition [part=" + entry.getKey() + ", partCntrSince=" + entry.getValue() + "]"); } - minPtr = getMinimalPointer(partsCntr, margin, minPtr, histPointerCandidate, null); + minPtr = getMinimalPointer(partsCounter, margin, minPtr, historyPointerCandidate, null); return minPtr; } @@ -513,23 +513,23 @@ private IgniteBiTuple calculateWalSegmentsCovered() { /** * Finds a minimal WAL pointer. * - * @param partsCntr Partition mapped to update counter. + * @param partsCounter Partition mapped to update counter. * @param margin Margin pointer. * @param minPtr Minimal WAL pointer which was determined before. - * @param histPointerCandidate Collection of candidates for a historical WAL pointer. + * @param historyPointerCandidate Collection of candidates for a historical WAL pointer. * @param cpEntry Checkpoint entry. * @return Minimal WAL pointer. */ private WALPointer getMinimalPointer( - Map partsCntr, + Map partsCounter, long margin, WALPointer minPtr, - LinkedList histPointerCandidate, + LinkedList historyPointerCandidate, CheckpointEntry cpEntry ) { - while (!F.isEmpty(histPointerCandidate)) { - WALPointer ptr = histPointerCandidate.poll() - .choose(cpEntry, margin, partsCntr); + while (!F.isEmpty(historyPointerCandidate)) { + WALPointer ptr = historyPointerCandidate.poll() + .choose(cpEntry, margin, partsCounter); if (minPtr == null || ptr.compareTo(minPtr) < 0) minPtr = ptr; 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 d297f05af0969..1b48f72d61c55 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 @@ -2008,9 +2008,6 @@ public SegmentArchiveResult archiveSegment(long absIdx) throws StorageException reservedWalArchiveSize.addAndGet(reservedSize); try { - // TODO: 11.12.2020 Can be remove after IGNITE-13815 - deleteArchiveFiles(false, dstTmpFile); - if (offs > 0 && offs < origLen) GridFileUtils.copy(ioFactory, origFile, ioFactory, dstTmpFile, offs); else From 626599e9eb06e4c4f06662b26bd9ee9b326ded93 Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Mon, 14 Dec 2020 17:17:39 +0300 Subject: [PATCH 12/21] IGNITE-13831 wip --- .../encryption/GridEncryptionManager.java | 10 +++++ .../wal/FileWriteAheadLogManager.java | 39 +++++++++++++------ .../encryption/AbstractEncryptionTest.java | 26 +++++++++++-- .../encryption/CacheGroupKeyChangeTest.java | 8 ++-- .../CacheGroupReencryptionTest.java | 6 ++- 5 files changed, 69 insertions(+), 20 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java index d0d467fefb123..666af34dac969 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java @@ -924,6 +924,16 @@ public void onDestroyPartitionStore(CacheGroupContext grp, int partId) { * @param segmentIdx WAL segment index. */ public void onWalSegmentRemoved(long segmentIdx) { + // TODO: Fix will be in IGNITE-13847 + ctx.getSystemExecutorService().submit(() -> releaseWalKeys(segmentIdx)); + } + + /** + * Cleanup keys reserved for WAL reading. + * + * @param segmentIdx WAL segment index. + */ + private void releaseWalKeys(long segmentIdx) { withMasterKeyChangeReadLock(() -> { synchronized (metaStorageMux) { Map> rmvKeys = grpKeys.releaseWalKeys(segmentIdx); 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 1b48f72d61c55..caee3521a0ac3 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 @@ -169,31 +169,46 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl public static final Pattern WAL_TEMP_NAME_PATTERN = Pattern.compile("\\d{16}\\.wal\\.tmp"); /** WAL segment file filter, see {@link #WAL_NAME_PATTERN} */ - public static final FileFilter WAL_SEGMENT_FILE_FILTER = file -> !file.isDirectory() && - WAL_NAME_PATTERN.matcher(file.getName()).matches(); + public static final FileFilter WAL_SEGMENT_FILE_FILTER = new FileFilter() { + @Override public boolean accept(File file) { + return !file.isDirectory() && WAL_NAME_PATTERN.matcher(file.getName()).matches(); + } + }; /** WAL segment temporary file filter, see {@link #WAL_TEMP_NAME_PATTERN} */ - private static final FileFilter WAL_SEGMENT_TEMP_FILE_FILTER = file -> !file.isDirectory() && - WAL_TEMP_NAME_PATTERN.matcher(file.getName()).matches(); + private static final FileFilter WAL_SEGMENT_TEMP_FILE_FILTER = new FileFilter() { + @Override public boolean accept(File file) { + return !file.isDirectory() && WAL_TEMP_NAME_PATTERN.matcher(file.getName()).matches(); + } + }; /** */ public static final Pattern WAL_SEGMENT_FILE_COMPACTED_PATTERN = Pattern.compile("\\d{16}\\.wal\\.zip"); /** WAL segment file filter, see {@link #WAL_NAME_PATTERN} */ - public static final FileFilter WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER = file -> !file.isDirectory() && - (WAL_NAME_PATTERN.matcher(file.getName()).matches() || - WAL_SEGMENT_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches()); + public static final FileFilter WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER = new FileFilter() { + @Override public boolean accept(File file) { + return !file.isDirectory() && (WAL_NAME_PATTERN.matcher(file.getName()).matches() || + WAL_SEGMENT_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches()); + } + }; /** */ private static final Pattern WAL_SEGMENT_TEMP_FILE_COMPACTED_PATTERN = Pattern.compile("\\d{16}\\.wal\\.zip\\.tmp"); /** */ - private static final FileFilter WAL_SEGMENT_FILE_COMPACTED_FILTER = file -> !file.isDirectory() && - WAL_SEGMENT_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches(); + private static final FileFilter WAL_SEGMENT_FILE_COMPACTED_FILTER = new FileFilter() { + @Override public boolean accept(File file) { + return !file.isDirectory() && WAL_SEGMENT_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches(); + } + }; /** */ - private static final FileFilter WAL_SEGMENT_TEMP_FILE_COMPACTED_FILTER = file -> !file.isDirectory() && - WAL_SEGMENT_TEMP_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches(); + private static final FileFilter WAL_SEGMENT_TEMP_FILE_COMPACTED_FILTER = new FileFilter() { + @Override public boolean accept(File file) { + return !file.isDirectory() && WAL_SEGMENT_TEMP_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches(); + } + }; /** Buffer size. */ private static final int BUF_SIZE = 1024 * 1024; @@ -3123,7 +3138,7 @@ else if (file.exists()) { * @throws IgniteCheckedException If failed. */ private void cleanWalArchive(long nextArchiveSegmentSize) throws IgniteCheckedException { - if (maxWalArchiveSize == Long.MAX_VALUE || + if (maxWalArchiveSize == DataStorageConfiguration.UNLIMITED_WAL_ARCHIVE || walArchiveSize.get() + reservedWalArchiveSize.get() + nextArchiveSegmentSize < maxWalArchiveSize) return; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/encryption/AbstractEncryptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/encryption/AbstractEncryptionTest.java index 8c66afe05d54c..fffdfae4ba3bc 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/encryption/AbstractEncryptionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/encryption/AbstractEncryptionTest.java @@ -69,6 +69,7 @@ import static org.apache.ignite.internal.pagemem.PageIdAllocator.INDEX_PARTITION; import static org.apache.ignite.spi.encryption.keystore.KeystoreEncryptionSpi.CIPHER_ALGO; import static org.apache.ignite.spi.encryption.keystore.KeystoreEncryptionSpi.DEFAULT_MASTER_KEY_NAME; +import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; /** * Abstract encryption test. @@ -230,7 +231,7 @@ protected void createEncryptedCache(IgniteEx grid0, @Nullable IgniteEx grid1, St IgniteCache cache = grid0.createCache(cacheConfiguration(cacheName, cacheGroup)); if (grid1 != null) - GridTestUtils.waitForCondition(() -> grid1.cachex(cacheName()) != null, 2_000L); + waitForCondition(() -> grid1.cachex(cacheName()) != null, 2_000L); if (putData) { for (long i = 0; i < 100; i++) @@ -379,7 +380,7 @@ protected void checkGroupKey(int grpId, int expKeyId, long timeout) throws Excep // The future will be completed after the checkpoint, forcecheckpoint does nothing // if the checkpoint has already been scheduled. - GridTestUtils.waitForCondition(() -> { + waitForCondition(() -> { if (fut.isDone()) return true; @@ -480,7 +481,7 @@ protected void awaitEncryption(List grids, int grpId, long timeout) thro IgniteInternalFuture fut0 = GridTestUtils.runAsync(() -> { boolean success = - GridTestUtils.waitForCondition(() -> !isReencryptionInProgress(grid, grpId), timeout); + waitForCondition(() -> !isReencryptionInProgress(grid, grpId), timeout); assertTrue(success); @@ -517,4 +518,23 @@ protected boolean isReencryptionInProgress(IgniteEx node, int grpId) { return ReencryptStateUtils.pageIndex(state) != ReencryptStateUtils.pageCount(state); } + + /** + * // TODO: Fix will be in IGNITE-13847 + * @param node Ignite node. + * @param grpId Cache group ID. + * @param keysCnt Expected keys count. + */ + protected void checkKeysCount( + IgniteEx node, + int grpId, + int keysCnt, + long timeout + ) throws IgniteInterruptedCheckedException { + GridEncryptionManager encMgr = node.context().encryption(); + + waitForCondition(() -> encMgr.groupKeyIds(grpId).size() == keysCnt, timeout); + + assertEquals(keysCnt, encMgr.groupKeyIds(grpId).size()); + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/encryption/CacheGroupKeyChangeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/encryption/CacheGroupKeyChangeTest.java index 810e05d109ace..769723ba56b40 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/encryption/CacheGroupKeyChangeTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/encryption/CacheGroupKeyChangeTest.java @@ -531,7 +531,8 @@ public void testNodeWithOlderKeyBecameCoordinator() throws Exception { } // Make sure the previous key has been removed. - assertEquals(1, encrMgr0.groupKeyIds(grpId).size()); + // TODO: Fix will be in IGNITE-13847 + checkKeysCount(node0, grpId, 1, MAX_AWAIT_MILLIS); assertEquals(encrMgr1.groupKeyIds(grpId), encrMgr0.groupKeyIds(grpId)); } @@ -930,8 +931,9 @@ public void testNodeJoinAfterRotation() throws Exception { encrMgr1.onWalSegmentRemoved(maxWalIdx); } - assertEquals(1, encrMgr1.groupKeyIds(grpId).size()); - assertEquals(encrMgr0.groupKeyIds(grpId), encrMgr1.groupKeyIds(grpId)); + // TODO: Fix will be in IGNITE-13847 + checkKeysCount(grid(GRID_1), grpId, 1, MAX_AWAIT_MILLIS); + checkKeysCount(grid(GRID_0), grpId, 1, MAX_AWAIT_MILLIS); startGrid(GRID_2); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/encryption/CacheGroupReencryptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/encryption/CacheGroupReencryptionTest.java index 19c8351184175..ae6bd34701cb2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/encryption/CacheGroupReencryptionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/encryption/CacheGroupReencryptionTest.java @@ -518,12 +518,14 @@ public void testNotBltNodeJoin() throws Exception { for (long segment = startIdx1; segment <= endIdx1; segment++) grid(GRID_0).context().encryption().onWalSegmentRemoved(segment); - assertEquals(1, grid(GRID_0).context().encryption().groupKeyIds(grpId).size()); + // TODO: Fix will be in IGNITE-13847 + checkKeysCount(grid(GRID_0), grpId, 1, MAX_AWAIT_MILLIS); for (long segment = startIdx2; segment <= endIdx2; segment++) grid(GRID_1).context().encryption().onWalSegmentRemoved(segment); - assertEquals(1, grid(GRID_1).context().encryption().groupKeyIds(grpId).size()); + // TODO: Fix will be in IGNITE-13847 + checkKeysCount(grid(GRID_1), grpId, 1, MAX_AWAIT_MILLIS); } /** From 7b35509ea2bdf3fbf5e5bcf5a435e93ea7a18eff Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Tue, 15 Dec 2020 19:25:10 +0300 Subject: [PATCH 13/21] IGNITE-13831 wip add SegmentArchiveSizeStorage --- .../wal/FileWriteAheadLogManager.java | 6 +- .../wal/aware/SegmentArchiveSizeStorage.java | 112 ++++++++++++++++++ .../persistence/wal/aware/SegmentAware.java | 48 +++++++- .../wal/aware/SegmentCompressStorage.java | 2 +- .../wal/aware/SegmentLockStorage.java | 2 +- .../wal/aware/SegmentObservable.java | 2 +- .../wal/aware/SegmentAwareTest.java | 110 ++++++++++++----- 7 files changed, 247 insertions(+), 35 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentArchiveSizeStorage.java 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 caee3521a0ac3..cac5884f8b391 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 @@ -508,7 +508,11 @@ public void setFileIOFactory(FileIOFactory ioFactory) { }); } - segmentAware = new SegmentAware(dsCfg.getWalSegments(), dsCfg.isWalCompactionEnabled()); + segmentAware = new SegmentAware( + dsCfg.getWalSegments(), + dsCfg.isWalCompactionEnabled(), + dsCfg.getMaxWalArchiveSize() + ); // We have to initialize compressor before archiver in order to setup already compressed segments. // Otherwise, FileArchiver initialization will trigger redundant work for FileCompressor. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentArchiveSizeStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentArchiveSizeStorage.java new file mode 100644 index 0000000000000..bf66aa5e1516a --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentArchiveSizeStorage.java @@ -0,0 +1,112 @@ +/* + * 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.wal.aware; + +import org.apache.ignite.internal.IgniteInterruptedCheckedException; + +/** + * Storage WAL archive size. + */ +class SegmentArchiveSizeStorage { + /** Maximum WAL archive size in bytes. */ + private final long max; + + /** Current WAL archive size in bytes. */ + private long curr; + + /** Reserved WAL archive size in bytes. */ + private long reserved; + + /** Flag of interrupt waiting on this object. */ + private volatile boolean interrupted; + + /** + * Constructor. + * + * @param max Maximum WAL archive size in bytes. + */ + SegmentArchiveSizeStorage(long max) { + this.max = max; + } + + /** + * Adding current WAL archive size in bytes. + * + * @param size Size in bytes. + */ + synchronized void addCurrentSize(long size) { + curr += size; + + if (size > 0) + notifyAll(); + } + + /** + * Adding reserved WAL archive size in bytes. + * + * @param size Size in bytes. + */ + synchronized void addReservedSize(long size) { + reserved += size; + + if (size > 0) + notifyAll(); + } + + /** + * Reset the current and reserved WAL archive sizes. + */ + synchronized void resetSizes() { + curr = 0; + reserved = 0; + } + + /** + * Waiting for exceeding the maximum WAL archive size. + * + * @throws IgniteInterruptedCheckedException If it was interrupted. + */ + synchronized void awaitExceedMaxSize() throws IgniteInterruptedCheckedException { + try { + while (max - (curr + reserved) > 0 && !interrupted) + wait(); + } + catch (InterruptedException e) { + throw new IgniteInterruptedCheckedException(e); + } + + if (interrupted) + throw new IgniteInterruptedCheckedException("Interrupt waiting of exceed max archive size"); + } + + /** + * Interrupt waiting on this object. + */ + synchronized void interrupt() { + interrupted = true; + + notifyAll(); + } + + /** + * Reset interrupted flag. + */ + void reset() { + interrupted = false; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAware.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAware.java index 89523db552f83..12e10de5bac85 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAware.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAware.java @@ -38,18 +38,24 @@ public class SegmentAware { /** Storage of absolute current segment index. */ private final SegmentCurrentStateStorage segmentCurrStateStorage; + /** Storage of archive size. */ + private final SegmentArchiveSizeStorage archiveSizeStorage; + /** * Constructor. * * @param walSegmentsCnt Total WAL segments count. * @param compactionEnabled Is wal compaction enabled. + * @param maxWalArchiveSize Maximum WAL archive size in bytes. */ - public SegmentAware(int walSegmentsCnt, boolean compactionEnabled) { + public SegmentAware(int walSegmentsCnt, boolean compactionEnabled, long maxWalArchiveSize) { segmentArchivedStorage = new SegmentArchivedStorage(segmentLockStorage); segmentCurrStateStorage = new SegmentCurrentStateStorage(walSegmentsCnt); segmentCompressStorage = new SegmentCompressStorage(compactionEnabled); + archiveSizeStorage = new SegmentArchiveSizeStorage(maxWalArchiveSize); + segmentArchivedStorage.addObserver(segmentCurrStateStorage::onSegmentArchived); segmentArchivedStorage.addObserver(segmentCompressStorage::onSegmentArchived); @@ -239,6 +245,8 @@ public void reset() { segmentCompressStorage.reset(); segmentCurrStateStorage.reset(); + + archiveSizeStorage.reset(); } /** @@ -250,6 +258,8 @@ public void interrupt() { segmentCompressStorage.interrupt(); segmentCurrStateStorage.interrupt(); + + archiveSizeStorage.interrupt(); } /** @@ -261,6 +271,8 @@ public void forceInterrupt() { segmentCompressStorage.interrupt(); segmentCurrStateStorage.forceInterrupt(); + + archiveSizeStorage.interrupt(); } /** @@ -286,4 +298,38 @@ public boolean minReserveIndex(long absIdx) { public boolean minLockIndex(long absIdx) { return segmentLockStorage.minLockIndex(absIdx); } + + /** + * Adding current WAL archive size in bytes. + * + * @param size Size in bytes. + */ + public void addCurrentWalArchiveSize(long size) { + archiveSizeStorage.addCurrentSize(size); + } + + /** + * Adding reserved WAL archive size in bytes. + * + * @param size Size in bytes. + */ + public void addReservedWalArchiveSize(long size) { + archiveSizeStorage.addReservedSize(size); + } + + /** + * Reset the current and reserved WAL archive sizes. + */ + public void resetWalArchiveSizes() { + archiveSizeStorage.resetSizes(); + } + + /** + * Waiting for exceeding the maximum WAL archive size. + * + * @throws IgniteInterruptedCheckedException If it was interrupted. + */ + public void awaitExceedMaxArchiveSize() throws IgniteInterruptedCheckedException { + archiveSizeStorage.awaitExceedMaxSize(); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentCompressStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentCompressStorage.java index 62fe69d7c1a47..7f2085dfdd786 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentCompressStorage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentCompressStorage.java @@ -26,7 +26,7 @@ /** * Storage of actual information about current index of compressed segments. */ -public class SegmentCompressStorage { +class SegmentCompressStorage { /** Flag of interrupt waiting on this object. */ private volatile boolean interrupted; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentLockStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentLockStorage.java index a5a79486ec4b4..189559acaa4fe 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentLockStorage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentLockStorage.java @@ -24,7 +24,7 @@ /** * Lock on segment protects from archiving segment. */ -public class SegmentLockStorage extends SegmentObservable { +class SegmentLockStorage extends SegmentObservable { /** * Maps absolute segment index to locks counter. Lock on segment protects from archiving segment and may come from * {@link FileWriteAheadLogManager.RecordsIterator} during WAL replay. Map itself is guarded by this. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentObservable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentObservable.java index 3e915044dd01b..f6cd9b04a2ae1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentObservable.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentObservable.java @@ -24,7 +24,7 @@ /** * Implementation of observer-observable pattern. For handling specific changes of segment. */ -public abstract class SegmentObservable { +abstract class SegmentObservable { /** Observers for handle changes of archived index. */ private final Queue> observers = new ConcurrentLinkedQueue<>(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java index 60663ef6e1f42..c85c781646269 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java @@ -21,9 +21,11 @@ import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.logger.NullLogger; import org.apache.ignite.testframework.GridTestUtils; import org.junit.Test; +import static org.apache.ignite.testframework.GridTestUtils.assertThrows; import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -43,7 +45,7 @@ public class SegmentAwareTest { */ @Test public void testAvoidDeadlockArchiverAndLockStorage() throws IgniteCheckedException { - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); int iterationCnt = 100_000; int segmentToHandle = 1; @@ -80,7 +82,7 @@ public void testAvoidDeadlockArchiverAndLockStorage() throws IgniteCheckedExcept @Test public void testFinishAwaitSegment_WhenExactWaitingSegmentWasSet() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); IgniteInternalFuture future = awaitThread(() -> aware.awaitSegment(5)); @@ -97,7 +99,7 @@ public void testFinishAwaitSegment_WhenExactWaitingSegmentWasSet() throws Ignite @Test public void testFinishAwaitSegment_WhenGreaterThanWaitingSegmentWasSet() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); IgniteInternalFuture future = awaitThread(() -> aware.awaitSegment(5)); @@ -114,7 +116,7 @@ public void testFinishAwaitSegment_WhenGreaterThanWaitingSegmentWasSet() throws @Test public void testFinishAwaitSegment_WhenNextSegmentEqualToWaitingOne() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); IgniteInternalFuture future = awaitThread(() -> aware.awaitSegment(5)); @@ -137,7 +139,7 @@ public void testFinishAwaitSegment_WhenNextSegmentEqualToWaitingOne() throws Ign @Test public void testFinishAwaitSegment_WhenInterruptWasCall() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); IgniteInternalFuture future = awaitThread(() -> aware.awaitSegment(5)); @@ -154,7 +156,7 @@ public void testFinishAwaitSegment_WhenInterruptWasCall() throws IgniteCheckedEx @Test public void testFinishWaitSegmentForArchive_WhenWorkSegmentIncremented() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); aware.curAbsWalIdx(5); aware.setLastArchivedAbsoluteIndex(4); @@ -174,7 +176,7 @@ public void testFinishWaitSegmentForArchive_WhenWorkSegmentIncremented() throws @Test public void testFinishWaitSegmentForArchive_WhenWorkSegmentGreaterValue() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); aware.curAbsWalIdx(5); aware.setLastArchivedAbsoluteIndex(4); @@ -194,7 +196,7 @@ public void testFinishWaitSegmentForArchive_WhenWorkSegmentGreaterValue() throws @Test public void testFinishWaitSegmentForArchive_WhenInterruptWasCall() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); aware.curAbsWalIdx(5); aware.setLastArchivedAbsoluteIndex(4); @@ -214,7 +216,7 @@ public void testFinishWaitSegmentForArchive_WhenInterruptWasCall() throws Ignite @Test public void testCorrectCalculateNextSegmentIndex() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); aware.curAbsWalIdx(5); @@ -231,7 +233,7 @@ public void testCorrectCalculateNextSegmentIndex() throws IgniteCheckedException @Test public void testFinishWaitNextAbsoluteIndex_WhenMarkAsArchivedFirstSegment() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(2, false); + SegmentAware aware = new SegmentAware(2, false, 0); aware.curAbsWalIdx(1); aware.setLastArchivedAbsoluteIndex(-1); @@ -251,7 +253,7 @@ public void testFinishWaitNextAbsoluteIndex_WhenMarkAsArchivedFirstSegment() thr @Test public void testFinishWaitNextAbsoluteIndex_WhenSetToArchivedFirst() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(2, false); + SegmentAware aware = new SegmentAware(2, false, 0); aware.curAbsWalIdx(1); aware.setLastArchivedAbsoluteIndex(-1); @@ -271,7 +273,7 @@ public void testFinishWaitNextAbsoluteIndex_WhenSetToArchivedFirst() throws Igni @Test public void testFinishWaitNextAbsoluteIndex_WhenOnlyForceInterruptWasCall() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(2, false); + SegmentAware aware = new SegmentAware(2, false, 0); aware.curAbsWalIdx(2); aware.setLastArchivedAbsoluteIndex(-1); @@ -297,7 +299,7 @@ public void testFinishWaitNextAbsoluteIndex_WhenOnlyForceInterruptWasCall() thro @Test public void testFinishSegmentArchived_WhenSetExactWaitingSegment() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); IgniteInternalFuture future = awaitThread(() -> aware.awaitSegmentArchived(5)); @@ -314,7 +316,7 @@ public void testFinishSegmentArchived_WhenSetExactWaitingSegment() throws Ignite @Test public void testFinishSegmentArchived_WhenMarkExactWaitingSegment() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); IgniteInternalFuture future = awaitThread(() -> aware.awaitSegmentArchived(5)); @@ -331,7 +333,7 @@ public void testFinishSegmentArchived_WhenMarkExactWaitingSegment() throws Ignit @Test public void testFinishSegmentArchived_WhenSetGreaterThanWaitingSegment() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); IgniteInternalFuture future = awaitThread(() -> aware.awaitSegmentArchived(5)); @@ -348,7 +350,7 @@ public void testFinishSegmentArchived_WhenSetGreaterThanWaitingSegment() throws @Test public void testFinishSegmentArchived_WhenMarkGreaterThanWaitingSegment() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); IgniteInternalFuture future = awaitThread(() -> aware.awaitSegmentArchived(5)); @@ -365,7 +367,7 @@ public void testFinishSegmentArchived_WhenMarkGreaterThanWaitingSegment() throws @Test public void testFinishSegmentArchived_WhenInterruptWasCall() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); aware.curAbsWalIdx(5); aware.setLastArchivedAbsoluteIndex(4); @@ -385,7 +387,7 @@ public void testFinishSegmentArchived_WhenInterruptWasCall() throws IgniteChecke @Test public void testMarkAsMovedToArchive_WhenReleaseLockedSegment() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); assertTrue(aware.lock(5)); @@ -404,7 +406,7 @@ public void testMarkAsMovedToArchive_WhenReleaseLockedSegment() throws IgniteChe @Test public void testMarkAsMovedToArchive_WhenInterruptWasCall() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); assertTrue(aware.lock(5)); @@ -426,7 +428,7 @@ public void testMarkAsMovedToArchive_WhenInterruptWasCall() throws IgniteChecked @Test public void testFinishWaitSegmentToCompress_WhenSetLastArchivedSegment() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, true); + SegmentAware aware = new SegmentAware(10, true, 0); aware.onSegmentCompressed(5); @@ -445,7 +447,7 @@ public void testFinishWaitSegmentToCompress_WhenSetLastArchivedSegment() throws @Test public void testFinishWaitSegmentToCompress_WhenMarkLastArchivedSegment() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, true); + SegmentAware aware = new SegmentAware(10, true, 0); aware.onSegmentCompressed(5); @@ -463,7 +465,7 @@ public void testFinishWaitSegmentToCompress_WhenMarkLastArchivedSegment() throws */ @Test public void testCorrectCalculateNextCompressSegment() throws IgniteCheckedException, InterruptedException { - SegmentAware aware = new SegmentAware(10, true); + SegmentAware aware = new SegmentAware(10, true, 0); aware.setLastArchivedAbsoluteIndex(6); @@ -477,7 +479,7 @@ public void testCorrectCalculateNextCompressSegment() throws IgniteCheckedExcept @Test public void testFinishWaitSegmentToCompress_WhenInterruptWasCall() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, true); + SegmentAware aware = new SegmentAware(10, true, 0); aware.onSegmentCompressed(5); IgniteInternalFuture future = awaitThread(aware::waitNextSegmentToCompress); @@ -494,7 +496,7 @@ public void testFinishWaitSegmentToCompress_WhenInterruptWasCall() throws Ignite */ @Test public void testLastCompressedIdxProperOrdering() throws IgniteInterruptedCheckedException { - SegmentAware aware = new SegmentAware(10, true); + SegmentAware aware = new SegmentAware(10, true, 0); for (int i = 0; i < 5; i++) { aware.setLastArchivedAbsoluteIndex(i); @@ -519,7 +521,7 @@ public void testLastCompressedIdxProperOrdering() throws IgniteInterruptedChecke @Test public void testReserveCorrectly() { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); // Set limits. aware.curAbsWalIdx(10); @@ -567,7 +569,7 @@ public void testReserveCorrectly() { @Test public void testAssertFail_WhenReleaseUnreservedSegment() { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); aware.reserve(5); try { @@ -587,7 +589,7 @@ public void testAssertFail_WhenReleaseUnreservedSegment() { @Test public void testReserveWorkSegmentCorrectly() { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); //when: lock one segment twice. assertTrue(aware.lock(5)); @@ -621,7 +623,7 @@ public void testReserveWorkSegmentCorrectly() { @Test public void testAssertFail_WhenReleaseUnreservedWorkSegment() { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); assertTrue(aware.lock(5)); try { @@ -639,7 +641,7 @@ public void testAssertFail_WhenReleaseUnreservedWorkSegment() { */ @Test public void testReservationBorder() { - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); assertTrue(aware.reserve(0)); assertTrue(aware.reserve(1)); @@ -661,7 +663,7 @@ public void testReservationBorder() { */ @Test public void testLockBorder() { - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, 0); assertTrue(aware.lock(0)); assertTrue(aware.lock(1)); @@ -678,6 +680,54 @@ public void testLockBorder() { assertTrue(aware.lock(1)); } + /** + * Checking the correctness of WAL archive size. + * + * @throws Exception If failed. + */ + @Test + public void testWalArchiveSize() throws Exception { + SegmentAware aware = new SegmentAware(10, false, 10); + + IgniteInternalFuture fut = awaitThread(aware::awaitExceedMaxArchiveSize); + + aware.addCurrentWalArchiveSize(4); + assertFutureIsNotFinish(fut); + + aware.addReservedWalArchiveSize(4); + assertFutureIsNotFinish(fut); + + aware.addCurrentWalArchiveSize(4); + fut.get(20); + + aware.resetWalArchiveSizes(); + + fut = awaitThread(aware::awaitExceedMaxArchiveSize); + + aware.addCurrentWalArchiveSize(4); + assertFutureIsNotFinish(fut); + + aware.addReservedWalArchiveSize(4); + assertFutureIsNotFinish(fut); + + aware.addReservedWalArchiveSize(4); + fut.get(20); + + aware.resetWalArchiveSizes(); + + fut = awaitThread(aware::awaitExceedMaxArchiveSize); + + aware.interrupt(); + assertTrue(fut.get(20) instanceof IgniteInterruptedCheckedException); + + aware.reset(); + + fut = awaitThread(aware::awaitExceedMaxArchiveSize); + + aware.forceInterrupt(); + assertTrue(fut.get(20) instanceof IgniteInterruptedCheckedException); + } + /** * Assert that future is still not finished. * From 99151dfbf28145f70b0608df081695c4e3e40751 Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Wed, 16 Dec 2020 11:55:59 +0300 Subject: [PATCH 14/21] IGNITE-13831 wip add FileCleaner --- .../wal/FileWriteAheadLogManager.java | 269 ++++++++++++------ .../wal/aware/SegmentArchiveSizeStorage.java | 3 + .../persistence/wal/aware/SegmentAware.java | 4 +- .../wal/aware/SegmentAwareTest.java | 2 - 4 files changed, 187 insertions(+), 91 deletions(-) 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 cac5884f8b391..914d8c54f0ce1 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 @@ -51,7 +51,6 @@ import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.regex.Pattern; import java.util.stream.Collectors; -import java.util.stream.Stream; import java.util.zip.ZipEntry; import java.util.zip.ZipInputStream; import java.util.zip.ZipOutputStream; @@ -334,6 +333,12 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl /** Decompressor. */ @Nullable private FileDecompressor decompressor; + /** + * Cleaner of segments from WAL archive when the maximum size is reached. + * Will not work if WAL archive size is {@link DataStorageConfiguration#UNLIMITED_WAL_ARCHIVE}. + */ + @Nullable private FileCleaner cleaner; + /** Current log segment handle. */ private volatile FileWriteHandle currHnd; @@ -403,15 +408,6 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl /** Pointer to the last successful checkpoint until which WAL segments can be safely deleted. */ private volatile WALPointer lastCheckpointPtr = new WALPointer(0, 0, 0); - /** Maximum WAL archive size in bytes. */ - private final long maxWalArchiveSize; - - /** Current size of WAL archive in bytes. */ - private final AtomicLong walArchiveSize = new AtomicLong(); - - /** Reserved size of WAL archive in bytes. */ - private final AtomicLong reservedWalArchiveSize = new AtomicLong(); - /** * Constructor. * @@ -446,8 +442,6 @@ public FileWriteAheadLogManager(final GridKernalContext ctx) { / dsCfg.getWalSegmentSize()); switchSegmentRecordOffset = isArchiverEnabled() ? new AtomicLongArray(dsCfg.getWalSegments()) : null; - - maxWalArchiveSize = dsCfg.getMaxWalArchiveSize(); } /** @@ -524,8 +518,9 @@ public void setFileIOFactory(FileIOFactory ioFactory) { if (isArchiverEnabled()) archiver = new FileArchiver(segmentAware, log); - else - archiver = null; + + if (!walArchiveUnlimited()) + cleaner = new FileCleaner(log); segmentRouter = new SegmentRouter(walWorkDir, walArchiveDir, segmentAware, dsCfg); @@ -564,11 +559,14 @@ public SegmentRouter getSegmentRouter() { } /** - * + * Running workers of WAL archive. */ - private void startArchiverAndCompressor() { + private void startArchiveWorkers() { segmentAware.reset(); + segmentAware.resetWalArchiveSizes(); + segmentAware.addCurrentWalArchiveSize(totalSize(walArchiveFiles())); + if (isArchiverEnabled()) { assert archiver != null : "FileArchiver should be initialized."; @@ -584,6 +582,12 @@ private void startArchiverAndCompressor() { decompressor.restart(); } + + if (!walArchiveUnlimited()) { + assert cleaner != null : "FileCleaner should be initialized."; + + cleaner.restart(); + } } /** @@ -684,6 +688,9 @@ private void checkWalConfiguration() throws IgniteCheckedException { if (decompressor != null) decompressor.shutdown(); + + if (cleaner != null) + cleaner.shutdown(); } catch (IgniteInterruptedCheckedException e) { U.error(log, "Failed to gracefully shutdown WAL components, thread was interrupted.", e); @@ -733,11 +740,7 @@ private void checkWalConfiguration() throws IgniteCheckedException { assert currHnd == null; - // It needs to be calculated before #startArchiverAndCompressor, to avoid possible race. - walArchiveSize.set(Stream.of(walArchiveFiles()).mapToLong(fd -> fd.file.length()).sum()); - reservedWalArchiveSize.set(0); - - startArchiverAndCompressor(); + startArchiveWorkers(); assert (isArchiverEnabled() && archiver != null) || (!isArchiverEnabled() && archiver == null) : "Trying to restore FileWriteHandle on deactivated write ahead log manager"; @@ -1095,7 +1098,7 @@ private boolean hasIndex(long absIdx) { deleted++; segmentSize.remove(desc.idx()); - walArchiveSize.addAndGet(-len); + segmentAware.addCurrentWalArchiveSize(-len); } // Bump up the oldest archive segment index. @@ -1293,38 +1296,42 @@ private FileWriteHandle rollOver(FileWriteHandle cur, @Nullable WALRecord rec) t if (metrics.metricsEnabled()) metrics.onWallRollOver(); - long nextArchiveSegmentSize = maxWalSegmentSize; - if (switchSegmentRecordOffset != null) { int idx = (int)(cur.getSegmentId() % dsCfg.getWalSegments()); switchSegmentRecordOffset.set(idx, hnd.getSwitchSegmentRecordOffset()); - nextArchiveSegmentSize = hnd.getSwitchSegmentRecordOffset(); } + if (archiver == null) + segmentAware.addReservedWalArchiveSize(maxWalSegmentSize); + FileWriteHandle next; try { - cleanWalArchive(nextArchiveSegmentSize); - - next = initNextWriteHandle(cur); - } - catch (IgniteCheckedException e) { - //Allow to avoid forever waiting in other threads. - cur.signalNextAvailable(); + try { + next = initNextWriteHandle(cur); + } + catch (IgniteCheckedException e) { + //Allow to avoid forever waiting in other threads. + cur.signalNextAvailable(); - throw e; - } + throw e; + } - if (rec != null) { - WALPointer ptr = next.addRecord(rec); + if (rec != null) { + WALPointer ptr = next.addRecord(rec); - assert ptr != null; - } + assert ptr != null; + } - segmentSize.put(next.getSegmentId(), maxWalSegmentSize); + segmentSize.put(next.getSegmentId(), maxWalSegmentSize); - if (archiver == null) - walArchiveSize.addAndGet(maxWalSegmentSize); + if (archiver == null) + segmentAware.addCurrentWalArchiveSize(maxWalSegmentSize); + } + finally { + if (archiver == null) + segmentAware.addReservedWalArchiveSize(-maxWalSegmentSize); + } if (next.getSegmentId() - lashCheckpointFileIdx() >= maxSegCountWithoutCheckpoint) cctx.database().forceCheckpoint("too big size of WAL without checkpoint"); @@ -2024,7 +2031,7 @@ public SegmentArchiveResult archiveSegment(long absIdx) throws StorageException long origLen = origFile.length(); long reservedSize = offs > 0 && offs < origLen ? offs : origLen; - reservedWalArchiveSize.addAndGet(reservedSize); + segmentAware.addReservedWalArchiveSize(reservedSize); try { if (offs > 0 && offs < origLen) @@ -2041,17 +2048,17 @@ public SegmentArchiveResult archiveSegment(long absIdx) throws StorageException } segmentSize.put(absIdx, dstFile.length()); - walArchiveSize.addAndGet(dstFile.length()); + segmentAware.addCurrentWalArchiveSize(dstFile.length()); } catch (IOException e) { - deleteArchiveFiles(false, dstFile, dstTmpFile); + deleteArchiveFiles(dstFile, dstTmpFile); throw new StorageException("Failed to archive WAL segment [" + "srcFile=" + origFile.getAbsolutePath() + ", dstFile=" + dstTmpFile.getAbsolutePath() + ']', e); } finally { - reservedWalArchiveSize.addAndGet(-reservedSize); + segmentAware.addReservedWalArchiveSize(-reservedSize); } if (log.isInfoEnabled()) { @@ -2247,7 +2254,7 @@ private void body0() { File raw = new File(walArchiveDir, segmentFileName); long reservedSize = raw.length(); - reservedWalArchiveSize.addAndGet(reservedSize); + segmentAware.addReservedWalArchiveSize(reservedSize); try { deleteObsoleteRawSegments(); @@ -2264,7 +2271,7 @@ private void body0() { } segmentSize.put(segIdx, zip.length()); - walArchiveSize.addAndGet(zip.length()); + segmentAware.addCurrentWalArchiveSize(zip.length()); segmentAware.onSegmentCompressed(segIdx); @@ -2272,7 +2279,7 @@ private void body0() { evt.record(new WalSegmentCompactedEvent(cctx.localNode(), segIdx, zip.getAbsoluteFile())); } catch (IgniteCheckedException | IOException e) { - deleteArchiveFiles(false, zip, tmpZip); + deleteArchiveFiles(zip, tmpZip); lastCompressionError = e; @@ -2282,7 +2289,7 @@ private void body0() { segmentAware.onSegmentCompressed(segIdx); } finally { - reservedWalArchiveSize.addAndGet(-reservedSize); + segmentAware.addReservedWalArchiveSize(-reservedSize); } } catch (IgniteInterruptedCheckedException ignore) { @@ -2393,7 +2400,7 @@ private void deleteObsoleteRawSegments() { return; if (desc.idx < lastCheckpointPtr.index() && duplicateIndices.contains(desc.idx)) - deleteArchiveFiles(true, desc.file); + segmentAware.addCurrentWalArchiveSize(-deleteArchiveFiles(desc.file)); } } } @@ -2449,7 +2456,7 @@ private class FileDecompressor extends GridWorker { File unzip = new File(walArchiveDir, segmentFileName); long reservedSize = U.uncompressedSize(zip); - reservedWalArchiveSize.addAndGet(reservedSize); + segmentAware.addReservedWalArchiveSize(reservedSize); IgniteCheckedException ex = null; @@ -2467,10 +2474,10 @@ private class FileDecompressor extends GridWorker { Files.move(unzipTmp.toPath(), unzip.toPath()); - walArchiveSize.addAndGet(unzip.length()); + segmentAware.addCurrentWalArchiveSize(unzip.length()); } catch (IOException e) { - deleteArchiveFiles(false, unzipTmp); + deleteArchiveFiles(unzipTmp); if (e instanceof FileAlreadyExistsException) { U.error(log, "Can't rename temporary unzipped segment: raw segment is already present " + @@ -2482,7 +2489,7 @@ else if (!isCancelled) { } } finally { - reservedWalArchiveSize.addAndGet(-reservedSize); + segmentAware.addReservedWalArchiveSize(-reservedSize); } updateHeartbeat(); @@ -3113,66 +3120,152 @@ public static boolean isSegmentFileName(@Nullable String name) { } /** - * Removing files from {@link #walArchiveDir} with updating {@link #walArchiveSize}. + * Check if WAL archive is unlimited. + * + * @return {@code True} if unlimited. + */ + private boolean walArchiveUnlimited() { + return dsCfg.getMaxWalArchiveSize() == DataStorageConfiguration.UNLIMITED_WAL_ARCHIVE; + } + + /** + * Total size of the segments in bytes. + * + * @return Size in bytes. + */ + private long totalSize(FileDescriptor... fileDescriptors) { + long len = 0; + + for (FileDescriptor descriptor : fileDescriptors) + len += descriptor.file.length(); + + return len; + } + + /** + * Removing files from {@link #walArchiveDir}. * - * @param updateArchiveSize Flag to update {@link #walArchiveSize}. * @param files Files from {@link #walArchiveDir}. + * @return Total deleted size in bytes. */ - private void deleteArchiveFiles(boolean updateArchiveSize, File... files) { + private long deleteArchiveFiles(File... files) { + long size = 0; + for (File file : files) { if (file.exists()) { long len = file.length(); - if (file.delete()) { - if (updateArchiveSize) - walArchiveSize.addAndGet(-len); - } + if (file.delete()) + size += len; else if (file.exists()) { U.warn(log, "Unable to delete file from WAL archive" + " (make sure the process has enough rights): " + file.getAbsolutePath()); } } } + + return size; } /** - * Clearing WAL archive when reaching the maximum. - * - * @param nextArchiveSegmentSize Size of next segment to be archived in bytes. - * @throws IgniteCheckedException If failed. + * Worker for an asynchronous WAL archive cleanup that starts when the maximum size is exceeded. + * {@link SegmentAware#awaitExceedMaxArchiveSize} is used to determine if the maximum is exceeded. */ - private void cleanWalArchive(long nextArchiveSegmentSize) throws IgniteCheckedException { - if (maxWalArchiveSize == DataStorageConfiguration.UNLIMITED_WAL_ARCHIVE || - walArchiveSize.get() + reservedWalArchiveSize.get() + nextArchiveSegmentSize < maxWalArchiveSize) - return; + private class FileCleaner extends GridWorker { + /** + * Constructor. + * + * @param log Logger. + */ + public FileCleaner(IgniteLogger log) { + super(cctx.igniteInstanceName(), "wal-file-cleaner%" + cctx.igniteInstanceName(), log); + + assert !walArchiveUnlimited(); + } - FileDescriptor[] walArchiveFiles = walArchiveFiles(); + /** {@inheritDoc} */ + @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException { + Throwable err = null; - FileDescriptor high = null; + try { + while (!isCancelled()) { + segmentAware.awaitExceedMaxArchiveSize(); - long size = 0; + FileDescriptor[] walArchiveFiles = walArchiveFiles(); - for (FileDescriptor fileDesc : walArchiveFiles) { - if (fileDesc.idx >= lashCheckpointFileIdx() || - (size += fileDesc.file.length()) > allowedThresholdWalArchiveSize) - break; - else - high = fileDesc; - } + FileDescriptor high = null; - if (high != null) { - WALPointer highPtr = new WALPointer(high.idx + 1, 0, 0); + long size = 0; - ((GridCacheDatabaseSharedManager)cctx.database()).onWalTruncated(highPtr); + for (FileDescriptor fileDesc : walArchiveFiles) { + if (fileDesc.idx >= lashCheckpointFileIdx() || + segmentAware.reserved(fileDesc.idx) || + (size += fileDesc.file.length()) > allowedThresholdWalArchiveSize) + break; + else + high = fileDesc; + } - int truncated = truncate(highPtr); + if (high != null) { + WALPointer highPtr = new WALPointer(high.idx + 1, 0, 0); - if (log.isInfoEnabled()) { - log.info("Cleaning WAL archive completed [highIdx=" + high.idx - + ", cleanCnt=" + truncated - + ", currSize=" + U.humanReadableByteCount(walArchiveSize.get()) - + ", maxSize=" + U.humanReadableByteCount(maxWalArchiveSize) + ']'); + if (log.isInfoEnabled()) { + log.info("Starting to clean WAL archive [highIdx=" + highPtr.index() + + ", currSize=" + U.humanReadableByteCount(totalSize(walArchiveFiles)) + + ", maxSize=" + U.humanReadableByteCount(dsCfg.getMaxWalArchiveSize()) + ']'); + } + + ((GridCacheDatabaseSharedManager)cctx.database()).onWalTruncated(highPtr); + + int truncated = truncate(highPtr); + + if (log.isInfoEnabled()) { + log.info("Finish clean WAL archive [cleanCnt=" + truncated + + ", currSize=" + U.humanReadableByteCount(totalSize(walArchiveFiles())) + + ", maxSize=" + U.humanReadableByteCount(dsCfg.getMaxWalArchiveSize()) + ']'); + } + } + } + } + catch (IgniteInterruptedCheckedException e) { + Thread.currentThread().interrupt(); + + isCancelled = true; + } + catch (Throwable t) { + err = t; + } + finally { + if (err == null && !isCancelled()) + err = new IllegalStateException("Worker " + name() + " is terminated unexpectedly"); + + if (err instanceof OutOfMemoryError) + failureProcessor.process(new FailureContext(CRITICAL_ERROR, err)); + else if (err != null) + failureProcessor.process(new FailureContext(SYSTEM_WORKER_TERMINATION, err)); } } + + /** + * Shutdown worker. + * + * @throws IgniteInterruptedCheckedException If failed to wait for worker shutdown. + */ + private void shutdown() throws IgniteInterruptedCheckedException { + isCancelled = true; + + U.join(this); + } + + /** + * Restart worker in IgniteThread. + */ + public void restart() { + assert runner() == null : "FileCleaner is still running"; + + isCancelled = false; + + new IgniteThread(archiver).start(); + } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentArchiveSizeStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentArchiveSizeStorage.java index bf66aa5e1516a..7720b25b25e98 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentArchiveSizeStorage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentArchiveSizeStorage.java @@ -21,6 +21,7 @@ /** * Storage WAL archive size. + * Allows to track the exceeding of the maximum archive size. */ class SegmentArchiveSizeStorage { /** Maximum WAL archive size in bytes. */ @@ -58,6 +59,7 @@ synchronized void addCurrentSize(long size) { /** * Adding reserved WAL archive size in bytes. + * Defines a hint to determine if the maximum size is exceeded before a new segment is archived. * * @param size Size in bytes. */ @@ -78,6 +80,7 @@ synchronized void resetSizes() { /** * Waiting for exceeding the maximum WAL archive size. + * To track size of WAL archive, need to use {@link #addCurrentSize} and {@link #addReservedSize}. * * @throws IgniteInterruptedCheckedException If it was interrupted. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAware.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAware.java index 12e10de5bac85..5f7d44d5a7a7b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAware.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAware.java @@ -310,6 +310,7 @@ public void addCurrentWalArchiveSize(long size) { /** * Adding reserved WAL archive size in bytes. + * Defines a hint to determine if the maximum size is exceeded before a new segment is archived. * * @param size Size in bytes. */ @@ -325,7 +326,8 @@ public void resetWalArchiveSizes() { } /** - * Waiting for exceeding the maximum WAL archive size. + * Waiting for exceeding the maximum WAL archive size. To track size of WAL archive, + * need to use {@link #addCurrentWalArchiveSize} and {@link #addReservedWalArchiveSize}. * * @throws IgniteInterruptedCheckedException If it was interrupted. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java index c85c781646269..bb5d605354e22 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java @@ -21,11 +21,9 @@ import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; -import org.apache.ignite.logger.NullLogger; import org.apache.ignite.testframework.GridTestUtils; import org.junit.Test; -import static org.apache.ignite.testframework.GridTestUtils.assertThrows; import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; From 17f4a4b9a547ab822864f959a055086ea7e23d3a Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Wed, 16 Dec 2020 12:41:32 +0300 Subject: [PATCH 15/21] IGNITE-13831 wip --- .../cache/persistence/wal/FileWriteAheadLogManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 914d8c54f0ce1..8f36484be2075 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 @@ -3265,7 +3265,7 @@ public void restart() { isCancelled = false; - new IgniteThread(archiver).start(); + new IgniteThread(this).start(); } } } From 2c3cc8d5433da16b53a02a12c36be849e58281aa Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Wed, 16 Dec 2020 15:13:50 +0300 Subject: [PATCH 16/21] IGNITE-13831 wip --- .../wal/FileWriteAheadLogManager.java | 19 +++++++++---- .../wal/WalDeletionArchiveAbstractTest.java | 27 ++++++++++--------- 2 files changed, 28 insertions(+), 18 deletions(-) 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 8f36484be2075..62773f40b3667 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 @@ -3120,12 +3120,12 @@ public static boolean isSegmentFileName(@Nullable String name) { } /** - * Check if WAL archive is unlimited. + * Getting last truncated segment. * - * @return {@code True} if unlimited. + * @return Absolut segment index. */ - private boolean walArchiveUnlimited() { - return dsCfg.getMaxWalArchiveSize() == DataStorageConfiguration.UNLIMITED_WAL_ARCHIVE; + public long lastTruncatedSegment() { + return segmentAware.lastTruncatedArchiveIdx(); } /** @@ -3133,7 +3133,7 @@ private boolean walArchiveUnlimited() { * * @return Size in bytes. */ - private long totalSize(FileDescriptor... fileDescriptors) { + public long totalSize(FileDescriptor... fileDescriptors) { long len = 0; for (FileDescriptor descriptor : fileDescriptors) @@ -3142,6 +3142,15 @@ private long totalSize(FileDescriptor... fileDescriptors) { return len; } + /** + * Check if WAL archive is unlimited. + * + * @return {@code True} if unlimited. + */ + private boolean walArchiveUnlimited() { + return dsCfg.getMaxWalArchiveSize() == DataStorageConfiguration.UNLIMITED_WAL_ARCHIVE; + } + /** * Removing files from {@link #walArchiveDir}. * diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java index 12293c1e4d7f3..9218b843377b5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java @@ -42,6 +42,8 @@ import org.junit.Test; import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE; +import static org.apache.ignite.testframework.GridTestUtils.getFieldValueHierarchy; +import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; /** * @@ -156,14 +158,14 @@ public void testCorrectDeletedArchivedWalFiles() throws Exception { cache.put(i, i); } - //then: total archive size less than half of maxWalArchiveSize(by current logic) + //then: total archive size less than of maxWalArchiveSize(by current logic) FileWriteAheadLogManager wal = wal(ignite); + assertTrue(waitForCondition(() -> wal.lastTruncatedSegment() >= 0, 10_000)); + FileDescriptor[] files = wal.walArchiveFiles(); - Long totalSize = Stream.of(files) - .map(desc -> desc.file().length()) - .reduce(0L, Long::sum); + long totalSize = wal.totalSize(files); assertTrue(files.length >= 1); assertTrue(totalSize <= maxWalArchiveSize && totalSize >= allowedThresholdWalArchiveSize); @@ -178,9 +180,7 @@ public void testCorrectDeletedArchivedWalFiles() throws Exception { @Test public void testCheckpointStarted_WhenWalHasTooBigSizeWithoutCheckpoint() throws Exception { //given: configured grid with max wal archive size = 1MB, wal segment size = 512KB - Ignite ignite = startGrid(dbCfg -> { - dbCfg.setMaxWalArchiveSize(1024 * 1024);// 1 Mbytes - }); + Ignite ignite = startGrid(dbCfg -> dbCfg.setMaxWalArchiveSize(1024 * 1024)); GridCacheDatabaseSharedManager dbMgr = gridDatabase(ignite); @@ -229,6 +229,9 @@ public void testCheckpointHistoryRemovingByTruncate() throws Exception { for (int i = 0; i < 6; i++) cache.put(i, new byte[ignite.configuration().getDataStorageConfiguration().getWalSegmentSize() / 2]); + FileWriteAheadLogManager wal = wal(ignite); + assertTrue(waitForCondition(() -> wal.lastTruncatedSegment() >= 0, 10_000)); + assertTrue(hist.checkpoints().size() < checkpointCnt + startHistSize); File[] cpFiles = dbMgr.checkpointDirectory().listFiles(); @@ -237,8 +240,8 @@ public void testCheckpointHistoryRemovingByTruncate() throws Exception { } /** - * Correct delete checkpoint history from memory depends on IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE. WAL files - * doesn't delete because deleting was disabled. + * Correct delete checkpoint history from memory depends on IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE. + * WAL files doesn't delete because deleting was disabled. */ @Test @WithSystemProperty(key = IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE, value = "2") @@ -265,13 +268,11 @@ public void testCorrectDeletedCheckpointHistoryButKeepWalFiles() throws Exceptio //then: WAL files was not deleted but some of checkpoint history was deleted. FileWriteAheadLogManager wal = wal(ignite); + assertNull(getFieldValueHierarchy(wal, "cleaner")); FileDescriptor[] files = wal.walArchiveFiles(); - boolean hasFirstSegment = Stream.of(files) - .anyMatch(desc -> desc.file().getName().endsWith("0001.wal")); - - assertTrue(hasFirstSegment); + assertTrue(Stream.of(files).anyMatch(desc -> desc.file().getName().endsWith("0001.wal"))); assertTrue(hist.checkpoints().size() == 2); } From 0ae12d6bb410b8f869e4d1b8b5b83fddff840472 Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Thu, 17 Dec 2020 12:45:17 +0300 Subject: [PATCH 17/21] IGNITE-13831 wip add SegmentTruncateStorage --- .../wal/FileWriteAheadLogManager.java | 23 ++- .../wal/aware/SegmentArchivedStorage.java | 17 -- .../persistence/wal/aware/SegmentAware.java | 50 +++++- .../wal/aware/SegmentReservationStorage.java | 67 ++++++-- .../wal/aware/SegmentTruncateStorage.java | 150 ++++++++++++++++++ .../wal/aware/SegmentAwareTest.java | 47 ++++++ 6 files changed, 307 insertions(+), 47 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentTruncateStorage.java 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 62773f40b3667..8f32035a85338 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 @@ -1128,6 +1128,8 @@ private boolean segmentReservedOrLocked(long absIdx) { /** {@inheritDoc} */ @Override public void notchLastCheckpointPtr(WALPointer ptr) { lastCheckpointPtr = ptr; + + segmentAware.lastCheckpointIdx(ptr.index()); } /** {@inheritDoc} */ @@ -1333,7 +1335,7 @@ private FileWriteHandle rollOver(FileWriteHandle cur, @Nullable WALRecord rec) t segmentAware.addReservedWalArchiveSize(-maxWalSegmentSize); } - if (next.getSegmentId() - lashCheckpointFileIdx() >= maxSegCountWithoutCheckpoint) + if (next.getSegmentId() - lastCheckpointPtr.index() >= maxSegCountWithoutCheckpoint) cctx.database().forceCheckpoint("too big size of WAL without checkpoint"); boolean updated = updateCurrentHandle(next, hnd); @@ -1352,15 +1354,6 @@ private FileWriteHandle rollOver(FileWriteHandle cur, @Nullable WALRecord rec) t return currentHandle(); } - /** - * Give last checkpoint file idx. - */ - private long lashCheckpointFileIdx() { - WALPointer lastCheckpointMark = cctx.database().lastCheckpointMarkWalPointer(); - - return lastCheckpointMark == null ? 0 : lastCheckpointMark.index(); - } - /** * @param lastReadPtr Last read WAL file pointer. * @return Initialized file write handle. @@ -1413,6 +1406,7 @@ private FileWriteHandle restoreWriteHandle(@Nullable WALPointer lastReadPtr) thr FileDescriptor[] walArchiveFiles = walArchiveFiles(); segmentAware.minReserveIndex(F.isEmpty(walArchiveFiles) ? -1 : walArchiveFiles[0].idx - 1); + segmentAware.lastTruncatedArchiveIdx(F.isEmpty(walArchiveFiles) ? -1 : walArchiveFiles[0].idx - 1); if (archiver0 == null) segmentAware.setLastArchivedAbsoluteIndex(absIdx - 1); @@ -3199,6 +3193,7 @@ public FileCleaner(IgniteLogger log) { try { while (!isCancelled()) { segmentAware.awaitExceedMaxArchiveSize(); + segmentAware.awaitAvailableTruncateArchive(); FileDescriptor[] walArchiveFiles = walArchiveFiles(); @@ -3207,7 +3202,7 @@ public FileCleaner(IgniteLogger log) { long size = 0; for (FileDescriptor fileDesc : walArchiveFiles) { - if (fileDesc.idx >= lashCheckpointFileIdx() || + if (fileDesc.idx >= lastCheckpointPtr.index() || segmentAware.reserved(fileDesc.idx) || (size += fileDesc.file.length()) > allowedThresholdWalArchiveSize) break; @@ -3229,8 +3224,12 @@ public FileCleaner(IgniteLogger log) { int truncated = truncate(highPtr); if (log.isInfoEnabled()) { + FileDescriptor[] newWalArchiveFiles = walArchiveFiles(); + long cleanSize = totalSize(walArchiveFiles) - totalSize(newWalArchiveFiles); + log.info("Finish clean WAL archive [cleanCnt=" + truncated - + ", currSize=" + U.humanReadableByteCount(totalSize(walArchiveFiles())) + + ", currSize=" + U.humanReadableByteCount(totalSize(newWalArchiveFiles)) + + ", cleanSize=" + U.humanReadableByteCount(cleanSize) + ", maxSize=" + U.humanReadableByteCount(dsCfg.getMaxWalArchiveSize()) + ']'); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentArchivedStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentArchivedStorage.java index 53b3b598a97eb..12a1e5f2fdc7b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentArchivedStorage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentArchivedStorage.java @@ -37,9 +37,6 @@ class SegmentArchivedStorage extends SegmentObservable { */ private volatile long lastAbsArchivedIdx = -1; - /** Latest truncated segment. */ - private volatile long lastTruncatedArchiveIdx = -1; - /** * @param segmentLockStorage Protects WAL work segments from moving. */ @@ -137,18 +134,4 @@ private void checkInterrupted() throws IgniteInterruptedCheckedException { synchronized void onSegmentUnlocked(long segmentId) { notifyAll(); } - - /** - * @param lastTruncatedArchiveIdx Last truncated segment. - */ - void lastTruncatedArchiveIdx(long lastTruncatedArchiveIdx) { - this.lastTruncatedArchiveIdx = lastTruncatedArchiveIdx; - } - - /** - * @return Last truncated segment. - */ - long lastTruncatedArchiveIdx() { - return lastTruncatedArchiveIdx; - } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAware.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAware.java index 5f7d44d5a7a7b..0b9032c635410 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAware.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAware.java @@ -41,6 +41,9 @@ public class SegmentAware { /** Storage of archive size. */ private final SegmentArchiveSizeStorage archiveSizeStorage; + /** Storage of truncated segments. */ + private final SegmentTruncateStorage truncateStorage; + /** * Constructor. * @@ -55,11 +58,15 @@ public SegmentAware(int walSegmentsCnt, boolean compactionEnabled, long maxWalAr segmentCompressStorage = new SegmentCompressStorage(compactionEnabled); archiveSizeStorage = new SegmentArchiveSizeStorage(maxWalArchiveSize); + truncateStorage = new SegmentTruncateStorage(); segmentArchivedStorage.addObserver(segmentCurrStateStorage::onSegmentArchived); segmentArchivedStorage.addObserver(segmentCompressStorage::onSegmentArchived); + segmentArchivedStorage.addObserver(truncateStorage::lastArchivedIdx); segmentLockStorage.addObserver(segmentArchivedStorage::onSegmentUnlocked); + + reservationStorage.addObserver(truncateStorage::minReservedIdx); } /** @@ -153,17 +160,21 @@ public void curAbsWalIdx(long curAbsWalIdx) { } /** - * @param lastTruncatedArchiveIdx Last truncated segment; + * Update last truncated segment. + * + * @param absIdx Absolut segment index. */ - public void lastTruncatedArchiveIdx(long lastTruncatedArchiveIdx) { - segmentArchivedStorage.lastTruncatedArchiveIdx(lastTruncatedArchiveIdx); + public void lastTruncatedArchiveIdx(long absIdx) { + truncateStorage.lastTruncatedIdx(absIdx); } /** - * @return Last truncated segment. + * Getting last truncated segment. + * + * @return Absolut segment index. */ public long lastTruncatedArchiveIdx() { - return segmentArchivedStorage.lastTruncatedArchiveIdx(); + return truncateStorage.lastTruncatedIdx(); } /** @@ -247,6 +258,8 @@ public void reset() { segmentCurrStateStorage.reset(); archiveSizeStorage.reset(); + + truncateStorage.reset(); } /** @@ -260,6 +273,8 @@ public void interrupt() { segmentCurrStateStorage.interrupt(); archiveSizeStorage.interrupt(); + + truncateStorage.interrupt(); } /** @@ -273,6 +288,8 @@ public void forceInterrupt() { segmentCurrStateStorage.forceInterrupt(); archiveSizeStorage.interrupt(); + + truncateStorage.interrupt(); } /** @@ -334,4 +351,27 @@ public void resetWalArchiveSizes() { public void awaitExceedMaxArchiveSize() throws IgniteInterruptedCheckedException { archiveSizeStorage.awaitExceedMaxSize(); } + + /** + * Update segment of last completed checkpoint. + * Required for binary recovery. + * + * @param absIdx Absolut segment index. + */ + public void lastCheckpointIdx(long absIdx) { + truncateStorage.lastCheckpointIdx(absIdx); + } + + /** + * Waiting for segment truncation to be available. To get the number of segments available for truncation, use + * {@link #lastTruncatedArchiveIdx}, {@link #lastCheckpointIdx}, {@link #reserve} and + * {@link #lastArchivedAbsoluteIndex} (to restart the node correctly) and is calculated as + * {@code lastTruncatedArchiveIdx} - {@code min(lastCheckpointIdx, reserve, lastArchivedAbsoluteIndex)}. + * + * @return Number of segments available to truncate. + * @throws IgniteInterruptedCheckedException If it was interrupted. + */ + public long awaitAvailableTruncateArchive() throws IgniteInterruptedCheckedException { + return truncateStorage.awaitAvailableTruncate(); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentReservationStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentReservationStorage.java index 42eece70761b2..453cc175d12d8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentReservationStorage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentReservationStorage.java @@ -16,13 +16,17 @@ */ package org.apache.ignite.internal.processors.cache.persistence.wal.aware; +import java.util.Map; import java.util.NavigableMap; +import java.util.Objects; import java.util.TreeMap; +import java.util.function.Consumer; +import org.jetbrains.annotations.Nullable; /** * Segment reservations storage: Protects WAL segments from deletion during WAL log cleanup. */ -class SegmentReservationStorage { +class SegmentReservationStorage extends SegmentObservable { /** * Maps absolute segment index to reservation counter. If counter > 0 then we wouldn't delete all segments which has * index >= reserved segment index. Guarded by {@code this}. @@ -38,14 +42,22 @@ class SegmentReservationStorage { * @param absIdx Index for reservation. * @return {@code True} if the reservation was successful. */ - synchronized boolean reserve(long absIdx) { - if (absIdx > minReserveIdx) { - reserved.merge(absIdx, 1, Integer::sum); + boolean reserve(long absIdx) { + boolean res = false; + Long minReservedIdx = null; - return true; + synchronized (this) { + if (absIdx > minReserveIdx) { + minReservedIdx = trackingMinReservedIdx(reserved -> reserved.merge(absIdx, 1, Integer::sum)); + + res = true; + } } - return false; + if (minReservedIdx != null) + notifyObservers(minReservedIdx); + + return res; } /** @@ -61,15 +73,24 @@ synchronized boolean reserved(long absIdx) { /** * @param absIdx Reserved index. */ - synchronized void release(long absIdx) { - Integer cur = reserved.get(absIdx); + void release(long absIdx) { + Long minReservedIdx; - assert cur != null && cur >= 1 : "cur=" + cur + ", absIdx=" + absIdx; + synchronized (this) { + minReservedIdx = trackingMinReservedIdx(reserved -> { + Integer cur = reserved.get(absIdx); - if (cur == 1) - reserved.remove(absIdx); - else - reserved.put(absIdx, cur - 1); + assert cur != null && cur >= 1 : "cur=" + cur + ", absIdx=" + absIdx; + + if (cur == 1) + reserved.remove(absIdx); + else + reserved.put(absIdx, cur - 1); + }); + } + + if (minReservedIdx != null) + notifyObservers(minReservedIdx); } /** @@ -88,4 +109,24 @@ synchronized boolean minReserveIndex(long absIdx) { return true; } + + /** + * Updating {@link #reserved} with tracking changes of minimum reserved segment. + * + * @param updateFun {@link #reserved} update function. + * @return New minimum reserved segment, {@code null} if there are no changes, + * {@code -1} if there are no reserved segments. + */ + @Nullable private synchronized Long trackingMinReservedIdx(Consumer> updateFun) { + Map.Entry oldMinE = reserved.firstEntry(); + + updateFun.accept(reserved); + + Map.Entry newMinE = reserved.firstEntry(); + + Long oldMin = oldMinE == null ? null : oldMinE.getKey(); + Long newMin = newMinE == null ? null : newMinE.getKey(); + + return Objects.equals(oldMin, newMin) ? null : newMin == null ? -1 : newMin; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentTruncateStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentTruncateStorage.java new file mode 100644 index 0000000000000..b2ee5eb29fcee --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentTruncateStorage.java @@ -0,0 +1,150 @@ +/* + * 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.wal.aware; + +import org.apache.ignite.internal.IgniteInterruptedCheckedException; + +/** + * Store the last truncated segment and allows to get the number of segments available for truncation. + * We cannot truncate the segments required for {@link #lastCpIdx binary recovery}, {@link #minReservedIdx reserved} + * and {@link #lastArchivedIdx last archived} (to restart the node correctly). Thus, we need to take account of these + * conditions in the calculation of the number of segments available for truncation. + */ +class SegmentTruncateStorage { + /** Flag of interrupt waiting on this object. */ + private volatile boolean interrupted; + + /** Latest truncated segment. */ + private long lastTruncatedIdx = -1; + + /** Minimum reserved segment. */ + private long minReservedIdx = -1; + + /** Segment of last completed checkpoint. */ + private long lastCpIdx = -1; + + /** Last archived segment. */ + private long lastArchivedIdx = -1; + + /** + * Update last truncated segment. + * + * @param absIdx Absolut segment index. + */ + synchronized void lastTruncatedIdx(long absIdx) { + lastTruncatedIdx = absIdx; + + notifyAll(); + } + + /** + * Update minimum reserved segment. + * Protected from deletion. + * + * @param absIdx Absolut segment index. + */ + synchronized void minReservedIdx(long absIdx) { + minReservedIdx = absIdx; + + notifyAll(); + } + + /** + * Update segment of last completed checkpoint. + * Required for binary recovery. + * + * @param absIdx Absolut segment index. + */ + synchronized void lastCheckpointIdx(long absIdx) { + lastCpIdx = absIdx; + + notifyAll(); + } + + /** + * Update last archived segment. + * Needed to restart the node correctly. + * + * @param absIdx Absolut segment index. + */ + synchronized void lastArchivedIdx(long absIdx) { + lastArchivedIdx = absIdx; + + notifyAll(); + } + + /** + * Getting last truncated segment. + * + * @return Absolut segment index. + */ + synchronized long lastTruncatedIdx() { + return lastTruncatedIdx; + } + + /** + * Waiting for segment truncation to be available. Use {@link #lastTruncatedIdx}, {@link #lastCpIdx}, + * {@link #minReservedIdx} and {@link #lastArchivedIdx} to determine the number of segments to truncate. + * + * @return Number of segments available to truncate. + * @throws IgniteInterruptedCheckedException If it was interrupted. + */ + synchronized long awaitAvailableTruncate() throws IgniteInterruptedCheckedException { + try { + while (availableTruncateCnt() <= 0 && !interrupted) + wait(); + } + catch (InterruptedException e) { + throw new IgniteInterruptedCheckedException(e); + } + + if (interrupted) + throw new IgniteInterruptedCheckedException("Interrupt waiting for truncation availability"); + + return availableTruncateCnt(); + } + + /** + * Interrupt waiting on this object. + */ + synchronized void interrupt() { + interrupted = true; + + notifyAll(); + } + + /** + * Resets interrupted flag. + */ + void reset() { + interrupted = false; + } + + /** + * Calculation the number of segments that can be truncated. + * + * @return Number of segments. + */ + private synchronized long availableTruncateCnt() { + long highIdx = minReservedIdx == -1 ? lastCpIdx : Math.min(minReservedIdx, lastCpIdx); + + highIdx = lastArchivedIdx == -1 ? highIdx : Math.min(lastArchivedIdx, highIdx); + + return highIdx == -1 ? 0 : highIdx - (lastTruncatedIdx + 1); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java index bb5d605354e22..db0aad60340f7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java @@ -726,6 +726,53 @@ public void testWalArchiveSize() throws Exception { assertTrue(fut.get(20) instanceof IgniteInterruptedCheckedException); } + /** + * Checking the correctness of truncate logic. + * + * @throws Exception If failed. + */ + @Test + public void testTruncate() throws Exception { + SegmentAware aware = new SegmentAware(10, false, 0); + + IgniteInternalFuture fut = awaitThread(aware::awaitAvailableTruncateArchive); + + aware.lastCheckpointIdx(5); + + fut.get(20); + assertEquals(5, aware.awaitAvailableTruncateArchive()); + + aware.reserve(4); + assertEquals(4, aware.awaitAvailableTruncateArchive()); + + aware.setLastArchivedAbsoluteIndex(3); + assertEquals(3, aware.awaitAvailableTruncateArchive()); + + aware.lastTruncatedArchiveIdx(0); + assertEquals(2, aware.awaitAvailableTruncateArchive()); + assertEquals(0, aware.lastTruncatedArchiveIdx()); + + aware.reserve(0); + fut = awaitThread(aware::awaitAvailableTruncateArchive); + + aware.release(0); + + fut.get(20); + assertEquals(2, aware.awaitAvailableTruncateArchive()); + + aware.setLastArchivedAbsoluteIndex(4); + assertEquals(3, aware.awaitAvailableTruncateArchive()); + + aware.release(4); + assertEquals(3, aware.awaitAvailableTruncateArchive()); + + aware.lastCheckpointIdx(6); + assertEquals(3, aware.awaitAvailableTruncateArchive()); + + aware.setLastArchivedAbsoluteIndex(6); + assertEquals(5, aware.awaitAvailableTruncateArchive()); + } + /** * Assert that future is still not finished. * From 93b53544fcb34cd80e7e1c15fe0855d2738a8d80 Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Thu, 17 Dec 2020 18:18:20 +0300 Subject: [PATCH 18/21] IGNITE-13831 wip modify clean logic --- .../apache/ignite/IgniteSystemProperties.java | 6 +- .../wal/FileWriteAheadLogManager.java | 18 ++--- .../wal/aware/SegmentArchiveSizeStorage.java | 15 +--- .../persistence/wal/aware/SegmentAware.java | 10 +-- .../wal/WalDeletionArchiveAbstractTest.java | 2 +- .../wal/aware/SegmentAwareTest.java | 72 +++++++++---------- 6 files changed, 52 insertions(+), 71 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index 6dd359241c600..aa12e541b06e2 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -1356,11 +1356,11 @@ public final class IgniteSystemProperties { public static final String IGNITE_CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE = "IGNITE_CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE"; /** - * Property for setting percentage of WAL archive size for calculating threshold - * to which old segments will be deleted when maximum is reached. Default value is 0.5 + * Property for setup percentage of WAL archive size to calculate threshold since which removing of old archive should be started. + * Default value is 0.5 */ @SystemProperty(value = "Percentage of WAL archive size to calculate threshold " + - "to which old segments will be deleted when maximum is reached", type = Double.class, + "since which removing of old archive should be started", type = Double.class, defaults = "" + DFLT_THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE) public static final String IGNITE_THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE = "IGNITE_THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE"; 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 8f32035a85338..f0cb4dffc684e 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 @@ -239,7 +239,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl DFLT_CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE); /** - * Percentage of WAL archive size to calculate threshold to which old segments will be deleted when maximum is reached. + * Percentage of WAL archive size to calculate threshold since which removing of old archive should be started. */ private static final double THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE = IgniteSystemProperties.getDouble(IGNITE_THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE, @@ -271,7 +271,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl */ private final long maxSegCountWithoutCheckpoint; - /** Size of wal archive since which removing of old archive should be started */ + /** Size of wal archive since which removing of old archive should be started. */ private final long allowedThresholdWalArchiveSize; /** */ @@ -502,11 +502,7 @@ public void setFileIOFactory(FileIOFactory ioFactory) { }); } - segmentAware = new SegmentAware( - dsCfg.getWalSegments(), - dsCfg.isWalCompactionEnabled(), - dsCfg.getMaxWalArchiveSize() - ); + segmentAware = new SegmentAware(dsCfg.getWalSegments(), dsCfg.isWalCompactionEnabled()); // We have to initialize compressor before archiver in order to setup already compressed segments. // Otherwise, FileArchiver initialization will trigger redundant work for FileCompressor. @@ -3192,7 +3188,7 @@ public FileCleaner(IgniteLogger log) { try { while (!isCancelled()) { - segmentAware.awaitExceedMaxArchiveSize(); + segmentAware.awaitExceedMaxArchiveSize(allowedThresholdWalArchiveSize); segmentAware.awaitAvailableTruncateArchive(); FileDescriptor[] walArchiveFiles = walArchiveFiles(); @@ -3224,12 +3220,8 @@ public FileCleaner(IgniteLogger log) { int truncated = truncate(highPtr); if (log.isInfoEnabled()) { - FileDescriptor[] newWalArchiveFiles = walArchiveFiles(); - long cleanSize = totalSize(walArchiveFiles) - totalSize(newWalArchiveFiles); - log.info("Finish clean WAL archive [cleanCnt=" + truncated - + ", currSize=" + U.humanReadableByteCount(totalSize(newWalArchiveFiles)) - + ", cleanSize=" + U.humanReadableByteCount(cleanSize) + + ", currSize=" + U.humanReadableByteCount(totalSize(walArchiveFiles())) + ", maxSize=" + U.humanReadableByteCount(dsCfg.getMaxWalArchiveSize()) + ']'); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentArchiveSizeStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentArchiveSizeStorage.java index 7720b25b25e98..76d6022bc1b0c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentArchiveSizeStorage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentArchiveSizeStorage.java @@ -24,9 +24,6 @@ * Allows to track the exceeding of the maximum archive size. */ class SegmentArchiveSizeStorage { - /** Maximum WAL archive size in bytes. */ - private final long max; - /** Current WAL archive size in bytes. */ private long curr; @@ -36,15 +33,6 @@ class SegmentArchiveSizeStorage { /** Flag of interrupt waiting on this object. */ private volatile boolean interrupted; - /** - * Constructor. - * - * @param max Maximum WAL archive size in bytes. - */ - SegmentArchiveSizeStorage(long max) { - this.max = max; - } - /** * Adding current WAL archive size in bytes. * @@ -82,9 +70,10 @@ synchronized void resetSizes() { * Waiting for exceeding the maximum WAL archive size. * To track size of WAL archive, need to use {@link #addCurrentSize} and {@link #addReservedSize}. * + * @param max Maximum WAL archive size in bytes. * @throws IgniteInterruptedCheckedException If it was interrupted. */ - synchronized void awaitExceedMaxSize() throws IgniteInterruptedCheckedException { + synchronized void awaitExceedMaxSize(long max) throws IgniteInterruptedCheckedException { try { while (max - (curr + reserved) > 0 && !interrupted) wait(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAware.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAware.java index 0b9032c635410..f068c81717261 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAware.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAware.java @@ -49,15 +49,14 @@ public class SegmentAware { * * @param walSegmentsCnt Total WAL segments count. * @param compactionEnabled Is wal compaction enabled. - * @param maxWalArchiveSize Maximum WAL archive size in bytes. */ - public SegmentAware(int walSegmentsCnt, boolean compactionEnabled, long maxWalArchiveSize) { + public SegmentAware(int walSegmentsCnt, boolean compactionEnabled) { segmentArchivedStorage = new SegmentArchivedStorage(segmentLockStorage); segmentCurrStateStorage = new SegmentCurrentStateStorage(walSegmentsCnt); segmentCompressStorage = new SegmentCompressStorage(compactionEnabled); - archiveSizeStorage = new SegmentArchiveSizeStorage(maxWalArchiveSize); + archiveSizeStorage = new SegmentArchiveSizeStorage(); truncateStorage = new SegmentTruncateStorage(); segmentArchivedStorage.addObserver(segmentCurrStateStorage::onSegmentArchived); @@ -346,10 +345,11 @@ public void resetWalArchiveSizes() { * Waiting for exceeding the maximum WAL archive size. To track size of WAL archive, * need to use {@link #addCurrentWalArchiveSize} and {@link #addReservedWalArchiveSize}. * + * @param max Maximum WAL archive size in bytes. * @throws IgniteInterruptedCheckedException If it was interrupted. */ - public void awaitExceedMaxArchiveSize() throws IgniteInterruptedCheckedException { - archiveSizeStorage.awaitExceedMaxSize(); + public void awaitExceedMaxArchiveSize(long max) throws IgniteInterruptedCheckedException { + archiveSizeStorage.awaitExceedMaxSize(max); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java index 9218b843377b5..e6830ad451c13 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java @@ -168,7 +168,7 @@ public void testCorrectDeletedArchivedWalFiles() throws Exception { long totalSize = wal.totalSize(files); assertTrue(files.length >= 1); - assertTrue(totalSize <= maxWalArchiveSize && totalSize >= allowedThresholdWalArchiveSize); + assertTrue(totalSize <= maxWalArchiveSize); assertFalse(Stream.of(files).anyMatch(desc -> desc.file().getName().endsWith("00001.wal"))); assertTrue(!hist.checkpoints().isEmpty()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java index db0aad60340f7..7891946af916a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java @@ -43,7 +43,7 @@ public class SegmentAwareTest { */ @Test public void testAvoidDeadlockArchiverAndLockStorage() throws IgniteCheckedException { - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); int iterationCnt = 100_000; int segmentToHandle = 1; @@ -80,7 +80,7 @@ public void testAvoidDeadlockArchiverAndLockStorage() throws IgniteCheckedExcept @Test public void testFinishAwaitSegment_WhenExactWaitingSegmentWasSet() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); IgniteInternalFuture future = awaitThread(() -> aware.awaitSegment(5)); @@ -97,7 +97,7 @@ public void testFinishAwaitSegment_WhenExactWaitingSegmentWasSet() throws Ignite @Test public void testFinishAwaitSegment_WhenGreaterThanWaitingSegmentWasSet() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); IgniteInternalFuture future = awaitThread(() -> aware.awaitSegment(5)); @@ -114,7 +114,7 @@ public void testFinishAwaitSegment_WhenGreaterThanWaitingSegmentWasSet() throws @Test public void testFinishAwaitSegment_WhenNextSegmentEqualToWaitingOne() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); IgniteInternalFuture future = awaitThread(() -> aware.awaitSegment(5)); @@ -137,7 +137,7 @@ public void testFinishAwaitSegment_WhenNextSegmentEqualToWaitingOne() throws Ign @Test public void testFinishAwaitSegment_WhenInterruptWasCall() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); IgniteInternalFuture future = awaitThread(() -> aware.awaitSegment(5)); @@ -154,7 +154,7 @@ public void testFinishAwaitSegment_WhenInterruptWasCall() throws IgniteCheckedEx @Test public void testFinishWaitSegmentForArchive_WhenWorkSegmentIncremented() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); aware.curAbsWalIdx(5); aware.setLastArchivedAbsoluteIndex(4); @@ -174,7 +174,7 @@ public void testFinishWaitSegmentForArchive_WhenWorkSegmentIncremented() throws @Test public void testFinishWaitSegmentForArchive_WhenWorkSegmentGreaterValue() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); aware.curAbsWalIdx(5); aware.setLastArchivedAbsoluteIndex(4); @@ -194,7 +194,7 @@ public void testFinishWaitSegmentForArchive_WhenWorkSegmentGreaterValue() throws @Test public void testFinishWaitSegmentForArchive_WhenInterruptWasCall() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); aware.curAbsWalIdx(5); aware.setLastArchivedAbsoluteIndex(4); @@ -214,7 +214,7 @@ public void testFinishWaitSegmentForArchive_WhenInterruptWasCall() throws Ignite @Test public void testCorrectCalculateNextSegmentIndex() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); aware.curAbsWalIdx(5); @@ -231,7 +231,7 @@ public void testCorrectCalculateNextSegmentIndex() throws IgniteCheckedException @Test public void testFinishWaitNextAbsoluteIndex_WhenMarkAsArchivedFirstSegment() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(2, false, 0); + SegmentAware aware = new SegmentAware(2, false); aware.curAbsWalIdx(1); aware.setLastArchivedAbsoluteIndex(-1); @@ -251,7 +251,7 @@ public void testFinishWaitNextAbsoluteIndex_WhenMarkAsArchivedFirstSegment() thr @Test public void testFinishWaitNextAbsoluteIndex_WhenSetToArchivedFirst() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(2, false, 0); + SegmentAware aware = new SegmentAware(2, false); aware.curAbsWalIdx(1); aware.setLastArchivedAbsoluteIndex(-1); @@ -271,7 +271,7 @@ public void testFinishWaitNextAbsoluteIndex_WhenSetToArchivedFirst() throws Igni @Test public void testFinishWaitNextAbsoluteIndex_WhenOnlyForceInterruptWasCall() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(2, false, 0); + SegmentAware aware = new SegmentAware(2, false); aware.curAbsWalIdx(2); aware.setLastArchivedAbsoluteIndex(-1); @@ -297,7 +297,7 @@ public void testFinishWaitNextAbsoluteIndex_WhenOnlyForceInterruptWasCall() thro @Test public void testFinishSegmentArchived_WhenSetExactWaitingSegment() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); IgniteInternalFuture future = awaitThread(() -> aware.awaitSegmentArchived(5)); @@ -314,7 +314,7 @@ public void testFinishSegmentArchived_WhenSetExactWaitingSegment() throws Ignite @Test public void testFinishSegmentArchived_WhenMarkExactWaitingSegment() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); IgniteInternalFuture future = awaitThread(() -> aware.awaitSegmentArchived(5)); @@ -331,7 +331,7 @@ public void testFinishSegmentArchived_WhenMarkExactWaitingSegment() throws Ignit @Test public void testFinishSegmentArchived_WhenSetGreaterThanWaitingSegment() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); IgniteInternalFuture future = awaitThread(() -> aware.awaitSegmentArchived(5)); @@ -348,7 +348,7 @@ public void testFinishSegmentArchived_WhenSetGreaterThanWaitingSegment() throws @Test public void testFinishSegmentArchived_WhenMarkGreaterThanWaitingSegment() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); IgniteInternalFuture future = awaitThread(() -> aware.awaitSegmentArchived(5)); @@ -365,7 +365,7 @@ public void testFinishSegmentArchived_WhenMarkGreaterThanWaitingSegment() throws @Test public void testFinishSegmentArchived_WhenInterruptWasCall() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); aware.curAbsWalIdx(5); aware.setLastArchivedAbsoluteIndex(4); @@ -385,7 +385,7 @@ public void testFinishSegmentArchived_WhenInterruptWasCall() throws IgniteChecke @Test public void testMarkAsMovedToArchive_WhenReleaseLockedSegment() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); assertTrue(aware.lock(5)); @@ -404,7 +404,7 @@ public void testMarkAsMovedToArchive_WhenReleaseLockedSegment() throws IgniteChe @Test public void testMarkAsMovedToArchive_WhenInterruptWasCall() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); assertTrue(aware.lock(5)); @@ -426,7 +426,7 @@ public void testMarkAsMovedToArchive_WhenInterruptWasCall() throws IgniteChecked @Test public void testFinishWaitSegmentToCompress_WhenSetLastArchivedSegment() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, true, 0); + SegmentAware aware = new SegmentAware(10, true); aware.onSegmentCompressed(5); @@ -445,7 +445,7 @@ public void testFinishWaitSegmentToCompress_WhenSetLastArchivedSegment() throws @Test public void testFinishWaitSegmentToCompress_WhenMarkLastArchivedSegment() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, true, 0); + SegmentAware aware = new SegmentAware(10, true); aware.onSegmentCompressed(5); @@ -463,7 +463,7 @@ public void testFinishWaitSegmentToCompress_WhenMarkLastArchivedSegment() throws */ @Test public void testCorrectCalculateNextCompressSegment() throws IgniteCheckedException, InterruptedException { - SegmentAware aware = new SegmentAware(10, true, 0); + SegmentAware aware = new SegmentAware(10, true); aware.setLastArchivedAbsoluteIndex(6); @@ -477,7 +477,7 @@ public void testCorrectCalculateNextCompressSegment() throws IgniteCheckedExcept @Test public void testFinishWaitSegmentToCompress_WhenInterruptWasCall() throws IgniteCheckedException, InterruptedException { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, true, 0); + SegmentAware aware = new SegmentAware(10, true); aware.onSegmentCompressed(5); IgniteInternalFuture future = awaitThread(aware::waitNextSegmentToCompress); @@ -494,7 +494,7 @@ public void testFinishWaitSegmentToCompress_WhenInterruptWasCall() throws Ignite */ @Test public void testLastCompressedIdxProperOrdering() throws IgniteInterruptedCheckedException { - SegmentAware aware = new SegmentAware(10, true, 0); + SegmentAware aware = new SegmentAware(10, true); for (int i = 0; i < 5; i++) { aware.setLastArchivedAbsoluteIndex(i); @@ -519,7 +519,7 @@ public void testLastCompressedIdxProperOrdering() throws IgniteInterruptedChecke @Test public void testReserveCorrectly() { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); // Set limits. aware.curAbsWalIdx(10); @@ -567,7 +567,7 @@ public void testReserveCorrectly() { @Test public void testAssertFail_WhenReleaseUnreservedSegment() { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); aware.reserve(5); try { @@ -587,7 +587,7 @@ public void testAssertFail_WhenReleaseUnreservedSegment() { @Test public void testReserveWorkSegmentCorrectly() { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); //when: lock one segment twice. assertTrue(aware.lock(5)); @@ -621,7 +621,7 @@ public void testReserveWorkSegmentCorrectly() { @Test public void testAssertFail_WhenReleaseUnreservedWorkSegment() { //given: thread which awaited segment. - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); assertTrue(aware.lock(5)); try { @@ -639,7 +639,7 @@ public void testAssertFail_WhenReleaseUnreservedWorkSegment() { */ @Test public void testReservationBorder() { - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); assertTrue(aware.reserve(0)); assertTrue(aware.reserve(1)); @@ -661,7 +661,7 @@ public void testReservationBorder() { */ @Test public void testLockBorder() { - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); assertTrue(aware.lock(0)); assertTrue(aware.lock(1)); @@ -685,9 +685,9 @@ public void testLockBorder() { */ @Test public void testWalArchiveSize() throws Exception { - SegmentAware aware = new SegmentAware(10, false, 10); + SegmentAware aware = new SegmentAware(10, false); - IgniteInternalFuture fut = awaitThread(aware::awaitExceedMaxArchiveSize); + IgniteInternalFuture fut = awaitThread(() -> aware.awaitExceedMaxArchiveSize(10)); aware.addCurrentWalArchiveSize(4); assertFutureIsNotFinish(fut); @@ -700,7 +700,7 @@ public void testWalArchiveSize() throws Exception { aware.resetWalArchiveSizes(); - fut = awaitThread(aware::awaitExceedMaxArchiveSize); + fut = awaitThread(() -> aware.awaitExceedMaxArchiveSize(10)); aware.addCurrentWalArchiveSize(4); assertFutureIsNotFinish(fut); @@ -713,14 +713,14 @@ public void testWalArchiveSize() throws Exception { aware.resetWalArchiveSizes(); - fut = awaitThread(aware::awaitExceedMaxArchiveSize); + fut = awaitThread(() -> aware.awaitExceedMaxArchiveSize(10)); aware.interrupt(); assertTrue(fut.get(20) instanceof IgniteInterruptedCheckedException); aware.reset(); - fut = awaitThread(aware::awaitExceedMaxArchiveSize); + fut = awaitThread(() -> aware.awaitExceedMaxArchiveSize(10)); aware.forceInterrupt(); assertTrue(fut.get(20) instanceof IgniteInterruptedCheckedException); @@ -733,7 +733,7 @@ public void testWalArchiveSize() throws Exception { */ @Test public void testTruncate() throws Exception { - SegmentAware aware = new SegmentAware(10, false, 0); + SegmentAware aware = new SegmentAware(10, false); IgniteInternalFuture fut = awaitThread(aware::awaitAvailableTruncateArchive); From 058b1342f301c6d8fca00a05d4cb38a3354003b6 Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Thu, 17 Dec 2020 18:35:37 +0300 Subject: [PATCH 19/21] IGNITE-13831 wip fix after merge --- .../persistence/db/wal/WalDeletionArchiveAbstractTest.java | 4 +--- .../cache/persistence/wal/aware/SegmentAwareTest.java | 4 ++-- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java index e6830ad451c13..e66abe361b264 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java @@ -146,8 +146,6 @@ public void testCorrectDeletedArchivedWalFiles() throws Exception { CheckpointHistory hist = dbMgr.checkpointHistory(); assertNotNull(hist); - long allowedThresholdWalArchiveSize = maxWalArchiveSize / 2; - IgniteCache cache = ignite.getOrCreateCache(cacheConfiguration()); //when: put to cache more than 2 MB @@ -168,7 +166,7 @@ public void testCorrectDeletedArchivedWalFiles() throws Exception { long totalSize = wal.totalSize(files); assertTrue(files.length >= 1); - assertTrue(totalSize <= maxWalArchiveSize); + assertTrue(totalSize < maxWalArchiveSize); assertFalse(Stream.of(files).anyMatch(desc -> desc.file().getName().endsWith("00001.wal"))); assertTrue(!hist.checkpoints().isEmpty()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java index c5b62b6c95b7d..18291c4999728 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java @@ -686,7 +686,7 @@ public void testLockBorder() { */ @Test public void testWalArchiveSize() throws Exception { - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, new NullLogger()); IgniteInternalFuture fut = awaitThread(() -> aware.awaitExceedMaxArchiveSize(10)); @@ -734,7 +734,7 @@ public void testWalArchiveSize() throws Exception { */ @Test public void testTruncate() throws Exception { - SegmentAware aware = new SegmentAware(10, false); + SegmentAware aware = new SegmentAware(10, false, new NullLogger()); IgniteInternalFuture fut = awaitThread(aware::awaitAvailableTruncateArchive); From ae82e60fa39e4f24b4151efe18a23a9f8a788a9f Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Thu, 17 Dec 2020 19:38:02 +0300 Subject: [PATCH 20/21] IGNITE-13831 wip after merge --- .../encryption/AbstractEncryptionTest.java | 25 +++---------------- 1 file changed, 3 insertions(+), 22 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/encryption/AbstractEncryptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/encryption/AbstractEncryptionTest.java index ae86da2e14f40..ed3b9d42e34cd 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/encryption/AbstractEncryptionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/encryption/AbstractEncryptionTest.java @@ -231,7 +231,7 @@ protected void createEncryptedCache(IgniteEx grid0, @Nullable IgniteEx grid1, St IgniteCache cache = grid0.createCache(cacheConfiguration(cacheName, cacheGroup)); if (grid1 != null) - waitForCondition(() -> grid1.cachex(cacheName()) != null, 2_000L); + GridTestUtils.waitForCondition(() -> grid1.cachex(cacheName()) != null, 2_000L); if (putData) { for (long i = 0; i < 100; i++) @@ -394,7 +394,7 @@ protected void checkGroupKey(int grpId, int expKeyId, long timeout) throws Excep // The future will be completed after the checkpoint, forcecheckpoint does nothing // if the checkpoint has already been scheduled. - waitForCondition(() -> { + GridTestUtils.waitForCondition(() -> { if (fut.isDone()) return true; @@ -495,7 +495,7 @@ protected void awaitEncryption(List grids, int grpId, long timeout) thro IgniteInternalFuture fut0 = GridTestUtils.runAsync(() -> { boolean success = - waitForCondition(() -> !isReencryptionInProgress(grid, grpId), timeout); + GridTestUtils.waitForCondition(() -> !isReencryptionInProgress(grid, grpId), timeout); assertTrue(success); @@ -532,23 +532,4 @@ protected boolean isReencryptionInProgress(IgniteEx node, int grpId) { return ReencryptStateUtils.pageIndex(state) != ReencryptStateUtils.pageCount(state); } - - /** - * // TODO: Fix will be in IGNITE-13847 - * @param node Ignite node. - * @param grpId Cache group ID. - * @param keysCnt Expected keys count. - */ - protected void checkKeysCount( - IgniteEx node, - int grpId, - int keysCnt, - long timeout - ) throws IgniteInterruptedCheckedException { - GridEncryptionManager encMgr = node.context().encryption(); - - waitForCondition(() -> encMgr.groupKeyIds(grpId).size() == keysCnt, timeout); - - assertEquals(keysCnt, encMgr.groupKeyIds(grpId).size()); - } } From 87a0e8b116ab58739901a342d5466040cfbdc8de Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Mon, 21 Dec 2020 08:04:04 +0300 Subject: [PATCH 21/21] IGNITE-13831 wip little changes --- .../persistence/wal/FileWriteAheadLogManager.java | 11 +++++++---- .../persistence/wal/aware/SegmentTruncateStorage.java | 5 +++-- 2 files changed, 10 insertions(+), 6 deletions(-) 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 96ca8e67bad6e..4b97487c93c88 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 @@ -3182,12 +3182,15 @@ public FileCleaner(IgniteLogger log) { long size = 0; for (FileDescriptor fileDesc : walArchiveFiles) { - if (fileDesc.idx >= lastCheckpointPtr.index() || - segmentAware.reserved(fileDesc.idx) || - (size += fileDesc.file.length()) > allowedThresholdWalArchiveSize) + if (fileDesc.idx >= lastCheckpointPtr.index() || segmentAware.reserved(fileDesc.idx)) break; - else + else { high = fileDesc; + + // Ensure that there will be exactly removed at least one segment. + if ((size += fileDesc.file.length()) > allowedThresholdWalArchiveSize) + break; + } } if (high != null) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentTruncateStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentTruncateStorage.java index b2ee5eb29fcee..a65ef8aab264e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentTruncateStorage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentTruncateStorage.java @@ -106,7 +106,7 @@ synchronized long lastTruncatedIdx() { */ synchronized long awaitAvailableTruncate() throws IgniteInterruptedCheckedException { try { - while (availableTruncateCnt() <= 0 && !interrupted) + while (availableTruncateCnt() == 0 && !interrupted) wait(); } catch (InterruptedException e) { @@ -143,8 +143,9 @@ void reset() { private synchronized long availableTruncateCnt() { long highIdx = minReservedIdx == -1 ? lastCpIdx : Math.min(minReservedIdx, lastCpIdx); + // Protection against deleting the last segment from WAL archive for correct restart the node. highIdx = lastArchivedIdx == -1 ? highIdx : Math.min(lastArchivedIdx, highIdx); - return highIdx == -1 ? 0 : highIdx - (lastTruncatedIdx + 1); + return Math.max(0, highIdx == -1 ? 0 : highIdx - (lastTruncatedIdx + 1)); } }