From 2022020738912ce5b79500228722fe5a38c08a4d Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Wed, 15 Nov 2017 00:36:57 +0300 Subject: [PATCH 01/16] IGNITE-5938 Working prototype. --- .../apache/ignite/IgniteSystemProperties.java | 5 + .../wal/IgniteWriteAheadLogManager.java | 5 + .../pagemem/wal/record/FilteredRecord.java | 31 + .../GridCacheDatabaseSharedManager.java | 3 + .../wal/AbstractWalRecordsIterator.java | 21 +- .../wal/FileWriteAheadLogManager.java | 1074 +++++++++++++++-- .../SingleSegmentLogicalRecordsIterator.java | 114 ++ .../reader/StandaloneWalRecordsIterator.java | 5 +- .../persistence/wal/record/RecordTypes.java | 69 ++ .../RecordDataSerializer.java | 3 +- .../serializer/RecordDataV1Serializer.java | 1 - .../serializer/RecordDataV2Serializer.java | 1 - .../{ => serializer}/RecordSerializer.java | 3 +- .../serializer/RecordSerializerFactory.java | 33 + .../RecordSerializerFactoryImpl.java | 98 ++ .../wal/serializer/RecordV1Serializer.java | 1 - .../wal/serializer/RecordV2Serializer.java | 30 +- .../wal/IgniteWalSerializerVersionTest.java | 2 +- .../persistence/pagemem/NoOpWALManager.java | 5 + 19 files changed, 1409 insertions(+), 95 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/FilteredRecord.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/record/RecordTypes.java rename modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/{ => serializer}/RecordDataSerializer.java (95%) rename modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/{ => serializer}/RecordSerializer.java (95%) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactory.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactoryImpl.java diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index 4294c713960f5..9406d7d75a226 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -712,6 +712,11 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_ENABLE_FORCIBLE_NODE_KILL = "IGNITE_ENABLE_FORCIBLE_NODE_KILL"; + /** + * + */ + public static final String IGNITE_WAL_ARCHIVE_COMPACT_SKIP_DELTA_RECORD = "IGNITE_WAL_ARCHIVE_COMPACT_SKIP_DELTA_RECORD"; + /** * Tasks stealing will be started if tasks queue size per data-streamer thread exceeds this threshold. *

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 ce28ff2d5e005..bdb43fa712fe3 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 @@ -101,6 +101,11 @@ public interface IgniteWriteAheadLogManager extends GridCacheSharedManager, Igni */ public int truncate(WALPointer ptr); + /** + * @param ptr Ptr. + */ + public void allowCompressionUntil(WALPointer ptr); + /** * @return Total number of segments in the WAL archive. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/FilteredRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/FilteredRecord.java new file mode 100644 index 0000000000000..519e825953d94 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/FilteredRecord.java @@ -0,0 +1,31 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ +package org.apache.ignite.internal.pagemem.wal.record; + +import org.apache.ignite.internal.processors.cache.persistence.wal.AbstractWalRecordsIterator; + +/** + * Special type of WAL record. Shouldn't be stored in file. + * Returned by deserializer if next record is not matched by filter. Automatically handled by + * {@link AbstractWalRecordsIterator}. + */ +public class FilteredRecord extends WALRecord { + /** {@inheritDoc} */ + @Override public RecordType type() { + return null; + } +} 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 e07aef70a9429..ca9bb9d5d727c 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 @@ -2973,6 +2973,9 @@ private void onCheckpointFinished(Checkpoint chp) { } chp.walFilesDeleted = deleted; + + if (!chp.cpPages.isEmpty()) + cctx.wal().allowCompressionUntil(chp.cpEntry.checkpointMark()); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java index 5be6e5534c7be..76e6b0d6d0ddb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java @@ -30,6 +30,9 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; +import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializer; +import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactory; +import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactoryImpl; import org.apache.ignite.internal.util.GridCloseableIteratorAdapter; import org.apache.ignite.lang.IgniteBiTuple; import org.jetbrains.annotations.NotNull; @@ -70,8 +73,8 @@ public abstract class AbstractWalRecordsIterator */ @NotNull protected final GridCacheSharedContext sharedCtx; - /** Serializer of current version to read headers. */ - @NotNull private final RecordSerializer serializer; + /** Serializer factory. */ + @NotNull private final RecordSerializerFactory serializerFactory; /** Factory to provide I/O interfaces for read/write operations with files */ @NotNull protected final FileIOFactory ioFactory; @@ -82,20 +85,20 @@ public abstract class AbstractWalRecordsIterator /** * @param log Logger. * @param sharedCtx Shared context. - * @param serializer Serializer of current version to read headers. + * @param serializerFactory Serializer of current version to read headers. * @param ioFactory ioFactory for file IO access. * @param bufSize buffer for reading records size. */ protected AbstractWalRecordsIterator( @NotNull final IgniteLogger log, @NotNull final GridCacheSharedContext sharedCtx, - @NotNull final RecordSerializer serializer, + @NotNull final RecordSerializerFactory serializerFactory, @NotNull final FileIOFactory ioFactory, final int bufSize ) { this.log = log; this.sharedCtx = sharedCtx; - this.serializer = serializer; + this.serializerFactory = serializerFactory; this.ioFactory = ioFactory; buf = new ByteBufferExpander(bufSize, ByteOrder.nativeOrder()); @@ -156,8 +159,12 @@ protected void advance() throws IgniteCheckedException { try { curRec = advanceRecord(currWalSegment); - if (curRec != null) + if (curRec != null) { + if (curRec.get2().type() == null) + continue; // Record was skipped by filter of current serializer, should read next record. + return; + } else { currWalSegment = advanceSegment(currWalSegment); @@ -277,7 +284,7 @@ protected FileWriteAheadLogManager.ReadFileHandle initReadHandle( try { int serVer = FileWriteAheadLogManager.readSerializerVersion(fileIO); - RecordSerializer ser = FileWriteAheadLogManager.forVersion(sharedCtx, serVer); + RecordSerializer ser = serializerFactory.createSerializer(serVer); FileInput in = new FileInput(fileIO, buf); 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 b4fc19257e875..9459cfd1a8bad 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 @@ -17,10 +17,14 @@ package org.apache.ignite.internal.processors.cache.persistence.wal; +import java.io.BufferedInputStream; +import java.io.BufferedOutputStream; import java.io.EOFException; import java.io.File; import java.io.FileFilter; +import java.io.FileInputStream; import java.io.FileNotFoundException; +import java.io.FileOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; @@ -31,6 +35,7 @@ import java.util.Map; import java.util.NavigableMap; import java.util.TreeMap; +import java.util.concurrent.PriorityBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; @@ -40,7 +45,11 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.regex.Pattern; +import java.util.zip.ZipEntry; +import java.util.zip.ZipInputStream; +import java.util.zip.ZipOutputStream; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.configuration.DataStorageConfiguration; @@ -49,6 +58,7 @@ import org.apache.ignite.events.EventType; import org.apache.ignite.events.WalSegmentArchivedEvent; import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager; import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; @@ -68,12 +78,15 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32; import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordDataV1Serializer; -import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordDataV2Serializer; +import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializer; +import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactory; +import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactoryImpl; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer; -import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV2Serializer; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.GridUnsafe; +import org.apache.ignite.internal.util.future.GridFinishedFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.S; @@ -123,8 +136,28 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl } }; + /** */ + private static final Pattern WAL_SEGMENT_FILE_COMPACTED_PATTERN = Pattern.compile("\\d{16}\\.wal\\.zip"); + + /** */ + private static final Pattern WAL_SEGMENT_TEMP_FILE_COMPACTED_PATTERN = Pattern.compile("\\d{16}\\.wal\\.zip\\.tmp"); + + /** */ + public 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(); + } + }; + + /** */ + public 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(); + } + }; + /** Latest serializer version to use. */ - public static final int LATEST_SERIALIZER_VERSION = 1; + public static final int LATEST_SERIALIZER_VERSION = 2; /** */ private final boolean alwaysWriteFullPages; @@ -169,8 +202,8 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl private final int serializerVersion = IgniteSystemProperties.getInteger(IGNITE_WAL_SERIALIZER_VERSION, LATEST_SERIALIZER_VERSION); - /** */ - private volatile long oldestArchiveSegmentIdx; + /** Latest segment cleared by {@link #truncate(WALPointer)}. */ + private volatile long lastTruncatedArchiveIdx; /** Factory to provide I/O interfaces for read/write operations with files */ private final FileIOFactory ioFactory; @@ -197,6 +230,12 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl /** */ private volatile FileArchiver archiver; + /** Compressor. */ + private volatile FileCompressor compressor; + + /** Decompressor. */ + private volatile FileDecompressor decompressor; + /** */ private final ThreadLocal lastWALPtr = new ThreadLocal<>(); @@ -276,7 +315,7 @@ public FileWriteAheadLogManager(@NotNull final GridKernalContext ctx) { "write ahead log archive directory" ); - serializer = forVersion(cctx, serializerVersion); + serializer = new RecordSerializerFactoryImpl(cctx).createSerializer(serializerVersion); GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager)cctx.database(); @@ -286,10 +325,14 @@ public FileWriteAheadLogManager(@NotNull final GridKernalContext ctx) { IgniteBiTuple tup = scanMinMaxArchiveIndices(); - oldestArchiveSegmentIdx = tup == null ? 0 : tup.get1(); + lastTruncatedArchiveIdx = tup == null ? 0 : tup.get1(); archiver = new FileArchiver(tup == null ? -1 : tup.get2()); + compressor = new FileCompressor(); + + decompressor = new FileDecompressor(); + if (mode != WALMode.NONE) { if (log.isInfoEnabled()) log.info("Started write-ahead log manager [mode=" + mode + ']'); @@ -338,6 +381,12 @@ private void checkWalConfiguration() throws IgniteCheckedException { if (archiver != null) archiver.shutdown(); + + if (compressor != null) + compressor.shutdown(); + + if (decompressor != null) + decompressor.shutdown(); } catch (Exception e) { U.error(log, "Failed to gracefully close WAL segment: " + currentHnd.fileIO, e); @@ -354,8 +403,13 @@ private void checkWalConfiguration() throws IgniteCheckedException { if (!cctx.kernalContext().clientNode()) { assert archiver != null; - archiver.start(); + + assert compressor != null; + compressor.start(); + + assert decompressor != null; + decompressor.start(); } } @@ -575,11 +629,10 @@ private void checkWalRolloverRequiredDuringInactivityPeriod() { (FileWALPointer)start, end, dsCfg, - serializer, + new RecordSerializerFactoryImpl(cctx), ioFactory, archiver, - log, - tlbSize + log ); } @@ -651,7 +704,12 @@ private boolean hasIndex(long absIdx) { // File pointer bound: older entries will be deleted from archive FileWALPointer fPtr = (FileWALPointer)ptr; - FileDescriptor[] descs = scan(walArchiveDir.listFiles(WAL_SEGMENT_FILE_FILTER)); + FileDescriptor[] descs; + + if (compressor == null) // If compression is enabled, FileCompressor is responsible for deleting raw segments. + descs = scan(walArchiveDir.listFiles(WAL_SEGMENT_FILE_FILTER)); + else + descs = scan(walArchiveDir.listFiles(WAL_SEGMENT_FILE_COMPACTED_FILTER)); int deleted = 0; @@ -671,17 +729,22 @@ private boolean hasIndex(long absIdx) { deleted++; // Bump up the oldest archive segment index. - if (oldestArchiveSegmentIdx < desc.idx) - oldestArchiveSegmentIdx = desc.idx; + if (lastTruncatedArchiveIdx < desc.idx) + lastTruncatedArchiveIdx = desc.idx; } } return deleted; } + /** {@inheritDoc} */ + @Override public void allowCompressionUntil(WALPointer ptr) { + compressor.allowCompressionUntil(((FileWALPointer)ptr).index()); + } + /** {@inheritDoc} */ @Override public int walArchiveSegments() { - long oldest = oldestArchiveSegmentIdx; + long oldest = lastTruncatedArchiveIdx; long lastArchived = archiver.lastArchivedAbsoluteIndex(); @@ -843,7 +906,7 @@ private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws Ig } } - RecordSerializer ser = forVersion(cctx, serVer); + RecordSerializer ser = new RecordSerializerFactoryImpl(cctx).createSerializer(serVer); if (log.isInfoEnabled()) log.info("Resuming logging to WAL segment [file=" + curFile.getAbsolutePath() + @@ -1021,37 +1084,6 @@ private File pollNextFile(long curIdx) throws IgniteCheckedException { return new File(walWorkDir, FileDescriptor.fileName(segmentIdx)); } - /** - * @param cctx Shared context. - * @param ver Serializer version. - * @return Entry serializer. - */ - public static RecordSerializer forVersion(GridCacheSharedContext cctx, int ver) throws IgniteCheckedException { - return forVersion(cctx, ver, false); - } - - /** - * @param ver Serializer version. - * @return Entry serializer. - */ - static RecordSerializer forVersion(GridCacheSharedContext cctx, int ver, boolean writePointer) throws IgniteCheckedException { - if (ver <= 0) - throw new IgniteCheckedException("Failed to create a serializer (corrupted WAL file)."); - - switch (ver) { - case 1: - return new RecordV1Serializer(new RecordDataV1Serializer(cctx), writePointer); - - case 2: - RecordDataV2Serializer dataV2Serializer = new RecordDataV2Serializer(new RecordDataV1Serializer(cctx)); - - return new RecordV2Serializer(dataV2Serializer, writePointer); - - default: - throw new IgniteCheckedException("Failed to create a serializer with the given version " + - "(forward compatibility is not supported): " + ver); - } - } /** * @return Sorted WAL files descriptors. @@ -1425,6 +1457,887 @@ private void allocateRemainingFiles() throws IgniteCheckedException { } } + /** + * Responsible for compressing WAL archive segments. + */ + private class FileCompressor extends Thread { + /** Current thread stopping advice. */ + private volatile boolean stopped; + + /** Last successfully compressed segment. */ + private volatile long lastCompressedSegmentIdx = -1L; + + /** All segments prior to this (inclusive) can be compressed. */ + private volatile long lastAllowedToCompressSegmentIdx; + + private void init() { + File[] toDel = walArchiveDir.listFiles(WAL_SEGMENT_TEMP_FILE_COMPACTED_FILTER); + + for (File f : toDel) { + if (stopped) + return; + + f.delete(); + } + + FileDescriptor[] alreadyCompressed = scan(walArchiveDir.listFiles(WAL_SEGMENT_FILE_COMPACTED_FILTER)); + + if (alreadyCompressed.length > 0) + lastCompressedSegmentIdx = alreadyCompressed[alreadyCompressed.length - 1].getIdx(); + } + + /** + * @param lastCpStartIdx Segment index to allow compression until (exclusively). + */ + public synchronized void allowCompressionUntil(long lastCpStartIdx) { + lastAllowedToCompressSegmentIdx = lastCpStartIdx - 1; + + notifyAll(); + } + + /** + * + */ + private synchronized long reserveNextSegmentOrWait() throws InterruptedException { + long segmentToCompress = lastCompressedSegmentIdx + 1; + + while (segmentToCompress > lastAllowedToCompressSegmentIdx) + wait(1000); // Next segment is still potentially needed for crash recovery. todo notify on checkpoint + + while (segmentToCompress > lastArchivedIndex() /* todo: use archiver's method */) + wait(1000); // Next segment has not been archived yet. todo notify on archivation of next segment + + boolean reserved = false; + + // Reserving segment to prevent its concurrent truncation. + while (!reserved) { + if (lastTruncatedArchiveIdx != 0) + segmentToCompress = Math.max(segmentToCompress, lastTruncatedArchiveIdx + 1); + + try { + reserved = reserve(new FileWALPointer(segmentToCompress, 0, 0)); + } + catch (IgniteCheckedException e) { + U.error(log, "Error while trying to reserve history", e); + } + } + + return segmentToCompress; + } + + /** + * + */ + public void deleteObsoleteRawSegments() { + FileDescriptor[] descs = scan(walArchiveDir.listFiles(WAL_SEGMENT_FILE_FILTER)); + + FileArchiver archiver0 = archiver; + + for (FileDescriptor desc : descs) { + // Do not delete reserved or locked segment and any segment after it. + if (archiver0 != null && archiver0.reserved(desc.idx)) + return; + + if (desc.idx < lastCompressedSegmentIdx) { + if (!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()); + } + } + } + + /** {@inheritDoc} */ + @Override public void run() { + init(); + + while (!Thread.currentThread().isInterrupted() || !stopped) { + try { + deleteObsoleteRawSegments(); + + long nextSegment = reserveNextSegmentOrWait(); + + File tmpZip = new File(walArchiveDir, FileDescriptor.fileName(nextSegment) + ".zip" + ".tmp"); + assert !tmpZip.exists(); + + File zip = new File(walArchiveDir, FileDescriptor.fileName(nextSegment) + ".zip"); + assert !zip.exists(); + + try (SingleSegmentLogicalRecordsIterator iter = new SingleSegmentLogicalRecordsIterator( + log, cctx, ioFactory, tlbSize, nextSegment, walArchiveDir); + ZipOutputStream zos = new ZipOutputStream(new BufferedOutputStream(new FileOutputStream(tmpZip)))) { + zos.putNextEntry(new ZipEntry(FileDescriptor.fileName(nextSegment))); + + zos.write(prepareSerializerVersionBuffer(nextSegment, serializerVersion).array()); // WAL segment header. + + ByteBuffer buf = ByteBuffer.allocate(tlbSize); + + for (IgniteBiTuple tuple : iter) { + FileWALPointer ptr = (FileWALPointer)tuple.get1(); + + if (ptr.length() > buf.capacity()) + buf = ByteBuffer.allocate(ptr.length()); + else + buf.clear(); + + WALRecord record = tuple.get2(); + + record.position(ptr); // Keep position from old file to make possible finding record by old WAL pointer. + + serializer.writeRecord(record, buf); + + zos.write(buf.array(), 0, buf.position()); + } + } + finally { + release(new FileWALPointer(nextSegment, 0, 0)); + } + + Files.move(tmpZip.toPath(), zip.toPath()); + + lastCompressedSegmentIdx = nextSegment; + } + catch (IgniteCheckedException | IOException e) { + U.error(log, "Unexpected error during WAL compression", e); // todo what do with ioexception? + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } + + /** + * @throws IgniteInterruptedCheckedException If failed to wait for thread shutdown. + */ + private void shutdown() throws IgniteInterruptedCheckedException { + synchronized (this) { + stopped = true; + + notifyAll(); + } + + U.join(this); + } + } + + /** + * Responsible for decompressing previously compressed segments of WAL archive if they are needed for replay. + */ + private class FileDecompressor extends Thread { + /** Current thread stopping advice. */ + private volatile boolean stopped; + + /** Decompression futures. */ + private Map> decompressionFutures = new HashMap<>(); + + /** Segments queue. */ + private PriorityBlockingQueue segmentsQueue = new PriorityBlockingQueue<>(); + + /** {@inheritDoc} */ + @Override public void run() { + while (!Thread.currentThread().isInterrupted() || !stopped) { + try { + long segmentToDecompress = segmentsQueue.take(); + + File zip = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress) + ".zip"); + assert zip.exists(); + + File unzipTmp = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress) + ".tmp"); + assert !unzipTmp.exists(); + + File unzip = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress)); + assert !unzip.exists(); + + try (ZipInputStream zis = new ZipInputStream(new BufferedInputStream(new FileInputStream(zip))); + FileIO io = ioFactory.create(unzipTmp)) { + + int bytesRead; + + while ((bytesRead = zis.read(tlb.get().array())) > 0) { + tlb.get().clear(); + + tlb.get().limit(bytesRead); + + io.write(tlb.get()); + } + } + + Files.move(unzipTmp.toPath(), unzip.toPath()); + + synchronized (this) { + decompressionFutures.remove(segmentToDecompress).onDone(); + } + } + catch (InterruptedException e){ + Thread.currentThread().interrupt(); + } + catch (IOException e) { + throw new IgniteException(e); // todo decide what do with FNF + } + } + } + + /** + * + */ + public synchronized IgniteInternalFuture ensureDecompressed(long idx) { + if (decompressionFutures.containsKey(idx)) + return decompressionFutures.get(idx); + + File f = new File(walArchiveDir, FileDescriptor.fileName(idx)); + + if (f.exists()) + return new GridFinishedFuture<>(); + + segmentsQueue.put(idx); + + GridFutureAdapter res = new GridFutureAdapter<>(); + + decompressionFutures.put(idx, res); + + return res; + } + + /** + * @throws IgniteInterruptedCheckedException If failed to wait for thread shutdown. + */ + private void shutdown() throws IgniteInterruptedCheckedException { + synchronized (this) { + stopped = true; + + notifyAll(); + } + + U.join(this); + } + } + +// /** +// * +// */ +// private class FileCompressor extends Thread { +// /** */ +// private Throwable exception; +// +// /** */ +// private volatile boolean stopped; +// +// /** */ +// private long lastCompressedSegment = -1L; +// +// /** */ +// private long lastSegmentWithCheckpointMarkerInArchive; +// +// /** */ +// private final FileArchiver fileArchiver; +// +// /** */ +// private final TreeSet checkpointMarkerSegments = new TreeSet<>(); +// +// /** */ +// private final CompactFactory compactFactory = new CompactFactoryImpl(); +// +// /** +// * +// */ +// private FileCompressor(FileArchiver archiver) { +// super("wal-file-compressor%" + cctx.igniteInstanceName()); +// +// assert archiver != null; +// +// fileArchiver = archiver; +// } +// +// private void init() { +// File[] toDelete = walArchiveDir.listFiles(WAL_SEGMENT_TEMP_FILE_COMPACTED_FILTER); +// +// for (File f : toDelete) { +// if (stopped) +// return; +// +// f.delete(); +// } +// +// // Find last compressed segment. +// File[] compactedArchives = walArchiveDir.listFiles(WAL_SEGMENT_FILE_COMPACTED_FILTER); +// +// if (!F.isEmpty(compactedArchives)) { +// long maxArchivedSegIdx = -1L; +// +// for (File f : compactedArchives) { +// CompactDescriptor desc = CompactDescriptor.parse(f.getAbsolutePath()); +// +// if (desc.endSeg > maxArchivedSegIdx) +// maxArchivedSegIdx = desc.endSeg; +// } +// +// assert maxArchivedSegIdx != -1L; +// +// synchronized (this) { +// lastCompressedSegment = maxArchivedSegIdx; +// } +// } +// } +// +// /** {@inheritDoc} */ +// @Override public void run() { +// init(); +// +// while (!Thread.currentThread().isInterrupted() || !stopped) { +// try { +// CompactDescriptor compactDesc = waitAvailableSegmentsForCompacting(); +// +// reserveSegments(compactDesc); +// +// CompactWriter compactWriter = compactFactory.createWriter(0, compactDesc); +// +// CompactionIterator iter = new CompactionIterator( +// compactDesc, +// walArchiveDir.getPath(), +// cctx.igniteInstanceName(), +// recSerFactory, +// serializer, +// ioFactory, +// compactWriter, +// igCfg.getPersistentStoreConfiguration().getTlbSize(), +// log +// ); +// +// try { +// while (iter.hasNext()) { +// iter.next(); +// +// if (stopped) +// break; +// } +// } +// finally { +// iter.closeCurrentWalSegment(); +// } +// +// synchronized (this) { +// lastCompressedSegment = compactDesc.endSeg; +// +// SortedSet toRemove = checkpointMarkerSegments.headSet(lastCompressedSegment); +// // Remove compressed segments with chp marker. +// checkpointMarkerSegments.removeAll(new ArrayList<>(toRemove)); +// } +// +// releaseSegments(compactDesc); +// +// for (long idx = compactDesc.startSeg; idx <= compactDesc.endSeg; idx++) { +// File segFile = new File(walArchiveDir, FileDescriptor.fileName(idx)); +// +// segFile.delete(); +// } +// +// } +// catch (Throwable e) { +// exception = e; +// +// e.printStackTrace(); +// } +// } +// +// if (log != null && log.isInfoEnabled()) +// log.info(getName() + " - stopped."); +// } +// +// /** +// * +// */ +// private CompactDescriptor waitAvailableSegmentsForCompacting() throws InterruptedException { +// long endSeg; +// +// long last; +// +// long startSeg; +// +// assert BATCH_SIZE >= 0; +// +// synchronized (this) { +// while ((last = lastCompressedSegment) + BATCH_SIZE >= (endSeg = readyForCompressingSegIdx())) +// wait(); +// +// startSeg = last == -1L ? 0 : last + 1; +// +// if (BATCH_SIZE != 0 && endSeg - last > BATCH_SIZE) +// endSeg = last + BATCH_SIZE; +// } +// +// return new CompactDescriptor(startSeg, endSeg, walArchiveDir.getPath()); +// } +// +// /** +// * +// */ +// private long readyForCompressingSegIdx() { +// return lastSegmentWithCheckpointMarkerInArchive - 1; +// } +// +// /** +// * +// */ +// private void reserveSegments(CompactDescriptor cDesc) { +// fileArchiver.reserve(cDesc.endSeg); +// } +// +// /** +// * +// */ +// private void releaseSegments(CompactDescriptor cDesc) { +// fileArchiver.release(cDesc.endSeg); +// } +// +// /** +// * +// */ +// public void shutdown() throws IgniteInterruptedCheckedException { +// synchronized (this) { +// stopped = true; +// +// notifyAll(); +// } +// +// U.join(this); +// } +// +// /** +// * +// */ +// private void advanceCheckpointMarker(long idx, long lastArchivedSegIdx) { +// synchronized (this) { +// checkpointMarkerSegments.add(idx); +// +// SortedSet head = checkpointMarkerSegments.headSet(lastArchivedSegIdx); +// +// if (!head.isEmpty()) { +// lastSegmentWithCheckpointMarkerInArchive = head.last(); +// +// notifyAll(); +// } +// } +// } +// } +// +// /** +// * +// */ +// private static class CompactDescriptor { +// +// /** */ +// private final long startSeg; +// +// /** */ +// private final long endSeg; +// +// /** */ +// private final String path; +// +// /** */ +// private final String name; +// +// /** +// * +// */ +// private CompactDescriptor(long startSeg, long endSeg, String path) { +// this.startSeg = startSeg; +// this.endSeg = endSeg; +// this.path = path; +// +// SB b1 = new SB(); +// +// String segmentStr = Long.toString(startSeg); +// +// for (int i = segmentStr.length(); i < 16; i++) +// b1.a('0'); +// +// SB b2 = new SB(); +// +// long cnt = endSeg - startSeg; +// +// String cntStr = Long.toString(cnt == 0 ? 1 : cnt); +// +// this.name = "seg-" + b1.a(segmentStr) + "-" + cntStr; +// } +// +// public static CompactDescriptor parse(String fullPath) { +// File f = new File(fullPath); +// +// String fileName = f.getName(); +// +// String[] parts = fileName.split("-"); +// +// Long startIdx = Long.valueOf(parts[1]); +// +// return new CompactDescriptor(startIdx, startIdx + Long.valueOf(parts[2]), f.getPath()); +// } +// } +// +// /** +// * +// */ +// private static class CompactionIterator extends AbstractWalRecordsIterator { +// /** */ +// private static final boolean SKIP_DELTA_RECORD_FILTER_ENABLE = +// IgniteSystemProperties.getBoolean(IGNITE_WAL_ARCHIVE_COMPACT_SKIP_DELTA_RECORD, true); +// +// /** */ +// private final IgniteBiTuple resFakeTuple = new IgniteBiTuple<>(); +// +// /** */ +// private final IgnitePredicate walDeltaRecordsFilter = new P1() { +// private final Set skip = RecordTypes.DELTA_TYPE_SET; +// +// @Override public boolean apply(WALRecord.RecordType type) { +// return skip.contains(type); +// } +// }; +// +// /** */ +// private final String directoryPath; +// +// /** */ +// private final long startSegIdx; +// +// /** */ +// private final long endSegIdx; +// +// /** */ +// private final CompactWriter compactWriter; +// +// /** +// * +// */ +// private CompactionIterator( +// CompactDescriptor desc, +// String directoryPath, +// String gridName, +// RecordSerializerFactory recSerFactory, +// RecordSerializer serializer, +// FileIOFactory ioFactory, +// CompactWriter compactWriter, +// int bufSize, +// IgniteLogger log +// ) throws IgniteCheckedException { +// super( +// gridName, +// recSerFactory, +// serializer, +// ioFactory, +// log, +// bufSize); +// +// assert !F.isEmpty(directoryPath); +// assert compactWriter != null; +// +// this.directoryPath = directoryPath; +// this.startSegIdx = desc.startSeg; +// this.endSegIdx = desc.endSeg; +// this.compactWriter = compactWriter; +// +// advance(); +// } +// +// @Override protected ReadFileHandle advanceSegment( +// @Nullable ReadFileHandle curWalSegment +// ) throws IgniteCheckedException { +// long idx = -1L; +// +// if (curWalSegment != null && curWalSegment.idx == endSegIdx) +// return null; +// +// if (curWalSegment == null) +// idx = startSegIdx; +// else if (curWalSegment.idx < endSegIdx) +// idx = curWalSegment.idx + 1; +// +// assert idx != -1L; +// +// try { +// String fileName = FileDescriptor.fileName(idx); +// +// compactWriter.advance(fileName); +// +// File file = new File(directoryPath, fileName); +// +// return initReadHandle(new FileDescriptor(file), null); +// } +// catch (FileNotFoundException e) { +// if (log.isInfoEnabled()) +// log.info("Missing " + idx + " WAL segment: " + e.getMessage()); +// +// return null; +// } +// } +// +// /** {@inheritDoc} */ +// @Override protected IgniteBiTuple advanceRecord( +// @Nullable final FileWriteAheadLogManager.ReadFileHandle hnd +// ) { +// if (hnd == null) +// return null; +// +// try { +// FileInput in = hnd.in; +// +// int startPos = (int)in.position(); +// +// final FileWALPointer ptr = new FileWALPointer(hnd.idx, startPos, 0); +// +// // Todo do not create real rec. +// final WALRecord rec = hnd.ser.readRecord(in, ptr); +// +// if (SKIP_DELTA_RECORD_FILTER_ENABLE && walDeltaRecordsFilter.apply(rec.type())) +// return resFakeTuple; +// +// assert compactWriter != null; +// +// // todo recalculate wal point. +// +// int recSize = rec.size(); +// +// in.seek(startPos); +// +// in.ensure(recSize); +// +// compactWriter.write(in.buffer(), recSize); +// +// in.seek(startPos + recSize); +// +// return resFakeTuple; +// } +// catch (IOException | IgniteCheckedException e) { +// if (!(e instanceof SegmentEofException)) { +// if (log != null && log.isInfoEnabled()) +// log.info(e.getMessage()); +// } +// +// return null; +// } +// } +// +// /** {@inheritDoc} */ +// @Nullable @Override protected ReadFileHandle closeCurrentWalSegment() throws IgniteCheckedException { +// ReadFileHandle h = super.closeCurrentWalSegment(); +// +// try { +// compactWriter.close(); +// } +// catch (Exception e) { +// throw new IgniteException(e); +// } +// +// return h; +// } +// } +// +// /** +// * +// */ +// private interface CompactFactory { +// /** +// * 0 - Standard java zip {@link ZipOutputStream}. +// */ +// public CompactWriter createWriter(int type, CompactDescriptor desc); +// +// /** +// * 0 - Standard java zip {@link ZipInputStream}. +// */ +// public CompactReader createReader(int type, CompactDescriptor desc); +// } +// +// /** +// * +// */ +// private interface CompactWriter extends AutoCloseable { +// /** +// * @param name Next entry name. +// */ +// public void advance(String name); +// +// /** +// * @param buf Wal record byte buffer. +// */ +// public void write(ByteBuffer buf, int size); +// } +// +// /** +// * +// */ +// private interface CompactReader extends AutoCloseable { +// +// /** +// * +// */ +// public String advance(); +// +// /** +// * +// */ +// public int read(ByteBuffer buf); +// } +// +// /** +// * +// */ +// private static class CompactFactoryImpl implements CompactFactory { +// private static final int DEFAULT_COMPACTION_LEVEL = 9; +// +// @Override public CompactWriter createWriter(int type, CompactDescriptor desc) { +// switch (type) { +// case 0: +// assert desc != null; +// +// return new StandardZipWriter(desc.path, desc.name, DEFAULT_COMPACTION_LEVEL); +// default: +// throw new UnsupportedOperationException("Unsupported compact writer type:" + type); +// } +// } +// +// @Override public CompactReader createReader(int type, CompactDescriptor desc) { +// switch (type) { +// case 0: +// assert desc != null; +// +// return new StandardZipReader(desc.path, desc.name); +// default: +// throw new UnsupportedOperationException("Unsupported compact reader type:" + type); +// } +// } +// } +// +// /** +// * +// */ +// private static class StandardZipWriter extends AbstractStandardZip implements CompactWriter { +// /** */ +// private static final String TMP_SUFFIX = SUFFIX + ".tmp"; +// +// /** */ +// private ZipOutputStream out; +// +// /** */ +// private File archiveTmp; +// +// /** +// * +// */ +// private StandardZipWriter(String dir, String name, int zipLevel) { +// assert dir != null; +// assert name != null; +// assert zipLevel >= 0 && zipLevel <= 9; +// +// try { +// archive = new File(dir, name + SUFFIX); +// archiveTmp = new File(dir, name + TMP_SUFFIX); +// +// assert !archive.exists(); +// assert !archiveTmp.exists(); +// +// archiveTmp.createNewFile(); +// +// out = new ZipOutputStream(new FileOutputStream(archiveTmp)); +// +// out.setLevel(zipLevel); +// } +// catch (IOException e) { +// throw new IgniteException(e); +// } +// } +// +// /** {@inheritDoc} */ +// @Override public void advance(String name) { +// try { +// out.putNextEntry(new ZipEntry(name)); +// } +// catch (IOException e) { +// throw new IgniteException(e); +// } +// } +// +// /** {@inheritDoc} */ +// @Override public void write(ByteBuffer buf, int size) { +// try { +// byte[] toWrite = buf.array(); +// +// out.write(toWrite, 0, size); +// } +// catch (IOException e) { +// throw new IgniteException(e); +// } +// } +// +// /** {@inheritDoc} */ +// @Override public void close() { +// try { +// out.close(); +// +// Files.move(archiveTmp.toPath(), archive.toPath()); +// } +// catch (IOException e) { +// throw new IgniteException(e); +// } +// } +// } +// +// private static class StandardZipReader extends AbstractStandardZip implements CompactReader { +// +// /** */ +// private ZipInputStream in; +// +// private StandardZipReader(String dir, String name) { +// try { +// in = new ZipInputStream(new FileInputStream(new File(dir, name + SUFFIX))); +// } +// catch (FileNotFoundException e) { +// e.printStackTrace(); +// } +// } +// +// @Override public String advance() { +// try { +// ZipEntry entry = in.getNextEntry(); +// +// return entry.getName(); +// } +// catch (IOException e) { +// e.printStackTrace(); +// } +// return null; +// } +// +// @Override public int read(ByteBuffer buf) { +// try { +// byte[] array = buf.array(); +// +// return in.read(array); +// } +// catch (IOException e) { +// e.printStackTrace(); +// } +// +// return 0; +// } +// +// @Override public void close() { +// try { +// in.close(); +// } +// catch (IOException e) { +// e.printStackTrace(); +// } +// } +// } +// +// private abstract static class AbstractStandardZip { +// /** */ +// protected static final String SUFFIX = "-archive.zip"; +// +// /** */ +// protected File archive; +// +// } + /** * Validate files depending on {@link DataStorageConfiguration#getWalSegments()} and create if need. * Check end when exit condition return false or all files are passed. @@ -1508,47 +2421,57 @@ public static int readSerializerVersion(FileIO io) * @throws IOException If failed to write serializer version. */ public static long writeSerializerVersion(FileIO io, long idx, int version) throws IOException { - ByteBuffer buffer = ByteBuffer.allocate(RecordV1Serializer.HEADER_RECORD_SIZE); - buffer.order(ByteOrder.nativeOrder()); + ByteBuffer buffer = prepareSerializerVersionBuffer(idx, version); + + do { + io.write(buffer); + } + while (buffer.hasRemaining()); + + // Flush + io.force(); + + return io.position(); + } + + /** + * @param idx Index. + * @param ver Version. + */ + @NotNull private static ByteBuffer prepareSerializerVersionBuffer(long idx, int ver) { + ByteBuffer buf = ByteBuffer.allocate(RecordV1Serializer.HEADER_RECORD_SIZE); + buf.order(ByteOrder.nativeOrder()); // Write record type. - buffer.put((byte) (WALRecord.RecordType.HEADER_RECORD.ordinal() + 1)); + buf.put((byte) (WALRecord.RecordType.HEADER_RECORD.ordinal() + 1)); // Write position. - RecordV1Serializer.putPosition(buffer, new FileWALPointer(idx, 0, 0)); + RecordV1Serializer.putPosition(buf, new FileWALPointer(idx, 0, 0)); // Place magic number. - buffer.putLong(HeaderRecord.MAGIC); + buf.putLong(HeaderRecord.MAGIC); // Place serializer version. - buffer.putInt(version); + buf.putInt(ver); // Place CRC if needed. if (!RecordV1Serializer.SKIP_CRC) { - int curPos = buffer.position(); + int curPos = buf.position(); - buffer.position(0); + buf.position(0); // This call will move buffer position to the end of the record again. - int crcVal = PureJavaCrc32.calcCrc32(buffer, curPos); + int crcVal = PureJavaCrc32.calcCrc32(buf, curPos); - buffer.putInt(crcVal); + buf.putInt(crcVal); } else - buffer.putInt(0); + buf.putInt(0); // Write header record through io. - buffer.position(0); + buf.position(0); - do { - io.write(buffer); - } - while (buffer.hasRemaining()); - - // Flush - io.force(); - - return io.position(); + return buf; } /** @@ -1579,11 +2502,9 @@ public FileDescriptor(@NotNull File file, @Nullable Long idx) { String fileName = file.getName(); - assert fileName.endsWith(WAL_SEGMENT_FILE_EXT); - - int end = fileName.length() - WAL_SEGMENT_FILE_EXT.length(); + assert fileName.contains(WAL_SEGMENT_FILE_EXT); - this.idx = idx == null ? Long.parseLong(fileName.substring(0, end)) : idx; + this.idx = idx == null ? Long.parseLong(fileName.substring(0, 16)) : idx; } /** @@ -2504,7 +3425,7 @@ private static class RecordsIterator extends AbstractWalRecordsIterator { * @param start Optional start pointer. * @param end Optional end pointer. * @param psCfg Database configuration. - * @param serializer Serializer of current version to read headers. + * @param serializerFactory Serializer factory. * @param archiver Archiver. * @param log Logger * @throws IgniteCheckedException If failed to initialize WAL segment. @@ -2516,15 +3437,14 @@ private RecordsIterator( @Nullable FileWALPointer start, @Nullable FileWALPointer end, DataStorageConfiguration psCfg, - @NotNull RecordSerializer serializer, + @NotNull RecordSerializerFactory serializerFactory, FileIOFactory ioFactory, FileArchiver archiver, - IgniteLogger log, - int tlbSize + IgniteLogger log ) throws IgniteCheckedException { super(log, cctx, - serializer, + serializerFactory, ioFactory, psCfg.getWalRecordIteratorBufferSize()); this.walWorkDir = walWorkDir; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java new file mode 100644 index 0000000000000..a1084deb27b86 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java @@ -0,0 +1,114 @@ +/* +* 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; + +import java.io.File; +import java.io.FileNotFoundException; +import java.util.Set; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; +import org.apache.ignite.internal.processors.cache.persistence.wal.record.RecordTypes; +import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactory; +import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactoryImpl; +import org.apache.ignite.internal.util.typedef.P1; +import org.apache.ignite.lang.IgnitePredicate; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +/** + * Iterates over logical records of one WAL segment from archive. Used for WAL archive compression. + */ +public class SingleSegmentLogicalRecordsIterator extends AbstractWalRecordsIterator { + /** Segment initialized flag. */ + private boolean segmentInitialized; + + /** Archived segment index. */ + private long archivedSegIdx; + + /** Archive directory. */ + private File archiveDir; + /** + * @param log Logger. + * @param sharedCtx Shared context. + * @param ioFactory Io factory. + * @param bufSize Buffer size. + * @param archivedSegIdx Archived seg index. + * @param archiveDir Directory with segment. + */ + public SingleSegmentLogicalRecordsIterator( + @NotNull IgniteLogger log, + @NotNull GridCacheSharedContext sharedCtx, + @NotNull FileIOFactory ioFactory, + int bufSize, + long archivedSegIdx, + File archiveDir + ) throws IgniteCheckedException { + super(log, sharedCtx, initFilteredSerializerFactory(sharedCtx), ioFactory, bufSize); + + this.archivedSegIdx = archivedSegIdx; + this.archiveDir = archiveDir; + + advance(); + } + + /** + * @param sharedCtx Shared context. + */ + private static RecordSerializerFactory initFilteredSerializerFactory(GridCacheSharedContext sharedCtx) + throws IgniteCheckedException { + IgnitePredicate logicalRecordsFilter = new P1() { + private final Set skip = RecordTypes.DELTA_TYPE_SET; + + @Override public boolean apply(WALRecord.RecordType type) { + return !skip.contains(type); + } + }; + + return new RecordSerializerFactoryImpl(sharedCtx).readTypeFilter(logicalRecordsFilter); + } + + /** {@inheritDoc} */ + @Override protected FileWriteAheadLogManager.ReadFileHandle advanceSegment( + @Nullable FileWriteAheadLogManager.ReadFileHandle curWalSegment) throws IgniteCheckedException { + if (segmentInitialized) { + closeCurrentWalSegment(); + // No advance as we iterate over single segment. + return null; + } + else { + segmentInitialized = true; + + FileWriteAheadLogManager.FileDescriptor fd = new FileWriteAheadLogManager.FileDescriptor( + new File(archiveDir, FileWriteAheadLogManager.FileDescriptor.fileName(archivedSegIdx))); + + try { + return initReadHandle(fd, null); + } + catch (FileNotFoundException e) { + throw new IgniteCheckedException("Missing WAL segment in the archive", e); + } + } + } + + /** {@inheritDoc} todo for debug only, remove */ + @Override protected void advance() throws IgniteCheckedException { + super.advance(); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java index a4d9e95c5a8a6..b36c2db44a12d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java @@ -45,6 +45,7 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.FileInput; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager; +import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactoryImpl; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer; import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor; import org.apache.ignite.internal.util.typedef.F; @@ -108,7 +109,7 @@ class StandaloneWalRecordsIterator extends AbstractWalRecordsIterator { ) throws IgniteCheckedException { super(log, sharedCtx, - FileWriteAheadLogManager.forVersion(sharedCtx, FileWriteAheadLogManager.LATEST_SERIALIZER_VERSION), + new RecordSerializerFactoryImpl(sharedCtx), ioFactory, BUF_SIZE); this.keepBinary = keepBinary; @@ -136,7 +137,7 @@ class StandaloneWalRecordsIterator extends AbstractWalRecordsIterator { @NotNull File... walFiles) throws IgniteCheckedException { super(log, sharedCtx, - FileWriteAheadLogManager.forVersion(sharedCtx, FileWriteAheadLogManager.LATEST_SERIALIZER_VERSION), + new RecordSerializerFactoryImpl(sharedCtx), ioFactory, BUF_SIZE); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/record/RecordTypes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/record/RecordTypes.java new file mode 100644 index 0000000000000..c1d39f46832a0 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/record/RecordTypes.java @@ -0,0 +1,69 @@ +/* + * 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.record; + +import java.util.HashSet; +import java.util.Set; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; + +/** + * + */ +public final class RecordTypes { + /** */ + public static final Set DELTA_TYPE_SET = new HashSet<>(); + + static { + DELTA_TYPE_SET.add(WALRecord.RecordType.PAGE_RECORD); + DELTA_TYPE_SET.add(WALRecord.RecordType.INIT_NEW_PAGE_RECORD); + DELTA_TYPE_SET.add(WALRecord.RecordType.DATA_PAGE_INSERT_RECORD); + DELTA_TYPE_SET.add(WALRecord.RecordType.DATA_PAGE_INSERT_FRAGMENT_RECORD); + DELTA_TYPE_SET.add(WALRecord.RecordType.DATA_PAGE_REMOVE_RECORD); + DELTA_TYPE_SET.add(WALRecord.RecordType.DATA_PAGE_SET_FREE_LIST_PAGE); + DELTA_TYPE_SET.add(WALRecord.RecordType.BTREE_META_PAGE_INIT_ROOT); + DELTA_TYPE_SET.add(WALRecord.RecordType.BTREE_META_PAGE_ADD_ROOT); + DELTA_TYPE_SET.add(WALRecord.RecordType.BTREE_META_PAGE_CUT_ROOT); + DELTA_TYPE_SET.add(WALRecord.RecordType.BTREE_INIT_NEW_ROOT); + DELTA_TYPE_SET.add(WALRecord.RecordType.BTREE_PAGE_RECYCLE); + DELTA_TYPE_SET.add(WALRecord.RecordType.BTREE_PAGE_INSERT); + DELTA_TYPE_SET.add(WALRecord.RecordType.BTREE_FIX_LEFTMOST_CHILD); + DELTA_TYPE_SET.add(WALRecord.RecordType.BTREE_FIX_COUNT); + DELTA_TYPE_SET.add(WALRecord.RecordType.BTREE_PAGE_REPLACE); + DELTA_TYPE_SET.add(WALRecord.RecordType.BTREE_PAGE_REMOVE); + DELTA_TYPE_SET.add(WALRecord.RecordType.BTREE_PAGE_INNER_REPLACE); + DELTA_TYPE_SET.add(WALRecord.RecordType.BTREE_FIX_REMOVE_ID); + DELTA_TYPE_SET.add(WALRecord.RecordType.BTREE_FORWARD_PAGE_SPLIT); + DELTA_TYPE_SET.add(WALRecord.RecordType.BTREE_EXISTING_PAGE_SPLIT); + DELTA_TYPE_SET.add(WALRecord.RecordType.BTREE_PAGE_MERGE); + DELTA_TYPE_SET.add(WALRecord.RecordType.PAGES_LIST_SET_NEXT); + DELTA_TYPE_SET.add(WALRecord.RecordType.PAGES_LIST_SET_PREVIOUS); + DELTA_TYPE_SET.add(WALRecord.RecordType.PAGES_LIST_INIT_NEW_PAGE); + DELTA_TYPE_SET.add(WALRecord.RecordType.PAGES_LIST_ADD_PAGE); + DELTA_TYPE_SET.add(WALRecord.RecordType.PAGES_LIST_REMOVE_PAGE); + DELTA_TYPE_SET.add(WALRecord.RecordType.META_PAGE_INIT); + DELTA_TYPE_SET.add(WALRecord.RecordType.PARTITION_META_PAGE_UPDATE_COUNTERS); + DELTA_TYPE_SET.add(WALRecord.RecordType.TRACKING_PAGE_DELTA); + DELTA_TYPE_SET.add(WALRecord.RecordType.META_PAGE_UPDATE_LAST_SUCCESSFUL_SNAPSHOT_ID); + DELTA_TYPE_SET.add(WALRecord.RecordType.META_PAGE_UPDATE_LAST_SUCCESSFUL_FULL_SNAPSHOT_ID); + DELTA_TYPE_SET.add(WALRecord.RecordType.META_PAGE_UPDATE_NEXT_SNAPSHOT_ID); + DELTA_TYPE_SET.add(WALRecord.RecordType.META_PAGE_UPDATE_LAST_ALLOCATED_INDEX); + DELTA_TYPE_SET.add(WALRecord.RecordType.PAGE_LIST_META_RESET_COUNT_RECORD); + DELTA_TYPE_SET.add(WALRecord.RecordType.DATA_PAGE_UPDATE_RECORD); + DELTA_TYPE_SET.add(WALRecord.RecordType.BTREE_META_PAGE_INIT_ROOT2); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/RecordDataSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataSerializer.java similarity index 95% rename from modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/RecordDataSerializer.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataSerializer.java index 5a140952bfd40..88c5f42bcddb7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/RecordDataSerializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataSerializer.java @@ -15,12 +15,13 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.cache.persistence.wal; +package org.apache.ignite.internal.processors.cache.persistence.wal.serializer; import java.io.IOException; import java.nio.ByteBuffer; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput; /** * Interface to provide size, read and write operations with WAL records diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java index e583df3eae0d7..6e708c0c9ede6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java @@ -87,7 +87,6 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.io.CacheVersionIO; import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer; -import org.apache.ignite.internal.processors.cache.persistence.wal.RecordDataSerializer; import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java index c02f36e925943..df63b9e508f6f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java @@ -26,7 +26,6 @@ import org.apache.ignite.internal.pagemem.wal.record.DataRecord; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput; -import org.apache.ignite.internal.processors.cache.persistence.wal.RecordDataSerializer; import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord; /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/RecordSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializer.java similarity index 95% rename from modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/RecordSerializer.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializer.java index 12e16a8aae792..c5760ab3bb30b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/RecordSerializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializer.java @@ -15,13 +15,14 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.cache.persistence.wal; +package org.apache.ignite.internal.processors.cache.persistence.wal.serializer; import java.io.IOException; import java.nio.ByteBuffer; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.pagemem.wal.WALPointer; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.processors.cache.persistence.wal.FileInput; /** * Record serializer. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactory.java new file mode 100644 index 0000000000000..8170424dbbde6 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactory.java @@ -0,0 +1,33 @@ +/* + * 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.serializer; + +import org.apache.ignite.IgniteCheckedException; + +/** + * + */ +public interface RecordSerializerFactory { + /** + * Factory method for creation {@link RecordSerializer}. + * + * @param ver Serializer version. + * @return record serializer. + */ + public RecordSerializer createSerializer(int ver) throws IgniteCheckedException; +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactoryImpl.java new file mode 100644 index 0000000000000..fc2453aec5064 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactoryImpl.java @@ -0,0 +1,98 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ +package org.apache.ignite.internal.processors.cache.persistence.wal.serializer; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.lang.IgnitePredicate; + +/** + * + */ +public class RecordSerializerFactoryImpl implements RecordSerializerFactory { + /** Context. */ + private GridCacheSharedContext cctx; + + /** Write pointer. */ + private boolean writePointer; + + /** Read type filter. */ + private IgnitePredicate readTypeFilter; + + /** + * @param cctx Cctx. + */ + public RecordSerializerFactoryImpl(GridCacheSharedContext cctx) { + this.cctx = cctx; + } + + /** {@inheritDoc} */ + @Override public RecordSerializer createSerializer(int ver) throws IgniteCheckedException { + if (ver <= 0) + throw new IgniteCheckedException("Failed to create a serializer (corrupted WAL file)."); + + switch (ver) { + case 1: + if (readTypeFilter != null) + throw new IgniteCheckedException("Read type filter is allowed only for version 2 or higher."); + + return new RecordV1Serializer(new RecordDataV1Serializer(cctx), writePointer); + + case 2: + RecordDataV2Serializer dataV2Serializer = new RecordDataV2Serializer(new RecordDataV1Serializer(cctx)); + + return new RecordV2Serializer(dataV2Serializer, writePointer, readTypeFilter); + + default: + throw new IgniteCheckedException("Failed to create a serializer with the given version " + + "(forward compatibility is not supported): " + ver); + } + } + + /** + * @return Write pointer. + */ + public boolean writePointer() { + return writePointer; + } + + /** + * @param writePointer New write pointer. + */ + public RecordSerializerFactoryImpl writePointer(boolean writePointer) { + this.writePointer = writePointer; + + return this; + } + + /** + * @return Read type filter. + */ + public IgnitePredicate readTypeFilter() { + return readTypeFilter; + } + + /** + * @param readTypeFilter New read type filter. + */ + public RecordSerializerFactoryImpl readTypeFilter(IgnitePredicate readTypeFilter) { + this.readTypeFilter = readTypeFilter; + + return this; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java index c4e1bf294de15..a8455695ef2f2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java @@ -30,7 +30,6 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput; import org.apache.ignite.internal.processors.cache.persistence.wal.FileInput; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer; -import org.apache.ignite.internal.processors.cache.persistence.wal.RecordSerializer; import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentEofException; import org.apache.ignite.internal.processors.cache.persistence.wal.WalSegmentTailReachedException; import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java index 5eb45ac3703d2..26a3d3f3cd614 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java @@ -22,15 +22,16 @@ import java.nio.ByteBuffer; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.pagemem.wal.WALPointer; +import org.apache.ignite.internal.pagemem.wal.record.FilteredRecord; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput; import org.apache.ignite.internal.processors.cache.persistence.wal.FileInput; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer; -import org.apache.ignite.internal.processors.cache.persistence.wal.RecordSerializer; import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentEofException; import org.apache.ignite.internal.processors.cache.persistence.wal.WalSegmentTailReachedException; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.io.RecordIO; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.lang.IgnitePredicate; import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.*; import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.CRC_SIZE; @@ -46,6 +47,8 @@ *

  • Data
  • *
  • CRC or zero padding
  • * + * Also, optimization for skipping deserialization of records of unwanted types. + * If {@link #readTypeFilter} is specified, {@link FilteredRecord} is read instead of unnecessary record. */ public class RecordV2Serializer implements RecordSerializer { /** Length of WAL Pointer: Index (8) + File offset (4) + Record length (4) */ @@ -57,6 +60,12 @@ public class RecordV2Serializer implements RecordSerializer { /** Write pointer. */ private final boolean writePointer; + /** + * Record type filter. + * {@link FilteredRecord} is deserialized instead of original record if type doesn't match filter. + */ + private final IgnitePredicate readTypeFilter; + /** Record read/write functional interface. */ private final RecordIO recordIO = new RecordIO() { @@ -77,7 +86,18 @@ public class RecordV2Serializer implements RecordSerializer { FileWALPointer ptr = readPositionAndCheckPoint(in, expPtr); - return dataSerializer.readRecord(recType, in); + if (readTypeFilter != null && !readTypeFilter.apply(recType)) { + int toSkip = ptr.length() - REC_TYPE_SIZE - FILE_WAL_POINTER_SIZE - CRC_SIZE; + + assert toSkip >= 0 : "Too small saved record length: " + ptr; + + in.readFully(new byte[toSkip]); + + return new FilteredRecord(); + } + else + return dataSerializer.readRecord(recType, in); + } /** {@inheritDoc} */ @@ -100,10 +120,14 @@ public class RecordV2Serializer implements RecordSerializer { * Create an instance of Record V2 serializer. * * @param dataSerializer V2 data serializer. + * @param readTypeFilter Record type filter. {@link FilteredRecord} is deserialized instead of original record + * if type doesn't match filter. */ - public RecordV2Serializer(RecordDataV2Serializer dataSerializer, boolean writePointer) { + public RecordV2Serializer(RecordDataV2Serializer dataSerializer, boolean writePointer, + IgnitePredicate readTypeFilter) { this.dataSerializer = dataSerializer; this.writePointer = writePointer; + this.readTypeFilter = readTypeFilter; } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java index 7500fdc4406e6..0b5b0e6a100d8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java @@ -34,7 +34,7 @@ import org.apache.ignite.internal.pagemem.wal.record.TimeStampRecord; import org.apache.ignite.internal.pagemem.wal.record.TxRecord; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; -import org.apache.ignite.internal.processors.cache.persistence.wal.RecordSerializer; +import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializer; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV2Serializer; import org.apache.ignite.internal.util.typedef.internal.GPC; 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 72450b861f08c..b1d9098244960 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 @@ -76,6 +76,11 @@ public class NoOpWALManager implements IgniteWriteAheadLogManager { return 0; } + /** {@inheritDoc} */ + @Override public void allowCompressionUntil(WALPointer ptr) { + + } + /** {@inheritDoc} */ @Override public boolean reserved(WALPointer ptr) { return false; From e1ba31aa81ac08bc5da106d6c71ab05da6796c90 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Wed, 15 Nov 2017 13:24:00 +0300 Subject: [PATCH 02/16] IGNITE-5938 Get rid of deserialization during compaction. --- .../pagemem/wal/record/MarshalledRecord.java | 63 ++++++++++++++++ .../GridCacheDatabaseSharedManager.java | 2 + .../wal/FileWriteAheadLogManager.java | 74 ++++++++++--------- .../SingleSegmentLogicalRecordsIterator.java | 2 +- .../RecordSerializerFactoryImpl.java | 46 +++++++++++- .../wal/serializer/RecordV2Serializer.java | 42 +++++++++-- 6 files changed, 185 insertions(+), 44 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MarshalledRecord.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MarshalledRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MarshalledRecord.java new file mode 100644 index 0000000000000..a338b41050837 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MarshalledRecord.java @@ -0,0 +1,63 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ +package org.apache.ignite.internal.pagemem.wal.record; + +import org.apache.ignite.internal.pagemem.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer; + +/** + * Special type of WAL record. Shouldn't be stored in file. + * Contains complete binary representation of record in {@link #data} and record position in {@link #pos}. + */ +public class MarshalledRecord extends WALRecord { + /** Type of marshalled record. */ + private WALRecord.RecordType type; + + /** Marshalled record bytes. */ + private byte[] data; + + /** + * @param type Type of marshalled record. + */ + public MarshalledRecord(WALRecord.RecordType type, WALPointer pos, byte[] data) { + this.type = type; + this.data = data; + + assert data.length == ((FileWALPointer)pos).length(); + + position(pos); + } + + /** {@inheritDoc} */ + @Override public RecordType type() { + return type; + } + + /** + * @return Marshalled record bytes. + */ + public byte[] data() { + return data; + } + + /** + * @param data New marshalled record bytes. + */ + public void data(byte[] data) { + this.data = data; + } +} 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 ca9bb9d5d727c..c0e59bc446d3c 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 @@ -1447,6 +1447,8 @@ private WALPointer restoreMemory(CheckpointStatus status) throws IgniteCheckedEx cctx.pageStore().beginRecover(); } + else + cctx.wal().allowCompressionUntil(status.startPtr); long start = U.currentTimeMillis(); int applied = 0; 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 9459cfd1a8bad..ce3cfc25e8867 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 @@ -66,6 +66,7 @@ import org.apache.ignite.internal.pagemem.wal.WALIterator; import org.apache.ignite.internal.pagemem.wal.WALPointer; import org.apache.ignite.internal.pagemem.wal.record.CheckpointRecord; +import org.apache.ignite.internal.pagemem.wal.record.MarshalledRecord; import org.apache.ignite.internal.pagemem.wal.record.SwitchSegmentRecord; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; @@ -203,7 +204,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl IgniteSystemProperties.getInteger(IGNITE_WAL_SERIALIZER_VERSION, LATEST_SERIALIZER_VERSION); /** Latest segment cleared by {@link #truncate(WALPointer)}. */ - private volatile long lastTruncatedArchiveIdx; + private volatile long lastTruncatedArchiveIdx = -1L; /** Factory to provide I/O interfaces for read/write operations with files */ private final FileIOFactory ioFactory; @@ -325,7 +326,7 @@ public FileWriteAheadLogManager(@NotNull final GridKernalContext ctx) { IgniteBiTuple tup = scanMinMaxArchiveIndices(); - lastTruncatedArchiveIdx = tup == null ? 0 : tup.get1(); + lastTruncatedArchiveIdx = tup == null ? -1 : tup.get1() - 1; archiver = new FileArchiver(tup == null ? -1 : tup.get2()); @@ -744,14 +745,14 @@ private boolean hasIndex(long absIdx) { /** {@inheritDoc} */ @Override public int walArchiveSegments() { - long oldest = lastTruncatedArchiveIdx; + long lastTruncated = lastTruncatedArchiveIdx; long lastArchived = archiver.lastArchivedAbsoluteIndex(); if (lastArchived == -1) return 0; - int res = (int)(lastArchived - oldest); + int res = (int)(lastArchived - lastTruncated); return res >= 0 ? res : 0; } @@ -1141,7 +1142,7 @@ private class FileArchiver extends Thread { private long curAbsWalIdx = -1; /** Last archived file index (absolute, 0-based). Guarded by this. */ - private long lastAbsArchivedIdx = -1; + private volatile long lastAbsArchivedIdx = -1; /** current thread stopping advice */ private volatile boolean stopped; @@ -1167,7 +1168,7 @@ private FileArchiver(long lastAbsArchivedIdx) { /** * @return Last archived segment absolute index. */ - private synchronized long lastArchivedAbsoluteIndex() { + private long lastArchivedAbsoluteIndex() { return lastAbsArchivedIdx; } @@ -1253,7 +1254,7 @@ private synchronized void release(long absIdx) { wait(); if (curAbsWalIdx != 0 && lastAbsArchivedIdx == -1) - lastAbsArchivedIdx = curAbsWalIdx - 1; + changeLastArchivedIndexAndWakeupCompressor(curAbsWalIdx - 1); } while (!Thread.currentThread().isInterrupted() && !stopped) { @@ -1284,7 +1285,7 @@ private synchronized void release(long absIdx) { formatFile(res.getOrigWorkFile()); // Then increase counter to allow rollover on clean working file - lastAbsArchivedIdx = toArchive; + changeLastArchivedIndexAndWakeupCompressor(toArchive); notifyAll(); } @@ -1306,6 +1307,16 @@ private synchronized void release(long absIdx) { } } + /** + * @param idx Index. + */ + private void changeLastArchivedIndexAndWakeupCompressor(long idx) { + lastAbsArchivedIdx = idx; + + if (compressor != null) + compressor.onNextSegmentArchived(); + } + /** * Gets the absolute index of the next WAL segment available to write. * Blocks till there are available file to write @@ -1333,7 +1344,7 @@ private long nextAbsoluteSegmentIndex(long curIdx) throws IgniteCheckedException return curAbsWalIdx; } } - catch (InterruptedException e) { + catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new IgniteInterruptedCheckedException(e); @@ -1468,7 +1479,7 @@ private class FileCompressor extends Thread { private volatile long lastCompressedSegmentIdx = -1L; /** All segments prior to this (inclusive) can be compressed. */ - private volatile long lastAllowedToCompressSegmentIdx; + private volatile long lastAllowedToCompressSegmentIdx = -1L; private void init() { File[] toDel = walArchiveDir.listFiles(WAL_SEGMENT_TEMP_FILE_COMPACTED_FILTER); @@ -1489,10 +1500,17 @@ private void init() { /** * @param lastCpStartIdx Segment index to allow compression until (exclusively). */ - public synchronized void allowCompressionUntil(long lastCpStartIdx) { + synchronized void allowCompressionUntil(long lastCpStartIdx) { lastAllowedToCompressSegmentIdx = lastCpStartIdx - 1; - notifyAll(); + notify(); + } + + /** + * Callback for waking up compressor when new segment is archived. + */ + synchronized void onNextSegmentArchived() { + notify(); } /** @@ -1501,18 +1519,14 @@ public synchronized void allowCompressionUntil(long lastCpStartIdx) { private synchronized long reserveNextSegmentOrWait() throws InterruptedException { long segmentToCompress = lastCompressedSegmentIdx + 1; - while (segmentToCompress > lastAllowedToCompressSegmentIdx) - wait(1000); // Next segment is still potentially needed for crash recovery. todo notify on checkpoint - - while (segmentToCompress > lastArchivedIndex() /* todo: use archiver's method */) - wait(1000); // Next segment has not been archived yet. todo notify on archivation of next segment + while (segmentToCompress > Math.max(lastAllowedToCompressSegmentIdx, archiver.lastArchivedAbsoluteIndex())) + wait(); boolean reserved = false; // Reserving segment to prevent its concurrent truncation. while (!reserved) { - if (lastTruncatedArchiveIdx != 0) - segmentToCompress = Math.max(segmentToCompress, lastTruncatedArchiveIdx + 1); + segmentToCompress = Math.max(segmentToCompress, lastTruncatedArchiveIdx + 1); try { reserved = reserve(new FileWALPointer(segmentToCompress, 0, 0)); @@ -1528,7 +1542,7 @@ private synchronized long reserveNextSegmentOrWait() throws InterruptedException /** * */ - public void deleteObsoleteRawSegments() { + private void deleteObsoleteRawSegments() { FileDescriptor[] descs = scan(walArchiveDir.listFiles(WAL_SEGMENT_FILE_FILTER)); FileArchiver archiver0 = archiver; @@ -1565,27 +1579,14 @@ public void deleteObsoleteRawSegments() { try (SingleSegmentLogicalRecordsIterator iter = new SingleSegmentLogicalRecordsIterator( log, cctx, ioFactory, tlbSize, nextSegment, walArchiveDir); ZipOutputStream zos = new ZipOutputStream(new BufferedOutputStream(new FileOutputStream(tmpZip)))) { - zos.putNextEntry(new ZipEntry(FileDescriptor.fileName(nextSegment))); + zos.putNextEntry(new ZipEntry("")); zos.write(prepareSerializerVersionBuffer(nextSegment, serializerVersion).array()); // WAL segment header. - ByteBuffer buf = ByteBuffer.allocate(tlbSize); - for (IgniteBiTuple tuple : iter) { - FileWALPointer ptr = (FileWALPointer)tuple.get1(); - - if (ptr.length() > buf.capacity()) - buf = ByteBuffer.allocate(ptr.length()); - else - buf.clear(); - - WALRecord record = tuple.get2(); - - record.position(ptr); // Keep position from old file to make possible finding record by old WAL pointer. - - serializer.writeRecord(record, buf); + WALRecord rec = tuple.get2(); - zos.write(buf.array(), 0, buf.position()); + zos.write(((MarshalledRecord)rec).data()); } } finally { @@ -1649,6 +1650,7 @@ private class FileDecompressor extends Thread { try (ZipInputStream zis = new ZipInputStream(new BufferedInputStream(new FileInputStream(zip))); FileIO io = ioFactory.create(unzipTmp)) { + zis.getNextEntry(); int bytesRead; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java index a1084deb27b86..b4f82dd97c832 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java @@ -81,7 +81,7 @@ private static RecordSerializerFactory initFilteredSerializerFactory(GridCacheSh } }; - return new RecordSerializerFactoryImpl(sharedCtx).readTypeFilter(logicalRecordsFilter); + return new RecordSerializerFactoryImpl(sharedCtx).readTypeFilter(logicalRecordsFilter).marshalledMode(true); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactoryImpl.java index fc2453aec5064..d06b26cf143a6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactoryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactoryImpl.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache.persistence.wal.serializer; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.record.MarshalledRecord; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.lang.IgnitePredicate; @@ -34,6 +35,15 @@ public class RecordSerializerFactoryImpl implements RecordSerializerFactory { /** Read type filter. */ private IgnitePredicate readTypeFilter; + /** + * Marshalled mode. + * Records are not deserialized in this mode, {@link MarshalledRecord} with binary representation are read instead. + */ + private boolean marshalledMode; + + /** Skip position check flag. Should be set for reading compacted wal file with skipped physical records. */ + private boolean skipPositionCheck; + /** * @param cctx Cctx. */ @@ -56,7 +66,7 @@ public RecordSerializerFactoryImpl(GridCacheSharedContext cctx) { case 2: RecordDataV2Serializer dataV2Serializer = new RecordDataV2Serializer(new RecordDataV1Serializer(cctx)); - return new RecordV2Serializer(dataV2Serializer, writePointer, readTypeFilter); + return new RecordV2Serializer(dataV2Serializer, writePointer, marshalledMode, skipPositionCheck, readTypeFilter); default: throw new IgniteCheckedException("Failed to create a serializer with the given version " + @@ -95,4 +105,38 @@ public RecordSerializerFactoryImpl readTypeFilter(IgnitePredicate readTypeFilter) { + boolean marshalledMode, boolean skipPositionCheck, IgnitePredicate readTypeFilter) { this.dataSerializer = dataSerializer; this.writePointer = writePointer; + this.marshalledMode = marshalledMode; + this.skipPositionCheck = skipPositionCheck; this.readTypeFilter = readTypeFilter; } @@ -157,12 +185,14 @@ public RecordV2Serializer(RecordDataV2Serializer dataSerializer, boolean writePo /** * @param in Data input to read pointer from. + * @param skipPositionCheck Flag for skipping position check. * @return Read file WAL pointer. * @throws IOException If failed to write. */ public static FileWALPointer readPositionAndCheckPoint( DataInput in, - WALPointer expPtr + WALPointer expPtr, + boolean skipPositionCheck ) throws IgniteCheckedException, IOException { long idx = in.readLong(); int fileOffset = in.readInt(); @@ -170,7 +200,7 @@ public static FileWALPointer readPositionAndCheckPoint( FileWALPointer p = (FileWALPointer)expPtr; - if (!F.eq(idx, p.index()) || !F.eq(fileOffset, p.fileOffset())) + if (!F.eq(idx, p.index()) || (skipPositionCheck && !F.eq(fileOffset, p.fileOffset()))) throw new WalSegmentTailReachedException( "WAL segment tail is reached. [ " + "Expected next state: {Index=" + p.index() + ",Offset=" + p.fileOffset() + "}, " + From 1de758dafd59a40e9af5f0bce28ca4aac97c7648 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Wed, 15 Nov 2017 19:01:05 +0300 Subject: [PATCH 03/16] IGNITE-5938 Several bugs fixed --- .../wal/AbstractWalRecordsIterator.java | 49 ++++++-- .../wal/FileWriteAheadLogManager.java | 108 ++++++++++++------ .../SingleSegmentLogicalRecordsIterator.java | 33 +++--- .../persistence/wal/record/HeaderRecord.java | 7 +- .../serializer/RecordDataV1Serializer.java | 7 +- .../serializer/RecordSerializerFactory.java | 23 ++++ .../RecordSerializerFactoryImpl.java | 28 ++--- .../wal/serializer/RecordV2Serializer.java | 14 +-- .../wal/IgniteWalHistoryReservationsTest.java | 3 +- 9 files changed, 185 insertions(+), 87 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java index 76e6b0d6d0ddb..ea8a023f11e28 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java @@ -30,10 +30,9 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; -import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializer; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactory; -import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactoryImpl; import org.apache.ignite.internal.util.GridCloseableIteratorAdapter; +import org.apache.ignite.internal.util.typedef.P2; import org.apache.ignite.lang.IgniteBiTuple; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -282,20 +281,27 @@ protected FileWriteAheadLogManager.ReadFileHandle initReadHandle( FileIO fileIO = ioFactory.create(desc.file); try { - int serVer = FileWriteAheadLogManager.readSerializerVersion(fileIO); + IgniteBiTuple tup = FileWriteAheadLogManager.readSerializerVersionAndCompactedFlag(fileIO); - RecordSerializer ser = serializerFactory.createSerializer(serVer); + int serVer = tup.get1(); + + boolean isCompacted = tup.get2(); FileInput in = new FileInput(fileIO, buf); if (start != null && desc.idx == start.index()) { - // Make sure we skip header with serializer version. - long startOffset = Math.max(start.fileOffset(), fileIO.position()); - - in.seek(startOffset); + if (isCompacted) + serializerFactory.recordDeserializeFilter(new StartSeekingFilter(start)); + else { + // Make sure we skip header with serializer version. + long startOff = Math.max(start.fileOffset(), fileIO.position()); + + in.seek(startOff); + } } - return new FileWriteAheadLogManager.ReadFileHandle(fileIO, desc.idx, sharedCtx.igniteInstanceName(), ser, in); + return new FileWriteAheadLogManager.ReadFileHandle( + fileIO, desc.idx, sharedCtx.igniteInstanceName(), serializerFactory.createSerializer(serVer), in); } catch (SegmentEofException | EOFException ignore) { try { @@ -327,4 +333,29 @@ protected FileWriteAheadLogManager.ReadFileHandle initReadHandle( } } + /** + * Filter that drops all records until given start pointer is reached. + */ + private static class StartSeekingFilter implements P2 { + /** Start pointer. */ + private final FileWALPointer start; + + /** Start reached flag. */ + private boolean startReached; + + /** + * @param start Start. + */ + StartSeekingFilter(FileWALPointer start) { + this.start = start; + } + + /** {@inheritDoc} */ + @Override public boolean apply(WALRecord.RecordType type, WALPointer pointer) { + if (start.fileOffset() == ((FileWALPointer)pointer).fileOffset()) + startReached = true; + + return startReached; + } + } } 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 ce3cfc25e8867..2bc53b3d7603d 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 @@ -633,6 +633,7 @@ private void checkWalRolloverRequiredDuringInactivityPeriod() { new RecordSerializerFactoryImpl(cctx), ioFactory, archiver, + decompressor, log ); } @@ -680,9 +681,12 @@ private void checkWalRolloverRequiredDuringInactivityPeriod() { * @return {@code true} if has this index. */ private boolean hasIndex(long absIdx) { - String name = FileDescriptor.fileName(absIdx); + String segmentName = FileDescriptor.fileName(absIdx); - boolean inArchive = new File(walArchiveDir, name).exists(); + String zipSegmentName = FileDescriptor.fileName(absIdx) + ".zip"; + + boolean inArchive = new File(walArchiveDir, segmentName).exists() || + new File(walArchiveDir, zipSegmentName).exists(); if (inArchive) return true; @@ -900,7 +904,7 @@ private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws Ig // If we have existing segment, try to read version from it. if (lastReadPtr != null) { try { - serVer = readSerializerVersion(fileIO); + serVer = readSerializerVersionAndCompactedFlag(fileIO).get1(); } catch (SegmentEofException | EOFException ignore) { serVer = serializerVersion; @@ -1516,27 +1520,21 @@ synchronized void onNextSegmentArchived() { /** * */ - private synchronized long reserveNextSegmentOrWait() throws InterruptedException { + private synchronized long tryReserveNextSegmentOrWait() throws InterruptedException, IgniteCheckedException { long segmentToCompress = lastCompressedSegmentIdx + 1; - while (segmentToCompress > Math.max(lastAllowedToCompressSegmentIdx, archiver.lastArchivedAbsoluteIndex())) + while (!stopped && segmentToCompress > Math.max( + lastAllowedToCompressSegmentIdx, archiver.lastArchivedAbsoluteIndex())) wait(); - boolean reserved = false; + if (stopped) + return -1; - // Reserving segment to prevent its concurrent truncation. - while (!reserved) { - segmentToCompress = Math.max(segmentToCompress, lastTruncatedArchiveIdx + 1); + segmentToCompress = Math.max(segmentToCompress, lastTruncatedArchiveIdx + 1); - try { - reserved = reserve(new FileWALPointer(segmentToCompress, 0, 0)); - } - catch (IgniteCheckedException e) { - U.error(log, "Error while trying to reserve history", e); - } - } + boolean reserved = reserve(new FileWALPointer(segmentToCompress, 0, 0)); - return segmentToCompress; + return reserved ? segmentToCompress : -1; } /** @@ -1564,11 +1562,13 @@ private void deleteObsoleteRawSegments() { @Override public void run() { init(); - while (!Thread.currentThread().isInterrupted() || !stopped) { + while (!Thread.currentThread().isInterrupted() && !stopped) { try { deleteObsoleteRawSegments(); - long nextSegment = reserveNextSegmentOrWait(); + long nextSegment = tryReserveNextSegmentOrWait(); + if (nextSegment == -1) + continue; File tmpZip = new File(walArchiveDir, FileDescriptor.fileName(nextSegment) + ".zip" + ".tmp"); assert !tmpZip.exists(); @@ -1581,7 +1581,7 @@ private void deleteObsoleteRawSegments() { ZipOutputStream zos = new ZipOutputStream(new BufferedOutputStream(new FileOutputStream(tmpZip)))) { zos.putNextEntry(new ZipEntry("")); - zos.write(prepareSerializerVersionBuffer(nextSegment, serializerVersion).array()); // WAL segment header. + zos.write(prepareSerializerVersionBuffer(nextSegment, serializerVersion, true).array()); // WAL segment header. for (IgniteBiTuple tuple : iter) { WALRecord rec = tuple.get2(); @@ -1635,10 +1635,13 @@ private class FileDecompressor extends Thread { /** {@inheritDoc} */ @Override public void run() { - while (!Thread.currentThread().isInterrupted() || !stopped) { + while (!Thread.currentThread().isInterrupted()) { try { long segmentToDecompress = segmentsQueue.take(); + if (stopped) + break; + File zip = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress) + ".zip"); assert zip.exists(); @@ -1681,7 +1684,7 @@ private class FileDecompressor extends Thread { /** * */ - public synchronized IgniteInternalFuture ensureDecompressed(long idx) { + synchronized IgniteInternalFuture decompressFile(long idx) { if (decompressionFutures.containsKey(idx)) return decompressionFutures.get(idx); @@ -1706,7 +1709,8 @@ private void shutdown() throws IgniteInterruptedCheckedException { synchronized (this) { stopped = true; - notifyAll(); + // Put fake -1 to wake thread from queue.take() + segmentsQueue.put(-1L); } U.join(this); @@ -2367,14 +2371,14 @@ else if (create) } /** - * Reads record serializer version from provided {@code io}. + * Reads record serializer version from provided {@code io} along with compacted flag. * NOTE: Method mutates position of {@code io}. * * @param io I/O interface for file. * @return Serializer version stored in the file. * @throws IgniteCheckedException If failed to read serializer version. */ - public static int readSerializerVersion(FileIO io) + public static IgniteBiTuple readSerializerVersionAndCompactedFlag(FileIO io) throws IgniteCheckedException, IOException { try (ByteBufferExpander buf = new ByteBufferExpander(RecordV1Serializer.HEADER_RECORD_SIZE, ByteOrder.nativeOrder())) { FileInput in = new FileInput(io, buf); @@ -2396,19 +2400,25 @@ public static int readSerializerVersion(FileIO io) assert ptr.fileOffset() == 0 : "Header record should be placed at the beginning of file " + ptr; - long headerMagicNumber = in.readLong(); + long hdrMagicNum = in.readLong(); - if (headerMagicNumber != HeaderRecord.MAGIC) - throw new IOException("Magic is corrupted [exp=" + U.hexLong(HeaderRecord.MAGIC) + - ", actual=" + U.hexLong(headerMagicNumber) + ']'); + boolean compacted; + if (hdrMagicNum == HeaderRecord.REGULAR_MAGIC) + compacted = true; + else if (hdrMagicNum == HeaderRecord.COMPACTED_MAGIC) + compacted = false; + else { + throw new IOException("Magic is corrupted [exp=" + U.hexLong(HeaderRecord.REGULAR_MAGIC) + + ", actual=" + U.hexLong(hdrMagicNum) + ']'); + } // Read serializer version. - int version = in.readInt(); + int ver = in.readInt(); // Read and skip CRC. in.readInt(); - return version; + return new IgniteBiTuple<>(ver, compacted); } } @@ -2423,7 +2433,7 @@ public static int readSerializerVersion(FileIO io) * @throws IOException If failed to write serializer version. */ public static long writeSerializerVersion(FileIO io, long idx, int version) throws IOException { - ByteBuffer buffer = prepareSerializerVersionBuffer(idx, version); + ByteBuffer buffer = prepareSerializerVersionBuffer(idx, version, false); do { io.write(buffer); @@ -2439,8 +2449,9 @@ public static long writeSerializerVersion(FileIO io, long idx, int version) thro /** * @param idx Index. * @param ver Version. + * @param compacted Compacted flag. */ - @NotNull private static ByteBuffer prepareSerializerVersionBuffer(long idx, int ver) { + @NotNull private static ByteBuffer prepareSerializerVersionBuffer(long idx, int ver, boolean compacted) { ByteBuffer buf = ByteBuffer.allocate(RecordV1Serializer.HEADER_RECORD_SIZE); buf.order(ByteOrder.nativeOrder()); @@ -2451,7 +2462,7 @@ public static long writeSerializerVersion(FileIO io, long idx, int version) thro RecordV1Serializer.putPosition(buf, new FileWALPointer(idx, 0, 0)); // Place magic number. - buf.putLong(HeaderRecord.MAGIC); + buf.putLong(compacted ? HeaderRecord.COMPACTED_MAGIC : HeaderRecord.REGULAR_MAGIC); // Place serializer version. buf.putInt(ver); @@ -3409,6 +3420,9 @@ private static class RecordsIterator extends AbstractWalRecordsIterator { /** */ private final FileArchiver archiver; + /** */ + private final FileDecompressor decompressor; + /** */ private final DataStorageConfiguration psCfg; @@ -3429,8 +3443,8 @@ private static class RecordsIterator extends AbstractWalRecordsIterator { * @param psCfg Database configuration. * @param serializerFactory Serializer factory. * @param archiver Archiver. - * @param log Logger - * @throws IgniteCheckedException If failed to initialize WAL segment. + * @param decompressor Decompressor. + *@param log Logger @throws IgniteCheckedException If failed to initialize WAL segment. */ private RecordsIterator( GridCacheSharedContext cctx, @@ -3442,6 +3456,7 @@ private RecordsIterator( @NotNull RecordSerializerFactory serializerFactory, FileIOFactory ioFactory, FileArchiver archiver, + FileDecompressor decompressor, IgniteLogger log ) throws IgniteCheckedException { super(log, @@ -3455,12 +3470,33 @@ private RecordsIterator( this.archiver = archiver; this.start = start; this.end = end; + this.decompressor = decompressor; init(); advance(); } + /** {@inheritDoc} */ + @Override protected ReadFileHandle initReadHandle( + @NotNull FileDescriptor desc, + @Nullable FileWALPointer start + ) throws IgniteCheckedException, FileNotFoundException { + if (decompressor != null && !desc.file.exists()) { + FileDescriptor zipFile = new FileDescriptor( + new File(walArchiveDir, FileDescriptor.fileName(desc.getIdx()) + ".zip")); + + if (!zipFile.file.exists()) { + throw new FileNotFoundException("Both compressed and raw segment files are missing in archive " + + "[segmentIdx=" + desc.idx + "]"); + } + + decompressor.decompressFile(desc.idx).get(); + } + + return super.initReadHandle(desc, start); + } + /** {@inheritDoc} */ @Override protected void onClose() throws IgniteCheckedException { super.onClose(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java index b4f82dd97c832..9465dd0210d27 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java @@ -21,14 +21,14 @@ import java.util.Set; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.pagemem.wal.WALPointer; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.wal.record.RecordTypes; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactory; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactoryImpl; -import org.apache.ignite.internal.util.typedef.P1; -import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.internal.util.typedef.P2; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -60,7 +60,7 @@ public SingleSegmentLogicalRecordsIterator( long archivedSegIdx, File archiveDir ) throws IgniteCheckedException { - super(log, sharedCtx, initFilteredSerializerFactory(sharedCtx), ioFactory, bufSize); + super(log, sharedCtx, initLogicalRecordsSerializerFactory(sharedCtx), ioFactory, bufSize); this.archivedSegIdx = archivedSegIdx; this.archiveDir = archiveDir; @@ -71,17 +71,12 @@ public SingleSegmentLogicalRecordsIterator( /** * @param sharedCtx Shared context. */ - private static RecordSerializerFactory initFilteredSerializerFactory(GridCacheSharedContext sharedCtx) + private static RecordSerializerFactory initLogicalRecordsSerializerFactory(GridCacheSharedContext sharedCtx) throws IgniteCheckedException { - IgnitePredicate logicalRecordsFilter = new P1() { - private final Set skip = RecordTypes.DELTA_TYPE_SET; - @Override public boolean apply(WALRecord.RecordType type) { - return !skip.contains(type); - } - }; - - return new RecordSerializerFactoryImpl(sharedCtx).readTypeFilter(logicalRecordsFilter).marshalledMode(true); + return new RecordSerializerFactoryImpl(sharedCtx) + .recordDeserializeFilter(new LogicalRecordsFilter()) + .marshalledMode(true); } /** {@inheritDoc} */ @@ -107,8 +102,16 @@ private static RecordSerializerFactory initFilteredSerializerFactory(GridCacheSh } } - /** {@inheritDoc} todo for debug only, remove */ - @Override protected void advance() throws IgniteCheckedException { - super.advance(); + /** + * + */ + private static class LogicalRecordsFilter implements P2 { + /** Records type to skip. */ + private final Set skip = RecordTypes.DELTA_TYPE_SET; + + /** {@inheritDoc} */ + @Override public boolean apply(WALRecord.RecordType type, WALPointer ptr) { + return !skip.contains(type); + } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/record/HeaderRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/record/HeaderRecord.java index 4fa62321bcb87..35c94a8b1a27e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/record/HeaderRecord.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/record/HeaderRecord.java @@ -24,8 +24,11 @@ * Header record. */ public class HeaderRecord extends WALRecord { - /** */ - public static final long MAGIC = 0xB0D045A_CE7ED045AL; + /** Magic of regular WAL segment. */ + public static final long REGULAR_MAGIC = 0xB0D045A_CE7ED045AL; + + /** Magic of WAL segment with skipped physical records. */ + public static final long COMPACTED_MAGIC = 0x4E07AE0_E573A694EL; /** Serializer version */ private final int ver; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java index 6e708c0c9ede6..5ccbcfaca1309 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java @@ -390,9 +390,8 @@ assert record instanceof PageSnapshot; case HEADER_RECORD: long magic = in.readLong(); - if (magic != HeaderRecord.MAGIC) - throw new EOFException("Magic is corrupted [exp=" + U.hexLong(HeaderRecord.MAGIC) + - ", actual=" + U.hexLong(magic) + ']'); + if (magic != HeaderRecord.REGULAR_MAGIC && magic != HeaderRecord.COMPACTED_MAGIC) + throw new EOFException("Magic is corrupted [actual=" + U.hexLong(magic) + ']'); int ver = in.readInt(); @@ -911,7 +910,7 @@ assert record instanceof PageSnapshot; break; case HEADER_RECORD: - buf.putLong(HeaderRecord.MAGIC); + buf.putLong(HeaderRecord.REGULAR_MAGIC); buf.putInt(((HeaderRecord)record).version()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactory.java index 8170424dbbde6..d04095cf4a353 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactory.java @@ -18,6 +18,9 @@ package org.apache.ignite.internal.processors.cache.persistence.wal.serializer; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.WALPointer; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.lang.IgniteBiPredicate; /** * @@ -30,4 +33,24 @@ public interface RecordSerializerFactory { * @return record serializer. */ public RecordSerializer createSerializer(int ver) throws IgniteCheckedException; + + /** + * @param writePointer Write pointer. + */ + public RecordSerializerFactory writePointer(boolean writePointer); + + /** + * @param readTypeFilter Read type filter. + */ + public RecordSerializerFactory recordDeserializeFilter(IgniteBiPredicate readTypeFilter); + + /** + * @param marshalledMode Marshalled mode. + */ + public RecordSerializerFactory marshalledMode(boolean marshalledMode); + + /** + * @param skipPositionCheck Skip position check. + */ + public RecordSerializerFactory skipPositionCheck(boolean skipPositionCheck); } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactoryImpl.java index d06b26cf143a6..b76ee491f0a38 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactoryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactoryImpl.java @@ -17,10 +17,11 @@ package org.apache.ignite.internal.processors.cache.persistence.wal.serializer; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.WALPointer; import org.apache.ignite.internal.pagemem.wal.record.MarshalledRecord; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; -import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.lang.IgniteBiPredicate; /** * @@ -32,11 +33,11 @@ public class RecordSerializerFactoryImpl implements RecordSerializerFactory { /** Write pointer. */ private boolean writePointer; - /** Read type filter. */ - private IgnitePredicate readTypeFilter; + /** Read record filter. */ + private IgniteBiPredicate recordDeserializeFilter; /** - * Marshalled mode. + * Marshalled mode flag. * Records are not deserialized in this mode, {@link MarshalledRecord} with binary representation are read instead. */ private boolean marshalledMode; @@ -58,7 +59,7 @@ public RecordSerializerFactoryImpl(GridCacheSharedContext cctx) { switch (ver) { case 1: - if (readTypeFilter != null) + if (recordDeserializeFilter != null) throw new IgniteCheckedException("Read type filter is allowed only for version 2 or higher."); return new RecordV1Serializer(new RecordDataV1Serializer(cctx), writePointer); @@ -66,7 +67,7 @@ public RecordSerializerFactoryImpl(GridCacheSharedContext cctx) { case 2: RecordDataV2Serializer dataV2Serializer = new RecordDataV2Serializer(new RecordDataV1Serializer(cctx)); - return new RecordV2Serializer(dataV2Serializer, writePointer, marshalledMode, skipPositionCheck, readTypeFilter); + return new RecordV2Serializer(dataV2Serializer, writePointer, marshalledMode, skipPositionCheck, recordDeserializeFilter); default: throw new IgniteCheckedException("Failed to create a serializer with the given version " + @@ -84,7 +85,7 @@ public boolean writePointer() { /** * @param writePointer New write pointer. */ - public RecordSerializerFactoryImpl writePointer(boolean writePointer) { + @Override public RecordSerializerFactoryImpl writePointer(boolean writePointer) { this.writePointer = writePointer; return this; @@ -93,15 +94,16 @@ public RecordSerializerFactoryImpl writePointer(boolean writePointer) { /** * @return Read type filter. */ - public IgnitePredicate readTypeFilter() { - return readTypeFilter; + public IgniteBiPredicate recordDeserializeFilter() { + return recordDeserializeFilter; } /** * @param readTypeFilter New read type filter. */ - public RecordSerializerFactoryImpl readTypeFilter(IgnitePredicate readTypeFilter) { - this.readTypeFilter = readTypeFilter; + @Override public RecordSerializerFactoryImpl recordDeserializeFilter( + IgniteBiPredicate readTypeFilter) { + this.recordDeserializeFilter = readTypeFilter; return this; } @@ -117,7 +119,7 @@ public boolean marshalledMode() { * @param marshalledMode New marshalled mode. Records are not deserialized in this mode, with binary representation * are read instead. */ - public RecordSerializerFactoryImpl marshalledMode(boolean marshalledMode) { + @Override public RecordSerializerFactoryImpl marshalledMode(boolean marshalledMode) { this.marshalledMode = marshalledMode; return this; @@ -134,7 +136,7 @@ public boolean skipPositionCheck() { * @param skipPositionCheck New skip position check flag. Should be set for reading compacted wal file with skipped * physical records. */ - public RecordSerializerFactoryImpl skipPositionCheck(boolean skipPositionCheck) { + @Override public RecordSerializerFactoryImpl skipPositionCheck(boolean skipPositionCheck) { this.skipPositionCheck = skipPositionCheck; return this; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java index 9fd71b8216243..aa9ac0fd440a3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java @@ -32,7 +32,7 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.WalSegmentTailReachedException; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.io.RecordIO; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.lang.IgniteBiPredicate; import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.*; import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.CRC_SIZE; @@ -49,7 +49,7 @@ *
  • CRC or zero padding
  • * * Also, optimization for skipping deserialization of records of unwanted types. - * If {@link #readTypeFilter} is specified, {@link FilteredRecord} is read instead of unnecessary record. + * If {@link #recordFilter} is specified, {@link FilteredRecord} is read instead of unnecessary record. */ public class RecordV2Serializer implements RecordSerializer { /** Length of WAL Pointer: Index (8) + File offset (4) + Record length (4) */ @@ -74,7 +74,7 @@ public class RecordV2Serializer implements RecordSerializer { * Record type filter. * {@link FilteredRecord} is deserialized instead of original record if type doesn't match filter. */ - private final IgnitePredicate readTypeFilter; + private final IgniteBiPredicate recordFilter; /** Record read/write functional interface. */ private final RecordIO recordIO = new RecordIO() { @@ -96,7 +96,7 @@ public class RecordV2Serializer implements RecordSerializer { FileWALPointer ptr = readPositionAndCheckPoint(in, expPtr, skipPositionCheck); - if (readTypeFilter != null && !readTypeFilter.apply(recType)) { + if (recordFilter != null && !recordFilter.apply(recType, ptr)) { int toSkip = ptr.length() - REC_TYPE_SIZE - FILE_WAL_POINTER_SIZE - CRC_SIZE; assert toSkip >= 0 : "Too small saved record length: " + ptr; @@ -147,15 +147,15 @@ else if (marshalledMode) { * @param dataSerializer V2 data serializer. * @param marshalledMode Marshalled mode. * @param skipPositionCheck Skip position check mode. - * @param readTypeFilter Record type filter. {@link FilteredRecord} is deserialized instead of original record + * @param recordFilter Record type filter. {@link FilteredRecord} is deserialized instead of original record */ public RecordV2Serializer(RecordDataV2Serializer dataSerializer, boolean writePointer, - boolean marshalledMode, boolean skipPositionCheck, IgnitePredicate readTypeFilter) { + boolean marshalledMode, boolean skipPositionCheck, IgniteBiPredicate recordFilter) { this.dataSerializer = dataSerializer; this.writePointer = writePointer; this.marshalledMode = marshalledMode; this.skipPositionCheck = skipPositionCheck; - this.readTypeFilter = readTypeFilter; + this.recordFilter = recordFilter; } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java index 35d85d1b63b28..97b19cd2e70ee 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java @@ -60,7 +60,8 @@ public class IgniteWalHistoryReservationsTest extends GridCommonAbstractTest { DataStorageConfiguration memCfg = new DataStorageConfiguration() .setDefaultDataRegionConfiguration( new DataRegionConfiguration().setMaxSize(200 * 1024 * 1024).setPersistenceEnabled(true)) - .setWalMode(WALMode.LOG_ONLY); + .setWalMode(WALMode.LOG_ONLY) + .setWalSegmentSize(512 * 1024); cfg.setDataStorageConfiguration(memCfg); From f890e7ac628f714f62ce6461b1518888ec92f1a2 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Thu, 16 Nov 2017 13:22:30 +0300 Subject: [PATCH 04/16] IGNITE-5938 Several bugs fixed --- .../wal/FileWriteAheadLogManager.java | 4 ++-- .../wal/serializer/RecordDataV2Serializer.java | 18 ++++++++++++++++++ 2 files changed, 20 insertions(+), 2 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 2bc53b3d7603d..315d11c617d63 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 @@ -2404,9 +2404,9 @@ public static IgniteBiTuple readSerializerVersionAndCompactedF boolean compacted; if (hdrMagicNum == HeaderRecord.REGULAR_MAGIC) - compacted = true; - else if (hdrMagicNum == HeaderRecord.COMPACTED_MAGIC) compacted = false; + else if (hdrMagicNum == HeaderRecord.COMPACTED_MAGIC) + compacted = true; else { throw new IOException("Magic is corrupted [exp=" + U.hexLong(HeaderRecord.REGULAR_MAGIC) + ", actual=" + U.hexLong(hdrMagicNum) + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java index df63b9e508f6f..16a81a4c5676c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java @@ -24,6 +24,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.pagemem.wal.record.DataEntry; import org.apache.ignite.internal.pagemem.wal.record.DataRecord; +import org.apache.ignite.internal.pagemem.wal.record.SnapshotRecord; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput; import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord; @@ -53,6 +54,9 @@ public RecordDataV2Serializer(RecordDataV1Serializer delegateSerializer) { case DATA_RECORD: return delegateSerializer.size(record) + 8/*timestamp*/; + case SNAPSHOT: + return 8 + 1; + default: return delegateSerializer.size(record); } @@ -75,6 +79,12 @@ public RecordDataV2Serializer(RecordDataV1Serializer delegateSerializer) { return new DataRecord(entries, timeStamp); + case SNAPSHOT: + long snpId = in.readLong(); + byte full = in.readByte(); + + return new SnapshotRecord(snpId, full == 1); + default: return delegateSerializer.readRecord(type, in); } @@ -97,6 +107,14 @@ public RecordDataV2Serializer(RecordDataV1Serializer delegateSerializer) { break; + case SNAPSHOT: + SnapshotRecord snpRec = (SnapshotRecord)record; + + buf.putLong(snpRec.getSnapshotId()); + buf.put(snpRec.isFull() ? (byte)1 : 0); + + break; + default: delegateSerializer.writeRecord(record, buf); } From c5f761307af3fc02cd2af83fb2c325a653abead5 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Thu, 16 Nov 2017 14:32:23 +0300 Subject: [PATCH 05/16] IGNITE-5938 Minor refactor --- .../wal/FileWriteAheadLogManager.java | 656 +----------------- 1 file changed, 15 insertions(+), 641 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 315d11c617d63..0918059b497fb 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 @@ -71,8 +71,8 @@ import org.apache.ignite.internal.pagemem.wal.record.WALRecord; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; -import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cache.persistence.DataStorageMetricsImpl; +import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings; @@ -1480,10 +1480,10 @@ private class FileCompressor extends Thread { private volatile boolean stopped; /** Last successfully compressed segment. */ - private volatile long lastCompressedSegmentIdx = -1L; + private volatile long lastCompressedIdx = -1L; /** All segments prior to this (inclusive) can be compressed. */ - private volatile long lastAllowedToCompressSegmentIdx = -1L; + private volatile long lastAllowedToCompressIdx = -1L; private void init() { File[] toDel = walArchiveDir.listFiles(WAL_SEGMENT_TEMP_FILE_COMPACTED_FILTER); @@ -1498,14 +1498,14 @@ private void init() { FileDescriptor[] alreadyCompressed = scan(walArchiveDir.listFiles(WAL_SEGMENT_FILE_COMPACTED_FILTER)); if (alreadyCompressed.length > 0) - lastCompressedSegmentIdx = alreadyCompressed[alreadyCompressed.length - 1].getIdx(); + lastCompressedIdx = alreadyCompressed[alreadyCompressed.length - 1].getIdx(); } /** * @param lastCpStartIdx Segment index to allow compression until (exclusively). */ synchronized void allowCompressionUntil(long lastCpStartIdx) { - lastAllowedToCompressSegmentIdx = lastCpStartIdx - 1; + lastAllowedToCompressIdx = lastCpStartIdx - 1; notify(); } @@ -1518,17 +1518,18 @@ synchronized void onNextSegmentArchived() { } /** - * + * Pessimistically tries to reserve segment for compression in order to avoid concurrent truncation. + * Waits if there's no segment to archive right now. */ private synchronized long tryReserveNextSegmentOrWait() throws InterruptedException, IgniteCheckedException { - long segmentToCompress = lastCompressedSegmentIdx + 1; + long segmentToCompress = lastCompressedIdx + 1; - while (!stopped && segmentToCompress > Math.max( - lastAllowedToCompressSegmentIdx, archiver.lastArchivedAbsoluteIndex())) + while (segmentToCompress > Math.max(lastAllowedToCompressIdx, archiver.lastArchivedAbsoluteIndex())) { wait(); - if (stopped) - return -1; + if (stopped) + return -1; + } segmentToCompress = Math.max(segmentToCompress, lastTruncatedArchiveIdx + 1); @@ -1550,7 +1551,7 @@ private void deleteObsoleteRawSegments() { if (archiver0 != null && archiver0.reserved(desc.idx)) return; - if (desc.idx < lastCompressedSegmentIdx) { + if (desc.idx < lastCompressedIdx) { if (!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()); @@ -1595,7 +1596,7 @@ private void deleteObsoleteRawSegments() { Files.move(tmpZip.toPath(), zip.toPath()); - lastCompressedSegmentIdx = nextSegment; + lastCompressedIdx = nextSegment; } catch (IgniteCheckedException | IOException e) { U.error(log, "Unexpected error during WAL compression", e); // todo what do with ioexception? @@ -1635,7 +1636,7 @@ private class FileDecompressor extends Thread { /** {@inheritDoc} */ @Override public void run() { - while (!Thread.currentThread().isInterrupted()) { + while (!Thread.currentThread().isInterrupted() && !stopped) { try { long segmentToDecompress = segmentsQueue.take(); @@ -1717,633 +1718,6 @@ private void shutdown() throws IgniteInterruptedCheckedException { } } -// /** -// * -// */ -// private class FileCompressor extends Thread { -// /** */ -// private Throwable exception; -// -// /** */ -// private volatile boolean stopped; -// -// /** */ -// private long lastCompressedSegment = -1L; -// -// /** */ -// private long lastSegmentWithCheckpointMarkerInArchive; -// -// /** */ -// private final FileArchiver fileArchiver; -// -// /** */ -// private final TreeSet checkpointMarkerSegments = new TreeSet<>(); -// -// /** */ -// private final CompactFactory compactFactory = new CompactFactoryImpl(); -// -// /** -// * -// */ -// private FileCompressor(FileArchiver archiver) { -// super("wal-file-compressor%" + cctx.igniteInstanceName()); -// -// assert archiver != null; -// -// fileArchiver = archiver; -// } -// -// private void init() { -// File[] toDelete = walArchiveDir.listFiles(WAL_SEGMENT_TEMP_FILE_COMPACTED_FILTER); -// -// for (File f : toDelete) { -// if (stopped) -// return; -// -// f.delete(); -// } -// -// // Find last compressed segment. -// File[] compactedArchives = walArchiveDir.listFiles(WAL_SEGMENT_FILE_COMPACTED_FILTER); -// -// if (!F.isEmpty(compactedArchives)) { -// long maxArchivedSegIdx = -1L; -// -// for (File f : compactedArchives) { -// CompactDescriptor desc = CompactDescriptor.parse(f.getAbsolutePath()); -// -// if (desc.endSeg > maxArchivedSegIdx) -// maxArchivedSegIdx = desc.endSeg; -// } -// -// assert maxArchivedSegIdx != -1L; -// -// synchronized (this) { -// lastCompressedSegment = maxArchivedSegIdx; -// } -// } -// } -// -// /** {@inheritDoc} */ -// @Override public void run() { -// init(); -// -// while (!Thread.currentThread().isInterrupted() || !stopped) { -// try { -// CompactDescriptor compactDesc = waitAvailableSegmentsForCompacting(); -// -// reserveSegments(compactDesc); -// -// CompactWriter compactWriter = compactFactory.createWriter(0, compactDesc); -// -// CompactionIterator iter = new CompactionIterator( -// compactDesc, -// walArchiveDir.getPath(), -// cctx.igniteInstanceName(), -// recSerFactory, -// serializer, -// ioFactory, -// compactWriter, -// igCfg.getPersistentStoreConfiguration().getTlbSize(), -// log -// ); -// -// try { -// while (iter.hasNext()) { -// iter.next(); -// -// if (stopped) -// break; -// } -// } -// finally { -// iter.closeCurrentWalSegment(); -// } -// -// synchronized (this) { -// lastCompressedSegment = compactDesc.endSeg; -// -// SortedSet toRemove = checkpointMarkerSegments.headSet(lastCompressedSegment); -// // Remove compressed segments with chp marker. -// checkpointMarkerSegments.removeAll(new ArrayList<>(toRemove)); -// } -// -// releaseSegments(compactDesc); -// -// for (long idx = compactDesc.startSeg; idx <= compactDesc.endSeg; idx++) { -// File segFile = new File(walArchiveDir, FileDescriptor.fileName(idx)); -// -// segFile.delete(); -// } -// -// } -// catch (Throwable e) { -// exception = e; -// -// e.printStackTrace(); -// } -// } -// -// if (log != null && log.isInfoEnabled()) -// log.info(getName() + " - stopped."); -// } -// -// /** -// * -// */ -// private CompactDescriptor waitAvailableSegmentsForCompacting() throws InterruptedException { -// long endSeg; -// -// long last; -// -// long startSeg; -// -// assert BATCH_SIZE >= 0; -// -// synchronized (this) { -// while ((last = lastCompressedSegment) + BATCH_SIZE >= (endSeg = readyForCompressingSegIdx())) -// wait(); -// -// startSeg = last == -1L ? 0 : last + 1; -// -// if (BATCH_SIZE != 0 && endSeg - last > BATCH_SIZE) -// endSeg = last + BATCH_SIZE; -// } -// -// return new CompactDescriptor(startSeg, endSeg, walArchiveDir.getPath()); -// } -// -// /** -// * -// */ -// private long readyForCompressingSegIdx() { -// return lastSegmentWithCheckpointMarkerInArchive - 1; -// } -// -// /** -// * -// */ -// private void reserveSegments(CompactDescriptor cDesc) { -// fileArchiver.reserve(cDesc.endSeg); -// } -// -// /** -// * -// */ -// private void releaseSegments(CompactDescriptor cDesc) { -// fileArchiver.release(cDesc.endSeg); -// } -// -// /** -// * -// */ -// public void shutdown() throws IgniteInterruptedCheckedException { -// synchronized (this) { -// stopped = true; -// -// notifyAll(); -// } -// -// U.join(this); -// } -// -// /** -// * -// */ -// private void advanceCheckpointMarker(long idx, long lastArchivedSegIdx) { -// synchronized (this) { -// checkpointMarkerSegments.add(idx); -// -// SortedSet head = checkpointMarkerSegments.headSet(lastArchivedSegIdx); -// -// if (!head.isEmpty()) { -// lastSegmentWithCheckpointMarkerInArchive = head.last(); -// -// notifyAll(); -// } -// } -// } -// } -// -// /** -// * -// */ -// private static class CompactDescriptor { -// -// /** */ -// private final long startSeg; -// -// /** */ -// private final long endSeg; -// -// /** */ -// private final String path; -// -// /** */ -// private final String name; -// -// /** -// * -// */ -// private CompactDescriptor(long startSeg, long endSeg, String path) { -// this.startSeg = startSeg; -// this.endSeg = endSeg; -// this.path = path; -// -// SB b1 = new SB(); -// -// String segmentStr = Long.toString(startSeg); -// -// for (int i = segmentStr.length(); i < 16; i++) -// b1.a('0'); -// -// SB b2 = new SB(); -// -// long cnt = endSeg - startSeg; -// -// String cntStr = Long.toString(cnt == 0 ? 1 : cnt); -// -// this.name = "seg-" + b1.a(segmentStr) + "-" + cntStr; -// } -// -// public static CompactDescriptor parse(String fullPath) { -// File f = new File(fullPath); -// -// String fileName = f.getName(); -// -// String[] parts = fileName.split("-"); -// -// Long startIdx = Long.valueOf(parts[1]); -// -// return new CompactDescriptor(startIdx, startIdx + Long.valueOf(parts[2]), f.getPath()); -// } -// } -// -// /** -// * -// */ -// private static class CompactionIterator extends AbstractWalRecordsIterator { -// /** */ -// private static final boolean SKIP_DELTA_RECORD_FILTER_ENABLE = -// IgniteSystemProperties.getBoolean(IGNITE_WAL_ARCHIVE_COMPACT_SKIP_DELTA_RECORD, true); -// -// /** */ -// private final IgniteBiTuple resFakeTuple = new IgniteBiTuple<>(); -// -// /** */ -// private final IgnitePredicate walDeltaRecordsFilter = new P1() { -// private final Set skip = RecordTypes.DELTA_TYPE_SET; -// -// @Override public boolean apply(WALRecord.RecordType type) { -// return skip.contains(type); -// } -// }; -// -// /** */ -// private final String directoryPath; -// -// /** */ -// private final long startSegIdx; -// -// /** */ -// private final long endSegIdx; -// -// /** */ -// private final CompactWriter compactWriter; -// -// /** -// * -// */ -// private CompactionIterator( -// CompactDescriptor desc, -// String directoryPath, -// String gridName, -// RecordSerializerFactory recSerFactory, -// RecordSerializer serializer, -// FileIOFactory ioFactory, -// CompactWriter compactWriter, -// int bufSize, -// IgniteLogger log -// ) throws IgniteCheckedException { -// super( -// gridName, -// recSerFactory, -// serializer, -// ioFactory, -// log, -// bufSize); -// -// assert !F.isEmpty(directoryPath); -// assert compactWriter != null; -// -// this.directoryPath = directoryPath; -// this.startSegIdx = desc.startSeg; -// this.endSegIdx = desc.endSeg; -// this.compactWriter = compactWriter; -// -// advance(); -// } -// -// @Override protected ReadFileHandle advanceSegment( -// @Nullable ReadFileHandle curWalSegment -// ) throws IgniteCheckedException { -// long idx = -1L; -// -// if (curWalSegment != null && curWalSegment.idx == endSegIdx) -// return null; -// -// if (curWalSegment == null) -// idx = startSegIdx; -// else if (curWalSegment.idx < endSegIdx) -// idx = curWalSegment.idx + 1; -// -// assert idx != -1L; -// -// try { -// String fileName = FileDescriptor.fileName(idx); -// -// compactWriter.advance(fileName); -// -// File file = new File(directoryPath, fileName); -// -// return initReadHandle(new FileDescriptor(file), null); -// } -// catch (FileNotFoundException e) { -// if (log.isInfoEnabled()) -// log.info("Missing " + idx + " WAL segment: " + e.getMessage()); -// -// return null; -// } -// } -// -// /** {@inheritDoc} */ -// @Override protected IgniteBiTuple advanceRecord( -// @Nullable final FileWriteAheadLogManager.ReadFileHandle hnd -// ) { -// if (hnd == null) -// return null; -// -// try { -// FileInput in = hnd.in; -// -// int startPos = (int)in.position(); -// -// final FileWALPointer ptr = new FileWALPointer(hnd.idx, startPos, 0); -// -// // Todo do not create real rec. -// final WALRecord rec = hnd.ser.readRecord(in, ptr); -// -// if (SKIP_DELTA_RECORD_FILTER_ENABLE && walDeltaRecordsFilter.apply(rec.type())) -// return resFakeTuple; -// -// assert compactWriter != null; -// -// // todo recalculate wal point. -// -// int recSize = rec.size(); -// -// in.seek(startPos); -// -// in.ensure(recSize); -// -// compactWriter.write(in.buffer(), recSize); -// -// in.seek(startPos + recSize); -// -// return resFakeTuple; -// } -// catch (IOException | IgniteCheckedException e) { -// if (!(e instanceof SegmentEofException)) { -// if (log != null && log.isInfoEnabled()) -// log.info(e.getMessage()); -// } -// -// return null; -// } -// } -// -// /** {@inheritDoc} */ -// @Nullable @Override protected ReadFileHandle closeCurrentWalSegment() throws IgniteCheckedException { -// ReadFileHandle h = super.closeCurrentWalSegment(); -// -// try { -// compactWriter.close(); -// } -// catch (Exception e) { -// throw new IgniteException(e); -// } -// -// return h; -// } -// } -// -// /** -// * -// */ -// private interface CompactFactory { -// /** -// * 0 - Standard java zip {@link ZipOutputStream}. -// */ -// public CompactWriter createWriter(int type, CompactDescriptor desc); -// -// /** -// * 0 - Standard java zip {@link ZipInputStream}. -// */ -// public CompactReader createReader(int type, CompactDescriptor desc); -// } -// -// /** -// * -// */ -// private interface CompactWriter extends AutoCloseable { -// /** -// * @param name Next entry name. -// */ -// public void advance(String name); -// -// /** -// * @param buf Wal record byte buffer. -// */ -// public void write(ByteBuffer buf, int size); -// } -// -// /** -// * -// */ -// private interface CompactReader extends AutoCloseable { -// -// /** -// * -// */ -// public String advance(); -// -// /** -// * -// */ -// public int read(ByteBuffer buf); -// } -// -// /** -// * -// */ -// private static class CompactFactoryImpl implements CompactFactory { -// private static final int DEFAULT_COMPACTION_LEVEL = 9; -// -// @Override public CompactWriter createWriter(int type, CompactDescriptor desc) { -// switch (type) { -// case 0: -// assert desc != null; -// -// return new StandardZipWriter(desc.path, desc.name, DEFAULT_COMPACTION_LEVEL); -// default: -// throw new UnsupportedOperationException("Unsupported compact writer type:" + type); -// } -// } -// -// @Override public CompactReader createReader(int type, CompactDescriptor desc) { -// switch (type) { -// case 0: -// assert desc != null; -// -// return new StandardZipReader(desc.path, desc.name); -// default: -// throw new UnsupportedOperationException("Unsupported compact reader type:" + type); -// } -// } -// } -// -// /** -// * -// */ -// private static class StandardZipWriter extends AbstractStandardZip implements CompactWriter { -// /** */ -// private static final String TMP_SUFFIX = SUFFIX + ".tmp"; -// -// /** */ -// private ZipOutputStream out; -// -// /** */ -// private File archiveTmp; -// -// /** -// * -// */ -// private StandardZipWriter(String dir, String name, int zipLevel) { -// assert dir != null; -// assert name != null; -// assert zipLevel >= 0 && zipLevel <= 9; -// -// try { -// archive = new File(dir, name + SUFFIX); -// archiveTmp = new File(dir, name + TMP_SUFFIX); -// -// assert !archive.exists(); -// assert !archiveTmp.exists(); -// -// archiveTmp.createNewFile(); -// -// out = new ZipOutputStream(new FileOutputStream(archiveTmp)); -// -// out.setLevel(zipLevel); -// } -// catch (IOException e) { -// throw new IgniteException(e); -// } -// } -// -// /** {@inheritDoc} */ -// @Override public void advance(String name) { -// try { -// out.putNextEntry(new ZipEntry(name)); -// } -// catch (IOException e) { -// throw new IgniteException(e); -// } -// } -// -// /** {@inheritDoc} */ -// @Override public void write(ByteBuffer buf, int size) { -// try { -// byte[] toWrite = buf.array(); -// -// out.write(toWrite, 0, size); -// } -// catch (IOException e) { -// throw new IgniteException(e); -// } -// } -// -// /** {@inheritDoc} */ -// @Override public void close() { -// try { -// out.close(); -// -// Files.move(archiveTmp.toPath(), archive.toPath()); -// } -// catch (IOException e) { -// throw new IgniteException(e); -// } -// } -// } -// -// private static class StandardZipReader extends AbstractStandardZip implements CompactReader { -// -// /** */ -// private ZipInputStream in; -// -// private StandardZipReader(String dir, String name) { -// try { -// in = new ZipInputStream(new FileInputStream(new File(dir, name + SUFFIX))); -// } -// catch (FileNotFoundException e) { -// e.printStackTrace(); -// } -// } -// -// @Override public String advance() { -// try { -// ZipEntry entry = in.getNextEntry(); -// -// return entry.getName(); -// } -// catch (IOException e) { -// e.printStackTrace(); -// } -// return null; -// } -// -// @Override public int read(ByteBuffer buf) { -// try { -// byte[] array = buf.array(); -// -// return in.read(array); -// } -// catch (IOException e) { -// e.printStackTrace(); -// } -// -// return 0; -// } -// -// @Override public void close() { -// try { -// in.close(); -// } -// catch (IOException e) { -// e.printStackTrace(); -// } -// } -// } -// -// private abstract static class AbstractStandardZip { -// /** */ -// protected static final String SUFFIX = "-archive.zip"; -// -// /** */ -// protected File archive; -// -// } - /** * Validate files depending on {@link DataStorageConfiguration#getWalSegments()} and create if need. * Check end when exit condition return false or all files are passed. From 8f10daa3b76bc7a5b266a0f6dbc1e62153308a84 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Thu, 16 Nov 2017 14:59:59 +0300 Subject: [PATCH 06/16] IGNITE-5938 serialVUID added --- .../cache/persistence/wal/AbstractWalRecordsIterator.java | 3 +++ .../wal/SingleSegmentLogicalRecordsIterator.java | 6 ++++++ 2 files changed, 9 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java index ea8a023f11e28..2a1ce2b971db1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java @@ -337,6 +337,9 @@ protected FileWriteAheadLogManager.ReadFileHandle initReadHandle( * Filter that drops all records until given start pointer is reached. */ private static class StartSeekingFilter implements P2 { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + /** Start pointer. */ private final FileWALPointer start; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java index 9465dd0210d27..bd28a5c432ad7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java @@ -36,6 +36,9 @@ * Iterates over logical records of one WAL segment from archive. Used for WAL archive compression. */ public class SingleSegmentLogicalRecordsIterator extends AbstractWalRecordsIterator { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + /** Segment initialized flag. */ private boolean segmentInitialized; @@ -106,6 +109,9 @@ private static RecordSerializerFactory initLogicalRecordsSerializerFactory(GridC * */ private static class LogicalRecordsFilter implements P2 { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + /** Records type to skip. */ private final Set skip = RecordTypes.DELTA_TYPE_SET; From 1d29758b8f380c8400d625976f2d2d22a319179c Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Thu, 16 Nov 2017 15:59:40 +0300 Subject: [PATCH 07/16] IGNITE-5938 Javadocs --- .../wal/IgniteWriteAheadLogManager.java | 5 ++++- .../wal/FileWriteAheadLogManager.java | 6 +++++- .../persistence/wal/record/RecordTypes.java | 2 +- .../serializer/RecordSerializerFactory.java | 19 +++++++++++++++++-- .../RecordSerializerFactoryImpl.java | 18 ++++-------------- .../wal/serializer/RecordV2Serializer.java | 2 -- .../wal/IgniteWalHistoryReservationsTest.java | 5 ++--- .../persistence/pagemem/NoOpWALManager.java | 2 +- 8 files changed, 34 insertions(+), 25 deletions(-) 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 bdb43fa712fe3..42d9611f9a5c8 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 @@ -102,7 +102,10 @@ public interface IgniteWriteAheadLogManager extends GridCacheSharedManager, Igni public int truncate(WALPointer ptr); /** - * @param ptr Ptr. + * Gives a hint to WAL manager to compact WAL until given pointer (exclusively). + * Compaction implies filtering out physical records and ZIP compression. + * + * @param ptr Pointer for which it is safe to compact the log. */ public void allowCompressionUntil(WALPointer 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 0918059b497fb..38e639f2861e3 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 @@ -744,7 +744,8 @@ private boolean hasIndex(long absIdx) { /** {@inheritDoc} */ @Override public void allowCompressionUntil(WALPointer ptr) { - compressor.allowCompressionUntil(((FileWALPointer)ptr).index()); + if (compressor != null) + compressor.allowCompressionUntil(((FileWALPointer)ptr).index()); } /** {@inheritDoc} */ @@ -1474,6 +1475,7 @@ private void allocateRemainingFiles() throws IgniteCheckedException { /** * Responsible for compressing WAL archive segments. + * Also responsible for deleting raw copies of already compressed WAL archive segments if they are not reserved. */ private class FileCompressor extends Thread { /** Current thread stopping advice. */ @@ -1683,7 +1685,9 @@ private class FileDecompressor extends Thread { } /** + * Asynchronously decompresses WAL segment which is present only in .zip file. * + * @return Future which is completed once file is decompressed. */ synchronized IgniteInternalFuture decompressFile(long idx) { if (decompressionFutures.containsKey(idx)) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/record/RecordTypes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/record/RecordTypes.java index c1d39f46832a0..9654748ada3f4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/record/RecordTypes.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/record/RecordTypes.java @@ -22,7 +22,7 @@ import org.apache.ignite.internal.pagemem.wal.record.WALRecord; /** - * + * Utility class for handling WAL record types. */ public final class RecordTypes { /** */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactory.java index d04095cf4a353..f46c315426516 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactory.java @@ -19,11 +19,13 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.pagemem.wal.WALPointer; +import org.apache.ignite.internal.pagemem.wal.record.FilteredRecord; +import org.apache.ignite.internal.pagemem.wal.record.MarshalledRecord; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; import org.apache.ignite.lang.IgniteBiPredicate; /** - * + * Factory for creating {@link RecordSerializer}. */ public interface RecordSerializerFactory { /** @@ -35,21 +37,34 @@ public interface RecordSerializerFactory { public RecordSerializer createSerializer(int ver) throws IgniteCheckedException; /** - * @param writePointer Write pointer. + * TODO: This flag was added under IGNITE-6029, but still unused. Should be either handled or removed. + * + * @param writePointer Write pointer flag. */ public RecordSerializerFactory writePointer(boolean writePointer); /** + * Specifies deserialization filter. Created serializer will read bulk {@link FilteredRecord} instead of actual + * record if record type/pointer doesn't satisfy filter. + * * @param readTypeFilter Read type filter. */ public RecordSerializerFactory recordDeserializeFilter(IgniteBiPredicate readTypeFilter); /** + * If marshalledMode is on, created serializer will read {@link MarshalledRecord} with raw binary data instead of + * actual record. + * Useful for copying binary data from WAL. + * * @param marshalledMode Marshalled mode. */ public RecordSerializerFactory marshalledMode(boolean marshalledMode); /** + * If skipPositionCheck is true, created serializer won't check that actual position of record in file is equal to + * position in saved record's WALPointer. + * Must be true if we are reading from compacted WAL segment. + * * @param skipPositionCheck Skip position check. */ public RecordSerializerFactory skipPositionCheck(boolean skipPositionCheck); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactoryImpl.java index b76ee491f0a38..286108160403c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactoryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactoryImpl.java @@ -82,9 +82,7 @@ public boolean writePointer() { return writePointer; } - /** - * @param writePointer New write pointer. - */ + /** {@inheritDoc} */ @Override public RecordSerializerFactoryImpl writePointer(boolean writePointer) { this.writePointer = writePointer; @@ -98,9 +96,7 @@ public IgniteBiPredicate recordDeserializeFilt return recordDeserializeFilter; } - /** - * @param readTypeFilter New read type filter. - */ + /** {@inheritDoc} */ @Override public RecordSerializerFactoryImpl recordDeserializeFilter( IgniteBiPredicate readTypeFilter) { this.recordDeserializeFilter = readTypeFilter; @@ -115,10 +111,7 @@ public boolean marshalledMode() { return marshalledMode; } - /** - * @param marshalledMode New marshalled mode. Records are not deserialized in this mode, with binary representation - * are read instead. - */ + /** {@inheritDoc} */ @Override public RecordSerializerFactoryImpl marshalledMode(boolean marshalledMode) { this.marshalledMode = marshalledMode; @@ -132,10 +125,7 @@ public boolean skipPositionCheck() { return skipPositionCheck; } - /** - * @param skipPositionCheck New skip position check flag. Should be set for reading compacted wal file with skipped - * physical records. - */ + /** {@inheritDoc} */ @Override public RecordSerializerFactoryImpl skipPositionCheck(boolean skipPositionCheck) { this.skipPositionCheck = skipPositionCheck; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java index aa9ac0fd440a3..4cc2a2f347eec 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java @@ -48,8 +48,6 @@ *
  • Data
  • *
  • CRC or zero padding
  • * - * Also, optimization for skipping deserialization of records of unwanted types. - * If {@link #recordFilter} is specified, {@link FilteredRecord} is read instead of unnecessary record. */ public class RecordV2Serializer implements RecordSerializer { /** Length of WAL Pointer: Index (8) + File offset (4) + Record length (4) */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java index 97b19cd2e70ee..c6d58e5b9f780 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java @@ -27,9 +27,9 @@ import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.DataRegionConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.configuration.DataRegionConfiguration; import org.apache.ignite.configuration.WALMode; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; @@ -60,8 +60,7 @@ public class IgniteWalHistoryReservationsTest extends GridCommonAbstractTest { DataStorageConfiguration memCfg = new DataStorageConfiguration() .setDefaultDataRegionConfiguration( new DataRegionConfiguration().setMaxSize(200 * 1024 * 1024).setPersistenceEnabled(true)) - .setWalMode(WALMode.LOG_ONLY) - .setWalSegmentSize(512 * 1024); + .setWalMode(WALMode.LOG_ONLY); cfg.setDataStorageConfiguration(memCfg); 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 b1d9098244960..b95208c1ee692 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 @@ -78,7 +78,7 @@ public class NoOpWALManager implements IgniteWriteAheadLogManager { /** {@inheritDoc} */ @Override public void allowCompressionUntil(WALPointer ptr) { - + // No-op. } /** {@inheritDoc} */ From 2e26a8ee69538c2ca29fe012a0d1d982ab9c7f0d Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Thu, 16 Nov 2017 16:19:35 +0300 Subject: [PATCH 08/16] IGNITE-5938 Exception handling --- .../wal/FileWriteAheadLogManager.java | 39 +++++++++++++++---- 1 file changed, 31 insertions(+), 8 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 38e639f2861e3..a528038ce635f 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 @@ -49,7 +49,6 @@ import java.util.zip.ZipInputStream; import java.util.zip.ZipOutputStream; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.configuration.DataStorageConfiguration; @@ -1566,6 +1565,9 @@ private void deleteObsoleteRawSegments() { init(); while (!Thread.currentThread().isInterrupted() && !stopped) { + File tmpZip = null; + File zip = null; + try { deleteObsoleteRawSegments(); @@ -1573,10 +1575,10 @@ private void deleteObsoleteRawSegments() { if (nextSegment == -1) continue; - File tmpZip = new File(walArchiveDir, FileDescriptor.fileName(nextSegment) + ".zip" + ".tmp"); + tmpZip = new File(walArchiveDir, FileDescriptor.fileName(nextSegment) + ".zip" + ".tmp"); assert !tmpZip.exists(); - File zip = new File(walArchiveDir, FileDescriptor.fileName(nextSegment) + ".zip"); + zip = new File(walArchiveDir, FileDescriptor.fileName(nextSegment) + ".zip"); assert !zip.exists(); try (SingleSegmentLogicalRecordsIterator iter = new SingleSegmentLogicalRecordsIterator( @@ -1601,7 +1603,13 @@ private void deleteObsoleteRawSegments() { lastCompressedIdx = nextSegment; } catch (IgniteCheckedException | IOException e) { - U.error(log, "Unexpected error during WAL compression", e); // todo what do with ioexception? + U.error(log, "Unexpected error during WAL compression", e); + + if (tmpZip != null) + tmpZip.delete(); + + if (zip != null) + zip.delete(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -1639,8 +1647,13 @@ private class FileDecompressor extends Thread { /** {@inheritDoc} */ @Override public void run() { while (!Thread.currentThread().isInterrupted() && !stopped) { + File unzipTmp = null; + File unzip = null; + + long segmentToDecompress = -1L; + try { - long segmentToDecompress = segmentsQueue.take(); + segmentToDecompress = segmentsQueue.take(); if (stopped) break; @@ -1648,10 +1661,10 @@ private class FileDecompressor extends Thread { File zip = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress) + ".zip"); assert zip.exists(); - File unzipTmp = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress) + ".tmp"); + unzipTmp = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress) + ".tmp"); assert !unzipTmp.exists(); - File unzip = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress)); + unzip = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress)); assert !unzip.exists(); try (ZipInputStream zis = new ZipInputStream(new BufferedInputStream(new FileInputStream(zip))); @@ -1679,7 +1692,17 @@ private class FileDecompressor extends Thread { Thread.currentThread().interrupt(); } catch (IOException e) { - throw new IgniteException(e); // todo decide what do with FNF + U.error(log, "Unexpected error during WAL compression", e); + + unzipTmp.delete(); + unzip.delete(); + + synchronized (this) { + GridFutureAdapter failFut = decompressionFutures.remove(segmentToDecompress); + + if (failFut != null) + failFut.onDone(e); + } } } } From e29e370bc344e5c0ab4f962f62c0351937c56458 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Thu, 16 Nov 2017 16:21:54 +0300 Subject: [PATCH 09/16] IGNITE-5938 Javadoc --- .../persistence/wal/SingleSegmentLogicalRecordsIterator.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java index bd28a5c432ad7..243924a1c6d1e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/SingleSegmentLogicalRecordsIterator.java @@ -22,6 +22,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.pagemem.wal.WALPointer; +import org.apache.ignite.internal.pagemem.wal.record.MarshalledRecord; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; @@ -34,6 +35,7 @@ /** * Iterates over logical records of one WAL segment from archive. Used for WAL archive compression. + * Doesn't deserialize actual record data, returns {@link MarshalledRecord} instances instead. */ public class SingleSegmentLogicalRecordsIterator extends AbstractWalRecordsIterator { /** Serial version uid. */ From dc5b554dae568c543b70ac386a3fcffd00dcfda8 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Thu, 16 Nov 2017 18:09:46 +0300 Subject: [PATCH 10/16] IGNITE-5938 bugs fixed --- .../wal/AbstractWalRecordsIterator.java | 4 +-- .../wal/FileWriteAheadLogManager.java | 29 +++++++++++-------- .../wal/serializer/RecordV2Serializer.java | 5 ++-- 3 files changed, 22 insertions(+), 16 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java index 2a1ce2b971db1..65b7ea460dc4a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java @@ -109,7 +109,7 @@ protected AbstractWalRecordsIterator( * @return found WAL file descriptors */ protected static FileWriteAheadLogManager.FileDescriptor[] loadFileDescriptors(@NotNull final File walFilesDir) throws IgniteCheckedException { - final File[] files = walFilesDir.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER); + final File[] files = walFilesDir.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER); if (files == null) { throw new IgniteCheckedException("WAL files directory does not not denote a " + @@ -291,7 +291,7 @@ protected FileWriteAheadLogManager.ReadFileHandle initReadHandle( if (start != null && desc.idx == start.index()) { if (isCompacted) - serializerFactory.recordDeserializeFilter(new StartSeekingFilter(start)); + serializerFactory.recordDeserializeFilter(new StartSeekingFilter(start)).skipPositionCheck(true); else { // Make sure we skip header with serializer version. long startOff = Math.max(start.fileOffset(), fileIO.position()); 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 a528038ce635f..b01441ad9caa7 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 @@ -139,25 +139,33 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl /** */ private 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()); + } + }; + /** */ private static final Pattern WAL_SEGMENT_TEMP_FILE_COMPACTED_PATTERN = Pattern.compile("\\d{16}\\.wal\\.zip\\.tmp"); /** */ - public static final FileFilter WAL_SEGMENT_FILE_COMPACTED_FILTER = new FileFilter() { + 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(); } }; /** */ - public static final FileFilter WAL_SEGMENT_TEMP_FILE_COMPACTED_FILTER = new FileFilter() { + 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(); } }; /** Latest serializer version to use. */ - public static final int LATEST_SERIALIZER_VERSION = 2; + private static final int LATEST_SERIALIZER_VERSION = 2; /** */ private final boolean alwaysWriteFullPages; @@ -1644,6 +1652,9 @@ private class FileDecompressor extends Thread { /** Segments queue. */ private PriorityBlockingQueue segmentsQueue = new PriorityBlockingQueue<>(); + /** Byte array for draining data. */ + private byte[] arr = new byte[tlbSize]; + /** {@inheritDoc} */ @Override public void run() { while (!Thread.currentThread().isInterrupted() && !stopped) { @@ -1672,14 +1683,8 @@ private class FileDecompressor extends Thread { zis.getNextEntry(); int bytesRead; - - while ((bytesRead = zis.read(tlb.get().array())) > 0) { - tlb.get().clear(); - - tlb.get().limit(bytesRead); - - io.write(tlb.get()); - } + while ((bytesRead = zis.read(arr)) > 0) + io.write(arr, 0, bytesRead); } Files.move(unzipTmp.toPath(), unzip.toPath()); @@ -1692,7 +1697,7 @@ private class FileDecompressor extends Thread { Thread.currentThread().interrupt(); } catch (IOException e) { - U.error(log, "Unexpected error during WAL compression", e); + U.error(log, "Unexpected error during WAL decompression", e); unzipTmp.delete(); unzip.delete(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java index 4cc2a2f347eec..20befe2ddc118 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java @@ -20,6 +20,7 @@ import java.io.DataInput; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.ByteOrder; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.pagemem.wal.WALPointer; import org.apache.ignite.internal.pagemem.wal.record.FilteredRecord; @@ -104,7 +105,7 @@ public class RecordV2Serializer implements RecordSerializer { return new FilteredRecord(); } else if (marshalledMode) { - ByteBuffer buf = ByteBuffer.allocate(ptr.length()); + ByteBuffer buf = ByteBuffer.allocate(ptr.length()).order(ByteOrder.nativeOrder()); buf.put((byte)(recType.ordinal() + 1)); @@ -198,7 +199,7 @@ public static FileWALPointer readPositionAndCheckPoint( FileWALPointer p = (FileWALPointer)expPtr; - if (!F.eq(idx, p.index()) || (skipPositionCheck && !F.eq(fileOffset, p.fileOffset()))) + if (!F.eq(idx, p.index()) || (!skipPositionCheck && !F.eq(fileOffset, p.fileOffset()))) throw new WalSegmentTailReachedException( "WAL segment tail is reached. [ " + "Expected next state: {Index=" + p.index() + ",Offset=" + p.fileOffset() + "}, " + From 2eaaa39c27b0fbcee5e25dec4e70a3088fb975c7 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Fri, 17 Nov 2017 12:46:33 +0300 Subject: [PATCH 11/16] IGNITE-5938 Bug fixed --- .../cache/persistence/wal/FileWriteAheadLogManager.java | 4 ++-- 1 file changed, 2 insertions(+), 2 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 b01441ad9caa7..8853162f15985 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 @@ -809,7 +809,7 @@ private IgniteBiTuple scanMinMaxArchiveIndices() { long minIdx = Integer.MAX_VALUE; long maxIdx = -1; - for (File file : walArchiveDir.listFiles(WAL_SEGMENT_FILE_FILTER)) { + for (File file : walArchiveDir.listFiles(WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER)) { try { long idx = Long.parseLong(file.getName().substring(0, 16)); @@ -1148,7 +1148,7 @@ private class FileArchiver extends Thread { private IgniteCheckedException cleanException; /** - * Absolute current segment index WAL Manger writes to. Guarded by this. + * Absolute current segment index WAL Manager writes to. Guarded by this. * Incremented during rollover. Also may be directly set if WAL is resuming logging after start. */ private long curAbsWalIdx = -1; From 535bbf0ab7890d4188534acc635de501a96fb0c6 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Fri, 17 Nov 2017 18:24:48 +0300 Subject: [PATCH 12/16] IGNITE-5938 Bug fixed --- .../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 8853162f15985..80d1a2fb685db 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 @@ -786,7 +786,7 @@ private boolean hasIndex(long absIdx) { private long lastArchivedIndex() { long lastIdx = -1; - for (File file : walArchiveDir.listFiles(WAL_SEGMENT_FILE_FILTER)) { + for (File file : walArchiveDir.listFiles(WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER)) { try { long idx = Long.parseLong(file.getName().substring(0, 16)); From 203b685ac6fcea0bdd779063752ba61f6e67ea81 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Fri, 17 Nov 2017 20:18:54 +0300 Subject: [PATCH 13/16] IGNITE-5938 debug info --- .../cache/persistence/wal/FileWriteAheadLogManager.java | 5 +++++ 1 file changed, 5 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 80d1a2fb685db..c2479bfae4bfa 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 @@ -316,6 +316,8 @@ public FileWriteAheadLogManager(@NotNull final GridKernalContext ctx) { "write ahead log work directory" ); + log.warning("Work Files:" + Arrays.toString(walWorkDir.listFiles())); + walArchiveDir = initDirectory( dsCfg.getWalArchivePath(), DataStorageConfiguration.DFLT_WAL_ARCHIVE_PATH, @@ -323,6 +325,9 @@ public FileWriteAheadLogManager(@NotNull final GridKernalContext ctx) { "write ahead log archive directory" ); + log.warning("Archive Files:" + Arrays.toString(walWorkDir.listFiles())); + + serializer = new RecordSerializerFactoryImpl(cctx).createSerializer(serializerVersion); GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager)cctx.database(); From 77a2c756a86887457386d188343a2038554e482c Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Fri, 17 Nov 2017 21:16:39 +0300 Subject: [PATCH 14/16] IGNITE-5938 debug info --- .../persistence/wal/FileWriteAheadLogManager.java | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 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 c2479bfae4bfa..385a94c48409f 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 @@ -325,8 +325,7 @@ public FileWriteAheadLogManager(@NotNull final GridKernalContext ctx) { "write ahead log archive directory" ); - log.warning("Archive Files:" + Arrays.toString(walWorkDir.listFiles())); - + log.warning("Archive Files:" + Arrays.toString(walArchiveDir.listFiles())); serializer = new RecordSerializerFactoryImpl(cctx).createSerializer(serializerVersion); @@ -1278,8 +1277,13 @@ private synchronized void release(long absIdx) { long toArchive; synchronized (this) { - assert lastAbsArchivedIdx <= curAbsWalIdx : "lastArchived=" + lastAbsArchivedIdx + - ", current=" + curAbsWalIdx; + if (lastAbsArchivedIdx > curAbsWalIdx) { + String msg = "lastArchived=" + lastAbsArchivedIdx + ", current=" + curAbsWalIdx; + msg += " archive: " + Arrays.toString(walArchiveDir.listFiles()); + msg += " work: " + Arrays.toString(walWorkDir.listFiles()); + + assert lastAbsArchivedIdx <= curAbsWalIdx : msg; + } while (lastAbsArchivedIdx >= curAbsWalIdx - 1 && !stopped) wait(); From 89ad24fecb9e46eed2c2af1aee235a9e9c3c9408 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Fri, 17 Nov 2017 21:20:11 +0300 Subject: [PATCH 15/16] IGNITE-5938 Disabling archivation. --- .../cache/persistence/wal/FileWriteAheadLogManager.java | 4 ++-- 1 file changed, 2 insertions(+), 2 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 385a94c48409f..77744f40670b5 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 @@ -341,9 +341,9 @@ public FileWriteAheadLogManager(@NotNull final GridKernalContext ctx) { archiver = new FileArchiver(tup == null ? -1 : tup.get2()); - compressor = new FileCompressor(); + //compressor = new FileCompressor(); - decompressor = new FileDecompressor(); + //decompressor = new FileDecompressor(); if (mode != WALMode.NONE) { if (log.isInfoEnabled()) From b7df90a085e9ffd360ef6727ec939f48e840faac Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Sat, 18 Nov 2017 00:11:27 +0300 Subject: [PATCH 16/16] IGNITE-5938 disabling archiver --- .../cache/persistence/wal/FileWriteAheadLogManager.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 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 77744f40670b5..5aeca0e817a45 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 @@ -417,11 +417,11 @@ private void checkWalConfiguration() throws IgniteCheckedException { assert archiver != null; archiver.start(); - assert compressor != null; - compressor.start(); + if (compressor != null) + compressor.start(); - assert decompressor != null; - decompressor.start(); + if (decompressor != null) + decompressor.start(); } }