From fdf709505407f3dd509d0b332c9b99effcdcf5b3 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Wed, 1 Jan 2020 20:34:52 -0800 Subject: [PATCH 01/53] [KAFKA-8522] Streamline tombstone and transaction marker removal --- checkstyle/checkstyle.xml | 4 +- .../record/AbstractLegacyRecordBatch.java | 30 ++++ .../common/record/DefaultRecordBatch.java | 95 ++++++++++-- .../kafka/common/record/MemoryRecords.java | 139 ++++++++++++++---- .../common/record/MemoryRecordsBuilder.java | 37 ++++- .../kafka/common/record/RecordBatch.java | 12 ++ .../common/record/MemoryRecordsTest.java | 39 +++++ core/src/main/scala/kafka/log/Log.scala | 3 +- .../src/main/scala/kafka/log/LogCleaner.scala | 102 +++++++++---- .../scala/kafka/log/LogCleanerManager.scala | 30 +++- .../kafka/log/LogCleanerIntegrationTest.scala | 55 ++++++- .../scala/unit/kafka/log/LogCleanerTest.scala | 46 +++--- 12 files changed, 497 insertions(+), 95 deletions(-) diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml index 13cfdb82bd0a8..8b28fcadb17c3 100644 --- a/checkstyle/checkstyle.xml +++ b/checkstyle/checkstyle.xml @@ -107,7 +107,7 @@ - + @@ -124,7 +124,7 @@ - + diff --git a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java index 83637640af49d..cf74f24f490f0 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java @@ -431,6 +431,16 @@ public LegacyRecord outerRecord() { return record; } + @Override + public long deleteHorizonMs() { + return RecordBatch.NO_TIMESTAMP; + } + + @Override + public boolean deleteHorizonSet() { + return false; + } + @Override public boolean equals(Object o) { if (this == o) @@ -468,6 +478,16 @@ public long offset() { return buffer.getLong(OFFSET_OFFSET); } + @Override + public long deleteHorizonMs() { + return RecordBatch.NO_TIMESTAMP; + } + + @Override + public boolean deleteHorizonSet() { + return false; + } + @Override public LegacyRecord outerRecord() { return record; @@ -557,6 +577,16 @@ public long baseOffset() { return loadFullBatch().baseOffset(); } + @Override + public long deleteHorizonMs() { + return RecordBatch.NO_TIMESTAMP; + } + + @Override + public boolean deleteHorizonSet() { + return false; + } + @Override public long lastOffset() { return offset; diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java index 6d79b268575ab..e0d7549fedebc 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java @@ -89,11 +89,15 @@ * by the broker and is preserved after compaction. Additionally, the MaxTimestamp of an empty batch always retains * the previous value prior to becoming empty. * + * The delete horizon flag for the sixth bit is used to determine if the first timestamp of the batch had been set to + * the time for which tombstones / transaction markers needs to be removed. If it is true, then the first timestamp is + * the delete horizon, otherwise, it is merely the first timestamp of the record batch. + * * The current attributes are given below: * - * ------------------------------------------------------------------------------------------------- - * | Unused (6-15) | Control (5) | Transactional (4) | Timestamp Type (3) | Compression Type (0-2) | - * ------------------------------------------------------------------------------------------------- + * --------------------------------------------------------------------------------------------------------------------------- + * | Unused (7-15) | Delete Horizon Flag (6) | Control (5) | Transactional (4) | Timestamp Type (3) | Compression Type (0-2) | + * --------------------------------------------------------------------------------------------------------------------------- */ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRecordBatch { static final int BASE_OFFSET_OFFSET = 0; @@ -128,6 +132,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe private static final byte COMPRESSION_CODEC_MASK = 0x07; private static final byte TRANSACTIONAL_FLAG_MASK = 0x10; private static final int CONTROL_FLAG_MASK = 0x20; + private static final byte DELETE_HORIZON_FLAG_MASK = 0x40; private static final byte TIMESTAMP_TYPE_MASK = 0x08; private static final int MAX_SKIP_BUFFER_SIZE = 2048; @@ -155,10 +160,15 @@ public void ensureValid() { } /** - * Get the timestamp of the first record in this batch. It is always the create time of the record even if the + * Get the timestamp of the first record in this batch. It is usually the create time of the record even if the * timestamp type of the batch is log append time. * - * @return The first timestamp or {@link RecordBatch#NO_TIMESTAMP} if the batch is empty + * There is the possibility that the first timestamp had been set to the delete horizon of the batch, + * in which case, the delete horizon will be returned instead. + * + * @return The first timestamp if the batch's delete horizon has not been set + * The delete horizon if the batch's delete horizon has been set + * {@link RecordBatch#NO_TIMESTAMP} if the batch is empty */ public long firstTimestamp() { return buffer.getLong(FIRST_TIMESTAMP_OFFSET); @@ -245,6 +255,18 @@ public boolean isTransactional() { return (attributes() & TRANSACTIONAL_FLAG_MASK) > 0; } + @Override + public boolean deleteHorizonSet() { + return (attributes() & DELETE_HORIZON_FLAG_MASK) > 0; + } + + @Override + public long deleteHorizonMs() { + if (deleteHorizonSet()) + return firstTimestamp(); + return RecordBatch.NO_TIMESTAMP; + } + @Override public boolean isControlBatch() { return (attributes() & CONTROL_FLAG_MASK) > 0; @@ -360,7 +382,7 @@ public void setMaxTimestamp(TimestampType timestampType, long maxTimestamp) { if (timestampType() == timestampType && currentMaxTimestamp == maxTimestamp) return; - byte attributes = computeAttributes(compressionType(), timestampType, isTransactional(), isControlBatch()); + byte attributes = computeAttributes(compressionType(), timestampType, isTransactional(), isControlBatch(), deleteHorizonSet()); buffer.putShort(ATTRIBUTES_OFFSET, attributes); buffer.putLong(MAX_TIMESTAMP_OFFSET, maxTimestamp); long crc = computeChecksum(); @@ -407,7 +429,7 @@ public int hashCode() { } private static byte computeAttributes(CompressionType type, TimestampType timestampType, - boolean isTransactional, boolean isControl) { + boolean isTransactional, boolean isControl, boolean isDeleteHorizonSet) { if (timestampType == TimestampType.NO_TIMESTAMP_TYPE) throw new IllegalArgumentException("Timestamp type must be provided to compute attributes for message " + "format v2 and above"); @@ -419,6 +441,8 @@ private static byte computeAttributes(CompressionType type, TimestampType timest attributes |= COMPRESSION_CODEC_MASK & type.id; if (timestampType == TimestampType.LOG_APPEND_TIME) attributes |= TIMESTAMP_TYPE_MASK; + if (isDeleteHorizonSet) + attributes |= DELETE_HORIZON_FLAG_MASK; return attributes; } @@ -435,9 +459,49 @@ public static void writeEmptyHeader(ByteBuffer buffer, boolean isTransactional, boolean isControlRecord) { int offsetDelta = (int) (lastOffset - baseOffset); + writeHeader(buffer, baseOffset, offsetDelta, DefaultRecordBatch.RECORD_BATCH_OVERHEAD, magic, + CompressionType.NONE, timestampType, RecordBatch.NO_TIMESTAMP, timestamp, producerId, + producerEpoch, baseSequence, isTransactional, isControlRecord, false, partitionLeaderEpoch, 0); + } + + public static void writeEmptyHeader(ByteBuffer buffer, + byte magic, + long producerId, + short producerEpoch, + int baseSequence, + long baseOffset, + long lastOffset, + int partitionLeaderEpoch, + TimestampType timestampType, + long timestamp, + boolean isTransactional, + boolean isControlRecord, + boolean isDeleteHorizonSet) { + int offsetDelta = (int) (lastOffset - baseOffset); writeHeader(buffer, baseOffset, offsetDelta, DefaultRecordBatch.RECORD_BATCH_OVERHEAD, magic, CompressionType.NONE, timestampType, RecordBatch.NO_TIMESTAMP, timestamp, producerId, - producerEpoch, baseSequence, isTransactional, isControlRecord, partitionLeaderEpoch, 0); + producerEpoch, baseSequence, isTransactional, isControlRecord, isDeleteHorizonSet, partitionLeaderEpoch, 0); + } + + static void writeHeader(ByteBuffer buffer, + long baseOffset, + int lastOffsetDelta, + int sizeInBytes, + byte magic, + CompressionType compressionType, + TimestampType timestampType, + long firstTimestamp, + long maxTimestamp, + long producerId, + short epoch, + int sequence, + boolean isTransactional, + boolean isControlBatch, + int partitionLeaderEpoch, + int numRecords) { + writeHeader(buffer, baseOffset, lastOffsetDelta, sizeInBytes, magic, compressionType, + timestampType, firstTimestamp, maxTimestamp, producerId, epoch, sequence, + isTransactional, isControlBatch, false, partitionLeaderEpoch, numRecords); } static void writeHeader(ByteBuffer buffer, @@ -454,6 +518,7 @@ static void writeHeader(ByteBuffer buffer, int sequence, boolean isTransactional, boolean isControlBatch, + boolean isDeleteHorizonSet, int partitionLeaderEpoch, int numRecords) { if (magic < RecordBatch.CURRENT_MAGIC_VALUE) @@ -461,7 +526,7 @@ static void writeHeader(ByteBuffer buffer, if (firstTimestamp < 0 && firstTimestamp != NO_TIMESTAMP) throw new IllegalArgumentException("Invalid message timestamp " + firstTimestamp); - short attributes = computeAttributes(compressionType, timestampType, isTransactional, isControlBatch); + short attributes = computeAttributes(compressionType, timestampType, isTransactional, isControlBatch, isDeleteHorizonSet); int position = buffer.position(); buffer.putLong(position + BASE_OFFSET_OFFSET, baseOffset); @@ -699,6 +764,18 @@ public boolean isTransactional() { return loadBatchHeader().isTransactional(); } + @Override + public boolean deleteHorizonSet() { + return loadBatchHeader().deleteHorizonSet(); + } + + @Override + public long deleteHorizonMs() { + if (deleteHorizonSet()) + return super.loadBatchHeader().deleteHorizonMs(); + return RecordBatch.NO_TIMESTAMP; + } + @Override public boolean isControlBatch() { return loadBatchHeader().isControlBatch(); diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index 8f73565d1b40f..7fb5fe951b9f0 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -150,15 +150,29 @@ public FilterResult filterTo(TopicPartition partition, RecordFilter filter, Byte return filterTo(partition, batches(), filter, destinationBuffer, maxRecordBatchSize, decompressionBufferSupplier); } + /** + * Note: This method is also used to convert the first timestamp of the batch (which is usually the timestamp of the first record) + * to the delete horizon of the tombstones which are present in the batch. + */ private static FilterResult filterTo(TopicPartition partition, Iterable batches, RecordFilter filter, ByteBuffer destinationBuffer, int maxRecordBatchSize, BufferSupplier decompressionBufferSupplier) { FilterResult filterResult = new FilterResult(destinationBuffer); ByteBufferOutputStream bufferOutputStream = new ByteBufferOutputStream(destinationBuffer); - for (MutableRecordBatch batch : batches) { long maxOffset = -1L; - BatchRetention batchRetention = filter.checkBatchRetention(batch); + // we first call this method here so that the flag in LogCleaner has been set + // which indicates if the control batch is empty or not + // we do this to avoid calling CleanedTransactionMetadata#onControlBatchRead + // more than once since each call is relatively expensive + filter.isControlBatchEmpty(batch); + long deleteHorizonMs = filter.retrieveDeleteHorizon(batch); + final BatchRetention batchRetention; + if (!batch.deleteHorizonSet()) + batchRetention = filter.checkBatchRetention(batch, deleteHorizonMs); + else + batchRetention = filter.checkBatchRetention(batch); + filterResult.bytesRead += batch.sizeInBytes(); if (batchRetention == BatchRetention.DELETE) @@ -170,36 +184,27 @@ private static FilterResult filterTo(TopicPartition partition, Iterable retainedRecords = new ArrayList<>(); - try (final CloseableIterator iterator = batch.streamingIterator(decompressionBufferSupplier)) { - while (iterator.hasNext()) { - Record record = iterator.next(); - filterResult.messagesRead += 1; - - if (filter.shouldRetainRecord(batch, record)) { - // Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite - // the corrupted batch with correct data. - if (!record.hasMagic(batchMagic)) - writeOriginalBatch = false; - - if (record.offset() > maxOffset) - maxOffset = record.offset(); - - retainedRecords.add(record); - } else { - writeOriginalBatch = false; - } - } - } + final BatchIterationResult iterationResult = iterateOverBatch(batch, decompressionBufferSupplier, filterResult, filter, + batchMagic, writeOriginalBatch, maxOffset, retainedRecords, + containsTombstonesOrMarker, deleteHorizonMs); + containsTombstonesOrMarker = iterationResult.containsTombstonesOrMarker(); + writeOriginalBatch = iterationResult.shouldWriteOriginalBatch(); + maxOffset = iterationResult.maxOffset(); if (!retainedRecords.isEmpty()) { - if (writeOriginalBatch) { + // we check if the delete horizon should be set to a new value + // in which case, we need to reset the base timestamp and overwrite the timestamp deltas + // if the batch does not contain tombstones, then we don't need to overwrite batch + if (writeOriginalBatch && (deleteHorizonMs == RecordBatch.NO_TIMESTAMP || deleteHorizonMs == batch.deleteHorizonMs() || !containsTombstonesOrMarker)) { batch.writeTo(bufferOutputStream); filterResult.updateRetainedBatchMetadata(batch, retainedRecords.size(), false); } else { - MemoryRecordsBuilder builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream); + MemoryRecordsBuilder builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, deleteHorizonMs); MemoryRecords records = builder.build(); int filteredBatchSize = records.sizeInBytes(); if (filteredBatchSize > batch.sizeInBytes() && filteredBatchSize > maxRecordBatchSize) @@ -236,9 +241,69 @@ private static FilterResult filterTo(TopicPartition partition, Iterable retainedRecords, + boolean containsTombstonesOrMarker, + long newBatchDeleteHorizonMs) { + try (final CloseableIterator iterator = batch.streamingIterator(decompressionBufferSupplier)) { + while (iterator.hasNext()) { + Record record = iterator.next(); + filterResult.messagesRead += 1; + + if (filter.shouldRetainRecord(batch, record, newBatchDeleteHorizonMs)) { + // Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite + // the corrupted batch with correct data. + if (!record.hasMagic(batchMagic)) + writeOriginalBatch = false; + + if (record.offset() > maxOffset) + maxOffset = record.offset(); + + retainedRecords.add(record); + + if (!record.hasValue()) { + containsTombstonesOrMarker = true; + } + } else { + writeOriginalBatch = false; + } + } + return new BatchIterationResult(writeOriginalBatch, containsTombstonesOrMarker, maxOffset); + } + } + + private static class BatchIterationResult { + private final boolean writeOriginalBatch; + private final boolean containsTombstonesOrMarker; + private final long maxOffset; + public BatchIterationResult(final boolean writeOriginalBatch, + final boolean containsTombstonesOrMarker, + final long maxOffset) { + this.writeOriginalBatch = writeOriginalBatch; + this.containsTombstonesOrMarker = containsTombstonesOrMarker; + this.maxOffset = maxOffset; + } + public boolean shouldWriteOriginalBatch() { + return this.writeOriginalBatch; + } + public boolean containsTombstonesOrMarker() { + return this.containsTombstonesOrMarker; + } + public long maxOffset() { + return this.maxOffset; + } + } + private static MemoryRecordsBuilder buildRetainedRecordsInto(RecordBatch originalBatch, List retainedRecords, - ByteBufferOutputStream bufferOutputStream) { + ByteBufferOutputStream bufferOutputStream, + final long deleteHorizonMs) { byte magic = originalBatch.magic(); TimestampType timestampType = originalBatch.timestampType(); long logAppendTime = timestampType == TimestampType.LOG_APPEND_TIME ? @@ -249,7 +314,7 @@ private static MemoryRecordsBuilder buildRetainedRecordsInto(RecordBatch origina MemoryRecordsBuilder builder = new MemoryRecordsBuilder(bufferOutputStream, magic, originalBatch.compressionType(), timestampType, baseOffset, logAppendTime, originalBatch.producerId(), originalBatch.producerEpoch(), originalBatch.baseSequence(), originalBatch.isTransactional(), - originalBatch.isControlBatch(), originalBatch.partitionLeaderEpoch(), bufferOutputStream.limit()); + originalBatch.isControlBatch(), originalBatch.partitionLeaderEpoch(), bufferOutputStream.limit(), deleteHorizonMs); for (Record record : retainedRecords) builder.append(record); @@ -312,12 +377,34 @@ public enum BatchRetention { */ protected abstract BatchRetention checkBatchRetention(RecordBatch batch); + protected BatchRetention checkBatchRetention(RecordBatch batch, long newBatchDeleteHorizonMs) { + return checkBatchRetention(batch); + } + /** * Check whether a record should be retained in the log. Note that {@link #checkBatchRetention(RecordBatch)} * is used prior to checking individual record retention. Only records from batches which were not * explicitly discarded with {@link BatchRetention#DELETE} will be considered. */ protected abstract boolean shouldRetainRecord(RecordBatch recordBatch, Record record); + + protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record, long newDeleteHorizonMs) { + return shouldRetainRecord(recordBatch, record); + } + + /** + * Retrieves the delete horizon ms for a specific batch + */ + protected long retrieveDeleteHorizon(RecordBatch recordBatch) { + return -1L; + } + + /** + * Checks if the control batch (if it is one) can be removed (making sure that it is empty) + */ + protected boolean isControlBatchEmpty(RecordBatch recordBatch) { + return true; + } } public static class FilterResult { diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java index 054fb86199884..be32a83905935 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java @@ -75,6 +75,7 @@ public void write(int b) { private int numRecords = 0; private float actualCompressionRatio = 1; private long maxTimestamp = RecordBatch.NO_TIMESTAMP; + private long deleteHorizonMs; private long offsetOfMaxTimestamp = -1; private Long lastOffset = null; private Long firstTimestamp = null; @@ -94,7 +95,8 @@ public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, boolean isTransactional, boolean isControlBatch, int partitionLeaderEpoch, - int writeLimit) { + int writeLimit, + long deleteHorizonMs) { if (magic > RecordBatch.MAGIC_VALUE_V0 && timestampType == TimestampType.NO_TIMESTAMP_TYPE) throw new IllegalArgumentException("TimestampType must be set for magic >= 0"); if (magic < RecordBatch.MAGIC_VALUE_V2) { @@ -120,6 +122,7 @@ public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, this.baseSequence = baseSequence; this.isTransactional = isTransactional; this.isControlBatch = isControlBatch; + this.deleteHorizonMs = deleteHorizonMs; this.partitionLeaderEpoch = partitionLeaderEpoch; this.writeLimit = writeLimit; this.initialPosition = bufferStream.position(); @@ -130,6 +133,24 @@ public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, this.appendStream = new DataOutputStream(compressionType.wrapForOutput(this.bufferStream, magic)); } + public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, + byte magic, + CompressionType compressionType, + TimestampType timestampType, + long baseOffset, + long logAppendTime, + long producerId, + short producerEpoch, + int baseSequence, + boolean isTransactional, + boolean isControlBatch, + int partitionLeaderEpoch, + int writeLimit) { + this(bufferStream, magic, compressionType, timestampType, baseOffset, logAppendTime, producerId, + producerEpoch, baseSequence, isTransactional, isControlBatch, partitionLeaderEpoch, writeLimit, + RecordBatch.NO_TIMESTAMP); + } + /** * Construct a new builder. * @@ -192,6 +213,10 @@ public boolean isTransactional() { return isTransactional; } + public boolean deleteHorizonSet() { + return deleteHorizonMs >= 0L; + } + /** * Close this builder and return the resulting buffer. * @return The built log buffer @@ -364,7 +389,7 @@ private int writeDefaultBatchHeader() { maxTimestamp = this.maxTimestamp; DefaultRecordBatch.writeHeader(buffer, baseOffset, offsetDelta, size, magic, compressionType, timestampType, - firstTimestamp, maxTimestamp, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, + firstTimestamp, maxTimestamp, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, deleteHorizonSet(), partitionLeaderEpoch, numRecords); buffer.position(pos); @@ -411,8 +436,12 @@ private Long appendWithOffset(long offset, boolean isControlRecord, long timesta if (magic < RecordBatch.MAGIC_VALUE_V2 && headers != null && headers.length > 0) throw new IllegalArgumentException("Magic v" + magic + " does not support record headers"); - if (firstTimestamp == null) - firstTimestamp = timestamp; + if (firstTimestamp == null) { + if (deleteHorizonSet()) + firstTimestamp = deleteHorizonMs; + else + firstTimestamp = timestamp; + } if (magic > RecordBatch.MAGIC_VALUE_V1) { appendDefaultRecord(offset, timestamp, key, value, headers); diff --git a/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java index 65a6a95fbe41f..45f1609e3bc29 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java @@ -210,6 +210,18 @@ public interface RecordBatch extends Iterable { */ boolean isTransactional(); + /** + * Whether or not the base timestamp has been set to the delete horizon + * @return true if it is, false otherwise + */ + boolean deleteHorizonSet(); + + /** + * Get the delete horizon, returns -1L if the first timestamp is not the delete horizon + * @return timestamp of the delete horizon + */ + long deleteHorizonMs(); + /** * Get the partition leader epoch of this record batch. * @return The leader epoch or -1 if it is unknown diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java index b8824d3a8276c..4677e40e388e1 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java @@ -247,6 +247,45 @@ public void testFilterToPreservesPartitionLeaderEpoch() { } } + /** + * This test is used to see if the first timestamp of the batch has been successfully + * converted to a delete horizon for the tombstones / transaction markers of the batch. + */ + @Test + public void testFirstTimestampToDeleteHorizonConversion() { + if (magic >= RecordBatch.MAGIC_VALUE_V2) { + ByteBuffer buffer = ByteBuffer.allocate(2048); + MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, + 0L, RecordBatch.NO_TIMESTAMP, partitionLeaderEpoch); + builder.append(10L, "1".getBytes(), null); + + ByteBuffer filtered = ByteBuffer.allocate(2048); + final long deleteHorizon = Integer.MAX_VALUE / 2; + builder.build().filterTo(new TopicPartition("random", 0), new MemoryRecords.RecordFilter() { + @Override + protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { + return true; + } + + @Override + protected BatchRetention checkBatchRetention(RecordBatch batch) { + return BatchRetention.RETAIN_EMPTY; + } + + @Override + protected long retrieveDeleteHorizon(RecordBatch batch) { + return deleteHorizon; // arbitrary value > 1 + } + }, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); + filtered.flip(); + MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); + + List batches = TestUtils.toList(filteredRecords.batches()); + assertEquals(1, batches.size()); + assertEquals(deleteHorizon, batches.get(0).deleteHorizonMs()); + } + } + @Test public void testFilterToEmptyBatchRetention() { if (magic >= RecordBatch.MAGIC_VALUE_V2) { diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 97fa743d50f15..2b1b292fb189f 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -221,7 +221,8 @@ class Log(@volatile var dir: File, val producerIdExpirationCheckIntervalMs: Int, val topicPartition: TopicPartition, val producerStateManager: ProducerStateManager, - logDirFailureChannel: LogDirFailureChannel) extends Logging with KafkaMetricsGroup { + logDirFailureChannel: LogDirFailureChannel, + @volatile var containsTombstones: Boolean = false) extends Logging with KafkaMetricsGroup { import kafka.log.Log._ diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index bcb358631c8c5..292139868bdf3 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -498,11 +498,10 @@ private[log] class Cleaner(val id: Int, case None => 0L case Some(seg) => seg.lastModified - cleanable.log.config.deleteRetentionMs } - - doClean(cleanable, deleteHorizonMs) + doClean(cleanable, time.milliseconds(), trackedHorizon = deleteHorizonMs) } - private[log] def doClean(cleanable: LogToClean, deleteHorizonMs: Long): (Long, CleanerStats) = { + private[log] def doClean(cleanable: LogToClean, currentTime: Long, trackedHorizon: Long = -1L): (Long, CleanerStats) = { info("Beginning cleaning of log %s.".format(cleanable.log.name)) val log = cleanable.log @@ -520,13 +519,13 @@ private[log] class Cleaner(val id: Int, val cleanableHorizonMs = log.logSegments(0, cleanable.firstUncleanableOffset).lastOption.map(_.lastModified).getOrElse(0L) // group the segments and clean the groups - info("Cleaning log %s (cleaning prior to %s, discarding tombstones prior to %s)...".format(log.name, new Date(cleanableHorizonMs), new Date(deleteHorizonMs))) + info("Cleaning log %s (cleaning prior to %s, discarding tombstones prior to %s)...".format(log.name, new Date(cleanableHorizonMs), new Date(trackedHorizon))) val transactionMetadata = new CleanedTransactionMetadata val groupedSegments = groupSegmentsBySize(log.logSegments(0, endOffset), log.config.segmentSize, log.config.maxIndexSize, cleanable.firstUncleanableOffset) for (group <- groupedSegments) - cleanSegments(log, group, offsetMap, deleteHorizonMs, stats, transactionMetadata) + cleanSegments(log, group, offsetMap, currentTime, stats, transactionMetadata, trackedHorizon = trackedHorizon) // record buffer utilization stats.bufferUtilization = offsetMap.utilization @@ -550,9 +549,10 @@ private[log] class Cleaner(val id: Int, private[log] def cleanSegments(log: Log, segments: Seq[LogSegment], map: OffsetMap, - deleteHorizonMs: Long, + currentTime: Long, stats: CleanerStats, - transactionMetadata: CleanedTransactionMetadata): Unit = { + transactionMetadata: CleanedTransactionMetadata, + trackedHorizon: Long = -1L): Unit = { // create a new segment with a suffix appended to the name of the log and indexes val cleaned = LogCleaner.createNewCleanedSegment(log, segments.head.baseOffset) transactionMetadata.cleanedIndex = Some(cleaned.txnIndex) @@ -572,14 +572,15 @@ private[log] class Cleaner(val id: Int, val abortedTransactions = log.collectAbortedTransactions(startOffset, upperBoundOffset) transactionMetadata.addAbortedTransactions(abortedTransactions) - val retainDeletesAndTxnMarkers = currentSegment.lastModified > deleteHorizonMs + val retainDeletesAndTxnMarkers = currentSegment.lastModified > trackedHorizon info(s"Cleaning $currentSegment in log ${log.name} into ${cleaned.baseOffset} " + - s"with deletion horizon $deleteHorizonMs, " + + s"with deletion horizon $trackedHorizon, " + s"${if(retainDeletesAndTxnMarkers) "retaining" else "discarding"} deletes.") try { - cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletesAndTxnMarkers, log.config.maxMessageSize, - transactionMetadata, lastOffsetOfActiveProducers, stats) + val containsTombstones: Boolean = cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletesAndTxnMarkers, log.config.maxMessageSize, + transactionMetadata, lastOffsetOfActiveProducers, stats, log.config.deleteRetentionMs, currentTime = currentTime) + log.containsTombstones = containsTombstones } catch { case e: LogSegmentOffsetOverflowException => // Split the current segment. It's also safest to abort the current cleaning process, so that we retry from @@ -623,6 +624,7 @@ private[log] class Cleaner(val id: Int, * @param retainDeletesAndTxnMarkers Should tombstones and markers be retained while cleaning this segment * @param maxLogMessageSize The maximum message size of the corresponding topic * @param stats Collector for cleaning statistics + * @param tombstoneRetentionMs Defines how long a tombstone should be kept as defined by log configuration */ private[log] def cleanInto(topicPartition: TopicPartition, sourceRecords: FileRecords, @@ -632,14 +634,33 @@ private[log] class Cleaner(val id: Int, maxLogMessageSize: Int, transactionMetadata: CleanedTransactionMetadata, lastRecordsOfActiveProducers: Map[Long, LastRecord], - stats: CleanerStats): Unit = { + stats: CleanerStats, + tombstoneRetentionMs: Long, + currentTime: Long = RecordBatch.NO_TIMESTAMP): Boolean = { + var containsTombstones: Boolean = false + val logCleanerFilter: RecordFilter = new RecordFilter { var discardBatchRecords: Boolean = _ + var isControlBatchEmpty: Boolean = _ - override def checkBatchRetention(batch: RecordBatch): BatchRetention = { + override def isControlBatchEmpty(batch: RecordBatch) : Boolean = { // we piggy-back on the tombstone retention logic to delay deletion of transaction markers. // note that we will never delete a marker until all the records from that transaction are removed. - discardBatchRecords = shouldDiscardBatch(batch, transactionMetadata, retainTxnMarkers = retainDeletesAndTxnMarkers) + val canDiscardBatch = shouldDiscardBatch(batch, transactionMetadata, retainTxnMarkers = retainDeletesAndTxnMarkers) + isControlBatchEmpty = canDiscardBatch + isControlBatchEmpty + } + + override def checkBatchRetention(batch: RecordBatch, newBatchDeleteHorizonMs : Long): BatchRetention = { + val canDiscardBatch = isControlBatchEmpty + + if (batch.isControlBatch) { + discardBatchRecords = canDiscardBatch && + ((batch.deleteHorizonSet() && batch.deleteHorizonMs() < currentTime) || + newBatchDeleteHorizonMs != RecordBatch.NO_TIMESTAMP && newBatchDeleteHorizonMs < currentTime) + } else { + discardBatchRecords = canDiscardBatch + } def isBatchLastRecordOfProducer: Boolean = { // We retain the batch in order to preserve the state of active producers. There are three cases: @@ -664,12 +685,33 @@ private[log] class Cleaner(val id: Int, BatchRetention.DELETE_EMPTY } - override def shouldRetainRecord(batch: RecordBatch, record: Record): Boolean = { + override def checkBatchRetention(batch: RecordBatch): BatchRetention = checkBatchRetention(batch, batch.deleteHorizonMs()) + + override def shouldRetainRecord(batch: RecordBatch, record: Record, newDeleteHorizonMs: Long): Boolean = { + var isRecordRetained: Boolean = true if (discardBatchRecords) // The batch is only retained to preserve producer sequence information; the records can be removed - false + isRecordRetained = false else - Cleaner.this.shouldRetainRecord(map, retainDeletesAndTxnMarkers, batch, record, stats) + isRecordRetained = Cleaner.this.shouldRetainRecord(map, retainDeletesAndTxnMarkers, batch, record, stats, newDeleteHorizonMs, currentTime = currentTime) + if (isRecordRetained && !record.hasValue()) + containsTombstones = true + isRecordRetained + } + + override def shouldRetainRecord(batch: RecordBatch, record: Record): Boolean = { + shouldRetainRecord(batch, record, RecordBatch.NO_TIMESTAMP) + } + + override def retrieveDeleteHorizon(batch: RecordBatch) : Long = { + if (batch.deleteHorizonSet()) + return batch.deleteHorizonMs() // means that we keep the old timestamp stored + + // check that the control batch has been emptied of records + // if not, then we do not set a delete horizon until that is true + if (batch.isControlBatch() && !isControlBatchEmpty) + return -1L + return time.milliseconds() + tombstoneRetentionMs; } } @@ -709,6 +751,7 @@ private[log] class Cleaner(val id: Int, growBuffersOrFail(sourceRecords, position, maxLogMessageSize, records) } restoreBuffers() + containsTombstones } @@ -747,20 +790,19 @@ private[log] class Cleaner(val id: Int, private def shouldDiscardBatch(batch: RecordBatch, transactionMetadata: CleanedTransactionMetadata, retainTxnMarkers: Boolean): Boolean = { - if (batch.isControlBatch) { - val canDiscardControlBatch = transactionMetadata.onControlBatchRead(batch) - canDiscardControlBatch && !retainTxnMarkers - } else { - val canDiscardBatch = transactionMetadata.onBatchRead(batch) - canDiscardBatch - } + if (batch.isControlBatch) + transactionMetadata.onControlBatchRead(batch) + else + transactionMetadata.onBatchRead(batch) } private def shouldRetainRecord(map: kafka.log.OffsetMap, retainDeletes: Boolean, batch: RecordBatch, record: Record, - stats: CleanerStats): Boolean = { + stats: CleanerStats, + newBatchDeleteHorizonMs: Long, + currentTime: Long = -1L): Boolean = { val pastLatestOffset = record.offset > map.latestOffset if (pastLatestOffset) return true @@ -774,7 +816,15 @@ private[log] class Cleaner(val id: Int, * 2) The message doesn't has value but it can't be deleted now. */ val latestOffsetForKey = record.offset() >= foundOffset - val isRetainedValue = record.hasValue || retainDeletes + val isLatestVersion = batch.magic() >= RecordBatch.MAGIC_VALUE_V2 + var shouldRetainDeletes = true + if (isLatestVersion) + shouldRetainDeletes = (batch.deleteHorizonSet() && currentTime < batch.deleteHorizonMs()) || + (!batch.deleteHorizonSet() && currentTime < newBatchDeleteHorizonMs) + else + shouldRetainDeletes = retainDeletes + val isRetainedValue = record.hasValue || shouldRetainDeletes + latestOffsetForKey && isRetainedValue } else { stats.invalidMessage() diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala index 4d49e1ffac678..561a6437d91ae 100755 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -181,6 +181,7 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], case (_, log) => log.config.compact // match logs that are marked as compacted }.filterNot { case (topicPartition, log) => + // skip any logs already in-progress and uncleanable partitions inProgress.contains(topicPartition) || isUncleanablePartition(log, topicPartition) }.map { @@ -204,7 +205,34 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], (ltc.needCompactionNow && ltc.cleanableBytes > 0) || ltc.cleanableRatio > ltc.log.config.minCleanableRatio } if(cleanableLogs.isEmpty) { - None + // in this case, we are probably in a low throughput situation + // therefore, we should take advantage of this fact and remove tombstones if we can + val logsContainingTombstones = logs.filter { + case (_, log) => log.containsTombstones + }.filterNot { + case (topicPartition, log) => + // skip any logs already in-progress and uncleanable partitions + inProgress.contains(topicPartition) || isUncleanablePartition(log, topicPartition) + }.map { + case (topicPartition, log) => // create a LogToClean instance for each + try { + val (firstDirtyOffset, firstUncleanableDirtyOffset) = cleanableOffsets(log, topicPartition, lastClean, now) + val compactionDelayMs = maxCompactionDelay(log, firstDirtyOffset, now) + preCleanStats.updateMaxCompactionDelay(compactionDelayMs) + + LogToClean(topicPartition, log, firstDirtyOffset, firstUncleanableDirtyOffset, compactionDelayMs > 0) + } catch { + case e: Throwable => throw new LogCleaningException(log, + s"Failed to calculate log cleaning stats for partition $topicPartition", e) + } + } + if (!logsContainingTombstones.isEmpty) { + val filthiest = logsContainingTombstones.max + inProgress.put(filthiest.topicPartition, LogCleaningInProgress) + Some(filthiest) + } else { + None + } } else { preCleanStats.recordCleanablePartitions(cleanableLogs.size) val filthiest = cleanableLogs.max diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 7aea6c1fcdd80..f51c3f126cf08 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -18,6 +18,7 @@ package kafka.log import java.io.PrintWriter +import java.util.Properties import com.yammer.metrics.Metrics import com.yammer.metrics.core.{Gauge, MetricName} @@ -178,6 +179,49 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K s"but lastCleaned=$lastCleaned2", lastCleaned2 >= secondBlockCleanableSegmentOffset) } + @Test + def testTombstoneCleanWithLowThoroughput() : Unit = { + val tombstoneRetentionMs = 1000 // this is in milliseconds -> 1 second + + val topicPartitions = Array(new TopicPartition("log-partition", 0)) + val props = new Properties() + props.put(LogConfig.DeleteRetentionMsProp, "1000") + cleaner = makeCleaner(partitions = topicPartitions, propertyOverrides = props) + + val log = cleaner.logs.get(topicPartitions(0)) + + val T0 = time.milliseconds + writeKeyDups(numKeys = 1, numDups = 1, log, CompressionType.NONE, timestamp = T0, + startValue = 0, step = 1, isRecordTombstone = true) + + val activeSegAtT0 = log.activeSegment + + // roll the active segment + log.roll() + + cleaner.startup() + Thread.sleep(100) + + import JavaConverters._ + var containsTombstones: Boolean = false + for (segment <- log.logSegments; record <- segment.log.records.asScala) { + containsTombstones = true + } + assertTrue(containsTombstones) + time.sleep(tombstoneRetentionMs + 1) + + val firstBlockCleanableSegmentOffset = activeSegAtT0.baseOffset + + // the first block should get cleaned + cleaner.awaitCleaned(new TopicPartition("log-partition", 0), + firstBlockCleanableSegmentOffset, maxWaitMs = tombstoneRetentionMs * 3) + + for (segment <- log.logSegments; record <- segment.log.records.asScala) { + fail ("The log should not contain record " + record + ", tombstone has expired its lifetime.") + } + assertFalse(log.containsTombstones) + } + private def readFromLog(log: Log): Iterable[(Int, Int)] = { import JavaConverters._ for (segment <- log.logSegments; record <- segment.log.records.asScala) yield { @@ -187,12 +231,17 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K } } - private def writeKeyDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionType, timestamp: Long, startValue: Int, step: Int): Seq[(Int, Int)] = { + private def writeKeyDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionType, timestamp: Long, + startValue: Int, step: Int, isRecordTombstone: Boolean = false): Seq[(Int, Int)] = { var valCounter = startValue for (_ <- 0 until numDups; key <- 0 until numKeys) yield { val curValue = valCounter - log.appendAsLeader(TestUtils.singletonRecords(value = curValue.toString.getBytes, codec = codec, - key = key.toString.getBytes, timestamp = timestamp), leaderEpoch = 0) + if (isRecordTombstone) + log.appendAsLeader(TestUtils.singletonRecords(value = null, codec = codec, + key = key.toString.getBytes, timestamp = timestamp), leaderEpoch = 0) + else + log.appendAsLeader(TestUtils.singletonRecords(value = curValue.toString.getBytes, codec = codec, + key = key.toString.getBytes, timestamp = timestamp), leaderEpoch = 0) valCounter += step (key, curValue) } diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index bb30287bb12bd..7d03b07fd7010 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -347,7 +347,7 @@ class LogCleanerTest { log.roll() // cannot remove the marker in this pass because there are still valid records - var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 + var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 assertEquals(List(1, 3, 2), LogTest.keysInLog(log)) assertEquals(List(0, 2, 3, 4, 5), offsetsInLog(log)) @@ -356,17 +356,17 @@ class LogCleanerTest { log.roll() // the first cleaning preserves the commit marker (at offset 3) since there were still records for the transaction - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) // delete horizon forced to 0 to verify marker is not removed early - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = 0L)._1 + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = 0L)._1 assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) // clean again with large delete horizon and verify the marker is removed - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) assertEquals(List(4, 5, 6, 7, 8), offsetsInLog(log)) } @@ -395,11 +395,11 @@ class LogCleanerTest { log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, isFromClient = false) log.roll() - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) + cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) assertEquals(List(2), LogTest.keysInLog(log)) assertEquals(List(1, 3, 4), offsetsInLog(log)) - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) + cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) assertEquals(List(2), LogTest.keysInLog(log)) assertEquals(List(3, 4), offsetsInLog(log)) } @@ -434,14 +434,14 @@ class LogCleanerTest { // first time through the records are removed // Expected State: [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}] - var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 + var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 assertEquals(List(2, 3), LogTest.keysInLog(log)) assertEquals(List(4, 5, 6, 7), offsetsInLog(log)) assertEquals(List(1, 3, 4, 5, 6, 7), lastOffsetsPerBatchInLog(log)) // the empty batch remains if cleaned again because it still holds the last sequence // Expected State: [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}] - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 assertEquals(List(2, 3), LogTest.keysInLog(log)) assertEquals(List(4, 5, 6, 7), offsetsInLog(log)) assertEquals(List(1, 3, 4, 5, 6, 7), lastOffsetsPerBatchInLog(log)) @@ -454,13 +454,13 @@ class LogCleanerTest { log.roll() // Expected State: [{Producer1: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) assertEquals(List(4, 5, 6, 7, 8, 9), offsetsInLog(log)) assertEquals(List(1, 4, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log)) // Expected State: [{Producer1: EmptyBatch}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) assertEquals(List(5, 6, 7, 8, 9), offsetsInLog(log)) assertEquals(List(1, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log)) @@ -484,14 +484,14 @@ class LogCleanerTest { // first time through the control batch is retained as an empty batch // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] - var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 + var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 assertEquals(List(2, 3), LogTest.keysInLog(log)) assertEquals(List(1, 2), offsetsInLog(log)) assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) // the empty control batch does not cause an exception when cleaned // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 assertEquals(List(2, 3), LogTest.keysInLog(log)) assertEquals(List(1, 2), offsetsInLog(log)) assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) @@ -515,7 +515,7 @@ class LogCleanerTest { log.roll() // Both the record and the marker should remain after cleaning - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) + cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) assertEquals(List(0, 1), offsetsInLog(log)) assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) } @@ -540,12 +540,12 @@ class LogCleanerTest { // Both the batch and the marker should remain after cleaning. The batch is retained // because it is the last entry for this producerId. The marker is retained because // there are still batches remaining from this transaction. - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) + cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) assertEquals(List(1), offsetsInLog(log)) assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) // The empty batch and the marker is still retained after a second cleaning. - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) + cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) assertEquals(List(1), offsetsInLog(log)) assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) } @@ -570,12 +570,12 @@ class LogCleanerTest { log.roll() // delete horizon set to 0 to verify marker is not removed early - val dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = 0L)._1 + val dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = 0L)._1 assertEquals(List(3), LogTest.keysInLog(log)) assertEquals(List(3, 4, 5), offsetsInLog(log)) // clean again with large delete horizon and verify the marker is removed - cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) + cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue) assertEquals(List(3), LogTest.keysInLog(log)) assertEquals(List(4, 5), offsetsInLog(log)) } @@ -609,12 +609,12 @@ class LogCleanerTest { // Both transactional batches will be cleaned. The last one will remain in the log // as an empty batch in order to preserve the producer sequence number and epoch - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) + cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) assertEquals(List(1, 3, 4, 5), offsetsInLog(log)) assertEquals(List(1, 2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) // On the second round of cleaning, the marker from the first transaction should be removed. - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) + cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) assertEquals(List(3, 4, 5), offsetsInLog(log)) assertEquals(List(2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) } @@ -646,14 +646,14 @@ class LogCleanerTest { assertAbortedTransactionIndexed() // first time through the records are removed - var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 + var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 assertAbortedTransactionIndexed() assertEquals(List(), LogTest.keysInLog(log)) assertEquals(List(2), offsetsInLog(log)) // abort marker is retained assertEquals(List(1, 2), lastOffsetsPerBatchInLog(log)) // empty batch is retained // the empty batch remains if cleaned again because it still holds the last sequence - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 assertAbortedTransactionIndexed() assertEquals(List(), LogTest.keysInLog(log)) assertEquals(List(2), offsetsInLog(log)) // abort marker is still retained @@ -663,13 +663,13 @@ class LogCleanerTest { appendProducer(Seq(1)) log.roll() - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 assertAbortedTransactionIndexed() assertEquals(List(1), LogTest.keysInLog(log)) assertEquals(List(2, 3), offsetsInLog(log)) // abort marker is not yet gone because we read the empty batch assertEquals(List(2, 3), lastOffsetsPerBatchInLog(log)) // but we do not preserve the empty batch - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 assertEquals(List(1), LogTest.keysInLog(log)) assertEquals(List(3), offsetsInLog(log)) // abort marker is gone assertEquals(List(3), lastOffsetsPerBatchInLog(log)) From f7dcaac3648de31a2f60933eb06fd8eef1a289c1 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Wed, 1 Jan 2020 20:48:30 -0800 Subject: [PATCH 02/53] Fixing stuff --- core/src/main/scala/kafka/log/LogCleaner.scala | 4 ++-- core/src/main/scala/kafka/log/LogCleanerManager.scala | 3 ++- .../scala/unit/kafka/log/LogCleanerIntegrationTest.scala | 6 ++++-- 3 files changed, 8 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 292139868bdf3..33bea6d997630 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -334,7 +334,8 @@ class LogCleaner(initialConfig: CleanerConfig, @throws(classOf[LogCleaningException]) private def cleanFilthiestLog(): Boolean = { val preCleanStats = new PreCleanStats() - val cleaned = cleanerManager.grabFilthiestCompactedLog(time, preCleanStats) match { + val ltc =cleanerManager.grabFilthiestCompactedLog(time, preCleanStats) + val cleaned = ltc match { case None => false case Some(cleanable) => @@ -824,7 +825,6 @@ private[log] class Cleaner(val id: Int, else shouldRetainDeletes = retainDeletes val isRetainedValue = record.hasValue || shouldRetainDeletes - latestOffsetForKey && isRetainedValue } else { stats.invalidMessage() diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala index 561a6437d91ae..2623e958f647e 100755 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -216,7 +216,8 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], }.map { case (topicPartition, log) => // create a LogToClean instance for each try { - val (firstDirtyOffset, firstUncleanableDirtyOffset) = cleanableOffsets(log, topicPartition, lastClean, now) + val lastCleanOffset = lastClean.get(topicPartition) + val (firstDirtyOffset, firstUncleanableDirtyOffset) = cleanableOffsets(log, lastCleanOffset, now) val compactionDelayMs = maxCompactionDelay(log, firstDirtyOffset, now) preCleanStats.updateMaxCompactionDelay(compactionDelayMs) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index f51c3f126cf08..74244cdc87398 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -200,7 +200,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K log.roll() cleaner.startup() - Thread.sleep(100) + Thread.sleep(400) import JavaConverters._ var containsTombstones: Boolean = false @@ -212,9 +212,11 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K val firstBlockCleanableSegmentOffset = activeSegAtT0.baseOffset + Thread.sleep(300) + // the first block should get cleaned cleaner.awaitCleaned(new TopicPartition("log-partition", 0), - firstBlockCleanableSegmentOffset, maxWaitMs = tombstoneRetentionMs * 3) + firstBlockCleanableSegmentOffset, maxWaitMs = tombstoneRetentionMs * 5) for (segment <- log.logSegments; record <- segment.log.records.asScala) { fail ("The log should not contain record " + record + ", tombstone has expired its lifetime.") From 2ab16afd932ed9e006dea1d7f183ac62a22ec5ed Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Thu, 2 Jan 2020 16:16:25 -0800 Subject: [PATCH 03/53] Fixing stuff --- .../kafka/log/LogCleanerIntegrationTest.scala | 23 +++++++++++++------ 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 74244cdc87398..2b951875ad326 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -194,13 +194,10 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K writeKeyDups(numKeys = 1, numDups = 1, log, CompressionType.NONE, timestamp = T0, startValue = 0, step = 1, isRecordTombstone = true) - val activeSegAtT0 = log.activeSegment - // roll the active segment log.roll() cleaner.startup() - Thread.sleep(400) import JavaConverters._ var containsTombstones: Boolean = false @@ -208,15 +205,27 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K containsTombstones = true } assertTrue(containsTombstones) - time.sleep(tombstoneRetentionMs + 1) - val firstBlockCleanableSegmentOffset = activeSegAtT0.baseOffset + Thread.sleep(400) + time.sleep(tombstoneRetentionMs + 1) - Thread.sleep(300) + val latestOffset: Long = log.latestEpoch match { + case None => + fail("There should be epoch defined.") + RecordBatch.NO_TIMESTAMP + case Some(epoch) => + log.endOffsetForEpoch(epoch) match { + case None => + fail("Offset should have been found.") + RecordBatch.NO_TIMESTAMP + case Some(offsetAndEpoch) => + offsetAndEpoch.offset + } + } // the first block should get cleaned cleaner.awaitCleaned(new TopicPartition("log-partition", 0), - firstBlockCleanableSegmentOffset, maxWaitMs = tombstoneRetentionMs * 5) + latestOffset + 1, maxWaitMs = tombstoneRetentionMs) for (segment <- log.logSegments; record <- segment.log.records.asScala) { fail ("The log should not contain record " + record + ", tombstone has expired its lifetime.") From d36f776605ed9738b151082f3ab417141d507997 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Wed, 15 Jan 2020 08:42:43 -0800 Subject: [PATCH 04/53] Resolving some comments --- core/src/main/scala/kafka/log/LogCleaner.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 33bea6d997630..7706d0f226a57 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -334,8 +334,7 @@ class LogCleaner(initialConfig: CleanerConfig, @throws(classOf[LogCleaningException]) private def cleanFilthiestLog(): Boolean = { val preCleanStats = new PreCleanStats() - val ltc =cleanerManager.grabFilthiestCompactedLog(time, preCleanStats) - val cleaned = ltc match { + val cleaned = cleanerManager.grabFilthiestCompactedLog(time, preCleanStats) match { case None => false case Some(cleanable) => @@ -542,10 +541,12 @@ private[log] class Cleaner(val id: Int, * @param log The log being cleaned * @param segments The group of segments being cleaned * @param map The offset map to use for cleaning segments + * @param currentTime The current time in milliseconds * @param deleteHorizonMs The time to retain delete tombstones * @param stats Collector for cleaning statistics * @param transactionMetadata State of ongoing transactions which is carried between the cleaning * of the grouped segments + * @param trackedHorizon The delete horizon used for tombstones whose version is less than 2 */ private[log] def cleanSegments(log: Log, segments: Seq[LogSegment], @@ -579,8 +580,8 @@ private[log] class Cleaner(val id: Int, s"${if(retainDeletesAndTxnMarkers) "retaining" else "discarding"} deletes.") try { - val containsTombstones: Boolean = cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletesAndTxnMarkers, log.config.maxMessageSize, - transactionMetadata, lastOffsetOfActiveProducers, stats, log.config.deleteRetentionMs, currentTime = currentTime) + val containsTombstones: Boolean = cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletesAndTxnMarkers, log.config.deleteRetentionMs, + log.config.maxMessageSize, transactionMetadata, lastOffsetOfActiveProducers, stats, currentTime = currentTime) log.containsTombstones = containsTombstones } catch { case e: LogSegmentOffsetOverflowException => @@ -622,7 +623,8 @@ private[log] class Cleaner(val id: Int, * @param sourceRecords The dirty log segment * @param dest The cleaned log segment * @param map The key=>offset mapping - * @param retainDeletesAndTxnMarkers Should tombstones and markers be retained while cleaning this segment + * @param retainDeletesAndTxnMarkers Should tombstones (lower than version 2) and markers be retained while cleaning this segment + * @param tombstoneRetentionMs How long we should retend the tombstones whose version is greater than equal to 2 * @param maxLogMessageSize The maximum message size of the corresponding topic * @param stats Collector for cleaning statistics * @param tombstoneRetentionMs Defines how long a tombstone should be kept as defined by log configuration @@ -632,11 +634,11 @@ private[log] class Cleaner(val id: Int, dest: LogSegment, map: OffsetMap, retainDeletesAndTxnMarkers: Boolean, + tombstoneRetentionMs: Long, maxLogMessageSize: Int, transactionMetadata: CleanedTransactionMetadata, lastRecordsOfActiveProducers: Map[Long, LastRecord], stats: CleanerStats, - tombstoneRetentionMs: Long, currentTime: Long = RecordBatch.NO_TIMESTAMP): Boolean = { var containsTombstones: Boolean = false From 3b2193bf595298498a7a01f97ca794e8fbb801c3 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Wed, 15 Jan 2020 15:40:19 -0800 Subject: [PATCH 05/53] Resolving remaining comments --- .../kafka/common/record/MemoryRecords.java | 5 ++-- .../src/main/scala/kafka/log/LogCleaner.scala | 25 ++++++++----------- 2 files changed, 13 insertions(+), 17 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index 7fb5fe951b9f0..2f42a7437426f 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -165,7 +165,6 @@ private static FilterResult filterTo(TopicPartition partition, Iterable retainedRecords, - boolean containsTombstonesOrMarker, long newBatchDeleteHorizonMs) { + boolean containsTombstonesOrMarker = false; try (final CloseableIterator iterator = batch.streamingIterator(decompressionBufferSupplier)) { while (iterator.hasNext()) { Record record = iterator.next(); diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 7706d0f226a57..7b5870e3e34ed 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -644,23 +644,19 @@ private[log] class Cleaner(val id: Int, val logCleanerFilter: RecordFilter = new RecordFilter { var discardBatchRecords: Boolean = _ - var isControlBatchEmpty: Boolean = _ - - override def isControlBatchEmpty(batch: RecordBatch) : Boolean = { - // we piggy-back on the tombstone retention logic to delay deletion of transaction markers. - // note that we will never delete a marker until all the records from that transaction are removed. - val canDiscardBatch = shouldDiscardBatch(batch, transactionMetadata, retainTxnMarkers = retainDeletesAndTxnMarkers) - isControlBatchEmpty = canDiscardBatch - isControlBatchEmpty - } + var isBatchEmpty: Boolean = _ override def checkBatchRetention(batch: RecordBatch, newBatchDeleteHorizonMs : Long): BatchRetention = { - val canDiscardBatch = isControlBatchEmpty + val canDiscardBatch = isBatchEmpty if (batch.isControlBatch) { - discardBatchRecords = canDiscardBatch && + if (batch.magic() < 2) { + discardBatchRecords = canDiscardBatch && !retainDeletesAndTxnMarkers + } else { + discardBatchRecords = canDiscardBatch && ((batch.deleteHorizonSet() && batch.deleteHorizonMs() < currentTime) || newBatchDeleteHorizonMs != RecordBatch.NO_TIMESTAMP && newBatchDeleteHorizonMs < currentTime) + } } else { discardBatchRecords = canDiscardBatch } @@ -707,12 +703,14 @@ private[log] class Cleaner(val id: Int, } override def retrieveDeleteHorizon(batch: RecordBatch) : Long = { + isBatchEmpty = shouldDiscardBatch(batch, transactionMetadata) + if (batch.deleteHorizonSet()) return batch.deleteHorizonMs() // means that we keep the old timestamp stored // check that the control batch has been emptied of records // if not, then we do not set a delete horizon until that is true - if (batch.isControlBatch() && !isControlBatchEmpty) + if (batch.isControlBatch() && !isBatchEmpty) return -1L return time.milliseconds() + tombstoneRetentionMs; } @@ -791,8 +789,7 @@ private[log] class Cleaner(val id: Int, } private def shouldDiscardBatch(batch: RecordBatch, - transactionMetadata: CleanedTransactionMetadata, - retainTxnMarkers: Boolean): Boolean = { + transactionMetadata: CleanedTransactionMetadata): Boolean = { if (batch.isControlBatch) transactionMetadata.onControlBatchRead(batch) else From dcc2f6507b4b8bd1d0c63ce62128b23f9f0168fa Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Sun, 26 Jan 2020 10:41:19 -0800 Subject: [PATCH 06/53] Adding two pass modification --- .../kafka/common/record/MemoryRecords.java | 33 ++++--------------- .../src/main/scala/kafka/log/LogCleaner.scala | 29 +++++++--------- .../scala/unit/kafka/log/LogCleanerTest.scala | 21 ++++++++---- 3 files changed, 32 insertions(+), 51 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index 2f42a7437426f..77c4d5d01b2e8 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -166,11 +166,7 @@ private static FilterResult filterTo(TopicPartition partition, Iterable retainedRecords = new ArrayList<>(); final BatchIterationResult iterationResult = iterateOverBatch(batch, decompressionBufferSupplier, filterResult, filter, - batchMagic, writeOriginalBatch, maxOffset, retainedRecords, - deleteHorizonMs); + batchMagic, writeOriginalBatch, maxOffset, retainedRecords); containsTombstonesOrMarker = iterationResult.containsTombstonesOrMarker(); writeOriginalBatch = iterationResult.shouldWriteOriginalBatch(); maxOffset = iterationResult.maxOffset(); @@ -199,7 +194,9 @@ private static FilterResult filterTo(TopicPartition partition, Iterable RecordBatch.NO_TIMESTAMP; + if (writeOriginalBatch && (deleteHorizonMs == RecordBatch.NO_TIMESTAMP || deleteHorizonMs == batch.deleteHorizonMs() + || (!containsTombstonesOrMarker && !canControlBatchBeRemoved))) { batch.writeTo(bufferOutputStream); filterResult.updateRetainedBatchMetadata(batch, retainedRecords.size(), false); } else { @@ -247,15 +244,14 @@ private static BatchIterationResult iterateOverBatch(RecordBatch batch, byte batchMagic, boolean writeOriginalBatch, long maxOffset, - List retainedRecords, - long newBatchDeleteHorizonMs) { + List retainedRecords) { boolean containsTombstonesOrMarker = false; try (final CloseableIterator iterator = batch.streamingIterator(decompressionBufferSupplier)) { while (iterator.hasNext()) { Record record = iterator.next(); filterResult.messagesRead += 1; - if (filter.shouldRetainRecord(batch, record, newBatchDeleteHorizonMs)) { + if (filter.shouldRetainRecord(batch, record)) { // Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite // the corrupted batch with correct data. if (!record.hasMagic(batchMagic)) @@ -376,10 +372,6 @@ public enum BatchRetention { */ protected abstract BatchRetention checkBatchRetention(RecordBatch batch); - protected BatchRetention checkBatchRetention(RecordBatch batch, long newBatchDeleteHorizonMs) { - return checkBatchRetention(batch); - } - /** * Check whether a record should be retained in the log. Note that {@link #checkBatchRetention(RecordBatch)} * is used prior to checking individual record retention. Only records from batches which were not @@ -387,23 +379,12 @@ protected BatchRetention checkBatchRetention(RecordBatch batch, long newBatchDel */ protected abstract boolean shouldRetainRecord(RecordBatch recordBatch, Record record); - protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record, long newDeleteHorizonMs) { - return shouldRetainRecord(recordBatch, record); - } - /** * Retrieves the delete horizon ms for a specific batch */ protected long retrieveDeleteHorizon(RecordBatch recordBatch) { return -1L; } - - /** - * Checks if the control batch (if it is one) can be removed (making sure that it is empty) - */ - protected boolean isControlBatchEmpty(RecordBatch recordBatch) { - return true; - } } public static class FilterResult { diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 7b5870e3e34ed..61799ae72bf45 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -624,10 +624,10 @@ private[log] class Cleaner(val id: Int, * @param dest The cleaned log segment * @param map The key=>offset mapping * @param retainDeletesAndTxnMarkers Should tombstones (lower than version 2) and markers be retained while cleaning this segment - * @param tombstoneRetentionMs How long we should retend the tombstones whose version is greater than equal to 2 * @param maxLogMessageSize The maximum message size of the corresponding topic * @param stats Collector for cleaning statistics * @param tombstoneRetentionMs Defines how long a tombstone should be kept as defined by log configuration + * @param currentTime The time at which the clean was initiated */ private[log] def cleanInto(topicPartition: TopicPartition, sourceRecords: FileRecords, @@ -644,18 +644,17 @@ private[log] class Cleaner(val id: Int, val logCleanerFilter: RecordFilter = new RecordFilter { var discardBatchRecords: Boolean = _ - var isBatchEmpty: Boolean = _ + var isBatchDiscardable: Boolean = _ - override def checkBatchRetention(batch: RecordBatch, newBatchDeleteHorizonMs : Long): BatchRetention = { - val canDiscardBatch = isBatchEmpty + override def checkBatchRetention(batch: RecordBatch): BatchRetention = { + val canDiscardBatch = isBatchDiscardable if (batch.isControlBatch) { if (batch.magic() < 2) { discardBatchRecords = canDiscardBatch && !retainDeletesAndTxnMarkers } else { discardBatchRecords = canDiscardBatch && - ((batch.deleteHorizonSet() && batch.deleteHorizonMs() < currentTime) || - newBatchDeleteHorizonMs != RecordBatch.NO_TIMESTAMP && newBatchDeleteHorizonMs < currentTime) + batch.deleteHorizonSet() && batch.deleteHorizonMs() < currentTime } } else { discardBatchRecords = canDiscardBatch @@ -684,34 +683,29 @@ private[log] class Cleaner(val id: Int, BatchRetention.DELETE_EMPTY } - override def checkBatchRetention(batch: RecordBatch): BatchRetention = checkBatchRetention(batch, batch.deleteHorizonMs()) - - override def shouldRetainRecord(batch: RecordBatch, record: Record, newDeleteHorizonMs: Long): Boolean = { + override def shouldRetainRecord(batch: RecordBatch, record: Record): Boolean = { var isRecordRetained: Boolean = true if (discardBatchRecords) // The batch is only retained to preserve producer sequence information; the records can be removed isRecordRetained = false else - isRecordRetained = Cleaner.this.shouldRetainRecord(map, retainDeletesAndTxnMarkers, batch, record, stats, newDeleteHorizonMs, currentTime = currentTime) + isRecordRetained = Cleaner.this.shouldRetainRecord(map, retainDeletesAndTxnMarkers, batch, record, stats, currentTime = currentTime) if (isRecordRetained && !record.hasValue()) containsTombstones = true isRecordRetained } - override def shouldRetainRecord(batch: RecordBatch, record: Record): Boolean = { - shouldRetainRecord(batch, record, RecordBatch.NO_TIMESTAMP) - } - override def retrieveDeleteHorizon(batch: RecordBatch) : Long = { - isBatchEmpty = shouldDiscardBatch(batch, transactionMetadata) + isBatchDiscardable = shouldDiscardBatch(batch, transactionMetadata) if (batch.deleteHorizonSet()) return batch.deleteHorizonMs() // means that we keep the old timestamp stored // check that the control batch has been emptied of records // if not, then we do not set a delete horizon until that is true - if (batch.isControlBatch() && !isBatchEmpty) + if (batch.isControlBatch() && !isBatchDiscardable) { return -1L + } return time.milliseconds() + tombstoneRetentionMs; } } @@ -801,7 +795,6 @@ private[log] class Cleaner(val id: Int, batch: RecordBatch, record: Record, stats: CleanerStats, - newBatchDeleteHorizonMs: Long, currentTime: Long = -1L): Boolean = { val pastLatestOffset = record.offset > map.latestOffset if (pastLatestOffset) @@ -820,7 +813,7 @@ private[log] class Cleaner(val id: Int, var shouldRetainDeletes = true if (isLatestVersion) shouldRetainDeletes = (batch.deleteHorizonSet() && currentTime < batch.deleteHorizonMs()) || - (!batch.deleteHorizonSet() && currentTime < newBatchDeleteHorizonMs) + !batch.deleteHorizonSet() else shouldRetainDeletes = retainDeletes val isRetainedValue = record.hasValue || shouldRetainDeletes diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 7d03b07fd7010..d4cfcabadfb86 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -399,7 +399,7 @@ class LogCleanerTest { assertEquals(List(2), LogTest.keysInLog(log)) assertEquals(List(1, 3, 4), offsetsInLog(log)) - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) assertEquals(List(2), LogTest.keysInLog(log)) assertEquals(List(3, 4), offsetsInLog(log)) } @@ -460,7 +460,7 @@ class LogCleanerTest { assertEquals(List(1, 4, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log)) // Expected State: [{Producer1: EmptyBatch}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue) assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) assertEquals(List(5, 6, 7, 8, 9), offsetsInLog(log)) assertEquals(List(1, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log)) @@ -484,14 +484,14 @@ class LogCleanerTest { // first time through the control batch is retained as an empty batch // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] - var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + var dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) assertEquals(List(2, 3), LogTest.keysInLog(log)) assertEquals(List(1, 2), offsetsInLog(log)) assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) // the empty control batch does not cause an exception when cleaned // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue) assertEquals(List(2, 3), LogTest.keysInLog(log)) assertEquals(List(1, 2), offsetsInLog(log)) assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) @@ -575,7 +575,7 @@ class LogCleanerTest { assertEquals(List(3, 4, 5), offsetsInLog(log)) // clean again with large delete horizon and verify the marker is removed - cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue) assertEquals(List(3), LogTest.keysInLog(log)) assertEquals(List(4, 5), offsetsInLog(log)) } @@ -614,7 +614,7 @@ class LogCleanerTest { assertEquals(List(1, 2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) // On the second round of cleaning, the marker from the first transaction should be removed. - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) assertEquals(List(3, 4, 5), offsetsInLog(log)) assertEquals(List(2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) } @@ -669,7 +669,7 @@ class LogCleanerTest { assertEquals(List(2, 3), offsetsInLog(log)) // abort marker is not yet gone because we read the empty batch assertEquals(List(2, 3), lastOffsetsPerBatchInLog(log)) // but we do not preserve the empty batch - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue) assertEquals(List(1), LogTest.keysInLog(log)) assertEquals(List(3), offsetsInLog(log)) // abort marker is gone assertEquals(List(3), lastOffsetsPerBatchInLog(log)) @@ -1544,6 +1544,7 @@ class LogCleanerTest { key = "0".getBytes, timestamp = time.milliseconds() - logConfig.deleteRetentionMs - 10000), leaderEpoch = 0) log.roll() + cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 1, log.activeSegment.baseOffset)) assertEquals("The tombstone should be retained.", 1, log.logSegments.head.log.batches.iterator.next().lastOffset) // Append a message and roll out another log segment. @@ -1674,6 +1675,12 @@ class LogCleanerTest { private def recoverAndCheck(config: LogConfig, expectedKeys: Iterable[Long]): Log = { LogTest.recoverAndCheck(dir, config, expectedKeys, new BrokerTopicStats(), time, time.scheduler) } + + private def runTwoPassClean(cleaner: Cleaner, logToClean: LogToClean, currentTime: Long) : Long = { + var offsetReturned: Long = cleaner.doClean(logToClean, currentTime = currentTime)._1 + offsetReturned = cleaner.doClean(logToClean, currentTime = currentTime)._1 + return offsetReturned + } } class FakeOffsetMap(val slots: Int) extends OffsetMap { From a9d8c4d51ac3952c5cb21f32d2c7f6634cfb0ac0 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Tue, 28 Jan 2020 17:08:02 -0800 Subject: [PATCH 07/53] Adding some last changes --- core/src/main/scala/kafka/log/Log.scala | 2 +- .../src/main/scala/kafka/log/LogCleaner.scala | 27 ++++++++++++------- .../scala/kafka/log/LogCleanerManager.scala | 5 +++- .../kafka/log/LogCleanerIntegrationTest.scala | 8 +++--- 4 files changed, 28 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 2b1b292fb189f..b7f2986defb63 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -222,7 +222,7 @@ class Log(@volatile var dir: File, val topicPartition: TopicPartition, val producerStateManager: ProducerStateManager, logDirFailureChannel: LogDirFailureChannel, - @volatile var containsTombstones: Boolean = false) extends Logging with KafkaMetricsGroup { + @volatile var latestDeleteHorizon: Long = RecordBatch.NO_TIMESTAMP) extends Logging with KafkaMetricsGroup { import kafka.log.Log._ diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 61799ae72bf45..bfdf36d6e47e6 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -580,9 +580,11 @@ private[log] class Cleaner(val id: Int, s"${if(retainDeletesAndTxnMarkers) "retaining" else "discarding"} deletes.") try { - val containsTombstones: Boolean = cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletesAndTxnMarkers, log.config.deleteRetentionMs, + val latestDeleteHorizon: Long = cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletesAndTxnMarkers, log.config.deleteRetentionMs, log.config.maxMessageSize, transactionMetadata, lastOffsetOfActiveProducers, stats, currentTime = currentTime) - log.containsTombstones = containsTombstones + if (log.latestDeleteHorizon < latestDeleteHorizon) { + log.latestDeleteHorizon = latestDeleteHorizon + } } catch { case e: LogSegmentOffsetOverflowException => // Split the current segment. It's also safest to abort the current cleaning process, so that we retry from @@ -639,8 +641,8 @@ private[log] class Cleaner(val id: Int, transactionMetadata: CleanedTransactionMetadata, lastRecordsOfActiveProducers: Map[Long, LastRecord], stats: CleanerStats, - currentTime: Long = RecordBatch.NO_TIMESTAMP): Boolean = { - var containsTombstones: Boolean = false + currentTime: Long = RecordBatch.NO_TIMESTAMP): Long = { + var latestDeleteHorizon: Long = RecordBatch.NO_TIMESTAMP val logCleanerFilter: RecordFilter = new RecordFilter { var discardBatchRecords: Boolean = _ @@ -690,23 +692,30 @@ private[log] class Cleaner(val id: Int, isRecordRetained = false else isRecordRetained = Cleaner.this.shouldRetainRecord(map, retainDeletesAndTxnMarkers, batch, record, stats, currentTime = currentTime) - if (isRecordRetained && !record.hasValue()) - containsTombstones = true isRecordRetained } override def retrieveDeleteHorizon(batch: RecordBatch) : Long = { isBatchDiscardable = shouldDiscardBatch(batch, transactionMetadata) - if (batch.deleteHorizonSet()) + if (batch.deleteHorizonSet()) { + if (batch.deleteHorizonMs() > latestDeleteHorizon) { + latestDeleteHorizon = batch.deleteHorizonMs() + } return batch.deleteHorizonMs() // means that we keep the old timestamp stored + } // check that the control batch has been emptied of records // if not, then we do not set a delete horizon until that is true if (batch.isControlBatch() && !isBatchDiscardable) { return -1L } - return time.milliseconds() + tombstoneRetentionMs; + + val newDeleteHorizon: Long = time.milliseconds() + tombstoneRetentionMs + if (newDeleteHorizon > latestDeleteHorizon) { + latestDeleteHorizon = newDeleteHorizon + } + newDeleteHorizon } } @@ -746,7 +755,7 @@ private[log] class Cleaner(val id: Int, growBuffersOrFail(sourceRecords, position, maxLogMessageSize, records) } restoreBuffers() - containsTombstones + latestDeleteHorizon } diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala index 2623e958f647e..28c7822b0ce3b 100755 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -207,8 +207,11 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], if(cleanableLogs.isEmpty) { // in this case, we are probably in a low throughput situation // therefore, we should take advantage of this fact and remove tombstones if we can + // under the condition that the log's latest delete horizon is less than the current time + // tracked val logsContainingTombstones = logs.filter { - case (_, log) => log.containsTombstones + case (_, log) => log.latestDeleteHorizon != -1L && + log.latestDeleteHorizon <= time.milliseconds() }.filterNot { case (topicPartition, log) => // skip any logs already in-progress and uncleanable partitions diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 2b951875ad326..3291310365a41 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -186,7 +186,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K val topicPartitions = Array(new TopicPartition("log-partition", 0)) val props = new Properties() props.put(LogConfig.DeleteRetentionMsProp, "1000") - cleaner = makeCleaner(partitions = topicPartitions, propertyOverrides = props) + cleaner = makeCleaner(partitions = topicPartitions, propertyOverrides = props, backOffMs = 100L) val log = cleaner.logs.get(topicPartitions(0)) @@ -206,7 +206,9 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K } assertTrue(containsTombstones) - Thread.sleep(400) + // We sleep a little bit, so that log cleaner has already gone through + // some iterations, ensures that delete horizons has been updated correctly + Thread.sleep(300L) time.sleep(tombstoneRetentionMs + 1) val latestOffset: Long = log.latestEpoch match { @@ -227,10 +229,10 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K cleaner.awaitCleaned(new TopicPartition("log-partition", 0), latestOffset + 1, maxWaitMs = tombstoneRetentionMs) + assertEquals(log.latestDeleteHorizon, T0 + tombstoneRetentionMs) for (segment <- log.logSegments; record <- segment.log.records.asScala) { fail ("The log should not contain record " + record + ", tombstone has expired its lifetime.") } - assertFalse(log.containsTombstones) } private def readFromLog(log: Log): Iterable[(Int, Int)] = { From dd9ca283012bf7be772d54ce6acf2f41681fb080 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Tue, 11 Feb 2020 19:29:03 -0800 Subject: [PATCH 08/53] Adding stuff --- .../common/record/DefaultRecordBatch.java | 40 - .../kafka/common/record/MemoryRecords.java | 13 +- ...he.kafka.connect.rest.ConnectRestExtension | 16 + connect/mirror/bin/.gitignore | 2 + connect/runtime/bin/.gitignore | 2 + .../src/main/scala/kafka/log/LogCleaner.scala | 9 +- .../scala/kafka/log/LogCleaner.scala.orig | 1199 +++++++++++++++++ .../scala/kafka/log/LogCleanerManager.scala | 41 +- .../kafka/log/LogCleanerManager.scala.orig | 626 +++++++++ .../kafka/log/LogCleanerIntegrationTest.scala | 6 +- diff.out | 1155 ++++++++++++++++ generator/bin/.gitignore | 2 + gradle.properties | 1 + .../upgrade-system-tests-0100/bin/.gitignore | 1 + .../upgrade-system-tests-0101/bin/.gitignore | 1 + .../upgrade-system-tests-0102/bin/.gitignore | 1 + .../upgrade-system-tests-0110/bin/.gitignore | 1 + .../upgrade-system-tests-10/bin/.gitignore | 1 + .../upgrade-system-tests-11/bin/.gitignore | 1 + .../upgrade-system-tests-20/bin/.gitignore | 1 + .../upgrade-system-tests-21/bin/.gitignore | 1 + .../upgrade-system-tests-22/bin/.gitignore | 1 + .../upgrade-system-tests-23/bin/.gitignore | 1 + .../upgrade-system-tests-24/bin/.gitignore | 1 + tools/bin/.gitignore | 2 + total.out | 1167 ++++++++++++++++ 26 files changed, 4210 insertions(+), 82 deletions(-) create mode 100644 connect/basic-auth-extension/bin/main/META-INF/services/org.apache.kafka.connect.rest.ConnectRestExtension create mode 100644 connect/mirror/bin/.gitignore create mode 100644 connect/runtime/bin/.gitignore create mode 100644 core/src/main/scala/kafka/log/LogCleaner.scala.orig create mode 100755 core/src/main/scala/kafka/log/LogCleanerManager.scala.orig create mode 100644 diff.out create mode 100644 generator/bin/.gitignore create mode 100644 streams/upgrade-system-tests-0100/bin/.gitignore create mode 100644 streams/upgrade-system-tests-0101/bin/.gitignore create mode 100644 streams/upgrade-system-tests-0102/bin/.gitignore create mode 100644 streams/upgrade-system-tests-0110/bin/.gitignore create mode 100644 streams/upgrade-system-tests-10/bin/.gitignore create mode 100644 streams/upgrade-system-tests-11/bin/.gitignore create mode 100644 streams/upgrade-system-tests-20/bin/.gitignore create mode 100644 streams/upgrade-system-tests-21/bin/.gitignore create mode 100644 streams/upgrade-system-tests-22/bin/.gitignore create mode 100644 streams/upgrade-system-tests-23/bin/.gitignore create mode 100644 streams/upgrade-system-tests-24/bin/.gitignore create mode 100644 tools/bin/.gitignore create mode 100644 total.out diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java index e0d7549fedebc..0d2157ae13da8 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java @@ -464,46 +464,6 @@ public static void writeEmptyHeader(ByteBuffer buffer, producerEpoch, baseSequence, isTransactional, isControlRecord, false, partitionLeaderEpoch, 0); } - public static void writeEmptyHeader(ByteBuffer buffer, - byte magic, - long producerId, - short producerEpoch, - int baseSequence, - long baseOffset, - long lastOffset, - int partitionLeaderEpoch, - TimestampType timestampType, - long timestamp, - boolean isTransactional, - boolean isControlRecord, - boolean isDeleteHorizonSet) { - int offsetDelta = (int) (lastOffset - baseOffset); - writeHeader(buffer, baseOffset, offsetDelta, DefaultRecordBatch.RECORD_BATCH_OVERHEAD, magic, - CompressionType.NONE, timestampType, RecordBatch.NO_TIMESTAMP, timestamp, producerId, - producerEpoch, baseSequence, isTransactional, isControlRecord, isDeleteHorizonSet, partitionLeaderEpoch, 0); - } - - static void writeHeader(ByteBuffer buffer, - long baseOffset, - int lastOffsetDelta, - int sizeInBytes, - byte magic, - CompressionType compressionType, - TimestampType timestampType, - long firstTimestamp, - long maxTimestamp, - long producerId, - short epoch, - int sequence, - boolean isTransactional, - boolean isControlBatch, - int partitionLeaderEpoch, - int numRecords) { - writeHeader(buffer, baseOffset, lastOffsetDelta, sizeInBytes, magic, compressionType, - timestampType, firstTimestamp, maxTimestamp, producerId, epoch, sequence, - isTransactional, isControlBatch, false, partitionLeaderEpoch, numRecords); - } - static void writeHeader(ByteBuffer buffer, long baseOffset, int lastOffsetDelta, diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index 77c4d5d01b2e8..e3ab8ddf12dbc 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -195,12 +195,11 @@ private static FilterResult filterTo(TopicPartition partition, Iterable RecordBatch.NO_TIMESTAMP; - if (writeOriginalBatch && (deleteHorizonMs == RecordBatch.NO_TIMESTAMP || deleteHorizonMs == batch.deleteHorizonMs() - || (!containsTombstonesOrMarker && !canControlBatchBeRemoved))) { + if (writeOriginalBatch && (batch.deleteHorizonSet() || (!containsTombstonesOrMarker && !canControlBatchBeRemoved))) { batch.writeTo(bufferOutputStream); filterResult.updateRetainedBatchMetadata(batch, retainedRecords.size(), false); } else { - MemoryRecordsBuilder builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, deleteHorizonMs); + final MemoryRecordsBuilder builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, deleteHorizonMs); MemoryRecords records = builder.build(); int filteredBatchSize = records.sizeInBytes(); if (filteredBatchSize > batch.sizeInBytes() && filteredBatchSize > maxRecordBatchSize) @@ -242,7 +241,7 @@ private static BatchIterationResult iterateOverBatch(RecordBatch batch, FilterResult filterResult, RecordFilter filter, byte batchMagic, - boolean writeOriginalBatch, + boolean recordsFiltered, long maxOffset, List retainedRecords) { boolean containsTombstonesOrMarker = false; @@ -255,7 +254,7 @@ private static BatchIterationResult iterateOverBatch(RecordBatch batch, // Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite // the corrupted batch with correct data. if (!record.hasMagic(batchMagic)) - writeOriginalBatch = false; + recordsFiltered = false; if (record.offset() > maxOffset) maxOffset = record.offset(); @@ -266,10 +265,10 @@ private static BatchIterationResult iterateOverBatch(RecordBatch batch, containsTombstonesOrMarker = true; } } else { - writeOriginalBatch = false; + recordsFiltered = false; } } - return new BatchIterationResult(writeOriginalBatch, containsTombstonesOrMarker, maxOffset); + return new BatchIterationResult(recordsFiltered, containsTombstonesOrMarker, maxOffset); } } diff --git a/connect/basic-auth-extension/bin/main/META-INF/services/org.apache.kafka.connect.rest.ConnectRestExtension b/connect/basic-auth-extension/bin/main/META-INF/services/org.apache.kafka.connect.rest.ConnectRestExtension new file mode 100644 index 0000000000000..ba7ae5b580d80 --- /dev/null +++ b/connect/basic-auth-extension/bin/main/META-INF/services/org.apache.kafka.connect.rest.ConnectRestExtension @@ -0,0 +1,16 @@ + # 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. + +org.apache.kafka.connect.rest.basic.auth.extension.BasicAuthSecurityRestExtension \ No newline at end of file diff --git a/connect/mirror/bin/.gitignore b/connect/mirror/bin/.gitignore new file mode 100644 index 0000000000000..7eed456bec8db --- /dev/null +++ b/connect/mirror/bin/.gitignore @@ -0,0 +1,2 @@ +/main/ +/test/ diff --git a/connect/runtime/bin/.gitignore b/connect/runtime/bin/.gitignore new file mode 100644 index 0000000000000..7eed456bec8db --- /dev/null +++ b/connect/runtime/bin/.gitignore @@ -0,0 +1,2 @@ +/main/ +/test/ diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index bfdf36d6e47e6..da19006017b8b 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -542,7 +542,6 @@ private[log] class Cleaner(val id: Int, * @param segments The group of segments being cleaned * @param map The offset map to use for cleaning segments * @param currentTime The current time in milliseconds - * @param deleteHorizonMs The time to retain delete tombstones * @param stats Collector for cleaning statistics * @param transactionMetadata State of ongoing transactions which is carried between the cleaning * of the grouped segments @@ -584,6 +583,8 @@ private[log] class Cleaner(val id: Int, log.config.maxMessageSize, transactionMetadata, lastOffsetOfActiveProducers, stats, currentTime = currentTime) if (log.latestDeleteHorizon < latestDeleteHorizon) { log.latestDeleteHorizon = latestDeleteHorizon + } else if (log.latestDeleteHorizon < currentTime) { + log.latestDeleteHorizon = RecordBatch.NO_TIMESTAMP } } catch { case e: LogSegmentOffsetOverflowException => @@ -626,9 +627,9 @@ private[log] class Cleaner(val id: Int, * @param dest The cleaned log segment * @param map The key=>offset mapping * @param retainDeletesAndTxnMarkers Should tombstones (lower than version 2) and markers be retained while cleaning this segment + * @param tombstoneRetentionMs Defines how long a tombstone should be kept as defined by log configuration * @param maxLogMessageSize The maximum message size of the corresponding topic * @param stats Collector for cleaning statistics - * @param tombstoneRetentionMs Defines how long a tombstone should be kept as defined by log configuration * @param currentTime The time at which the clean was initiated */ private[log] def cleanInto(topicPartition: TopicPartition, @@ -818,9 +819,9 @@ private[log] class Cleaner(val id: Int, * 2) The message doesn't has value but it can't be deleted now. */ val latestOffsetForKey = record.offset() >= foundOffset - val isLatestVersion = batch.magic() >= RecordBatch.MAGIC_VALUE_V2 + val supportDeleteHorizon = batch.magic() >= RecordBatch.MAGIC_VALUE_V2 var shouldRetainDeletes = true - if (isLatestVersion) + if (supportDeleteHorizon) shouldRetainDeletes = (batch.deleteHorizonSet() && currentTime < batch.deleteHorizonMs()) || !batch.deleteHorizonSet() else diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala.orig b/core/src/main/scala/kafka/log/LogCleaner.scala.orig new file mode 100644 index 0000000000000..292139868bdf3 --- /dev/null +++ b/core/src/main/scala/kafka/log/LogCleaner.scala.orig @@ -0,0 +1,1199 @@ +/** + * 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 kafka.log + +import java.io.{File, IOException} +import java.nio._ +import java.util.Date +import java.util.concurrent.TimeUnit + +import com.yammer.metrics.core.Gauge +import kafka.common._ +import kafka.metrics.KafkaMetricsGroup +import kafka.server.{BrokerReconfigurable, KafkaConfig, LogDirFailureChannel} +import kafka.utils._ +import org.apache.kafka.common.{KafkaException, TopicPartition} +import org.apache.kafka.common.config.ConfigException +import org.apache.kafka.common.errors.{CorruptRecordException, KafkaStorageException} +import org.apache.kafka.common.record.MemoryRecords.RecordFilter +import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention +import org.apache.kafka.common.record._ +import org.apache.kafka.common.utils.Time + +import scala.collection.JavaConverters._ +import scala.collection.{Iterable, Seq, Set, mutable} +import scala.util.control.ControlThrowable + +/** + * The cleaner is responsible for removing obsolete records from logs which have the "compact" retention strategy. + * A message with key K and offset O is obsolete if there exists a message with key K and offset O' such that O < O'. + * + * Each log can be thought of being split into two sections of segments: a "clean" section which has previously been cleaned followed by a + * "dirty" section that has not yet been cleaned. The dirty section is further divided into the "cleanable" section followed by an "uncleanable" section. + * The uncleanable section is excluded from cleaning. The active log segment is always uncleanable. If there is a + * compaction lag time set, segments whose largest message timestamp is within the compaction lag time of the cleaning operation are also uncleanable. + * + * The cleaning is carried out by a pool of background threads. Each thread chooses the dirtiest log that has the "compact" retention policy + * and cleans that. The dirtiness of the log is guessed by taking the ratio of bytes in the dirty section of the log to the total bytes in the log. + * + * To clean a log the cleaner first builds a mapping of key=>last_offset for the dirty section of the log. See kafka.log.OffsetMap for details of + * the implementation of the mapping. + * + * Once the key=>last_offset map is built, the log is cleaned by recopying each log segment but omitting any key that appears in the offset map with a + * higher offset than what is found in the segment (i.e. messages with a key that appears in the dirty section of the log). + * + * To avoid segments shrinking to very small sizes with repeated cleanings we implement a rule by which if we will merge successive segments when + * doing a cleaning if their log and index size are less than the maximum log and index size prior to the clean beginning. + * + * Cleaned segments are swapped into the log as they become available. + * + * One nuance that the cleaner must handle is log truncation. If a log is truncated while it is being cleaned the cleaning of that log is aborted. + * + * Messages with null payload are treated as deletes for the purpose of log compaction. This means that they receive special treatment by the cleaner. + * The cleaner will only retain delete records for a period of time to avoid accumulating space indefinitely. This period of time is configurable on a per-topic + * basis and is measured from the time the segment enters the clean portion of the log (at which point any prior message with that key has been removed). + * Delete markers in the clean section of the log that are older than this time will not be retained when log segments are being recopied as part of cleaning. + * + * Note that cleaning is more complicated with the idempotent/transactional producer capabilities. The following + * are the key points: + * + * 1. In order to maintain sequence number continuity for active producers, we always retain the last batch + * from each producerId, even if all the records from the batch have been removed. The batch will be removed + * once the producer either writes a new batch or is expired due to inactivity. + * 2. We do not clean beyond the last stable offset. This ensures that all records observed by the cleaner have + * been decided (i.e. committed or aborted). In particular, this allows us to use the transaction index to + * collect the aborted transactions ahead of time. + * 3. Records from aborted transactions are removed by the cleaner immediately without regard to record keys. + * 4. Transaction markers are retained until all record batches from the same transaction have been removed and + * a sufficient amount of time has passed to reasonably ensure that an active consumer wouldn't consume any + * data from the transaction prior to reaching the offset of the marker. This follows the same logic used for + * tombstone deletion. + * + * @param initialConfig Initial configuration parameters for the cleaner. Actual config may be dynamically updated. + * @param logDirs The directories where offset checkpoints reside + * @param logs The pool of logs + * @param time A way to control the passage of time + */ +class LogCleaner(initialConfig: CleanerConfig, + val logDirs: Seq[File], + val logs: Pool[TopicPartition, Log], + val logDirFailureChannel: LogDirFailureChannel, + time: Time = Time.SYSTEM) extends Logging with KafkaMetricsGroup with BrokerReconfigurable +{ + + /* Log cleaner configuration which may be dynamically updated */ + @volatile private var config = initialConfig + + /* for managing the state of partitions being cleaned. package-private to allow access in tests */ + private[log] val cleanerManager = new LogCleanerManager(logDirs, logs, logDirFailureChannel) + + /* a throttle used to limit the I/O of all the cleaner threads to a user-specified maximum rate */ + private val throttler = new Throttler(desiredRatePerSec = config.maxIoBytesPerSecond, + checkIntervalMs = 300, + throttleDown = true, + "cleaner-io", + "bytes", + time = time) + + private[log] val cleaners = mutable.ArrayBuffer[CleanerThread]() + + /* a metric to track the maximum utilization of any thread's buffer in the last cleaning */ + newGauge("max-buffer-utilization-percent", + new Gauge[Int] { + def value: Int = cleaners.map(_.lastStats).map(100 * _.bufferUtilization).max.toInt + }) + /* a metric to track the recopy rate of each thread's last cleaning */ + newGauge("cleaner-recopy-percent", + new Gauge[Int] { + def value: Int = { + val stats = cleaners.map(_.lastStats) + val recopyRate = stats.map(_.bytesWritten).sum.toDouble / math.max(stats.map(_.bytesRead).sum, 1) + (100 * recopyRate).toInt + } + }) + /* a metric to track the maximum cleaning time for the last cleaning from each thread */ + newGauge("max-clean-time-secs", + new Gauge[Int] { + def value: Int = cleaners.map(_.lastStats).map(_.elapsedSecs).max.toInt + }) + // a metric to track delay between the time when a log is required to be compacted + // as determined by max compaction lag and the time of last cleaner run. + newGauge("max-compaction-delay-secs", + new Gauge[Int] { + def value: Int = Math.max(0, (cleaners.map(_.lastPreCleanStats).map(_.maxCompactionDelayMs).max / 1000).toInt) + }) + + newGauge("DeadThreadCount", + new Gauge[Int] { + def value: Int = deadThreadCount + }) + + private[log] def deadThreadCount: Int = cleaners.count(_.isThreadFailed) + + /** + * Start the background cleaning + */ + def startup(): Unit = { + info("Starting the log cleaner") + (0 until config.numThreads).foreach { i => + val cleaner = new CleanerThread(i) + cleaners += cleaner + cleaner.start() + } + } + + /** + * Stop the background cleaning + */ + def shutdown(): Unit = { + info("Shutting down the log cleaner.") + cleaners.foreach(_.shutdown()) + cleaners.clear() + } + + override def reconfigurableConfigs: Set[String] = { + LogCleaner.ReconfigurableConfigs + } + + override def validateReconfiguration(newConfig: KafkaConfig): Unit = { + val newCleanerConfig = LogCleaner.cleanerConfig(newConfig) + val numThreads = newCleanerConfig.numThreads + val currentThreads = config.numThreads + if (numThreads < 1) + throw new ConfigException(s"Log cleaner threads should be at least 1") + if (numThreads < currentThreads / 2) + throw new ConfigException(s"Log cleaner threads cannot be reduced to less than half the current value $currentThreads") + if (numThreads > currentThreads * 2) + throw new ConfigException(s"Log cleaner threads cannot be increased to more than double the current value $currentThreads") + + } + + /** + * Reconfigure log clean config. This simply stops current log cleaners and creates new ones. + * That ensures that if any of the cleaners had failed, new cleaners are created to match the new config. + */ + override def reconfigure(oldConfig: KafkaConfig, newConfig: KafkaConfig): Unit = { + config = LogCleaner.cleanerConfig(newConfig) + shutdown() + startup() + } + + /** + * Abort the cleaning of a particular partition, if it's in progress. This call blocks until the cleaning of + * the partition is aborted. + */ + def abortCleaning(topicPartition: TopicPartition): Unit = { + cleanerManager.abortCleaning(topicPartition) + } + + /** + * Update checkpoint file, removing topics and partitions that no longer exist + */ + def updateCheckpoints(dataDir: File): Unit = { + cleanerManager.updateCheckpoints(dataDir, update=None) + } + + def alterCheckpointDir(topicPartition: TopicPartition, sourceLogDir: File, destLogDir: File): Unit = { + cleanerManager.alterCheckpointDir(topicPartition, sourceLogDir, destLogDir) + } + + def handleLogDirFailure(dir: String): Unit = { + cleanerManager.handleLogDirFailure(dir) + } + + /** + * Truncate cleaner offset checkpoint for the given partition if its checkpointed offset is larger than the given offset + */ + def maybeTruncateCheckpoint(dataDir: File, topicPartition: TopicPartition, offset: Long): Unit = { + cleanerManager.maybeTruncateCheckpoint(dataDir, topicPartition, offset) + } + + /** + * Abort the cleaning of a particular partition if it's in progress, and pause any future cleaning of this partition. + * This call blocks until the cleaning of the partition is aborted and paused. + */ + def abortAndPauseCleaning(topicPartition: TopicPartition): Unit = { + cleanerManager.abortAndPauseCleaning(topicPartition) + } + + /** + * Resume the cleaning of paused partitions. + */ + def resumeCleaning(topicPartitions: Iterable[TopicPartition]): Unit = { + cleanerManager.resumeCleaning(topicPartitions) + } + + /** + * For testing, a way to know when work has completed. This method waits until the + * cleaner has processed up to the given offset on the specified topic/partition + * + * @param topicPartition The topic and partition to be cleaned + * @param offset The first dirty offset that the cleaner doesn't have to clean + * @param maxWaitMs The maximum time in ms to wait for cleaner + * + * @return A boolean indicating whether the work has completed before timeout + */ + def awaitCleaned(topicPartition: TopicPartition, offset: Long, maxWaitMs: Long = 60000L): Boolean = { + def isCleaned = cleanerManager.allCleanerCheckpoints.get(topicPartition).fold(false)(_ >= offset) + var remainingWaitMs = maxWaitMs + while (!isCleaned && remainingWaitMs > 0) { + val sleepTime = math.min(100, remainingWaitMs) + Thread.sleep(sleepTime) + remainingWaitMs -= sleepTime + } + isCleaned + } + + /** + * To prevent race between retention and compaction, + * retention threads need to make this call to obtain: + * @return A list of log partitions that retention threads can safely work on + */ + def pauseCleaningForNonCompactedPartitions(): Iterable[(TopicPartition, Log)] = { + cleanerManager.pauseCleaningForNonCompactedPartitions() + } + + // Only for testing + private[kafka] def currentConfig: CleanerConfig = config + + // Only for testing + private[log] def cleanerCount: Int = cleaners.size + + /** + * The cleaner threads do the actual log cleaning. Each thread processes does its cleaning repeatedly by + * choosing the dirtiest log, cleaning it, and then swapping in the cleaned segments. + */ + private[log] class CleanerThread(threadId: Int) + extends ShutdownableThread(name = s"kafka-log-cleaner-thread-$threadId", isInterruptible = false) { + + protected override def loggerName = classOf[LogCleaner].getName + + if (config.dedupeBufferSize / config.numThreads > Int.MaxValue) + warn("Cannot use more than 2G of cleaner buffer space per cleaner thread, ignoring excess buffer space...") + + val cleaner = new Cleaner(id = threadId, + offsetMap = new SkimpyOffsetMap(memory = math.min(config.dedupeBufferSize / config.numThreads, Int.MaxValue).toInt, + hashAlgorithm = config.hashAlgorithm), + ioBufferSize = config.ioBufferSize / config.numThreads / 2, + maxIoBufferSize = config.maxMessageSize, + dupBufferLoadFactor = config.dedupeBufferLoadFactor, + throttler = throttler, + time = time, + checkDone = checkDone) + + @volatile var lastStats: CleanerStats = new CleanerStats() + @volatile var lastPreCleanStats: PreCleanStats = new PreCleanStats() + + private def checkDone(topicPartition: TopicPartition): Unit = { + if (!isRunning) + throw new ThreadShutdownException + cleanerManager.checkCleaningAborted(topicPartition) + } + + /** + * The main loop for the cleaner thread + * Clean a log if there is a dirty log available, otherwise sleep for a bit + */ + override def doWork(): Unit = { + val cleaned = tryCleanFilthiestLog() + if (!cleaned) + pause(config.backOffMs, TimeUnit.MILLISECONDS) + } + + /** + * Cleans a log if there is a dirty log available + * @return whether a log was cleaned + */ + private def tryCleanFilthiestLog(): Boolean = { + try { + cleanFilthiestLog() + } catch { + case e: LogCleaningException => + warn(s"Unexpected exception thrown when cleaning log ${e.log}. Marking its partition (${e.log.topicPartition}) as uncleanable", e) + cleanerManager.markPartitionUncleanable(e.log.dir.getParent, e.log.topicPartition) + + false + } + } + + @throws(classOf[LogCleaningException]) + private def cleanFilthiestLog(): Boolean = { + val preCleanStats = new PreCleanStats() + val cleaned = cleanerManager.grabFilthiestCompactedLog(time, preCleanStats) match { + case None => + false + case Some(cleanable) => + // there's a log, clean it + this.lastPreCleanStats = preCleanStats + try { + cleanLog(cleanable) + true + } catch { + case e @ (_: ThreadShutdownException | _: ControlThrowable) => throw e + case e: Exception => throw new LogCleaningException(cleanable.log, e.getMessage, e) + } + } + val deletable: Iterable[(TopicPartition, Log)] = cleanerManager.deletableLogs() + try { + deletable.foreach { case (_, log) => + try { + log.deleteOldSegments() + } catch { + case e @ (_: ThreadShutdownException | _: ControlThrowable) => throw e + case e: Exception => throw new LogCleaningException(log, e.getMessage, e) + } + } + } finally { + cleanerManager.doneDeleting(deletable.map(_._1)) + } + + cleaned + } + + private def cleanLog(cleanable: LogToClean): Unit = { + var endOffset = cleanable.firstDirtyOffset + try { + val (nextDirtyOffset, cleanerStats) = cleaner.clean(cleanable) + recordStats(cleaner.id, cleanable.log.name, cleanable.firstDirtyOffset, endOffset, cleanerStats) + endOffset = nextDirtyOffset + } catch { + case _: LogCleaningAbortedException => // task can be aborted, let it go. + case _: KafkaStorageException => // partition is already offline. let it go. + case e: IOException => + val logDirectory = cleanable.log.dir.getParent + val msg = s"Failed to clean up log for ${cleanable.topicPartition} in dir ${logDirectory} due to IOException" + logDirFailureChannel.maybeAddOfflineLogDir(logDirectory, msg, e) + } finally { + cleanerManager.doneCleaning(cleanable.topicPartition, cleanable.log.dir.getParentFile, endOffset) + } + } + + /** + * Log out statistics on a single run of the cleaner. + */ + def recordStats(id: Int, name: String, from: Long, to: Long, stats: CleanerStats): Unit = { + this.lastStats = stats + def mb(bytes: Double) = bytes / (1024*1024) + val message = + "%n\tLog cleaner thread %d cleaned log %s (dirty section = [%d, %d])%n".format(id, name, from, to) + + "\t%,.1f MB of log processed in %,.1f seconds (%,.1f MB/sec).%n".format(mb(stats.bytesRead), + stats.elapsedSecs, + mb(stats.bytesRead/stats.elapsedSecs)) + + "\tIndexed %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.mapBytesRead), + stats.elapsedIndexSecs, + mb(stats.mapBytesRead)/stats.elapsedIndexSecs, + 100 * stats.elapsedIndexSecs/stats.elapsedSecs) + + "\tBuffer utilization: %.1f%%%n".format(100 * stats.bufferUtilization) + + "\tCleaned %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.bytesRead), + stats.elapsedSecs - stats.elapsedIndexSecs, + mb(stats.bytesRead)/(stats.elapsedSecs - stats.elapsedIndexSecs), 100 * (stats.elapsedSecs - stats.elapsedIndexSecs).toDouble/stats.elapsedSecs) + + "\tStart size: %,.1f MB (%,d messages)%n".format(mb(stats.bytesRead), stats.messagesRead) + + "\tEnd size: %,.1f MB (%,d messages)%n".format(mb(stats.bytesWritten), stats.messagesWritten) + + "\t%.1f%% size reduction (%.1f%% fewer messages)%n".format(100.0 * (1.0 - stats.bytesWritten.toDouble/stats.bytesRead), + 100.0 * (1.0 - stats.messagesWritten.toDouble/stats.messagesRead)) + info(message) + if (lastPreCleanStats.delayedPartitions > 0) { + info("\tCleanable partitions: %d, Delayed partitions: %d, max delay: %d".format(lastPreCleanStats.cleanablePartitions, lastPreCleanStats.delayedPartitions, lastPreCleanStats.maxCompactionDelayMs)) + } + if (stats.invalidMessagesRead > 0) { + warn("\tFound %d invalid messages during compaction.".format(stats.invalidMessagesRead)) + } + } + + } +} + +object LogCleaner { + val ReconfigurableConfigs = Set( + KafkaConfig.LogCleanerThreadsProp, + KafkaConfig.LogCleanerDedupeBufferSizeProp, + KafkaConfig.LogCleanerDedupeBufferLoadFactorProp, + KafkaConfig.LogCleanerIoBufferSizeProp, + KafkaConfig.MessageMaxBytesProp, + KafkaConfig.LogCleanerIoMaxBytesPerSecondProp, + KafkaConfig.LogCleanerBackoffMsProp + ) + + def cleanerConfig(config: KafkaConfig): CleanerConfig = { + CleanerConfig(numThreads = config.logCleanerThreads, + dedupeBufferSize = config.logCleanerDedupeBufferSize, + dedupeBufferLoadFactor = config.logCleanerDedupeBufferLoadFactor, + ioBufferSize = config.logCleanerIoBufferSize, + maxMessageSize = config.messageMaxBytes, + maxIoBytesPerSecond = config.logCleanerIoMaxBytesPerSecond, + backOffMs = config.logCleanerBackoffMs, + enableCleaner = config.logCleanerEnable) + + } + + def createNewCleanedSegment(log: Log, baseOffset: Long): LogSegment = { + LogSegment.deleteIfExists(log.dir, baseOffset, fileSuffix = Log.CleanedFileSuffix) + LogSegment.open(log.dir, baseOffset, log.config, Time.SYSTEM, fileAlreadyExists = false, + fileSuffix = Log.CleanedFileSuffix, initFileSize = log.initFileSize, preallocate = log.config.preallocate) + } + +} + +/** + * This class holds the actual logic for cleaning a log + * @param id An identifier used for logging + * @param offsetMap The map used for deduplication + * @param ioBufferSize The size of the buffers to use. Memory usage will be 2x this number as there is a read and write buffer. + * @param maxIoBufferSize The maximum size of a message that can appear in the log + * @param dupBufferLoadFactor The maximum percent full for the deduplication buffer + * @param throttler The throttler instance to use for limiting I/O rate. + * @param time The time instance + * @param checkDone Check if the cleaning for a partition is finished or aborted. + */ +private[log] class Cleaner(val id: Int, + val offsetMap: OffsetMap, + ioBufferSize: Int, + maxIoBufferSize: Int, + dupBufferLoadFactor: Double, + throttler: Throttler, + time: Time, + checkDone: TopicPartition => Unit) extends Logging { + + protected override def loggerName = classOf[LogCleaner].getName + + this.logIdent = s"Cleaner $id: " + + /* buffer used for read i/o */ + private var readBuffer = ByteBuffer.allocate(ioBufferSize) + + /* buffer used for write i/o */ + private var writeBuffer = ByteBuffer.allocate(ioBufferSize) + + private val decompressionBufferSupplier = BufferSupplier.create(); + + require(offsetMap.slots * dupBufferLoadFactor > 1, "offset map is too small to fit in even a single message, so log cleaning will never make progress. You can increase log.cleaner.dedupe.buffer.size or decrease log.cleaner.threads") + + /** + * Clean the given log + * + * @param cleanable The log to be cleaned + * + * @return The first offset not cleaned and the statistics for this round of cleaning + */ + private[log] def clean(cleanable: LogToClean): (Long, CleanerStats) = { + // figure out the timestamp below which it is safe to remove delete tombstones + // this position is defined to be a configurable time beneath the last modified time of the last clean segment + val deleteHorizonMs = + cleanable.log.logSegments(0, cleanable.firstDirtyOffset).lastOption match { + case None => 0L + case Some(seg) => seg.lastModified - cleanable.log.config.deleteRetentionMs + } + doClean(cleanable, time.milliseconds(), trackedHorizon = deleteHorizonMs) + } + + private[log] def doClean(cleanable: LogToClean, currentTime: Long, trackedHorizon: Long = -1L): (Long, CleanerStats) = { + info("Beginning cleaning of log %s.".format(cleanable.log.name)) + + val log = cleanable.log + val stats = new CleanerStats() + + // build the offset map + info("Building offset map for %s...".format(cleanable.log.name)) + val upperBoundOffset = cleanable.firstUncleanableOffset + buildOffsetMap(log, cleanable.firstDirtyOffset, upperBoundOffset, offsetMap, stats) + val endOffset = offsetMap.latestOffset + 1 + stats.indexDone() + + // determine the timestamp up to which the log will be cleaned + // this is the lower of the last active segment and the compaction lag + val cleanableHorizonMs = log.logSegments(0, cleanable.firstUncleanableOffset).lastOption.map(_.lastModified).getOrElse(0L) + + // group the segments and clean the groups + info("Cleaning log %s (cleaning prior to %s, discarding tombstones prior to %s)...".format(log.name, new Date(cleanableHorizonMs), new Date(trackedHorizon))) + val transactionMetadata = new CleanedTransactionMetadata + + val groupedSegments = groupSegmentsBySize(log.logSegments(0, endOffset), log.config.segmentSize, + log.config.maxIndexSize, cleanable.firstUncleanableOffset) + for (group <- groupedSegments) + cleanSegments(log, group, offsetMap, currentTime, stats, transactionMetadata, trackedHorizon = trackedHorizon) + + // record buffer utilization + stats.bufferUtilization = offsetMap.utilization + + stats.allDone() + + (endOffset, stats) + } + + /** + * Clean a group of segments into a single replacement segment + * + * @param log The log being cleaned + * @param segments The group of segments being cleaned + * @param map The offset map to use for cleaning segments + * @param deleteHorizonMs The time to retain delete tombstones + * @param stats Collector for cleaning statistics + * @param transactionMetadata State of ongoing transactions which is carried between the cleaning + * of the grouped segments + */ + private[log] def cleanSegments(log: Log, + segments: Seq[LogSegment], + map: OffsetMap, + currentTime: Long, + stats: CleanerStats, + transactionMetadata: CleanedTransactionMetadata, + trackedHorizon: Long = -1L): Unit = { + // create a new segment with a suffix appended to the name of the log and indexes + val cleaned = LogCleaner.createNewCleanedSegment(log, segments.head.baseOffset) + transactionMetadata.cleanedIndex = Some(cleaned.txnIndex) + + try { + // clean segments into the new destination segment + val iter = segments.iterator + var currentSegmentOpt: Option[LogSegment] = Some(iter.next()) + val lastOffsetOfActiveProducers = log.lastRecordsOfActiveProducers + + while (currentSegmentOpt.isDefined) { + val currentSegment = currentSegmentOpt.get + val nextSegmentOpt = if (iter.hasNext) Some(iter.next()) else None + + val startOffset = currentSegment.baseOffset + val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(map.latestOffset + 1) + val abortedTransactions = log.collectAbortedTransactions(startOffset, upperBoundOffset) + transactionMetadata.addAbortedTransactions(abortedTransactions) + + val retainDeletesAndTxnMarkers = currentSegment.lastModified > trackedHorizon + info(s"Cleaning $currentSegment in log ${log.name} into ${cleaned.baseOffset} " + + s"with deletion horizon $trackedHorizon, " + + s"${if(retainDeletesAndTxnMarkers) "retaining" else "discarding"} deletes.") + + try { + val containsTombstones: Boolean = cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletesAndTxnMarkers, log.config.maxMessageSize, + transactionMetadata, lastOffsetOfActiveProducers, stats, log.config.deleteRetentionMs, currentTime = currentTime) + log.containsTombstones = containsTombstones + } catch { + case e: LogSegmentOffsetOverflowException => + // Split the current segment. It's also safest to abort the current cleaning process, so that we retry from + // scratch once the split is complete. + info(s"Caught segment overflow error during cleaning: ${e.getMessage}") + log.splitOverflowedSegment(currentSegment) + throw new LogCleaningAbortedException() + } + currentSegmentOpt = nextSegmentOpt + } + + cleaned.onBecomeInactiveSegment() + // flush new segment to disk before swap + cleaned.flush() + + // update the modification date to retain the last modified date of the original files + val modified = segments.last.lastModified + cleaned.lastModified = modified + + // swap in new segment + info(s"Swapping in cleaned segment $cleaned for segment(s) $segments in log $log") + log.replaceSegments(List(cleaned), segments) + } catch { + case e: LogCleaningAbortedException => + try cleaned.deleteIfExists() + catch { + case deleteException: Exception => + e.addSuppressed(deleteException) + } finally throw e + } + } + + /** + * Clean the given source log segment into the destination segment using the key=>offset mapping + * provided + * + * @param topicPartition The topic and partition of the log segment to clean + * @param sourceRecords The dirty log segment + * @param dest The cleaned log segment + * @param map The key=>offset mapping + * @param retainDeletesAndTxnMarkers Should tombstones and markers be retained while cleaning this segment + * @param maxLogMessageSize The maximum message size of the corresponding topic + * @param stats Collector for cleaning statistics + * @param tombstoneRetentionMs Defines how long a tombstone should be kept as defined by log configuration + */ + private[log] def cleanInto(topicPartition: TopicPartition, + sourceRecords: FileRecords, + dest: LogSegment, + map: OffsetMap, + retainDeletesAndTxnMarkers: Boolean, + maxLogMessageSize: Int, + transactionMetadata: CleanedTransactionMetadata, + lastRecordsOfActiveProducers: Map[Long, LastRecord], + stats: CleanerStats, + tombstoneRetentionMs: Long, + currentTime: Long = RecordBatch.NO_TIMESTAMP): Boolean = { + var containsTombstones: Boolean = false + + val logCleanerFilter: RecordFilter = new RecordFilter { + var discardBatchRecords: Boolean = _ + var isControlBatchEmpty: Boolean = _ + + override def isControlBatchEmpty(batch: RecordBatch) : Boolean = { + // we piggy-back on the tombstone retention logic to delay deletion of transaction markers. + // note that we will never delete a marker until all the records from that transaction are removed. + val canDiscardBatch = shouldDiscardBatch(batch, transactionMetadata, retainTxnMarkers = retainDeletesAndTxnMarkers) + isControlBatchEmpty = canDiscardBatch + isControlBatchEmpty + } + + override def checkBatchRetention(batch: RecordBatch, newBatchDeleteHorizonMs : Long): BatchRetention = { + val canDiscardBatch = isControlBatchEmpty + + if (batch.isControlBatch) { + discardBatchRecords = canDiscardBatch && + ((batch.deleteHorizonSet() && batch.deleteHorizonMs() < currentTime) || + newBatchDeleteHorizonMs != RecordBatch.NO_TIMESTAMP && newBatchDeleteHorizonMs < currentTime) + } else { + discardBatchRecords = canDiscardBatch + } + + def isBatchLastRecordOfProducer: Boolean = { + // We retain the batch in order to preserve the state of active producers. There are three cases: + // 1) The producer is no longer active, which means we can delete all records for that producer. + // 2) The producer is still active and has a last data offset. We retain the batch that contains + // this offset since it also contains the last sequence number for this producer. + // 3) The last entry in the log is a transaction marker. We retain this marker since it has the + // last producer epoch, which is needed to ensure fencing. + lastRecordsOfActiveProducers.get(batch.producerId).exists { lastRecord => + lastRecord.lastDataOffset match { + case Some(offset) => batch.lastOffset == offset + case None => batch.isControlBatch && batch.producerEpoch == lastRecord.producerEpoch + } + } + } + + if (batch.hasProducerId && isBatchLastRecordOfProducer) + BatchRetention.RETAIN_EMPTY + else if (discardBatchRecords) + BatchRetention.DELETE + else + BatchRetention.DELETE_EMPTY + } + + override def checkBatchRetention(batch: RecordBatch): BatchRetention = checkBatchRetention(batch, batch.deleteHorizonMs()) + + override def shouldRetainRecord(batch: RecordBatch, record: Record, newDeleteHorizonMs: Long): Boolean = { + var isRecordRetained: Boolean = true + if (discardBatchRecords) + // The batch is only retained to preserve producer sequence information; the records can be removed + isRecordRetained = false + else + isRecordRetained = Cleaner.this.shouldRetainRecord(map, retainDeletesAndTxnMarkers, batch, record, stats, newDeleteHorizonMs, currentTime = currentTime) + if (isRecordRetained && !record.hasValue()) + containsTombstones = true + isRecordRetained + } + + override def shouldRetainRecord(batch: RecordBatch, record: Record): Boolean = { + shouldRetainRecord(batch, record, RecordBatch.NO_TIMESTAMP) + } + + override def retrieveDeleteHorizon(batch: RecordBatch) : Long = { + if (batch.deleteHorizonSet()) + return batch.deleteHorizonMs() // means that we keep the old timestamp stored + + // check that the control batch has been emptied of records + // if not, then we do not set a delete horizon until that is true + if (batch.isControlBatch() && !isControlBatchEmpty) + return -1L + return time.milliseconds() + tombstoneRetentionMs; + } + } + + var position = 0 + while (position < sourceRecords.sizeInBytes) { + checkDone(topicPartition) + // read a chunk of messages and copy any that are to be retained to the write buffer to be written out + readBuffer.clear() + writeBuffer.clear() + + sourceRecords.readInto(readBuffer, position) + val records = MemoryRecords.readableRecords(readBuffer) + throttler.maybeThrottle(records.sizeInBytes) + val result = records.filterTo(topicPartition, logCleanerFilter, writeBuffer, maxLogMessageSize, decompressionBufferSupplier) + stats.readMessages(result.messagesRead, result.bytesRead) + stats.recopyMessages(result.messagesRetained, result.bytesRetained) + + position += result.bytesRead + + // if any messages are to be retained, write them out + val outputBuffer = result.outputBuffer + if (outputBuffer.position() > 0) { + outputBuffer.flip() + val retained = MemoryRecords.readableRecords(outputBuffer) + // it's OK not to hold the Log's lock in this case, because this segment is only accessed by other threads + // after `Log.replaceSegments` (which acquires the lock) is called + dest.append(largestOffset = result.maxOffset, + largestTimestamp = result.maxTimestamp, + shallowOffsetOfMaxTimestamp = result.shallowOffsetOfMaxTimestamp, + records = retained) + throttler.maybeThrottle(outputBuffer.limit()) + } + + // if we read bytes but didn't get even one complete batch, our I/O buffer is too small, grow it and try again + // `result.bytesRead` contains bytes from `messagesRead` and any discarded batches. + if (readBuffer.limit() > 0 && result.bytesRead == 0) + growBuffersOrFail(sourceRecords, position, maxLogMessageSize, records) + } + restoreBuffers() + containsTombstones + } + + + /** + * Grow buffers to process next batch of records from `sourceRecords.` Buffers are doubled in size + * up to a maximum of `maxLogMessageSize`. In some scenarios, a record could be bigger than the + * current maximum size configured for the log. For example: + * 1. A compacted topic using compression may contain a message set slightly larger than max.message.bytes + * 2. max.message.bytes of a topic could have been reduced after writing larger messages + * In these cases, grow the buffer to hold the next batch. + */ + private def growBuffersOrFail(sourceRecords: FileRecords, + position: Int, + maxLogMessageSize: Int, + memoryRecords: MemoryRecords): Unit = { + + val maxSize = if (readBuffer.capacity >= maxLogMessageSize) { + val nextBatchSize = memoryRecords.firstBatchSize + val logDesc = s"log segment ${sourceRecords.file} at position $position" + if (nextBatchSize == null) + throw new IllegalStateException(s"Could not determine next batch size for $logDesc") + if (nextBatchSize <= 0) + throw new IllegalStateException(s"Invalid batch size $nextBatchSize for $logDesc") + if (nextBatchSize <= readBuffer.capacity) + throw new IllegalStateException(s"Batch size $nextBatchSize < buffer size ${readBuffer.capacity}, but not processed for $logDesc") + val bytesLeft = sourceRecords.channel.size - position + if (nextBatchSize > bytesLeft) + throw new CorruptRecordException(s"Log segment may be corrupt, batch size $nextBatchSize > $bytesLeft bytes left in segment for $logDesc") + nextBatchSize.intValue + } else + maxLogMessageSize + + growBuffers(maxSize) + } + + private def shouldDiscardBatch(batch: RecordBatch, + transactionMetadata: CleanedTransactionMetadata, + retainTxnMarkers: Boolean): Boolean = { + if (batch.isControlBatch) + transactionMetadata.onControlBatchRead(batch) + else + transactionMetadata.onBatchRead(batch) + } + + private def shouldRetainRecord(map: kafka.log.OffsetMap, + retainDeletes: Boolean, + batch: RecordBatch, + record: Record, + stats: CleanerStats, + newBatchDeleteHorizonMs: Long, + currentTime: Long = -1L): Boolean = { + val pastLatestOffset = record.offset > map.latestOffset + if (pastLatestOffset) + return true + + if (record.hasKey) { + val key = record.key + val foundOffset = map.get(key) + /* First,the message must have the latest offset for the key + * then there are two cases in which we can retain a message: + * 1) The message has value + * 2) The message doesn't has value but it can't be deleted now. + */ + val latestOffsetForKey = record.offset() >= foundOffset + val isLatestVersion = batch.magic() >= RecordBatch.MAGIC_VALUE_V2 + var shouldRetainDeletes = true + if (isLatestVersion) + shouldRetainDeletes = (batch.deleteHorizonSet() && currentTime < batch.deleteHorizonMs()) || + (!batch.deleteHorizonSet() && currentTime < newBatchDeleteHorizonMs) + else + shouldRetainDeletes = retainDeletes + val isRetainedValue = record.hasValue || shouldRetainDeletes + + latestOffsetForKey && isRetainedValue + } else { + stats.invalidMessage() + false + } + } + + /** + * Double the I/O buffer capacity + */ + def growBuffers(maxLogMessageSize: Int): Unit = { + val maxBufferSize = math.max(maxLogMessageSize, maxIoBufferSize) + if(readBuffer.capacity >= maxBufferSize || writeBuffer.capacity >= maxBufferSize) + throw new IllegalStateException("This log contains a message larger than maximum allowable size of %s.".format(maxBufferSize)) + val newSize = math.min(this.readBuffer.capacity * 2, maxBufferSize) + info(s"Growing cleaner I/O buffers from ${readBuffer.capacity} bytes to $newSize bytes.") + this.readBuffer = ByteBuffer.allocate(newSize) + this.writeBuffer = ByteBuffer.allocate(newSize) + } + + /** + * Restore the I/O buffer capacity to its original size + */ + def restoreBuffers(): Unit = { + if(this.readBuffer.capacity > this.ioBufferSize) + this.readBuffer = ByteBuffer.allocate(this.ioBufferSize) + if(this.writeBuffer.capacity > this.ioBufferSize) + this.writeBuffer = ByteBuffer.allocate(this.ioBufferSize) + } + + /** + * Group the segments in a log into groups totaling less than a given size. the size is enforced separately for the log data and the index data. + * We collect a group of such segments together into a single + * destination segment. This prevents segment sizes from shrinking too much. + * + * @param segments The log segments to group + * @param maxSize the maximum size in bytes for the total of all log data in a group + * @param maxIndexSize the maximum size in bytes for the total of all index data in a group + * + * @return A list of grouped segments + */ + private[log] def groupSegmentsBySize(segments: Iterable[LogSegment], maxSize: Int, maxIndexSize: Int, firstUncleanableOffset: Long): List[Seq[LogSegment]] = { + var grouped = List[List[LogSegment]]() + var segs = segments.toList + while(segs.nonEmpty) { + var group = List(segs.head) + var logSize = segs.head.size.toLong + var indexSize = segs.head.offsetIndex.sizeInBytes.toLong + var timeIndexSize = segs.head.timeIndex.sizeInBytes.toLong + segs = segs.tail + while(segs.nonEmpty && + logSize + segs.head.size <= maxSize && + indexSize + segs.head.offsetIndex.sizeInBytes <= maxIndexSize && + timeIndexSize + segs.head.timeIndex.sizeInBytes <= maxIndexSize && + lastOffsetForFirstSegment(segs, firstUncleanableOffset) - group.last.baseOffset <= Int.MaxValue) { + group = segs.head :: group + logSize += segs.head.size + indexSize += segs.head.offsetIndex.sizeInBytes + timeIndexSize += segs.head.timeIndex.sizeInBytes + segs = segs.tail + } + grouped ::= group.reverse + } + grouped.reverse + } + + /** + * We want to get the last offset in the first log segment in segs. + * LogSegment.nextOffset() gives the exact last offset in a segment, but can be expensive since it requires + * scanning the segment from the last index entry. + * Therefore, we estimate the last offset of the first log segment by using + * the base offset of the next segment in the list. + * If the next segment doesn't exist, first Uncleanable Offset will be used. + * + * @param segs - remaining segments to group. + * @return The estimated last offset for the first segment in segs + */ + private def lastOffsetForFirstSegment(segs: List[LogSegment], firstUncleanableOffset: Long): Long = { + if (segs.size > 1) { + /* if there is a next segment, use its base offset as the bounding offset to guarantee we know + * the worst case offset */ + segs(1).baseOffset - 1 + } else { + //for the last segment in the list, use the first uncleanable offset. + firstUncleanableOffset - 1 + } + } + + /** + * Build a map of key_hash => offset for the keys in the cleanable dirty portion of the log to use in cleaning. + * @param log The log to use + * @param start The offset at which dirty messages begin + * @param end The ending offset for the map that is being built + * @param map The map in which to store the mappings + * @param stats Collector for cleaning statistics + */ + private[log] def buildOffsetMap(log: Log, + start: Long, + end: Long, + map: OffsetMap, + stats: CleanerStats): Unit = { + map.clear() + val dirty = log.logSegments(start, end).toBuffer + info("Building offset map for log %s for %d segments in offset range [%d, %d).".format(log.name, dirty.size, start, end)) + + val transactionMetadata = new CleanedTransactionMetadata + val abortedTransactions = log.collectAbortedTransactions(start, end) + transactionMetadata.addAbortedTransactions(abortedTransactions) + + // Add all the cleanable dirty segments. We must take at least map.slots * load_factor, + // but we may be able to fit more (if there is lots of duplication in the dirty section of the log) + var full = false + for (segment <- dirty if !full) { + checkDone(log.topicPartition) + + full = buildOffsetMapForSegment(log.topicPartition, segment, map, start, log.config.maxMessageSize, + transactionMetadata, stats) + if (full) + debug("Offset map is full, %d segments fully mapped, segment with base offset %d is partially mapped".format(dirty.indexOf(segment), segment.baseOffset)) + } + info("Offset map for log %s complete.".format(log.name)) + } + + /** + * Add the messages in the given segment to the offset map + * + * @param segment The segment to index + * @param map The map in which to store the key=>offset mapping + * @param stats Collector for cleaning statistics + * + * @return If the map was filled whilst loading from this segment + */ + private def buildOffsetMapForSegment(topicPartition: TopicPartition, + segment: LogSegment, + map: OffsetMap, + startOffset: Long, + maxLogMessageSize: Int, + transactionMetadata: CleanedTransactionMetadata, + stats: CleanerStats): Boolean = { + var position = segment.offsetIndex.lookup(startOffset).position + val maxDesiredMapSize = (map.slots * this.dupBufferLoadFactor).toInt + while (position < segment.log.sizeInBytes) { + checkDone(topicPartition) + readBuffer.clear() + try { + segment.log.readInto(readBuffer, position) + } catch { + case e: Exception => + throw new KafkaException(s"Failed to read from segment $segment of partition $topicPartition " + + "while loading offset map", e) + } + val records = MemoryRecords.readableRecords(readBuffer) + throttler.maybeThrottle(records.sizeInBytes) + + val startPosition = position + for (batch <- records.batches.asScala) { + if (batch.isControlBatch) { + transactionMetadata.onControlBatchRead(batch) + stats.indexMessagesRead(1) + } else { + val isAborted = transactionMetadata.onBatchRead(batch) + if (isAborted) { + // If the batch is aborted, do not bother populating the offset map. + // Note that abort markers are supported in v2 and above, which means count is defined. + stats.indexMessagesRead(batch.countOrNull) + } else { + for (record <- batch.asScala) { + if (record.hasKey && record.offset >= startOffset) { + if (map.size < maxDesiredMapSize) + map.put(record.key, record.offset) + else + return true + } + stats.indexMessagesRead(1) + } + } + } + + if (batch.lastOffset >= startOffset) + map.updateLatestOffset(batch.lastOffset) + } + val bytesRead = records.validBytes + position += bytesRead + stats.indexBytesRead(bytesRead) + + // if we didn't read even one complete message, our read buffer may be too small + if(position == startPosition) + growBuffersOrFail(segment.log, position, maxLogMessageSize, records) + } + restoreBuffers() + false + } +} + +/** + * A simple struct for collecting pre-clean stats + */ +private class PreCleanStats() { + var maxCompactionDelayMs = 0L + var delayedPartitions = 0 + var cleanablePartitions = 0 + + def updateMaxCompactionDelay(delayMs: Long): Unit = { + maxCompactionDelayMs = Math.max(maxCompactionDelayMs, delayMs) + if (delayMs > 0) { + delayedPartitions += 1 + } + } + def recordCleanablePartitions(numOfCleanables: Int): Unit = { + cleanablePartitions = numOfCleanables + } +} + +/** + * A simple struct for collecting stats about log cleaning + */ +private class CleanerStats(time: Time = Time.SYSTEM) { + val startTime = time.milliseconds + var mapCompleteTime = -1L + var endTime = -1L + var bytesRead = 0L + var bytesWritten = 0L + var mapBytesRead = 0L + var mapMessagesRead = 0L + var messagesRead = 0L + var invalidMessagesRead = 0L + var messagesWritten = 0L + var bufferUtilization = 0.0d + + def readMessages(messagesRead: Int, bytesRead: Int): Unit = { + this.messagesRead += messagesRead + this.bytesRead += bytesRead + } + + def invalidMessage(): Unit = { + invalidMessagesRead += 1 + } + + def recopyMessages(messagesWritten: Int, bytesWritten: Int): Unit = { + this.messagesWritten += messagesWritten + this.bytesWritten += bytesWritten + } + + def indexMessagesRead(size: Int): Unit = { + mapMessagesRead += size + } + + def indexBytesRead(size: Int): Unit = { + mapBytesRead += size + } + + def indexDone(): Unit = { + mapCompleteTime = time.milliseconds + } + + def allDone(): Unit = { + endTime = time.milliseconds + } + + def elapsedSecs = (endTime - startTime)/1000.0 + + def elapsedIndexSecs = (mapCompleteTime - startTime)/1000.0 + +} + +/** + * Helper class for a log, its topic/partition, the first cleanable position, the first uncleanable dirty position, + * and whether it needs compaction immediately. + */ +private case class LogToClean(topicPartition: TopicPartition, + log: Log, + firstDirtyOffset: Long, + uncleanableOffset: Long, + needCompactionNow: Boolean = false) extends Ordered[LogToClean] { + val cleanBytes = log.logSegments(-1, firstDirtyOffset).map(_.size.toLong).sum + val (firstUncleanableOffset, cleanableBytes) = LogCleanerManager.calculateCleanableBytes(log, firstDirtyOffset, uncleanableOffset) + val totalBytes = cleanBytes + cleanableBytes + val cleanableRatio = cleanableBytes / totalBytes.toDouble + override def compare(that: LogToClean): Int = math.signum(this.cleanableRatio - that.cleanableRatio).toInt +} + +/** + * This is a helper class to facilitate tracking transaction state while cleaning the log. It maintains a set + * of the ongoing aborted and committed transactions as the cleaner is working its way through the log. This + * class is responsible for deciding when transaction markers can be removed and is therefore also responsible + * for updating the cleaned transaction index accordingly. + */ +private[log] class CleanedTransactionMetadata { + private val ongoingCommittedTxns = mutable.Set.empty[Long] + private val ongoingAbortedTxns = mutable.Map.empty[Long, AbortedTransactionMetadata] + // Minheap of aborted transactions sorted by the transaction first offset + private val abortedTransactions = mutable.PriorityQueue.empty[AbortedTxn](new Ordering[AbortedTxn] { + override def compare(x: AbortedTxn, y: AbortedTxn): Int = x.firstOffset compare y.firstOffset + }.reverse) + + // Output cleaned index to write retained aborted transactions + var cleanedIndex: Option[TransactionIndex] = None + + def addAbortedTransactions(abortedTransactions: List[AbortedTxn]): Unit = { + this.abortedTransactions ++= abortedTransactions + } + + /** + * Update the cleaned transaction state with a control batch that has just been traversed by the cleaner. + * Return true if the control batch can be discarded. + */ + def onControlBatchRead(controlBatch: RecordBatch): Boolean = { + consumeAbortedTxnsUpTo(controlBatch.lastOffset) + + val controlRecordIterator = controlBatch.iterator + if (controlRecordIterator.hasNext) { + val controlRecord = controlRecordIterator.next() + val controlType = ControlRecordType.parse(controlRecord.key) + val producerId = controlBatch.producerId + controlType match { + case ControlRecordType.ABORT => + ongoingAbortedTxns.remove(producerId) match { + // Retain the marker until all batches from the transaction have been removed. + // We may retain a record from an aborted transaction if it is the last entry + // written by a given producerId. + case Some(abortedTxnMetadata) if abortedTxnMetadata.lastObservedBatchOffset.isDefined => + cleanedIndex.foreach(_.append(abortedTxnMetadata.abortedTxn)) + false + case _ => true + } + + case ControlRecordType.COMMIT => + // This marker is eligible for deletion if we didn't traverse any batches from the transaction + !ongoingCommittedTxns.remove(producerId) + + case _ => false + } + } else { + // An empty control batch was already cleaned, so it's safe to discard + true + } + } + + private def consumeAbortedTxnsUpTo(offset: Long): Unit = { + while (abortedTransactions.headOption.exists(_.firstOffset <= offset)) { + val abortedTxn = abortedTransactions.dequeue() + ongoingAbortedTxns.getOrElseUpdate(abortedTxn.producerId, new AbortedTransactionMetadata(abortedTxn)) + } + } + + /** + * Update the transactional state for the incoming non-control batch. If the batch is part of + * an aborted transaction, return true to indicate that it is safe to discard. + */ + def onBatchRead(batch: RecordBatch): Boolean = { + consumeAbortedTxnsUpTo(batch.lastOffset) + if (batch.isTransactional) { + ongoingAbortedTxns.get(batch.producerId) match { + case Some(abortedTransactionMetadata) => + abortedTransactionMetadata.lastObservedBatchOffset = Some(batch.lastOffset) + true + case None => + ongoingCommittedTxns += batch.producerId + false + } + } else { + false + } + } + +} + +private class AbortedTransactionMetadata(val abortedTxn: AbortedTxn) { + var lastObservedBatchOffset: Option[Long] = None + + override def toString: String = s"(txn: $abortedTxn, lastOffset: $lastObservedBatchOffset)" +} diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala index 28c7822b0ce3b..b0c3bbccf032f 100755 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -177,12 +177,11 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], val now = time.milliseconds this.timeOfLastRun = now val lastClean = allCleanerCheckpoints + val dirtyLogs = logs.filter { - case (_, log) => log.config.compact // match logs that are marked as compacted + case (_, log) => log.config.compact }.filterNot { case (topicPartition, log) => - - // skip any logs already in-progress and uncleanable partitions inProgress.contains(topicPartition) || isUncleanablePartition(log, topicPartition) }.map { case (topicPartition, log) => // create a LogToClean instance for each @@ -204,34 +203,18 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], val cleanableLogs = dirtyLogs.filter { ltc => (ltc.needCompactionNow && ltc.cleanableBytes > 0) || ltc.cleanableRatio > ltc.log.config.minCleanableRatio } + if(cleanableLogs.isEmpty) { - // in this case, we are probably in a low throughput situation - // therefore, we should take advantage of this fact and remove tombstones if we can - // under the condition that the log's latest delete horizon is less than the current time - // tracked - val logsContainingTombstones = logs.filter { - case (_, log) => log.latestDeleteHorizon != -1L && - log.latestDeleteHorizon <= time.milliseconds() - }.filterNot { - case (topicPartition, log) => - // skip any logs already in-progress and uncleanable partitions - inProgress.contains(topicPartition) || isUncleanablePartition(log, topicPartition) - }.map { - case (topicPartition, log) => // create a LogToClean instance for each - try { - val lastCleanOffset = lastClean.get(topicPartition) - val (firstDirtyOffset, firstUncleanableDirtyOffset) = cleanableOffsets(log, lastCleanOffset, now) - val compactionDelayMs = maxCompactionDelay(log, firstDirtyOffset, now) - preCleanStats.updateMaxCompactionDelay(compactionDelayMs) - - LogToClean(topicPartition, log, firstDirtyOffset, firstUncleanableDirtyOffset, compactionDelayMs > 0) - } catch { - case e: Throwable => throw new LogCleaningException(log, - s"Failed to calculate log cleaning stats for partition $topicPartition", e) - } + val logsWithTombstonesExpired = dirtyLogs.filter { + case ltc => + // in this case, we are probably in a low throughput situation + // therefore, we should take advantage of this fact and remove tombstones if we can + // under the condition that the log's latest delete horizon is less than the current time + // tracked + ltc.log.latestDeleteHorizon != -1L && ltc.log.latestDeleteHorizon <= time.milliseconds() } - if (!logsContainingTombstones.isEmpty) { - val filthiest = logsContainingTombstones.max + if (!logsWithTombstonesExpired.isEmpty) { + val filthiest = logsWithTombstonesExpired.max inProgress.put(filthiest.topicPartition, LogCleaningInProgress) Some(filthiest) } else { diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala.orig b/core/src/main/scala/kafka/log/LogCleanerManager.scala.orig new file mode 100755 index 0000000000000..561a6437d91ae --- /dev/null +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala.orig @@ -0,0 +1,626 @@ +/** + * 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 kafka.log + +import java.io.File +import java.util.concurrent.TimeUnit +import java.util.concurrent.locks.ReentrantLock + +import com.yammer.metrics.core.Gauge +import kafka.common.{KafkaException, LogCleaningAbortedException} +import kafka.metrics.KafkaMetricsGroup +import kafka.server.LogDirFailureChannel +import kafka.server.checkpoints.OffsetCheckpointFile +import kafka.utils.CoreUtils._ +import kafka.utils.{Logging, Pool} +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.utils.Time +import org.apache.kafka.common.errors.KafkaStorageException + +import scala.collection.{Iterable, Seq, mutable} + +private[log] sealed trait LogCleaningState +private[log] case object LogCleaningInProgress extends LogCleaningState +private[log] case object LogCleaningAborted extends LogCleaningState +private[log] case class LogCleaningPaused(pausedCount: Int) extends LogCleaningState + +private[log] class LogCleaningException(val log: Log, + private val message: String, + private val cause: Throwable) extends KafkaException(message, cause) + +/** + * This class manages the state of each partition being cleaned. + * LogCleaningState defines the cleaning states that a TopicPartition can be in. + * 1. None : No cleaning state in a TopicPartition. In this state, it can become LogCleaningInProgress + * or LogCleaningPaused(1). Valid previous state are LogCleaningInProgress and LogCleaningPaused(1) + * 2. LogCleaningInProgress : The cleaning is currently in progress. In this state, it can become None when log cleaning is finished + * or become LogCleaningAborted. Valid previous state is None. + * 3. LogCleaningAborted : The cleaning abort is requested. In this state, it can become LogCleaningPaused(1). + * Valid previous state is LogCleaningInProgress. + * 4-a. LogCleaningPaused(1) : The cleaning is paused once. No log cleaning can be done in this state. + * In this state, it can become None or LogCleaningPaused(2). + * Valid previous state is None, LogCleaningAborted or LogCleaningPaused(2). + * 4-b. LogCleaningPaused(i) : The cleaning is paused i times where i>= 2. No log cleaning can be done in this state. + * In this state, it can become LogCleaningPaused(i-1) or LogCleaningPaused(i+1). + * Valid previous state is LogCleaningPaused(i-1) or LogCleaningPaused(i+1). + */ +private[log] class LogCleanerManager(val logDirs: Seq[File], + val logs: Pool[TopicPartition, Log], + val logDirFailureChannel: LogDirFailureChannel) extends Logging with KafkaMetricsGroup { + import LogCleanerManager._ + + + protected override def loggerName = classOf[LogCleaner].getName + + // package-private for testing + private[log] val offsetCheckpointFile = "cleaner-offset-checkpoint" + + /* the offset checkpoints holding the last cleaned point for each log */ + @volatile private var checkpoints = logDirs.map(dir => + (dir, new OffsetCheckpointFile(new File(dir, offsetCheckpointFile), logDirFailureChannel))).toMap + + /* the set of logs currently being cleaned */ + private val inProgress = mutable.HashMap[TopicPartition, LogCleaningState]() + + /* the set of uncleanable partitions (partitions that have raised an unexpected error during cleaning) + * for each log directory */ + private val uncleanablePartitions = mutable.HashMap[String, mutable.Set[TopicPartition]]() + + /* a global lock used to control all access to the in-progress set and the offset checkpoints */ + private val lock = new ReentrantLock + + /* for coordinating the pausing and the cleaning of a partition */ + private val pausedCleaningCond = lock.newCondition() + + /* gauges for tracking the number of partitions marked as uncleanable for each log directory */ + for (dir <- logDirs) { + newGauge( + "uncleanable-partitions-count", + new Gauge[Int] { def value = inLock(lock) { uncleanablePartitions.get(dir.getAbsolutePath).map(_.size).getOrElse(0) } }, + Map("logDirectory" -> dir.getAbsolutePath) + ) + } + + /* gauges for tracking the number of uncleanable bytes from uncleanable partitions for each log directory */ + for (dir <- logDirs) { + newGauge( + "uncleanable-bytes", + new Gauge[Long] { + def value = { + inLock(lock) { + uncleanablePartitions.get(dir.getAbsolutePath) match { + case Some(partitions) => { + val lastClean = allCleanerCheckpoints + val now = Time.SYSTEM.milliseconds + partitions.map { tp => + val log = logs.get(tp) + val lastCleanOffset = lastClean.get(tp) + val (firstDirtyOffset, firstUncleanableDirtyOffset) = cleanableOffsets(log, lastCleanOffset, now) + val (_, uncleanableBytes) = calculateCleanableBytes(log, firstDirtyOffset, firstUncleanableDirtyOffset) + uncleanableBytes + }.sum + } + case _ => 0 + } + } + } + }, + Map("logDirectory" -> dir.getAbsolutePath) + ) + } + + /* a gauge for tracking the cleanable ratio of the dirtiest log */ + @volatile private var dirtiestLogCleanableRatio = 0.0 + newGauge("max-dirty-percent", new Gauge[Int] { def value = (100 * dirtiestLogCleanableRatio).toInt }) + + /* a gauge for tracking the time since the last log cleaner run, in milli seconds */ + @volatile private var timeOfLastRun : Long = Time.SYSTEM.milliseconds + newGauge("time-since-last-run-ms", new Gauge[Long] { def value = Time.SYSTEM.milliseconds - timeOfLastRun }) + + /** + * @return the position processed for all logs. + */ + def allCleanerCheckpoints: Map[TopicPartition, Long] = { + inLock(lock) { + checkpoints.values.flatMap(checkpoint => { + try { + checkpoint.read() + } catch { + case e: KafkaStorageException => + error(s"Failed to access checkpoint file ${checkpoint.file.getName} in dir ${checkpoint.file.getParentFile.getAbsolutePath}", e) + Map.empty[TopicPartition, Long] + } + }).toMap + } + } + + /** + * Package private for unit test. Get the cleaning state of the partition. + */ + private[log] def cleaningState(tp: TopicPartition): Option[LogCleaningState] = { + inLock(lock) { + inProgress.get(tp) + } + } + + /** + * Package private for unit test. Set the cleaning state of the partition. + */ + private[log] def setCleaningState(tp: TopicPartition, state: LogCleaningState): Unit = { + inLock(lock) { + inProgress.put(tp, state) + } + } + + /** + * Choose the log to clean next and add it to the in-progress set. We recompute this + * each time from the full set of logs to allow logs to be dynamically added to the pool of logs + * the log manager maintains. + */ + def grabFilthiestCompactedLog(time: Time, preCleanStats: PreCleanStats = new PreCleanStats()): Option[LogToClean] = { + inLock(lock) { + val now = time.milliseconds + this.timeOfLastRun = now + val lastClean = allCleanerCheckpoints + val dirtyLogs = logs.filter { + case (_, log) => log.config.compact // match logs that are marked as compacted + }.filterNot { + case (topicPartition, log) => + + // skip any logs already in-progress and uncleanable partitions + inProgress.contains(topicPartition) || isUncleanablePartition(log, topicPartition) + }.map { + case (topicPartition, log) => // create a LogToClean instance for each + try { + val lastCleanOffset = lastClean.get(topicPartition) + val (firstDirtyOffset, firstUncleanableDirtyOffset) = cleanableOffsets(log, lastCleanOffset, now) + val compactionDelayMs = maxCompactionDelay(log, firstDirtyOffset, now) + preCleanStats.updateMaxCompactionDelay(compactionDelayMs) + + LogToClean(topicPartition, log, firstDirtyOffset, firstUncleanableDirtyOffset, compactionDelayMs > 0) + } catch { + case e: Throwable => throw new LogCleaningException(log, + s"Failed to calculate log cleaning stats for partition $topicPartition", e) + } + }.filter(ltc => ltc.totalBytes > 0) // skip any empty logs + + this.dirtiestLogCleanableRatio = if (dirtyLogs.nonEmpty) dirtyLogs.max.cleanableRatio else 0 + // and must meet the minimum threshold for dirty byte ratio or have some bytes required to be compacted + val cleanableLogs = dirtyLogs.filter { ltc => + (ltc.needCompactionNow && ltc.cleanableBytes > 0) || ltc.cleanableRatio > ltc.log.config.minCleanableRatio + } + if(cleanableLogs.isEmpty) { + // in this case, we are probably in a low throughput situation + // therefore, we should take advantage of this fact and remove tombstones if we can + val logsContainingTombstones = logs.filter { + case (_, log) => log.containsTombstones + }.filterNot { + case (topicPartition, log) => + // skip any logs already in-progress and uncleanable partitions + inProgress.contains(topicPartition) || isUncleanablePartition(log, topicPartition) + }.map { + case (topicPartition, log) => // create a LogToClean instance for each + try { + val (firstDirtyOffset, firstUncleanableDirtyOffset) = cleanableOffsets(log, topicPartition, lastClean, now) + val compactionDelayMs = maxCompactionDelay(log, firstDirtyOffset, now) + preCleanStats.updateMaxCompactionDelay(compactionDelayMs) + + LogToClean(topicPartition, log, firstDirtyOffset, firstUncleanableDirtyOffset, compactionDelayMs > 0) + } catch { + case e: Throwable => throw new LogCleaningException(log, + s"Failed to calculate log cleaning stats for partition $topicPartition", e) + } + } + if (!logsContainingTombstones.isEmpty) { + val filthiest = logsContainingTombstones.max + inProgress.put(filthiest.topicPartition, LogCleaningInProgress) + Some(filthiest) + } else { + None + } + } else { + preCleanStats.recordCleanablePartitions(cleanableLogs.size) + val filthiest = cleanableLogs.max + inProgress.put(filthiest.topicPartition, LogCleaningInProgress) + Some(filthiest) + } + } + } + + /** + * Pause logs cleaning for logs that do not have compaction enabled + * and do not have other deletion or compaction in progress. + * This is to handle potential race between retention and cleaner threads when users + * switch topic configuration between compacted and non-compacted topic. + * @return retention logs that have log cleaning successfully paused + */ + def pauseCleaningForNonCompactedPartitions(): Iterable[(TopicPartition, Log)] = { + inLock(lock) { + val deletableLogs = logs.filter { + case (_, log) => !log.config.compact // pick non-compacted logs + }.filterNot { + case (topicPartition, _) => inProgress.contains(topicPartition) // skip any logs already in-progress + } + + deletableLogs.foreach { + case (topicPartition, _) => inProgress.put(topicPartition, LogCleaningPaused(1)) + } + deletableLogs + } + } + + /** + * Find any logs that have compaction enabled. Mark them as being cleaned + * Include logs without delete enabled, as they may have segments + * that precede the start offset. + */ + def deletableLogs(): Iterable[(TopicPartition, Log)] = { + inLock(lock) { + val toClean = logs.filter { case (topicPartition, log) => + !inProgress.contains(topicPartition) && log.config.compact && + !isUncleanablePartition(log, topicPartition) + } + toClean.foreach { case (tp, _) => inProgress.put(tp, LogCleaningInProgress) } + toClean + } + + } + + /** + * Abort the cleaning of a particular partition, if it's in progress. This call blocks until the cleaning of + * the partition is aborted. + * This is implemented by first abortAndPausing and then resuming the cleaning of the partition. + */ + def abortCleaning(topicPartition: TopicPartition): Unit = { + inLock(lock) { + abortAndPauseCleaning(topicPartition) + resumeCleaning(Seq(topicPartition)) + } + info(s"The cleaning for partition $topicPartition is aborted") + } + + /** + * Abort the cleaning of a particular partition if it's in progress, and pause any future cleaning of this partition. + * This call blocks until the cleaning of the partition is aborted and paused. + * 1. If the partition is not in progress, mark it as paused. + * 2. Otherwise, first mark the state of the partition as aborted. + * 3. The cleaner thread checks the state periodically and if it sees the state of the partition is aborted, it + * throws a LogCleaningAbortedException to stop the cleaning task. + * 4. When the cleaning task is stopped, doneCleaning() is called, which sets the state of the partition as paused. + * 5. abortAndPauseCleaning() waits until the state of the partition is changed to paused. + * 6. If the partition is already paused, a new call to this function + * will increase the paused count by one. + */ + def abortAndPauseCleaning(topicPartition: TopicPartition): Unit = { + inLock(lock) { + inProgress.get(topicPartition) match { + case None => + inProgress.put(topicPartition, LogCleaningPaused(1)) + case Some(LogCleaningInProgress) => + inProgress.put(topicPartition, LogCleaningAborted) + case Some(LogCleaningPaused(count)) => + inProgress.put(topicPartition, LogCleaningPaused(count + 1)) + case Some(s) => + throw new IllegalStateException(s"Compaction for partition $topicPartition cannot be aborted and paused since it is in $s state.") + } + + while(!isCleaningInStatePaused(topicPartition)) + pausedCleaningCond.await(100, TimeUnit.MILLISECONDS) + } + info(s"The cleaning for partition $topicPartition is aborted and paused") + } + + /** + * Resume the cleaning of paused partitions. + * Each call of this function will undo one pause. + */ + def resumeCleaning(topicPartitions: Iterable[TopicPartition]): Unit = { + inLock(lock) { + topicPartitions.foreach { + topicPartition => + inProgress.get(topicPartition) match { + case None => + throw new IllegalStateException(s"Compaction for partition $topicPartition cannot be resumed since it is not paused.") + case Some(state) => + state match { + case LogCleaningPaused(count) if count == 1 => + inProgress.remove(topicPartition) + case LogCleaningPaused(count) if count > 1 => + inProgress.put(topicPartition, LogCleaningPaused(count - 1)) + case s => + throw new IllegalStateException(s"Compaction for partition $topicPartition cannot be resumed since it is in $s state.") + } + } + } + } + } + + /** + * Check if the cleaning for a partition is in a particular state. The caller is expected to hold lock while making the call. + */ + private def isCleaningInState(topicPartition: TopicPartition, expectedState: LogCleaningState): Boolean = { + inProgress.get(topicPartition) match { + case None => false + case Some(state) => + if (state == expectedState) + true + else + false + } + } + + /** + * Check if the cleaning for a partition is paused. The caller is expected to hold lock while making the call. + */ + private def isCleaningInStatePaused(topicPartition: TopicPartition): Boolean = { + inProgress.get(topicPartition) match { + case None => false + case Some(state) => + state match { + case _: LogCleaningPaused => + true + case _ => + false + } + } + } + + /** + * Check if the cleaning for a partition is aborted. If so, throw an exception. + */ + def checkCleaningAborted(topicPartition: TopicPartition): Unit = { + inLock(lock) { + if (isCleaningInState(topicPartition, LogCleaningAborted)) + throw new LogCleaningAbortedException() + } + } + + def updateCheckpoints(dataDir: File, update: Option[(TopicPartition,Long)]): Unit = { + inLock(lock) { + val checkpoint = checkpoints(dataDir) + if (checkpoint != null) { + try { + val existing = checkpoint.read().filter { case (k, _) => logs.keys.contains(k) } ++ update + checkpoint.write(existing) + } catch { + case e: KafkaStorageException => + error(s"Failed to access checkpoint file ${checkpoint.file.getName} in dir ${checkpoint.file.getParentFile.getAbsolutePath}", e) + } + } + } + } + + def alterCheckpointDir(topicPartition: TopicPartition, sourceLogDir: File, destLogDir: File): Unit = { + inLock(lock) { + try { + checkpoints.get(sourceLogDir).flatMap(_.read().get(topicPartition)) match { + case Some(offset) => + // Remove this partition from the checkpoint file in the source log directory + updateCheckpoints(sourceLogDir, None) + // Add offset for this partition to the checkpoint file in the source log directory + updateCheckpoints(destLogDir, Option(topicPartition, offset)) + case None => + } + } catch { + case e: KafkaStorageException => + error(s"Failed to access checkpoint file in dir ${sourceLogDir.getAbsolutePath}", e) + } + + val logUncleanablePartitions = uncleanablePartitions.getOrElse(sourceLogDir.toString, mutable.Set[TopicPartition]()) + if (logUncleanablePartitions.contains(topicPartition)) { + logUncleanablePartitions.remove(topicPartition) + markPartitionUncleanable(destLogDir.toString, topicPartition) + } + } + } + + def handleLogDirFailure(dir: String): Unit = { + warn(s"Stopping cleaning logs in dir $dir") + inLock(lock) { + checkpoints = checkpoints.filter { case (k, _) => k.getAbsolutePath != dir } + } + } + + def maybeTruncateCheckpoint(dataDir: File, topicPartition: TopicPartition, offset: Long): Unit = { + inLock(lock) { + if (logs.get(topicPartition).config.compact) { + val checkpoint = checkpoints(dataDir) + if (checkpoint != null) { + val existing = checkpoint.read() + if (existing.getOrElse(topicPartition, 0L) > offset) + checkpoint.write(existing + (topicPartition -> offset)) + } + } + } + } + + /** + * Save out the endOffset and remove the given log from the in-progress set, if not aborted. + */ + def doneCleaning(topicPartition: TopicPartition, dataDir: File, endOffset: Long): Unit = { + inLock(lock) { + inProgress.get(topicPartition) match { + case Some(LogCleaningInProgress) => + updateCheckpoints(dataDir, Option(topicPartition, endOffset)) + inProgress.remove(topicPartition) + case Some(LogCleaningAborted) => + inProgress.put(topicPartition, LogCleaningPaused(1)) + pausedCleaningCond.signalAll() + case None => + throw new IllegalStateException(s"State for partition $topicPartition should exist.") + case s => + throw new IllegalStateException(s"In-progress partition $topicPartition cannot be in $s state.") + } + } + } + + def doneDeleting(topicPartitions: Iterable[TopicPartition]): Unit = { + inLock(lock) { + topicPartitions.foreach { + topicPartition => + inProgress.get(topicPartition) match { + case Some(LogCleaningInProgress) => + inProgress.remove(topicPartition) + case Some(LogCleaningAborted) => + inProgress.put(topicPartition, LogCleaningPaused(1)) + pausedCleaningCond.signalAll() + case None => + throw new IllegalStateException(s"State for partition $topicPartition should exist.") + case s => + throw new IllegalStateException(s"In-progress partition $topicPartition cannot be in $s state.") + } + } + } + } + + /** + * Returns an immutable set of the uncleanable partitions for a given log directory + * Only used for testing + */ + private[log] def uncleanablePartitions(logDir: String): Set[TopicPartition] = { + var partitions: Set[TopicPartition] = Set() + inLock(lock) { partitions ++= uncleanablePartitions.getOrElse(logDir, partitions) } + partitions + } + + def markPartitionUncleanable(logDir: String, partition: TopicPartition): Unit = { + inLock(lock) { + uncleanablePartitions.get(logDir) match { + case Some(partitions) => + partitions.add(partition) + case None => + uncleanablePartitions.put(logDir, mutable.Set(partition)) + } + } + } + + private def isUncleanablePartition(log: Log, topicPartition: TopicPartition): Boolean = { + inLock(lock) { + uncleanablePartitions.get(log.dir.getParent).exists(partitions => partitions.contains(topicPartition)) + } + } +} + +private[log] object LogCleanerManager extends Logging { + + def isCompactAndDelete(log: Log): Boolean = { + log.config.compact && log.config.delete + } + + /** + * get max delay between the time when log is required to be compacted as determined + * by maxCompactionLagMs and the current time. + */ + def maxCompactionDelay(log: Log, firstDirtyOffset: Long, now: Long) : Long = { + val dirtyNonActiveSegments = log.nonActiveLogSegmentsFrom(firstDirtyOffset) + val firstBatchTimestamps = log.getFirstBatchTimestampForSegments(dirtyNonActiveSegments).filter(_ > 0) + + val earliestDirtySegmentTimestamp = { + if (firstBatchTimestamps.nonEmpty) + firstBatchTimestamps.min + else Long.MaxValue + } + + val maxCompactionLagMs = math.max(log.config.maxCompactionLagMs, 0L) + val cleanUntilTime = now - maxCompactionLagMs + + if (earliestDirtySegmentTimestamp < cleanUntilTime) + cleanUntilTime - earliestDirtySegmentTimestamp + else + 0L + } + + /** + * Returns the range of dirty offsets that can be cleaned. + * + * @param log the log + * @param lastCleanOffset the last checkpointed offset + * @param now the current time in milliseconds of the cleaning operation + * @return the lower (inclusive) and upper (exclusive) offsets + */ + def cleanableOffsets(log: Log, lastCleanOffset: Option[Long], now: Long): (Long, Long) = { + // If the log segments are abnormally truncated and hence the checkpointed offset is no longer valid; + // reset to the log starting offset and log the error + val firstDirtyOffset = { + val logStartOffset = log.logStartOffset + val checkpointDirtyOffset = lastCleanOffset.getOrElse(logStartOffset) + + if (checkpointDirtyOffset < logStartOffset) { + // Don't bother with the warning if compact and delete are enabled. + if (!isCompactAndDelete(log)) + warn(s"Resetting first dirty offset of ${log.name} to log start offset $logStartOffset " + + s"since the checkpointed offset $checkpointDirtyOffset is invalid.") + logStartOffset + } else if (checkpointDirtyOffset > log.logEndOffset) { + // The dirty offset has gotten ahead of the log end offset. This could happen if there was data + // corruption at the end of the log. We conservatively assume that the full log needs cleaning. + warn(s"The last checkpoint dirty offset for partition ${log.name} is $checkpointDirtyOffset, " + + s"which is larger than the log end offset ${log.logEndOffset}. Resetting to the log start offset $logStartOffset.") + logStartOffset + } else { + checkpointDirtyOffset + } + } + + val minCompactionLagMs = math.max(log.config.compactionLagMs, 0L) + + // find first segment that cannot be cleaned + // neither the active segment, nor segments with any messages closer to the head of the log than the minimum compaction lag time + // may be cleaned + val firstUncleanableDirtyOffset: Long = Seq( + + // we do not clean beyond the first unstable offset + log.firstUnstableOffset, + + // the active segment is always uncleanable + Option(log.activeSegment.baseOffset), + + // the first segment whose largest message timestamp is within a minimum time lag from now + if (minCompactionLagMs > 0) { + // dirty log segments + val dirtyNonActiveSegments = log.nonActiveLogSegmentsFrom(firstDirtyOffset) + dirtyNonActiveSegments.find { s => + val isUncleanable = s.largestTimestamp > now - minCompactionLagMs + debug(s"Checking if log segment may be cleaned: log='${log.name}' segment.baseOffset=${s.baseOffset} " + + s"segment.largestTimestamp=${s.largestTimestamp}; now - compactionLag=${now - minCompactionLagMs}; " + + s"is uncleanable=$isUncleanable") + isUncleanable + }.map(_.baseOffset) + } else None + ).flatten.min + + debug(s"Finding range of cleanable offsets for log=${log.name}. Last clean offset=$lastCleanOffset " + + s"now=$now => firstDirtyOffset=$firstDirtyOffset firstUncleanableOffset=$firstUncleanableDirtyOffset " + + s"activeSegment.baseOffset=${log.activeSegment.baseOffset}") + + (firstDirtyOffset, math.max(firstDirtyOffset, firstUncleanableDirtyOffset)) + } + + /** + * Given the first dirty offset and an uncleanable offset, calculates the total cleanable bytes for this log + * @return the biggest uncleanable offset and the total amount of cleanable bytes + */ + def calculateCleanableBytes(log: Log, firstDirtyOffset: Long, uncleanableOffset: Long): (Long, Long) = { + val firstUncleanableSegment = log.nonActiveLogSegmentsFrom(uncleanableOffset).headOption.getOrElse(log.activeSegment) + val firstUncleanableOffset = firstUncleanableSegment.baseOffset + val cleanableBytes = log.logSegments(firstDirtyOffset, math.max(firstDirtyOffset, firstUncleanableOffset)).map(_.size.toLong).sum + + (firstUncleanableOffset, cleanableBytes) + } + +} diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 3291310365a41..2c3a75a3cade2 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -208,7 +208,9 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K // We sleep a little bit, so that log cleaner has already gone through // some iterations, ensures that delete horizons has been updated correctly - Thread.sleep(300L) + Thread.sleep(400L) + assertEquals(log.latestDeleteHorizon, T0 + tombstoneRetentionMs) + time.sleep(tombstoneRetentionMs + 1) val latestOffset: Long = log.latestEpoch match { @@ -229,10 +231,10 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K cleaner.awaitCleaned(new TopicPartition("log-partition", 0), latestOffset + 1, maxWaitMs = tombstoneRetentionMs) - assertEquals(log.latestDeleteHorizon, T0 + tombstoneRetentionMs) for (segment <- log.logSegments; record <- segment.log.records.asScala) { fail ("The log should not contain record " + record + ", tombstone has expired its lifetime.") } + assertEquals(log.latestDeleteHorizon, -1L) } private def readFromLog(log: Log): Iterable[(Int, Int)] = { diff --git a/diff.out b/diff.out new file mode 100644 index 0000000000000..9b932c690de3b --- /dev/null +++ b/diff.out @@ -0,0 +1,1155 @@ +diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml +index 13cfdb82bd0..8b28fcadb17 100644 +--- a/checkstyle/checkstyle.xml ++++ b/checkstyle/checkstyle.xml +@@ -107,7 +107,7 @@ + + + +- ++ + + + +@@ -124,7 +124,7 @@ + + + +- ++ + + + +diff --git a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java +index 83637640af4..cf74f24f490 100644 +--- a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java ++++ b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java +@@ -431,6 +431,16 @@ public LegacyRecord outerRecord() { + return record; + } + ++ @Override ++ public long deleteHorizonMs() { ++ return RecordBatch.NO_TIMESTAMP; ++ } ++ ++ @Override ++ public boolean deleteHorizonSet() { ++ return false; ++ } ++ + @Override + public boolean equals(Object o) { + if (this == o) +@@ -468,6 +478,16 @@ public long offset() { + return buffer.getLong(OFFSET_OFFSET); + } + ++ @Override ++ public long deleteHorizonMs() { ++ return RecordBatch.NO_TIMESTAMP; ++ } ++ ++ @Override ++ public boolean deleteHorizonSet() { ++ return false; ++ } ++ + @Override + public LegacyRecord outerRecord() { + return record; +@@ -557,6 +577,16 @@ public long baseOffset() { + return loadFullBatch().baseOffset(); + } + ++ @Override ++ public long deleteHorizonMs() { ++ return RecordBatch.NO_TIMESTAMP; ++ } ++ ++ @Override ++ public boolean deleteHorizonSet() { ++ return false; ++ } ++ + @Override + public long lastOffset() { + return offset; +diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java +index 6d79b268575..e0d7549fede 100644 +--- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java ++++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java +@@ -89,11 +89,15 @@ + * by the broker and is preserved after compaction. Additionally, the MaxTimestamp of an empty batch always retains + * the previous value prior to becoming empty. + * ++ * The delete horizon flag for the sixth bit is used to determine if the first timestamp of the batch had been set to ++ * the time for which tombstones / transaction markers needs to be removed. If it is true, then the first timestamp is ++ * the delete horizon, otherwise, it is merely the first timestamp of the record batch. ++ * + * The current attributes are given below: + * +- * ------------------------------------------------------------------------------------------------- +- * | Unused (6-15) | Control (5) | Transactional (4) | Timestamp Type (3) | Compression Type (0-2) | +- * ------------------------------------------------------------------------------------------------- ++ * --------------------------------------------------------------------------------------------------------------------------- ++ * | Unused (7-15) | Delete Horizon Flag (6) | Control (5) | Transactional (4) | Timestamp Type (3) | Compression Type (0-2) | ++ * --------------------------------------------------------------------------------------------------------------------------- + */ + public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRecordBatch { + static final int BASE_OFFSET_OFFSET = 0; +@@ -128,6 +132,7 @@ + private static final byte COMPRESSION_CODEC_MASK = 0x07; + private static final byte TRANSACTIONAL_FLAG_MASK = 0x10; + private static final int CONTROL_FLAG_MASK = 0x20; ++ private static final byte DELETE_HORIZON_FLAG_MASK = 0x40; + private static final byte TIMESTAMP_TYPE_MASK = 0x08; + + private static final int MAX_SKIP_BUFFER_SIZE = 2048; +@@ -155,10 +160,15 @@ public void ensureValid() { + } + + /** +- * Get the timestamp of the first record in this batch. It is always the create time of the record even if the ++ * Get the timestamp of the first record in this batch. It is usually the create time of the record even if the + * timestamp type of the batch is log append time. + * +- * @return The first timestamp or {@link RecordBatch#NO_TIMESTAMP} if the batch is empty ++ * There is the possibility that the first timestamp had been set to the delete horizon of the batch, ++ * in which case, the delete horizon will be returned instead. ++ * ++ * @return The first timestamp if the batch's delete horizon has not been set ++ * The delete horizon if the batch's delete horizon has been set ++ * {@link RecordBatch#NO_TIMESTAMP} if the batch is empty + */ + public long firstTimestamp() { + return buffer.getLong(FIRST_TIMESTAMP_OFFSET); +@@ -245,6 +255,18 @@ public boolean isTransactional() { + return (attributes() & TRANSACTIONAL_FLAG_MASK) > 0; + } + ++ @Override ++ public boolean deleteHorizonSet() { ++ return (attributes() & DELETE_HORIZON_FLAG_MASK) > 0; ++ } ++ ++ @Override ++ public long deleteHorizonMs() { ++ if (deleteHorizonSet()) ++ return firstTimestamp(); ++ return RecordBatch.NO_TIMESTAMP; ++ } ++ + @Override + public boolean isControlBatch() { + return (attributes() & CONTROL_FLAG_MASK) > 0; +@@ -360,7 +382,7 @@ public void setMaxTimestamp(TimestampType timestampType, long maxTimestamp) { + if (timestampType() == timestampType && currentMaxTimestamp == maxTimestamp) + return; + +- byte attributes = computeAttributes(compressionType(), timestampType, isTransactional(), isControlBatch()); ++ byte attributes = computeAttributes(compressionType(), timestampType, isTransactional(), isControlBatch(), deleteHorizonSet()); + buffer.putShort(ATTRIBUTES_OFFSET, attributes); + buffer.putLong(MAX_TIMESTAMP_OFFSET, maxTimestamp); + long crc = computeChecksum(); +@@ -407,7 +429,7 @@ public int hashCode() { + } + + private static byte computeAttributes(CompressionType type, TimestampType timestampType, +- boolean isTransactional, boolean isControl) { ++ boolean isTransactional, boolean isControl, boolean isDeleteHorizonSet) { + if (timestampType == TimestampType.NO_TIMESTAMP_TYPE) + throw new IllegalArgumentException("Timestamp type must be provided to compute attributes for message " + + "format v2 and above"); +@@ -419,6 +441,8 @@ private static byte computeAttributes(CompressionType type, TimestampType timest + attributes |= COMPRESSION_CODEC_MASK & type.id; + if (timestampType == TimestampType.LOG_APPEND_TIME) + attributes |= TIMESTAMP_TYPE_MASK; ++ if (isDeleteHorizonSet) ++ attributes |= DELETE_HORIZON_FLAG_MASK; + return attributes; + } + +@@ -435,9 +459,49 @@ public static void writeEmptyHeader(ByteBuffer buffer, + boolean isTransactional, + boolean isControlRecord) { + int offsetDelta = (int) (lastOffset - baseOffset); ++ writeHeader(buffer, baseOffset, offsetDelta, DefaultRecordBatch.RECORD_BATCH_OVERHEAD, magic, ++ CompressionType.NONE, timestampType, RecordBatch.NO_TIMESTAMP, timestamp, producerId, ++ producerEpoch, baseSequence, isTransactional, isControlRecord, false, partitionLeaderEpoch, 0); ++ } ++ ++ public static void writeEmptyHeader(ByteBuffer buffer, ++ byte magic, ++ long producerId, ++ short producerEpoch, ++ int baseSequence, ++ long baseOffset, ++ long lastOffset, ++ int partitionLeaderEpoch, ++ TimestampType timestampType, ++ long timestamp, ++ boolean isTransactional, ++ boolean isControlRecord, ++ boolean isDeleteHorizonSet) { ++ int offsetDelta = (int) (lastOffset - baseOffset); + writeHeader(buffer, baseOffset, offsetDelta, DefaultRecordBatch.RECORD_BATCH_OVERHEAD, magic, + CompressionType.NONE, timestampType, RecordBatch.NO_TIMESTAMP, timestamp, producerId, +- producerEpoch, baseSequence, isTransactional, isControlRecord, partitionLeaderEpoch, 0); ++ producerEpoch, baseSequence, isTransactional, isControlRecord, isDeleteHorizonSet, partitionLeaderEpoch, 0); ++ } ++ ++ static void writeHeader(ByteBuffer buffer, ++ long baseOffset, ++ int lastOffsetDelta, ++ int sizeInBytes, ++ byte magic, ++ CompressionType compressionType, ++ TimestampType timestampType, ++ long firstTimestamp, ++ long maxTimestamp, ++ long producerId, ++ short epoch, ++ int sequence, ++ boolean isTransactional, ++ boolean isControlBatch, ++ int partitionLeaderEpoch, ++ int numRecords) { ++ writeHeader(buffer, baseOffset, lastOffsetDelta, sizeInBytes, magic, compressionType, ++ timestampType, firstTimestamp, maxTimestamp, producerId, epoch, sequence, ++ isTransactional, isControlBatch, false, partitionLeaderEpoch, numRecords); + } + + static void writeHeader(ByteBuffer buffer, +@@ -454,6 +518,7 @@ static void writeHeader(ByteBuffer buffer, + int sequence, + boolean isTransactional, + boolean isControlBatch, ++ boolean isDeleteHorizonSet, + int partitionLeaderEpoch, + int numRecords) { + if (magic < RecordBatch.CURRENT_MAGIC_VALUE) +@@ -461,7 +526,7 @@ static void writeHeader(ByteBuffer buffer, + if (firstTimestamp < 0 && firstTimestamp != NO_TIMESTAMP) + throw new IllegalArgumentException("Invalid message timestamp " + firstTimestamp); + +- short attributes = computeAttributes(compressionType, timestampType, isTransactional, isControlBatch); ++ short attributes = computeAttributes(compressionType, timestampType, isTransactional, isControlBatch, isDeleteHorizonSet); + + int position = buffer.position(); + buffer.putLong(position + BASE_OFFSET_OFFSET, baseOffset); +@@ -699,6 +764,18 @@ public boolean isTransactional() { + return loadBatchHeader().isTransactional(); + } + ++ @Override ++ public boolean deleteHorizonSet() { ++ return loadBatchHeader().deleteHorizonSet(); ++ } ++ ++ @Override ++ public long deleteHorizonMs() { ++ if (deleteHorizonSet()) ++ return super.loadBatchHeader().deleteHorizonMs(); ++ return RecordBatch.NO_TIMESTAMP; ++ } ++ + @Override + public boolean isControlBatch() { + return loadBatchHeader().isControlBatch(); +diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +index 8f73565d1b4..7fb5fe951b9 100644 +--- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java ++++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +@@ -150,15 +150,29 @@ public FilterResult filterTo(TopicPartition partition, RecordFilter filter, Byte + return filterTo(partition, batches(), filter, destinationBuffer, maxRecordBatchSize, decompressionBufferSupplier); + } + ++ /** ++ * Note: This method is also used to convert the first timestamp of the batch (which is usually the timestamp of the first record) ++ * to the delete horizon of the tombstones which are present in the batch. ++ */ + private static FilterResult filterTo(TopicPartition partition, Iterable batches, + RecordFilter filter, ByteBuffer destinationBuffer, int maxRecordBatchSize, + BufferSupplier decompressionBufferSupplier) { + FilterResult filterResult = new FilterResult(destinationBuffer); + ByteBufferOutputStream bufferOutputStream = new ByteBufferOutputStream(destinationBuffer); +- + for (MutableRecordBatch batch : batches) { + long maxOffset = -1L; +- BatchRetention batchRetention = filter.checkBatchRetention(batch); ++ // we first call this method here so that the flag in LogCleaner has been set ++ // which indicates if the control batch is empty or not ++ // we do this to avoid calling CleanedTransactionMetadata#onControlBatchRead ++ // more than once since each call is relatively expensive ++ filter.isControlBatchEmpty(batch); ++ long deleteHorizonMs = filter.retrieveDeleteHorizon(batch); ++ final BatchRetention batchRetention; ++ if (!batch.deleteHorizonSet()) ++ batchRetention = filter.checkBatchRetention(batch, deleteHorizonMs); ++ else ++ batchRetention = filter.checkBatchRetention(batch); ++ + filterResult.bytesRead += batch.sizeInBytes(); + + if (batchRetention == BatchRetention.DELETE) +@@ -170,36 +184,27 @@ private static FilterResult filterTo(TopicPartition partition, Iterable retainedRecords = new ArrayList<>(); + +- try (final CloseableIterator iterator = batch.streamingIterator(decompressionBufferSupplier)) { +- while (iterator.hasNext()) { +- Record record = iterator.next(); +- filterResult.messagesRead += 1; +- +- if (filter.shouldRetainRecord(batch, record)) { +- // Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite +- // the corrupted batch with correct data. +- if (!record.hasMagic(batchMagic)) +- writeOriginalBatch = false; +- +- if (record.offset() > maxOffset) +- maxOffset = record.offset(); +- +- retainedRecords.add(record); +- } else { +- writeOriginalBatch = false; +- } +- } +- } ++ final BatchIterationResult iterationResult = iterateOverBatch(batch, decompressionBufferSupplier, filterResult, filter, ++ batchMagic, writeOriginalBatch, maxOffset, retainedRecords, ++ containsTombstonesOrMarker, deleteHorizonMs); ++ containsTombstonesOrMarker = iterationResult.containsTombstonesOrMarker(); ++ writeOriginalBatch = iterationResult.shouldWriteOriginalBatch(); ++ maxOffset = iterationResult.maxOffset(); + + if (!retainedRecords.isEmpty()) { +- if (writeOriginalBatch) { ++ // we check if the delete horizon should be set to a new value ++ // in which case, we need to reset the base timestamp and overwrite the timestamp deltas ++ // if the batch does not contain tombstones, then we don't need to overwrite batch ++ if (writeOriginalBatch && (deleteHorizonMs == RecordBatch.NO_TIMESTAMP || deleteHorizonMs == batch.deleteHorizonMs() || !containsTombstonesOrMarker)) { + batch.writeTo(bufferOutputStream); + filterResult.updateRetainedBatchMetadata(batch, retainedRecords.size(), false); + } else { +- MemoryRecordsBuilder builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream); ++ MemoryRecordsBuilder builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, deleteHorizonMs); + MemoryRecords records = builder.build(); + int filteredBatchSize = records.sizeInBytes(); + if (filteredBatchSize > batch.sizeInBytes() && filteredBatchSize > maxRecordBatchSize) +@@ -236,9 +241,69 @@ private static FilterResult filterTo(TopicPartition partition, Iterable retainedRecords, ++ boolean containsTombstonesOrMarker, ++ long newBatchDeleteHorizonMs) { ++ try (final CloseableIterator iterator = batch.streamingIterator(decompressionBufferSupplier)) { ++ while (iterator.hasNext()) { ++ Record record = iterator.next(); ++ filterResult.messagesRead += 1; ++ ++ if (filter.shouldRetainRecord(batch, record, newBatchDeleteHorizonMs)) { ++ // Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite ++ // the corrupted batch with correct data. ++ if (!record.hasMagic(batchMagic)) ++ writeOriginalBatch = false; ++ ++ if (record.offset() > maxOffset) ++ maxOffset = record.offset(); ++ ++ retainedRecords.add(record); ++ ++ if (!record.hasValue()) { ++ containsTombstonesOrMarker = true; ++ } ++ } else { ++ writeOriginalBatch = false; ++ } ++ } ++ return new BatchIterationResult(writeOriginalBatch, containsTombstonesOrMarker, maxOffset); ++ } ++ } ++ ++ private static class BatchIterationResult { ++ private final boolean writeOriginalBatch; ++ private final boolean containsTombstonesOrMarker; ++ private final long maxOffset; ++ public BatchIterationResult(final boolean writeOriginalBatch, ++ final boolean containsTombstonesOrMarker, ++ final long maxOffset) { ++ this.writeOriginalBatch = writeOriginalBatch; ++ this.containsTombstonesOrMarker = containsTombstonesOrMarker; ++ this.maxOffset = maxOffset; ++ } ++ public boolean shouldWriteOriginalBatch() { ++ return this.writeOriginalBatch; ++ } ++ public boolean containsTombstonesOrMarker() { ++ return this.containsTombstonesOrMarker; ++ } ++ public long maxOffset() { ++ return this.maxOffset; ++ } ++ } ++ + private static MemoryRecordsBuilder buildRetainedRecordsInto(RecordBatch originalBatch, + List retainedRecords, +- ByteBufferOutputStream bufferOutputStream) { ++ ByteBufferOutputStream bufferOutputStream, ++ final long deleteHorizonMs) { + byte magic = originalBatch.magic(); + TimestampType timestampType = originalBatch.timestampType(); + long logAppendTime = timestampType == TimestampType.LOG_APPEND_TIME ? +@@ -249,7 +314,7 @@ private static MemoryRecordsBuilder buildRetainedRecordsInto(RecordBatch origina + MemoryRecordsBuilder builder = new MemoryRecordsBuilder(bufferOutputStream, magic, + originalBatch.compressionType(), timestampType, baseOffset, logAppendTime, originalBatch.producerId(), + originalBatch.producerEpoch(), originalBatch.baseSequence(), originalBatch.isTransactional(), +- originalBatch.isControlBatch(), originalBatch.partitionLeaderEpoch(), bufferOutputStream.limit()); ++ originalBatch.isControlBatch(), originalBatch.partitionLeaderEpoch(), bufferOutputStream.limit(), deleteHorizonMs); + + for (Record record : retainedRecords) + builder.append(record); +@@ -312,12 +377,34 @@ public int hashCode() { + */ + protected abstract BatchRetention checkBatchRetention(RecordBatch batch); + ++ protected BatchRetention checkBatchRetention(RecordBatch batch, long newBatchDeleteHorizonMs) { ++ return checkBatchRetention(batch); ++ } ++ + /** + * Check whether a record should be retained in the log. Note that {@link #checkBatchRetention(RecordBatch)} + * is used prior to checking individual record retention. Only records from batches which were not + * explicitly discarded with {@link BatchRetention#DELETE} will be considered. + */ + protected abstract boolean shouldRetainRecord(RecordBatch recordBatch, Record record); ++ ++ protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record, long newDeleteHorizonMs) { ++ return shouldRetainRecord(recordBatch, record); ++ } ++ ++ /** ++ * Retrieves the delete horizon ms for a specific batch ++ */ ++ protected long retrieveDeleteHorizon(RecordBatch recordBatch) { ++ return -1L; ++ } ++ ++ /** ++ * Checks if the control batch (if it is one) can be removed (making sure that it is empty) ++ */ ++ protected boolean isControlBatchEmpty(RecordBatch recordBatch) { ++ return true; ++ } + } + + public static class FilterResult { +diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java +index 054fb861998..be32a839059 100644 +--- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java ++++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java +@@ -75,6 +75,7 @@ public void write(int b) { + private int numRecords = 0; + private float actualCompressionRatio = 1; + private long maxTimestamp = RecordBatch.NO_TIMESTAMP; ++ private long deleteHorizonMs; + private long offsetOfMaxTimestamp = -1; + private Long lastOffset = null; + private Long firstTimestamp = null; +@@ -94,7 +95,8 @@ public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, + boolean isTransactional, + boolean isControlBatch, + int partitionLeaderEpoch, +- int writeLimit) { ++ int writeLimit, ++ long deleteHorizonMs) { + if (magic > RecordBatch.MAGIC_VALUE_V0 && timestampType == TimestampType.NO_TIMESTAMP_TYPE) + throw new IllegalArgumentException("TimestampType must be set for magic >= 0"); + if (magic < RecordBatch.MAGIC_VALUE_V2) { +@@ -120,6 +122,7 @@ public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, + this.baseSequence = baseSequence; + this.isTransactional = isTransactional; + this.isControlBatch = isControlBatch; ++ this.deleteHorizonMs = deleteHorizonMs; + this.partitionLeaderEpoch = partitionLeaderEpoch; + this.writeLimit = writeLimit; + this.initialPosition = bufferStream.position(); +@@ -130,6 +133,24 @@ public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, + this.appendStream = new DataOutputStream(compressionType.wrapForOutput(this.bufferStream, magic)); + } + ++ public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, ++ byte magic, ++ CompressionType compressionType, ++ TimestampType timestampType, ++ long baseOffset, ++ long logAppendTime, ++ long producerId, ++ short producerEpoch, ++ int baseSequence, ++ boolean isTransactional, ++ boolean isControlBatch, ++ int partitionLeaderEpoch, ++ int writeLimit) { ++ this(bufferStream, magic, compressionType, timestampType, baseOffset, logAppendTime, producerId, ++ producerEpoch, baseSequence, isTransactional, isControlBatch, partitionLeaderEpoch, writeLimit, ++ RecordBatch.NO_TIMESTAMP); ++ } ++ + /** + * Construct a new builder. + * +@@ -192,6 +213,10 @@ public boolean isTransactional() { + return isTransactional; + } + ++ public boolean deleteHorizonSet() { ++ return deleteHorizonMs >= 0L; ++ } ++ + /** + * Close this builder and return the resulting buffer. + * @return The built log buffer +@@ -364,7 +389,7 @@ private int writeDefaultBatchHeader() { + maxTimestamp = this.maxTimestamp; + + DefaultRecordBatch.writeHeader(buffer, baseOffset, offsetDelta, size, magic, compressionType, timestampType, +- firstTimestamp, maxTimestamp, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, ++ firstTimestamp, maxTimestamp, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, deleteHorizonSet(), + partitionLeaderEpoch, numRecords); + + buffer.position(pos); +@@ -411,8 +436,12 @@ private Long appendWithOffset(long offset, boolean isControlRecord, long timesta + if (magic < RecordBatch.MAGIC_VALUE_V2 && headers != null && headers.length > 0) + throw new IllegalArgumentException("Magic v" + magic + " does not support record headers"); + +- if (firstTimestamp == null) +- firstTimestamp = timestamp; ++ if (firstTimestamp == null) { ++ if (deleteHorizonSet()) ++ firstTimestamp = deleteHorizonMs; ++ else ++ firstTimestamp = timestamp; ++ } + + if (magic > RecordBatch.MAGIC_VALUE_V1) { + appendDefaultRecord(offset, timestamp, key, value, headers); +diff --git a/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java +index 65a6a95fbe4..45f1609e3bc 100644 +--- a/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java ++++ b/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java +@@ -210,6 +210,18 @@ + */ + boolean isTransactional(); + ++ /** ++ * Whether or not the base timestamp has been set to the delete horizon ++ * @return true if it is, false otherwise ++ */ ++ boolean deleteHorizonSet(); ++ ++ /** ++ * Get the delete horizon, returns -1L if the first timestamp is not the delete horizon ++ * @return timestamp of the delete horizon ++ */ ++ long deleteHorizonMs(); ++ + /** + * Get the partition leader epoch of this record batch. + * @return The leader epoch or -1 if it is unknown +diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java +index b8824d3a827..4677e40e388 100644 +--- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java ++++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java +@@ -247,6 +247,45 @@ public void testFilterToPreservesPartitionLeaderEpoch() { + } + } + ++ /** ++ * This test is used to see if the first timestamp of the batch has been successfully ++ * converted to a delete horizon for the tombstones / transaction markers of the batch. ++ */ ++ @Test ++ public void testFirstTimestampToDeleteHorizonConversion() { ++ if (magic >= RecordBatch.MAGIC_VALUE_V2) { ++ ByteBuffer buffer = ByteBuffer.allocate(2048); ++ MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, ++ 0L, RecordBatch.NO_TIMESTAMP, partitionLeaderEpoch); ++ builder.append(10L, "1".getBytes(), null); ++ ++ ByteBuffer filtered = ByteBuffer.allocate(2048); ++ final long deleteHorizon = Integer.MAX_VALUE / 2; ++ builder.build().filterTo(new TopicPartition("random", 0), new MemoryRecords.RecordFilter() { ++ @Override ++ protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { ++ return true; ++ } ++ ++ @Override ++ protected BatchRetention checkBatchRetention(RecordBatch batch) { ++ return BatchRetention.RETAIN_EMPTY; ++ } ++ ++ @Override ++ protected long retrieveDeleteHorizon(RecordBatch batch) { ++ return deleteHorizon; // arbitrary value > 1 ++ } ++ }, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); ++ filtered.flip(); ++ MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); ++ ++ List batches = TestUtils.toList(filteredRecords.batches()); ++ assertEquals(1, batches.size()); ++ assertEquals(deleteHorizon, batches.get(0).deleteHorizonMs()); ++ } ++ } ++ + @Test + public void testFilterToEmptyBatchRetention() { + if (magic >= RecordBatch.MAGIC_VALUE_V2) { +diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala +index 5e04c3cb55c..9e221ffb19a 100644 +--- a/core/src/main/scala/kafka/log/Log.scala ++++ b/core/src/main/scala/kafka/log/Log.scala +@@ -221,7 +221,8 @@ class Log(@volatile var dir: File, + val producerIdExpirationCheckIntervalMs: Int, + val topicPartition: TopicPartition, + val producerStateManager: ProducerStateManager, +- logDirFailureChannel: LogDirFailureChannel) extends Logging with KafkaMetricsGroup { ++ logDirFailureChannel: LogDirFailureChannel, ++ @volatile var containsTombstones: Boolean = false) extends Logging with KafkaMetricsGroup { + + import kafka.log.Log._ + +diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala +index 219f49716f0..a59dfadb147 100644 +--- a/core/src/main/scala/kafka/log/LogCleaner.scala ++++ b/core/src/main/scala/kafka/log/LogCleaner.scala +@@ -509,11 +509,10 @@ private[log] class Cleaner(val id: Int, + case None => 0L + case Some(seg) => seg.lastModified - cleanable.log.config.deleteRetentionMs + } +- +- doClean(cleanable, deleteHorizonMs) ++ doClean(cleanable, time.milliseconds(), trackedHorizon = deleteHorizonMs) + } + +- private[log] def doClean(cleanable: LogToClean, deleteHorizonMs: Long): (Long, CleanerStats) = { ++ private[log] def doClean(cleanable: LogToClean, currentTime: Long, trackedHorizon: Long = -1L): (Long, CleanerStats) = { + info("Beginning cleaning of log %s.".format(cleanable.log.name)) + + val log = cleanable.log +@@ -531,13 +530,13 @@ private[log] class Cleaner(val id: Int, + val cleanableHorizonMs = log.logSegments(0, cleanable.firstUncleanableOffset).lastOption.map(_.lastModified).getOrElse(0L) + + // group the segments and clean the groups +- info("Cleaning log %s (cleaning prior to %s, discarding tombstones prior to %s)...".format(log.name, new Date(cleanableHorizonMs), new Date(deleteHorizonMs))) ++ info("Cleaning log %s (cleaning prior to %s, discarding tombstones prior to %s)...".format(log.name, new Date(cleanableHorizonMs), new Date(trackedHorizon))) + val transactionMetadata = new CleanedTransactionMetadata + + val groupedSegments = groupSegmentsBySize(log.logSegments(0, endOffset), log.config.segmentSize, + log.config.maxIndexSize, cleanable.firstUncleanableOffset) + for (group <- groupedSegments) +- cleanSegments(log, group, offsetMap, deleteHorizonMs, stats, transactionMetadata) ++ cleanSegments(log, group, offsetMap, currentTime, stats, transactionMetadata, trackedHorizon = trackedHorizon) + + // record buffer utilization + stats.bufferUtilization = offsetMap.utilization +@@ -561,9 +560,10 @@ private[log] class Cleaner(val id: Int, + private[log] def cleanSegments(log: Log, + segments: Seq[LogSegment], + map: OffsetMap, +- deleteHorizonMs: Long, ++ currentTime: Long, + stats: CleanerStats, +- transactionMetadata: CleanedTransactionMetadata): Unit = { ++ transactionMetadata: CleanedTransactionMetadata, ++ trackedHorizon: Long = -1L): Unit = { + // create a new segment with a suffix appended to the name of the log and indexes + val cleaned = LogCleaner.createNewCleanedSegment(log, segments.head.baseOffset) + transactionMetadata.cleanedIndex = Some(cleaned.txnIndex) +@@ -583,14 +583,15 @@ private[log] class Cleaner(val id: Int, + val abortedTransactions = log.collectAbortedTransactions(startOffset, upperBoundOffset) + transactionMetadata.addAbortedTransactions(abortedTransactions) + +- val retainDeletesAndTxnMarkers = currentSegment.lastModified > deleteHorizonMs ++ val retainDeletesAndTxnMarkers = currentSegment.lastModified > trackedHorizon + info(s"Cleaning $currentSegment in log ${log.name} into ${cleaned.baseOffset} " + +- s"with deletion horizon $deleteHorizonMs, " + ++ s"with deletion horizon $trackedHorizon, " + + s"${if(retainDeletesAndTxnMarkers) "retaining" else "discarding"} deletes.") + + try { +- cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletesAndTxnMarkers, log.config.maxMessageSize, +- transactionMetadata, lastOffsetOfActiveProducers, stats) ++ val containsTombstones: Boolean = cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletesAndTxnMarkers, log.config.maxMessageSize, ++ transactionMetadata, lastOffsetOfActiveProducers, stats, log.config.deleteRetentionMs, currentTime = currentTime) ++ log.containsTombstones = containsTombstones + } catch { + case e: LogSegmentOffsetOverflowException => + // Split the current segment. It's also safest to abort the current cleaning process, so that we retry from +@@ -634,6 +635,7 @@ private[log] class Cleaner(val id: Int, + * @param retainDeletesAndTxnMarkers Should tombstones and markers be retained while cleaning this segment + * @param maxLogMessageSize The maximum message size of the corresponding topic + * @param stats Collector for cleaning statistics ++ * @param tombstoneRetentionMs Defines how long a tombstone should be kept as defined by log configuration + */ + private[log] def cleanInto(topicPartition: TopicPartition, + sourceRecords: FileRecords, +@@ -643,14 +645,33 @@ private[log] class Cleaner(val id: Int, + maxLogMessageSize: Int, + transactionMetadata: CleanedTransactionMetadata, + lastRecordsOfActiveProducers: Map[Long, LastRecord], +- stats: CleanerStats): Unit = { ++ stats: CleanerStats, ++ tombstoneRetentionMs: Long, ++ currentTime: Long = RecordBatch.NO_TIMESTAMP): Boolean = { ++ var containsTombstones: Boolean = false ++ + val logCleanerFilter: RecordFilter = new RecordFilter { + var discardBatchRecords: Boolean = _ ++ var isControlBatchEmpty: Boolean = _ + +- override def checkBatchRetention(batch: RecordBatch): BatchRetention = { ++ override def isControlBatchEmpty(batch: RecordBatch) : Boolean = { + // we piggy-back on the tombstone retention logic to delay deletion of transaction markers. + // note that we will never delete a marker until all the records from that transaction are removed. +- discardBatchRecords = shouldDiscardBatch(batch, transactionMetadata, retainTxnMarkers = retainDeletesAndTxnMarkers) ++ val canDiscardBatch = shouldDiscardBatch(batch, transactionMetadata, retainTxnMarkers = retainDeletesAndTxnMarkers) ++ isControlBatchEmpty = canDiscardBatch ++ isControlBatchEmpty ++ } ++ ++ override def checkBatchRetention(batch: RecordBatch, newBatchDeleteHorizonMs : Long): BatchRetention = { ++ val canDiscardBatch = isControlBatchEmpty ++ ++ if (batch.isControlBatch) { ++ discardBatchRecords = canDiscardBatch && ++ ((batch.deleteHorizonSet() && batch.deleteHorizonMs() < currentTime) || ++ newBatchDeleteHorizonMs != RecordBatch.NO_TIMESTAMP && newBatchDeleteHorizonMs < currentTime) ++ } else { ++ discardBatchRecords = canDiscardBatch ++ } + + def isBatchLastRecordOfProducer: Boolean = { + // We retain the batch in order to preserve the state of active producers. There are three cases: +@@ -675,12 +696,33 @@ private[log] class Cleaner(val id: Int, + BatchRetention.DELETE_EMPTY + } + +- override def shouldRetainRecord(batch: RecordBatch, record: Record): Boolean = { ++ override def checkBatchRetention(batch: RecordBatch): BatchRetention = checkBatchRetention(batch, batch.deleteHorizonMs()) ++ ++ override def shouldRetainRecord(batch: RecordBatch, record: Record, newDeleteHorizonMs: Long): Boolean = { ++ var isRecordRetained: Boolean = true + if (discardBatchRecords) + // The batch is only retained to preserve producer sequence information; the records can be removed +- false ++ isRecordRetained = false + else +- Cleaner.this.shouldRetainRecord(map, retainDeletesAndTxnMarkers, batch, record, stats) ++ isRecordRetained = Cleaner.this.shouldRetainRecord(map, retainDeletesAndTxnMarkers, batch, record, stats, newDeleteHorizonMs, currentTime = currentTime) ++ if (isRecordRetained && !record.hasValue()) ++ containsTombstones = true ++ isRecordRetained ++ } ++ ++ override def shouldRetainRecord(batch: RecordBatch, record: Record): Boolean = { ++ shouldRetainRecord(batch, record, RecordBatch.NO_TIMESTAMP) ++ } ++ ++ override def retrieveDeleteHorizon(batch: RecordBatch) : Long = { ++ if (batch.deleteHorizonSet()) ++ return batch.deleteHorizonMs() // means that we keep the old timestamp stored ++ ++ // check that the control batch has been emptied of records ++ // if not, then we do not set a delete horizon until that is true ++ if (batch.isControlBatch() && !isControlBatchEmpty) ++ return -1L ++ return time.milliseconds() + tombstoneRetentionMs; + } + } + +@@ -720,6 +762,7 @@ private[log] class Cleaner(val id: Int, + growBuffersOrFail(sourceRecords, position, maxLogMessageSize, records) + } + restoreBuffers() ++ containsTombstones + } + + +@@ -758,20 +801,19 @@ private[log] class Cleaner(val id: Int, + private def shouldDiscardBatch(batch: RecordBatch, + transactionMetadata: CleanedTransactionMetadata, + retainTxnMarkers: Boolean): Boolean = { +- if (batch.isControlBatch) { +- val canDiscardControlBatch = transactionMetadata.onControlBatchRead(batch) +- canDiscardControlBatch && !retainTxnMarkers +- } else { +- val canDiscardBatch = transactionMetadata.onBatchRead(batch) +- canDiscardBatch +- } ++ if (batch.isControlBatch) ++ transactionMetadata.onControlBatchRead(batch) ++ else ++ transactionMetadata.onBatchRead(batch) + } + + private def shouldRetainRecord(map: kafka.log.OffsetMap, + retainDeletes: Boolean, + batch: RecordBatch, + record: Record, +- stats: CleanerStats): Boolean = { ++ stats: CleanerStats, ++ newBatchDeleteHorizonMs: Long, ++ currentTime: Long = -1L): Boolean = { + val pastLatestOffset = record.offset > map.latestOffset + if (pastLatestOffset) + return true +@@ -785,7 +827,15 @@ private[log] class Cleaner(val id: Int, + * 2) The message doesn't has value but it can't be deleted now. + */ + val latestOffsetForKey = record.offset() >= foundOffset +- val isRetainedValue = record.hasValue || retainDeletes ++ val isLatestVersion = batch.magic() >= RecordBatch.MAGIC_VALUE_V2 ++ var shouldRetainDeletes = true ++ if (isLatestVersion) ++ shouldRetainDeletes = (batch.deleteHorizonSet() && currentTime < batch.deleteHorizonMs()) || ++ (!batch.deleteHorizonSet() && currentTime < newBatchDeleteHorizonMs) ++ else ++ shouldRetainDeletes = retainDeletes ++ val isRetainedValue = record.hasValue || shouldRetainDeletes ++ + latestOffsetForKey && isRetainedValue + } else { + stats.invalidMessage() +diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala +index a5cfed5c094..6edd73530a9 100755 +--- a/core/src/main/scala/kafka/log/LogCleanerManager.scala ++++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala +@@ -180,6 +180,7 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], + case (_, log) => log.config.compact // match logs that are marked as compacted + }.filterNot { + case (topicPartition, log) => ++ + // skip any logs already in-progress and uncleanable partitions + inProgress.contains(topicPartition) || isUncleanablePartition(log, topicPartition) + }.map { +@@ -202,7 +203,34 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], + (ltc.needCompactionNow && ltc.cleanableBytes > 0) || ltc.cleanableRatio > ltc.log.config.minCleanableRatio + } + if(cleanableLogs.isEmpty) { +- None ++ // in this case, we are probably in a low throughput situation ++ // therefore, we should take advantage of this fact and remove tombstones if we can ++ val logsContainingTombstones = logs.filter { ++ case (_, log) => log.containsTombstones ++ }.filterNot { ++ case (topicPartition, log) => ++ // skip any logs already in-progress and uncleanable partitions ++ inProgress.contains(topicPartition) || isUncleanablePartition(log, topicPartition) ++ }.map { ++ case (topicPartition, log) => // create a LogToClean instance for each ++ try { ++ val (firstDirtyOffset, firstUncleanableDirtyOffset) = cleanableOffsets(log, topicPartition, lastClean, now) ++ val compactionDelayMs = maxCompactionDelay(log, firstDirtyOffset, now) ++ preCleanStats.updateMaxCompactionDelay(compactionDelayMs) ++ ++ LogToClean(topicPartition, log, firstDirtyOffset, firstUncleanableDirtyOffset, compactionDelayMs > 0) ++ } catch { ++ case e: Throwable => throw new LogCleaningException(log, ++ s"Failed to calculate log cleaning stats for partition $topicPartition", e) ++ } ++ } ++ if (!logsContainingTombstones.isEmpty) { ++ val filthiest = logsContainingTombstones.max ++ inProgress.put(filthiest.topicPartition, LogCleaningInProgress) ++ Some(filthiest) ++ } else { ++ None ++ } + } else { + preCleanStats.recordCleanablePartitions(cleanableLogs.size) + val filthiest = cleanableLogs.max +diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +index d148c3f8959..aa908c4c787 100644 +--- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala ++++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +@@ -18,6 +18,7 @@ + package kafka.log + + import java.io.PrintWriter ++import java.util.Properties + + import com.yammer.metrics.Metrics + import com.yammer.metrics.core.{Gauge, MetricName} +@@ -178,6 +179,49 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K + s"but lastCleaned=$lastCleaned2", lastCleaned2 >= secondBlockCleanableSegmentOffset) + } + ++ @Test ++ def testTombstoneCleanWithLowThoroughput() : Unit = { ++ val tombstoneRetentionMs = 1000 // this is in milliseconds -> 1 second ++ ++ val topicPartitions = Array(new TopicPartition("log-partition", 0)) ++ val props = new Properties() ++ props.put(LogConfig.DeleteRetentionMsProp, "1000") ++ cleaner = makeCleaner(partitions = topicPartitions, propertyOverrides = props) ++ ++ val log = cleaner.logs.get(topicPartitions(0)) ++ ++ val T0 = time.milliseconds ++ writeKeyDups(numKeys = 1, numDups = 1, log, CompressionType.NONE, timestamp = T0, ++ startValue = 0, step = 1, isRecordTombstone = true) ++ ++ val activeSegAtT0 = log.activeSegment ++ ++ // roll the active segment ++ log.roll() ++ ++ cleaner.startup() ++ Thread.sleep(100) ++ ++ import JavaConverters._ ++ var containsTombstones: Boolean = false ++ for (segment <- log.logSegments; record <- segment.log.records.asScala) { ++ containsTombstones = true ++ } ++ assertTrue(containsTombstones) ++ time.sleep(tombstoneRetentionMs + 1) ++ ++ val firstBlockCleanableSegmentOffset = activeSegAtT0.baseOffset ++ ++ // the first block should get cleaned ++ cleaner.awaitCleaned(new TopicPartition("log-partition", 0), ++ firstBlockCleanableSegmentOffset, maxWaitMs = tombstoneRetentionMs * 3) ++ ++ for (segment <- log.logSegments; record <- segment.log.records.asScala) { ++ fail ("The log should not contain record " + record + ", tombstone has expired its lifetime.") ++ } ++ assertFalse(log.containsTombstones) ++ } ++ + private def readFromLog(log: Log): Iterable[(Int, Int)] = { + import JavaConverters._ + for (segment <- log.logSegments; record <- segment.log.records.asScala) yield { +@@ -187,12 +231,17 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K + } + } + +- private def writeKeyDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionType, timestamp: Long, startValue: Int, step: Int): Seq[(Int, Int)] = { ++ private def writeKeyDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionType, timestamp: Long, ++ startValue: Int, step: Int, isRecordTombstone: Boolean = false): Seq[(Int, Int)] = { + var valCounter = startValue + for (_ <- 0 until numDups; key <- 0 until numKeys) yield { + val curValue = valCounter +- log.appendAsLeader(TestUtils.singletonRecords(value = curValue.toString.getBytes, codec = codec, +- key = key.toString.getBytes, timestamp = timestamp), leaderEpoch = 0) ++ if (isRecordTombstone) ++ log.appendAsLeader(TestUtils.singletonRecords(value = null, codec = codec, ++ key = key.toString.getBytes, timestamp = timestamp), leaderEpoch = 0) ++ else ++ log.appendAsLeader(TestUtils.singletonRecords(value = curValue.toString.getBytes, codec = codec, ++ key = key.toString.getBytes, timestamp = timestamp), leaderEpoch = 0) + valCounter += step + (key, curValue) + } +diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +index bb30287bb12..7d03b07fd70 100755 +--- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala ++++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +@@ -347,7 +347,7 @@ class LogCleanerTest { + log.roll() + + // cannot remove the marker in this pass because there are still valid records +- var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertEquals(List(1, 3, 2), LogTest.keysInLog(log)) + assertEquals(List(0, 2, 3, 4, 5), offsetsInLog(log)) + +@@ -356,17 +356,17 @@ class LogCleanerTest { + log.roll() + + // the first cleaning preserves the commit marker (at offset 3) since there were still records for the transaction +- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) + assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) + + // delete horizon forced to 0 to verify marker is not removed early +- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = 0L)._1 ++ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = 0L)._1 + assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) + assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) + + // clean again with large delete horizon and verify the marker is removed +- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) + assertEquals(List(4, 5, 6, 7, 8), offsetsInLog(log)) + } +@@ -395,11 +395,11 @@ class LogCleanerTest { + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, isFromClient = false) + log.roll() + +- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) ++ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + assertEquals(List(2), LogTest.keysInLog(log)) + assertEquals(List(1, 3, 4), offsetsInLog(log)) + +- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) ++ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + assertEquals(List(2), LogTest.keysInLog(log)) + assertEquals(List(3, 4), offsetsInLog(log)) + } +@@ -434,14 +434,14 @@ class LogCleanerTest { + + // first time through the records are removed + // Expected State: [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}] +- var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertEquals(List(2, 3), LogTest.keysInLog(log)) + assertEquals(List(4, 5, 6, 7), offsetsInLog(log)) + assertEquals(List(1, 3, 4, 5, 6, 7), lastOffsetsPerBatchInLog(log)) + + // the empty batch remains if cleaned again because it still holds the last sequence + // Expected State: [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}] +- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertEquals(List(2, 3), LogTest.keysInLog(log)) + assertEquals(List(4, 5, 6, 7), offsetsInLog(log)) + assertEquals(List(1, 3, 4, 5, 6, 7), lastOffsetsPerBatchInLog(log)) +@@ -454,13 +454,13 @@ class LogCleanerTest { + log.roll() + + // Expected State: [{Producer1: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] +- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) + assertEquals(List(4, 5, 6, 7, 8, 9), offsetsInLog(log)) + assertEquals(List(1, 4, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log)) + + // Expected State: [{Producer1: EmptyBatch}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] +- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) + assertEquals(List(5, 6, 7, 8, 9), offsetsInLog(log)) + assertEquals(List(1, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log)) +@@ -484,14 +484,14 @@ class LogCleanerTest { + + // first time through the control batch is retained as an empty batch + // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] +- var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertEquals(List(2, 3), LogTest.keysInLog(log)) + assertEquals(List(1, 2), offsetsInLog(log)) + assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) + + // the empty control batch does not cause an exception when cleaned + // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] +- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertEquals(List(2, 3), LogTest.keysInLog(log)) + assertEquals(List(1, 2), offsetsInLog(log)) + assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) +@@ -515,7 +515,7 @@ class LogCleanerTest { + log.roll() + + // Both the record and the marker should remain after cleaning +- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) ++ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + assertEquals(List(0, 1), offsetsInLog(log)) + assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) + } +@@ -540,12 +540,12 @@ class LogCleanerTest { + // Both the batch and the marker should remain after cleaning. The batch is retained + // because it is the last entry for this producerId. The marker is retained because + // there are still batches remaining from this transaction. +- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) ++ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + assertEquals(List(1), offsetsInLog(log)) + assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) + + // The empty batch and the marker is still retained after a second cleaning. +- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) ++ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + assertEquals(List(1), offsetsInLog(log)) + assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) + } +@@ -570,12 +570,12 @@ class LogCleanerTest { + log.roll() + + // delete horizon set to 0 to verify marker is not removed early +- val dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = 0L)._1 ++ val dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = 0L)._1 + assertEquals(List(3), LogTest.keysInLog(log)) + assertEquals(List(3, 4, 5), offsetsInLog(log)) + + // clean again with large delete horizon and verify the marker is removed +- cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) ++ cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + assertEquals(List(3), LogTest.keysInLog(log)) + assertEquals(List(4, 5), offsetsInLog(log)) + } +@@ -609,12 +609,12 @@ class LogCleanerTest { + + // Both transactional batches will be cleaned. The last one will remain in the log + // as an empty batch in order to preserve the producer sequence number and epoch +- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) ++ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + assertEquals(List(1, 3, 4, 5), offsetsInLog(log)) + assertEquals(List(1, 2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) + + // On the second round of cleaning, the marker from the first transaction should be removed. +- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) ++ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + assertEquals(List(3, 4, 5), offsetsInLog(log)) + assertEquals(List(2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) + } +@@ -646,14 +646,14 @@ class LogCleanerTest { + assertAbortedTransactionIndexed() + + // first time through the records are removed +- var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertAbortedTransactionIndexed() + assertEquals(List(), LogTest.keysInLog(log)) + assertEquals(List(2), offsetsInLog(log)) // abort marker is retained + assertEquals(List(1, 2), lastOffsetsPerBatchInLog(log)) // empty batch is retained + + // the empty batch remains if cleaned again because it still holds the last sequence +- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertAbortedTransactionIndexed() + assertEquals(List(), LogTest.keysInLog(log)) + assertEquals(List(2), offsetsInLog(log)) // abort marker is still retained +@@ -663,13 +663,13 @@ class LogCleanerTest { + appendProducer(Seq(1)) + log.roll() + +- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertAbortedTransactionIndexed() + assertEquals(List(1), LogTest.keysInLog(log)) + assertEquals(List(2, 3), offsetsInLog(log)) // abort marker is not yet gone because we read the empty batch + assertEquals(List(2, 3), lastOffsetsPerBatchInLog(log)) // but we do not preserve the empty batch + +- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertEquals(List(1), LogTest.keysInLog(log)) + assertEquals(List(3), offsetsInLog(log)) // abort marker is gone + assertEquals(List(3), lastOffsetsPerBatchInLog(log)) diff --git a/generator/bin/.gitignore b/generator/bin/.gitignore new file mode 100644 index 0000000000000..7eed456bec8db --- /dev/null +++ b/generator/bin/.gitignore @@ -0,0 +1,2 @@ +/main/ +/test/ diff --git a/gradle.properties b/gradle.properties index a063f5c5b64e8..9d232a0c90d46 100644 --- a/gradle.properties +++ b/gradle.properties @@ -23,4 +23,5 @@ group=org.apache.kafka version=2.5.0-SNAPSHOT scalaVersion=2.12.10 task=build +org.gradle.caching=true org.gradle.jvmargs=-Xmx1024m -Xss2m diff --git a/streams/upgrade-system-tests-0100/bin/.gitignore b/streams/upgrade-system-tests-0100/bin/.gitignore new file mode 100644 index 0000000000000..1933786028b34 --- /dev/null +++ b/streams/upgrade-system-tests-0100/bin/.gitignore @@ -0,0 +1 @@ +/test/ diff --git a/streams/upgrade-system-tests-0101/bin/.gitignore b/streams/upgrade-system-tests-0101/bin/.gitignore new file mode 100644 index 0000000000000..1933786028b34 --- /dev/null +++ b/streams/upgrade-system-tests-0101/bin/.gitignore @@ -0,0 +1 @@ +/test/ diff --git a/streams/upgrade-system-tests-0102/bin/.gitignore b/streams/upgrade-system-tests-0102/bin/.gitignore new file mode 100644 index 0000000000000..1933786028b34 --- /dev/null +++ b/streams/upgrade-system-tests-0102/bin/.gitignore @@ -0,0 +1 @@ +/test/ diff --git a/streams/upgrade-system-tests-0110/bin/.gitignore b/streams/upgrade-system-tests-0110/bin/.gitignore new file mode 100644 index 0000000000000..1933786028b34 --- /dev/null +++ b/streams/upgrade-system-tests-0110/bin/.gitignore @@ -0,0 +1 @@ +/test/ diff --git a/streams/upgrade-system-tests-10/bin/.gitignore b/streams/upgrade-system-tests-10/bin/.gitignore new file mode 100644 index 0000000000000..1933786028b34 --- /dev/null +++ b/streams/upgrade-system-tests-10/bin/.gitignore @@ -0,0 +1 @@ +/test/ diff --git a/streams/upgrade-system-tests-11/bin/.gitignore b/streams/upgrade-system-tests-11/bin/.gitignore new file mode 100644 index 0000000000000..1933786028b34 --- /dev/null +++ b/streams/upgrade-system-tests-11/bin/.gitignore @@ -0,0 +1 @@ +/test/ diff --git a/streams/upgrade-system-tests-20/bin/.gitignore b/streams/upgrade-system-tests-20/bin/.gitignore new file mode 100644 index 0000000000000..1933786028b34 --- /dev/null +++ b/streams/upgrade-system-tests-20/bin/.gitignore @@ -0,0 +1 @@ +/test/ diff --git a/streams/upgrade-system-tests-21/bin/.gitignore b/streams/upgrade-system-tests-21/bin/.gitignore new file mode 100644 index 0000000000000..1933786028b34 --- /dev/null +++ b/streams/upgrade-system-tests-21/bin/.gitignore @@ -0,0 +1 @@ +/test/ diff --git a/streams/upgrade-system-tests-22/bin/.gitignore b/streams/upgrade-system-tests-22/bin/.gitignore new file mode 100644 index 0000000000000..1933786028b34 --- /dev/null +++ b/streams/upgrade-system-tests-22/bin/.gitignore @@ -0,0 +1 @@ +/test/ diff --git a/streams/upgrade-system-tests-23/bin/.gitignore b/streams/upgrade-system-tests-23/bin/.gitignore new file mode 100644 index 0000000000000..1933786028b34 --- /dev/null +++ b/streams/upgrade-system-tests-23/bin/.gitignore @@ -0,0 +1 @@ +/test/ diff --git a/streams/upgrade-system-tests-24/bin/.gitignore b/streams/upgrade-system-tests-24/bin/.gitignore new file mode 100644 index 0000000000000..1933786028b34 --- /dev/null +++ b/streams/upgrade-system-tests-24/bin/.gitignore @@ -0,0 +1 @@ +/test/ diff --git a/tools/bin/.gitignore b/tools/bin/.gitignore new file mode 100644 index 0000000000000..7eed456bec8db --- /dev/null +++ b/tools/bin/.gitignore @@ -0,0 +1,2 @@ +/main/ +/test/ diff --git a/total.out b/total.out new file mode 100644 index 0000000000000..87ef3a795b864 --- /dev/null +++ b/total.out @@ -0,0 +1,1167 @@ +diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml +index 13cfdb82b..8b28fcadb 100644 +--- a/checkstyle/checkstyle.xml ++++ b/checkstyle/checkstyle.xml +@@ -107,7 +107,7 @@ + + + +- ++ + + + +@@ -124,7 +124,7 @@ + + + +- ++ + + + +diff --git a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java +index 83637640a..cf74f24f4 100644 +--- a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java ++++ b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java +@@ -431,6 +431,16 @@ public abstract class AbstractLegacyRecordBatch extends AbstractRecordBatch impl + return record; + } + ++ @Override ++ public long deleteHorizonMs() { ++ return RecordBatch.NO_TIMESTAMP; ++ } ++ ++ @Override ++ public boolean deleteHorizonSet() { ++ return false; ++ } ++ + @Override + public boolean equals(Object o) { + if (this == o) +@@ -468,6 +478,16 @@ public abstract class AbstractLegacyRecordBatch extends AbstractRecordBatch impl + return buffer.getLong(OFFSET_OFFSET); + } + ++ @Override ++ public long deleteHorizonMs() { ++ return RecordBatch.NO_TIMESTAMP; ++ } ++ ++ @Override ++ public boolean deleteHorizonSet() { ++ return false; ++ } ++ + @Override + public LegacyRecord outerRecord() { + return record; +@@ -557,6 +577,16 @@ public abstract class AbstractLegacyRecordBatch extends AbstractRecordBatch impl + return loadFullBatch().baseOffset(); + } + ++ @Override ++ public long deleteHorizonMs() { ++ return RecordBatch.NO_TIMESTAMP; ++ } ++ ++ @Override ++ public boolean deleteHorizonSet() { ++ return false; ++ } ++ + @Override + public long lastOffset() { + return offset; +diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java +index 6d79b2685..e0d7549fe 100644 +--- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java ++++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java +@@ -89,11 +89,15 @@ import static org.apache.kafka.common.record.Records.LOG_OVERHEAD; + * by the broker and is preserved after compaction. Additionally, the MaxTimestamp of an empty batch always retains + * the previous value prior to becoming empty. + * ++ * The delete horizon flag for the sixth bit is used to determine if the first timestamp of the batch had been set to ++ * the time for which tombstones / transaction markers needs to be removed. If it is true, then the first timestamp is ++ * the delete horizon, otherwise, it is merely the first timestamp of the record batch. ++ * + * The current attributes are given below: + * +- * ------------------------------------------------------------------------------------------------- +- * | Unused (6-15) | Control (5) | Transactional (4) | Timestamp Type (3) | Compression Type (0-2) | +- * ------------------------------------------------------------------------------------------------- ++ * --------------------------------------------------------------------------------------------------------------------------- ++ * | Unused (7-15) | Delete Horizon Flag (6) | Control (5) | Transactional (4) | Timestamp Type (3) | Compression Type (0-2) | ++ * --------------------------------------------------------------------------------------------------------------------------- + */ + public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRecordBatch { + static final int BASE_OFFSET_OFFSET = 0; +@@ -128,6 +132,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe + private static final byte COMPRESSION_CODEC_MASK = 0x07; + private static final byte TRANSACTIONAL_FLAG_MASK = 0x10; + private static final int CONTROL_FLAG_MASK = 0x20; ++ private static final byte DELETE_HORIZON_FLAG_MASK = 0x40; + private static final byte TIMESTAMP_TYPE_MASK = 0x08; + + private static final int MAX_SKIP_BUFFER_SIZE = 2048; +@@ -155,10 +160,15 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe + } + + /** +- * Get the timestamp of the first record in this batch. It is always the create time of the record even if the ++ * Get the timestamp of the first record in this batch. It is usually the create time of the record even if the + * timestamp type of the batch is log append time. + * +- * @return The first timestamp or {@link RecordBatch#NO_TIMESTAMP} if the batch is empty ++ * There is the possibility that the first timestamp had been set to the delete horizon of the batch, ++ * in which case, the delete horizon will be returned instead. ++ * ++ * @return The first timestamp if the batch's delete horizon has not been set ++ * The delete horizon if the batch's delete horizon has been set ++ * {@link RecordBatch#NO_TIMESTAMP} if the batch is empty + */ + public long firstTimestamp() { + return buffer.getLong(FIRST_TIMESTAMP_OFFSET); +@@ -245,6 +255,18 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe + return (attributes() & TRANSACTIONAL_FLAG_MASK) > 0; + } + ++ @Override ++ public boolean deleteHorizonSet() { ++ return (attributes() & DELETE_HORIZON_FLAG_MASK) > 0; ++ } ++ ++ @Override ++ public long deleteHorizonMs() { ++ if (deleteHorizonSet()) ++ return firstTimestamp(); ++ return RecordBatch.NO_TIMESTAMP; ++ } ++ + @Override + public boolean isControlBatch() { + return (attributes() & CONTROL_FLAG_MASK) > 0; +@@ -360,7 +382,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe + if (timestampType() == timestampType && currentMaxTimestamp == maxTimestamp) + return; + +- byte attributes = computeAttributes(compressionType(), timestampType, isTransactional(), isControlBatch()); ++ byte attributes = computeAttributes(compressionType(), timestampType, isTransactional(), isControlBatch(), deleteHorizonSet()); + buffer.putShort(ATTRIBUTES_OFFSET, attributes); + buffer.putLong(MAX_TIMESTAMP_OFFSET, maxTimestamp); + long crc = computeChecksum(); +@@ -407,7 +429,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe + } + + private static byte computeAttributes(CompressionType type, TimestampType timestampType, +- boolean isTransactional, boolean isControl) { ++ boolean isTransactional, boolean isControl, boolean isDeleteHorizonSet) { + if (timestampType == TimestampType.NO_TIMESTAMP_TYPE) + throw new IllegalArgumentException("Timestamp type must be provided to compute attributes for message " + + "format v2 and above"); +@@ -419,6 +441,8 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe + attributes |= COMPRESSION_CODEC_MASK & type.id; + if (timestampType == TimestampType.LOG_APPEND_TIME) + attributes |= TIMESTAMP_TYPE_MASK; ++ if (isDeleteHorizonSet) ++ attributes |= DELETE_HORIZON_FLAG_MASK; + return attributes; + } + +@@ -435,9 +459,49 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe + boolean isTransactional, + boolean isControlRecord) { + int offsetDelta = (int) (lastOffset - baseOffset); ++ writeHeader(buffer, baseOffset, offsetDelta, DefaultRecordBatch.RECORD_BATCH_OVERHEAD, magic, ++ CompressionType.NONE, timestampType, RecordBatch.NO_TIMESTAMP, timestamp, producerId, ++ producerEpoch, baseSequence, isTransactional, isControlRecord, false, partitionLeaderEpoch, 0); ++ } ++ ++ public static void writeEmptyHeader(ByteBuffer buffer, ++ byte magic, ++ long producerId, ++ short producerEpoch, ++ int baseSequence, ++ long baseOffset, ++ long lastOffset, ++ int partitionLeaderEpoch, ++ TimestampType timestampType, ++ long timestamp, ++ boolean isTransactional, ++ boolean isControlRecord, ++ boolean isDeleteHorizonSet) { ++ int offsetDelta = (int) (lastOffset - baseOffset); + writeHeader(buffer, baseOffset, offsetDelta, DefaultRecordBatch.RECORD_BATCH_OVERHEAD, magic, + CompressionType.NONE, timestampType, RecordBatch.NO_TIMESTAMP, timestamp, producerId, +- producerEpoch, baseSequence, isTransactional, isControlRecord, partitionLeaderEpoch, 0); ++ producerEpoch, baseSequence, isTransactional, isControlRecord, isDeleteHorizonSet, partitionLeaderEpoch, 0); ++ } ++ ++ static void writeHeader(ByteBuffer buffer, ++ long baseOffset, ++ int lastOffsetDelta, ++ int sizeInBytes, ++ byte magic, ++ CompressionType compressionType, ++ TimestampType timestampType, ++ long firstTimestamp, ++ long maxTimestamp, ++ long producerId, ++ short epoch, ++ int sequence, ++ boolean isTransactional, ++ boolean isControlBatch, ++ int partitionLeaderEpoch, ++ int numRecords) { ++ writeHeader(buffer, baseOffset, lastOffsetDelta, sizeInBytes, magic, compressionType, ++ timestampType, firstTimestamp, maxTimestamp, producerId, epoch, sequence, ++ isTransactional, isControlBatch, false, partitionLeaderEpoch, numRecords); + } + + static void writeHeader(ByteBuffer buffer, +@@ -454,6 +518,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe + int sequence, + boolean isTransactional, + boolean isControlBatch, ++ boolean isDeleteHorizonSet, + int partitionLeaderEpoch, + int numRecords) { + if (magic < RecordBatch.CURRENT_MAGIC_VALUE) +@@ -461,7 +526,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe + if (firstTimestamp < 0 && firstTimestamp != NO_TIMESTAMP) + throw new IllegalArgumentException("Invalid message timestamp " + firstTimestamp); + +- short attributes = computeAttributes(compressionType, timestampType, isTransactional, isControlBatch); ++ short attributes = computeAttributes(compressionType, timestampType, isTransactional, isControlBatch, isDeleteHorizonSet); + + int position = buffer.position(); + buffer.putLong(position + BASE_OFFSET_OFFSET, baseOffset); +@@ -699,6 +764,18 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe + return loadBatchHeader().isTransactional(); + } + ++ @Override ++ public boolean deleteHorizonSet() { ++ return loadBatchHeader().deleteHorizonSet(); ++ } ++ ++ @Override ++ public long deleteHorizonMs() { ++ if (deleteHorizonSet()) ++ return super.loadBatchHeader().deleteHorizonMs(); ++ return RecordBatch.NO_TIMESTAMP; ++ } ++ + @Override + public boolean isControlBatch() { + return loadBatchHeader().isControlBatch(); +diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +index 8f73565d1..7fb5fe951 100644 +--- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java ++++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +@@ -150,15 +150,29 @@ public class MemoryRecords extends AbstractRecords { + return filterTo(partition, batches(), filter, destinationBuffer, maxRecordBatchSize, decompressionBufferSupplier); + } + ++ /** ++ * Note: This method is also used to convert the first timestamp of the batch (which is usually the timestamp of the first record) ++ * to the delete horizon of the tombstones which are present in the batch. ++ */ + private static FilterResult filterTo(TopicPartition partition, Iterable batches, + RecordFilter filter, ByteBuffer destinationBuffer, int maxRecordBatchSize, + BufferSupplier decompressionBufferSupplier) { + FilterResult filterResult = new FilterResult(destinationBuffer); + ByteBufferOutputStream bufferOutputStream = new ByteBufferOutputStream(destinationBuffer); +- + for (MutableRecordBatch batch : batches) { + long maxOffset = -1L; +- BatchRetention batchRetention = filter.checkBatchRetention(batch); ++ // we first call this method here so that the flag in LogCleaner has been set ++ // which indicates if the control batch is empty or not ++ // we do this to avoid calling CleanedTransactionMetadata#onControlBatchRead ++ // more than once since each call is relatively expensive ++ filter.isControlBatchEmpty(batch); ++ long deleteHorizonMs = filter.retrieveDeleteHorizon(batch); ++ final BatchRetention batchRetention; ++ if (!batch.deleteHorizonSet()) ++ batchRetention = filter.checkBatchRetention(batch, deleteHorizonMs); ++ else ++ batchRetention = filter.checkBatchRetention(batch); ++ + filterResult.bytesRead += batch.sizeInBytes(); + + if (batchRetention == BatchRetention.DELETE) +@@ -170,36 +184,27 @@ public class MemoryRecords extends AbstractRecords { + // recopy the messages to the destination buffer. + + byte batchMagic = batch.magic(); ++ // we want to check if the delete horizon has been set or stayed the same + boolean writeOriginalBatch = true; ++ boolean containsTombstonesOrMarker = false; + List retainedRecords = new ArrayList<>(); + +- try (final CloseableIterator iterator = batch.streamingIterator(decompressionBufferSupplier)) { +- while (iterator.hasNext()) { +- Record record = iterator.next(); +- filterResult.messagesRead += 1; +- +- if (filter.shouldRetainRecord(batch, record)) { +- // Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite +- // the corrupted batch with correct data. +- if (!record.hasMagic(batchMagic)) +- writeOriginalBatch = false; +- +- if (record.offset() > maxOffset) +- maxOffset = record.offset(); +- +- retainedRecords.add(record); +- } else { +- writeOriginalBatch = false; +- } +- } +- } ++ final BatchIterationResult iterationResult = iterateOverBatch(batch, decompressionBufferSupplier, filterResult, filter, ++ batchMagic, writeOriginalBatch, maxOffset, retainedRecords, ++ containsTombstonesOrMarker, deleteHorizonMs); ++ containsTombstonesOrMarker = iterationResult.containsTombstonesOrMarker(); ++ writeOriginalBatch = iterationResult.shouldWriteOriginalBatch(); ++ maxOffset = iterationResult.maxOffset(); + + if (!retainedRecords.isEmpty()) { +- if (writeOriginalBatch) { ++ // we check if the delete horizon should be set to a new value ++ // in which case, we need to reset the base timestamp and overwrite the timestamp deltas ++ // if the batch does not contain tombstones, then we don't need to overwrite batch ++ if (writeOriginalBatch && (deleteHorizonMs == RecordBatch.NO_TIMESTAMP || deleteHorizonMs == batch.deleteHorizonMs() || !containsTombstonesOrMarker)) { + batch.writeTo(bufferOutputStream); + filterResult.updateRetainedBatchMetadata(batch, retainedRecords.size(), false); + } else { +- MemoryRecordsBuilder builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream); ++ MemoryRecordsBuilder builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, deleteHorizonMs); + MemoryRecords records = builder.build(); + int filteredBatchSize = records.sizeInBytes(); + if (filteredBatchSize > batch.sizeInBytes() && filteredBatchSize > maxRecordBatchSize) +@@ -236,9 +241,69 @@ public class MemoryRecords extends AbstractRecords { + return filterResult; + } + ++ private static BatchIterationResult iterateOverBatch(RecordBatch batch, ++ BufferSupplier decompressionBufferSupplier, ++ FilterResult filterResult, ++ RecordFilter filter, ++ byte batchMagic, ++ boolean writeOriginalBatch, ++ long maxOffset, ++ List retainedRecords, ++ boolean containsTombstonesOrMarker, ++ long newBatchDeleteHorizonMs) { ++ try (final CloseableIterator iterator = batch.streamingIterator(decompressionBufferSupplier)) { ++ while (iterator.hasNext()) { ++ Record record = iterator.next(); ++ filterResult.messagesRead += 1; ++ ++ if (filter.shouldRetainRecord(batch, record, newBatchDeleteHorizonMs)) { ++ // Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite ++ // the corrupted batch with correct data. ++ if (!record.hasMagic(batchMagic)) ++ writeOriginalBatch = false; ++ ++ if (record.offset() > maxOffset) ++ maxOffset = record.offset(); ++ ++ retainedRecords.add(record); ++ ++ if (!record.hasValue()) { ++ containsTombstonesOrMarker = true; ++ } ++ } else { ++ writeOriginalBatch = false; ++ } ++ } ++ return new BatchIterationResult(writeOriginalBatch, containsTombstonesOrMarker, maxOffset); ++ } ++ } ++ ++ private static class BatchIterationResult { ++ private final boolean writeOriginalBatch; ++ private final boolean containsTombstonesOrMarker; ++ private final long maxOffset; ++ public BatchIterationResult(final boolean writeOriginalBatch, ++ final boolean containsTombstonesOrMarker, ++ final long maxOffset) { ++ this.writeOriginalBatch = writeOriginalBatch; ++ this.containsTombstonesOrMarker = containsTombstonesOrMarker; ++ this.maxOffset = maxOffset; ++ } ++ public boolean shouldWriteOriginalBatch() { ++ return this.writeOriginalBatch; ++ } ++ public boolean containsTombstonesOrMarker() { ++ return this.containsTombstonesOrMarker; ++ } ++ public long maxOffset() { ++ return this.maxOffset; ++ } ++ } ++ + private static MemoryRecordsBuilder buildRetainedRecordsInto(RecordBatch originalBatch, + List retainedRecords, +- ByteBufferOutputStream bufferOutputStream) { ++ ByteBufferOutputStream bufferOutputStream, ++ final long deleteHorizonMs) { + byte magic = originalBatch.magic(); + TimestampType timestampType = originalBatch.timestampType(); + long logAppendTime = timestampType == TimestampType.LOG_APPEND_TIME ? +@@ -249,7 +314,7 @@ public class MemoryRecords extends AbstractRecords { + MemoryRecordsBuilder builder = new MemoryRecordsBuilder(bufferOutputStream, magic, + originalBatch.compressionType(), timestampType, baseOffset, logAppendTime, originalBatch.producerId(), + originalBatch.producerEpoch(), originalBatch.baseSequence(), originalBatch.isTransactional(), +- originalBatch.isControlBatch(), originalBatch.partitionLeaderEpoch(), bufferOutputStream.limit()); ++ originalBatch.isControlBatch(), originalBatch.partitionLeaderEpoch(), bufferOutputStream.limit(), deleteHorizonMs); + + for (Record record : retainedRecords) + builder.append(record); +@@ -312,12 +377,34 @@ public class MemoryRecords extends AbstractRecords { + */ + protected abstract BatchRetention checkBatchRetention(RecordBatch batch); + ++ protected BatchRetention checkBatchRetention(RecordBatch batch, long newBatchDeleteHorizonMs) { ++ return checkBatchRetention(batch); ++ } ++ + /** + * Check whether a record should be retained in the log. Note that {@link #checkBatchRetention(RecordBatch)} + * is used prior to checking individual record retention. Only records from batches which were not + * explicitly discarded with {@link BatchRetention#DELETE} will be considered. + */ + protected abstract boolean shouldRetainRecord(RecordBatch recordBatch, Record record); ++ ++ protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record, long newDeleteHorizonMs) { ++ return shouldRetainRecord(recordBatch, record); ++ } ++ ++ /** ++ * Retrieves the delete horizon ms for a specific batch ++ */ ++ protected long retrieveDeleteHorizon(RecordBatch recordBatch) { ++ return -1L; ++ } ++ ++ /** ++ * Checks if the control batch (if it is one) can be removed (making sure that it is empty) ++ */ ++ protected boolean isControlBatchEmpty(RecordBatch recordBatch) { ++ return true; ++ } + } + + public static class FilterResult { +diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java +index 054fb8619..be32a8390 100644 +--- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java ++++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java +@@ -75,6 +75,7 @@ public class MemoryRecordsBuilder implements AutoCloseable { + private int numRecords = 0; + private float actualCompressionRatio = 1; + private long maxTimestamp = RecordBatch.NO_TIMESTAMP; ++ private long deleteHorizonMs; + private long offsetOfMaxTimestamp = -1; + private Long lastOffset = null; + private Long firstTimestamp = null; +@@ -94,7 +95,8 @@ public class MemoryRecordsBuilder implements AutoCloseable { + boolean isTransactional, + boolean isControlBatch, + int partitionLeaderEpoch, +- int writeLimit) { ++ int writeLimit, ++ long deleteHorizonMs) { + if (magic > RecordBatch.MAGIC_VALUE_V0 && timestampType == TimestampType.NO_TIMESTAMP_TYPE) + throw new IllegalArgumentException("TimestampType must be set for magic >= 0"); + if (magic < RecordBatch.MAGIC_VALUE_V2) { +@@ -120,6 +122,7 @@ public class MemoryRecordsBuilder implements AutoCloseable { + this.baseSequence = baseSequence; + this.isTransactional = isTransactional; + this.isControlBatch = isControlBatch; ++ this.deleteHorizonMs = deleteHorizonMs; + this.partitionLeaderEpoch = partitionLeaderEpoch; + this.writeLimit = writeLimit; + this.initialPosition = bufferStream.position(); +@@ -130,6 +133,24 @@ public class MemoryRecordsBuilder implements AutoCloseable { + this.appendStream = new DataOutputStream(compressionType.wrapForOutput(this.bufferStream, magic)); + } + ++ public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, ++ byte magic, ++ CompressionType compressionType, ++ TimestampType timestampType, ++ long baseOffset, ++ long logAppendTime, ++ long producerId, ++ short producerEpoch, ++ int baseSequence, ++ boolean isTransactional, ++ boolean isControlBatch, ++ int partitionLeaderEpoch, ++ int writeLimit) { ++ this(bufferStream, magic, compressionType, timestampType, baseOffset, logAppendTime, producerId, ++ producerEpoch, baseSequence, isTransactional, isControlBatch, partitionLeaderEpoch, writeLimit, ++ RecordBatch.NO_TIMESTAMP); ++ } ++ + /** + * Construct a new builder. + * +@@ -192,6 +213,10 @@ public class MemoryRecordsBuilder implements AutoCloseable { + return isTransactional; + } + ++ public boolean deleteHorizonSet() { ++ return deleteHorizonMs >= 0L; ++ } ++ + /** + * Close this builder and return the resulting buffer. + * @return The built log buffer +@@ -364,7 +389,7 @@ public class MemoryRecordsBuilder implements AutoCloseable { + maxTimestamp = this.maxTimestamp; + + DefaultRecordBatch.writeHeader(buffer, baseOffset, offsetDelta, size, magic, compressionType, timestampType, +- firstTimestamp, maxTimestamp, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, ++ firstTimestamp, maxTimestamp, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, deleteHorizonSet(), + partitionLeaderEpoch, numRecords); + + buffer.position(pos); +@@ -411,8 +436,12 @@ public class MemoryRecordsBuilder implements AutoCloseable { + if (magic < RecordBatch.MAGIC_VALUE_V2 && headers != null && headers.length > 0) + throw new IllegalArgumentException("Magic v" + magic + " does not support record headers"); + +- if (firstTimestamp == null) +- firstTimestamp = timestamp; ++ if (firstTimestamp == null) { ++ if (deleteHorizonSet()) ++ firstTimestamp = deleteHorizonMs; ++ else ++ firstTimestamp = timestamp; ++ } + + if (magic > RecordBatch.MAGIC_VALUE_V1) { + appendDefaultRecord(offset, timestamp, key, value, headers); +diff --git a/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java +index 65a6a95fb..45f1609e3 100644 +--- a/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java ++++ b/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java +@@ -210,6 +210,18 @@ public interface RecordBatch extends Iterable { + */ + boolean isTransactional(); + ++ /** ++ * Whether or not the base timestamp has been set to the delete horizon ++ * @return true if it is, false otherwise ++ */ ++ boolean deleteHorizonSet(); ++ ++ /** ++ * Get the delete horizon, returns -1L if the first timestamp is not the delete horizon ++ * @return timestamp of the delete horizon ++ */ ++ long deleteHorizonMs(); ++ + /** + * Get the partition leader epoch of this record batch. + * @return The leader epoch or -1 if it is unknown +diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java +index b8824d3a8..4677e40e3 100644 +--- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java ++++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java +@@ -247,6 +247,45 @@ public class MemoryRecordsTest { + } + } + ++ /** ++ * This test is used to see if the first timestamp of the batch has been successfully ++ * converted to a delete horizon for the tombstones / transaction markers of the batch. ++ */ ++ @Test ++ public void testFirstTimestampToDeleteHorizonConversion() { ++ if (magic >= RecordBatch.MAGIC_VALUE_V2) { ++ ByteBuffer buffer = ByteBuffer.allocate(2048); ++ MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, ++ 0L, RecordBatch.NO_TIMESTAMP, partitionLeaderEpoch); ++ builder.append(10L, "1".getBytes(), null); ++ ++ ByteBuffer filtered = ByteBuffer.allocate(2048); ++ final long deleteHorizon = Integer.MAX_VALUE / 2; ++ builder.build().filterTo(new TopicPartition("random", 0), new MemoryRecords.RecordFilter() { ++ @Override ++ protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { ++ return true; ++ } ++ ++ @Override ++ protected BatchRetention checkBatchRetention(RecordBatch batch) { ++ return BatchRetention.RETAIN_EMPTY; ++ } ++ ++ @Override ++ protected long retrieveDeleteHorizon(RecordBatch batch) { ++ return deleteHorizon; // arbitrary value > 1 ++ } ++ }, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); ++ filtered.flip(); ++ MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); ++ ++ List batches = TestUtils.toList(filteredRecords.batches()); ++ assertEquals(1, batches.size()); ++ assertEquals(deleteHorizon, batches.get(0).deleteHorizonMs()); ++ } ++ } ++ + @Test + public void testFilterToEmptyBatchRetention() { + if (magic >= RecordBatch.MAGIC_VALUE_V2) { +diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala +index 97fa743d5..2b1b292fb 100644 +--- a/core/src/main/scala/kafka/log/Log.scala ++++ b/core/src/main/scala/kafka/log/Log.scala +@@ -221,7 +221,8 @@ class Log(@volatile var dir: File, + val producerIdExpirationCheckIntervalMs: Int, + val topicPartition: TopicPartition, + val producerStateManager: ProducerStateManager, +- logDirFailureChannel: LogDirFailureChannel) extends Logging with KafkaMetricsGroup { ++ logDirFailureChannel: LogDirFailureChannel, ++ @volatile var containsTombstones: Boolean = false) extends Logging with KafkaMetricsGroup { + + import kafka.log.Log._ + +diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala +index bcb358631..33bea6d99 100644 +--- a/core/src/main/scala/kafka/log/LogCleaner.scala ++++ b/core/src/main/scala/kafka/log/LogCleaner.scala +@@ -334,7 +334,8 @@ class LogCleaner(initialConfig: CleanerConfig, + @throws(classOf[LogCleaningException]) + private def cleanFilthiestLog(): Boolean = { + val preCleanStats = new PreCleanStats() +- val cleaned = cleanerManager.grabFilthiestCompactedLog(time, preCleanStats) match { ++ val ltc =cleanerManager.grabFilthiestCompactedLog(time, preCleanStats) ++ val cleaned = ltc match { + case None => + false + case Some(cleanable) => +@@ -498,11 +499,10 @@ private[log] class Cleaner(val id: Int, + case None => 0L + case Some(seg) => seg.lastModified - cleanable.log.config.deleteRetentionMs + } +- +- doClean(cleanable, deleteHorizonMs) ++ doClean(cleanable, time.milliseconds(), trackedHorizon = deleteHorizonMs) + } + +- private[log] def doClean(cleanable: LogToClean, deleteHorizonMs: Long): (Long, CleanerStats) = { ++ private[log] def doClean(cleanable: LogToClean, currentTime: Long, trackedHorizon: Long = -1L): (Long, CleanerStats) = { + info("Beginning cleaning of log %s.".format(cleanable.log.name)) + + val log = cleanable.log +@@ -520,13 +520,13 @@ private[log] class Cleaner(val id: Int, + val cleanableHorizonMs = log.logSegments(0, cleanable.firstUncleanableOffset).lastOption.map(_.lastModified).getOrElse(0L) + + // group the segments and clean the groups +- info("Cleaning log %s (cleaning prior to %s, discarding tombstones prior to %s)...".format(log.name, new Date(cleanableHorizonMs), new Date(deleteHorizonMs))) ++ info("Cleaning log %s (cleaning prior to %s, discarding tombstones prior to %s)...".format(log.name, new Date(cleanableHorizonMs), new Date(trackedHorizon))) + val transactionMetadata = new CleanedTransactionMetadata + + val groupedSegments = groupSegmentsBySize(log.logSegments(0, endOffset), log.config.segmentSize, + log.config.maxIndexSize, cleanable.firstUncleanableOffset) + for (group <- groupedSegments) +- cleanSegments(log, group, offsetMap, deleteHorizonMs, stats, transactionMetadata) ++ cleanSegments(log, group, offsetMap, currentTime, stats, transactionMetadata, trackedHorizon = trackedHorizon) + + // record buffer utilization + stats.bufferUtilization = offsetMap.utilization +@@ -550,9 +550,10 @@ private[log] class Cleaner(val id: Int, + private[log] def cleanSegments(log: Log, + segments: Seq[LogSegment], + map: OffsetMap, +- deleteHorizonMs: Long, ++ currentTime: Long, + stats: CleanerStats, +- transactionMetadata: CleanedTransactionMetadata): Unit = { ++ transactionMetadata: CleanedTransactionMetadata, ++ trackedHorizon: Long = -1L): Unit = { + // create a new segment with a suffix appended to the name of the log and indexes + val cleaned = LogCleaner.createNewCleanedSegment(log, segments.head.baseOffset) + transactionMetadata.cleanedIndex = Some(cleaned.txnIndex) +@@ -572,14 +573,15 @@ private[log] class Cleaner(val id: Int, + val abortedTransactions = log.collectAbortedTransactions(startOffset, upperBoundOffset) + transactionMetadata.addAbortedTransactions(abortedTransactions) + +- val retainDeletesAndTxnMarkers = currentSegment.lastModified > deleteHorizonMs ++ val retainDeletesAndTxnMarkers = currentSegment.lastModified > trackedHorizon + info(s"Cleaning $currentSegment in log ${log.name} into ${cleaned.baseOffset} " + +- s"with deletion horizon $deleteHorizonMs, " + ++ s"with deletion horizon $trackedHorizon, " + + s"${if(retainDeletesAndTxnMarkers) "retaining" else "discarding"} deletes.") + + try { +- cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletesAndTxnMarkers, log.config.maxMessageSize, +- transactionMetadata, lastOffsetOfActiveProducers, stats) ++ val containsTombstones: Boolean = cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletesAndTxnMarkers, log.config.maxMessageSize, ++ transactionMetadata, lastOffsetOfActiveProducers, stats, log.config.deleteRetentionMs, currentTime = currentTime) ++ log.containsTombstones = containsTombstones + } catch { + case e: LogSegmentOffsetOverflowException => + // Split the current segment. It's also safest to abort the current cleaning process, so that we retry from +@@ -623,6 +625,7 @@ private[log] class Cleaner(val id: Int, + * @param retainDeletesAndTxnMarkers Should tombstones and markers be retained while cleaning this segment + * @param maxLogMessageSize The maximum message size of the corresponding topic + * @param stats Collector for cleaning statistics ++ * @param tombstoneRetentionMs Defines how long a tombstone should be kept as defined by log configuration + */ + private[log] def cleanInto(topicPartition: TopicPartition, + sourceRecords: FileRecords, +@@ -632,14 +635,33 @@ private[log] class Cleaner(val id: Int, + maxLogMessageSize: Int, + transactionMetadata: CleanedTransactionMetadata, + lastRecordsOfActiveProducers: Map[Long, LastRecord], +- stats: CleanerStats): Unit = { ++ stats: CleanerStats, ++ tombstoneRetentionMs: Long, ++ currentTime: Long = RecordBatch.NO_TIMESTAMP): Boolean = { ++ var containsTombstones: Boolean = false ++ + val logCleanerFilter: RecordFilter = new RecordFilter { + var discardBatchRecords: Boolean = _ ++ var isControlBatchEmpty: Boolean = _ + +- override def checkBatchRetention(batch: RecordBatch): BatchRetention = { ++ override def isControlBatchEmpty(batch: RecordBatch) : Boolean = { + // we piggy-back on the tombstone retention logic to delay deletion of transaction markers. + // note that we will never delete a marker until all the records from that transaction are removed. +- discardBatchRecords = shouldDiscardBatch(batch, transactionMetadata, retainTxnMarkers = retainDeletesAndTxnMarkers) ++ val canDiscardBatch = shouldDiscardBatch(batch, transactionMetadata, retainTxnMarkers = retainDeletesAndTxnMarkers) ++ isControlBatchEmpty = canDiscardBatch ++ isControlBatchEmpty ++ } ++ ++ override def checkBatchRetention(batch: RecordBatch, newBatchDeleteHorizonMs : Long): BatchRetention = { ++ val canDiscardBatch = isControlBatchEmpty ++ ++ if (batch.isControlBatch) { ++ discardBatchRecords = canDiscardBatch && ++ ((batch.deleteHorizonSet() && batch.deleteHorizonMs() < currentTime) || ++ newBatchDeleteHorizonMs != RecordBatch.NO_TIMESTAMP && newBatchDeleteHorizonMs < currentTime) ++ } else { ++ discardBatchRecords = canDiscardBatch ++ } + + def isBatchLastRecordOfProducer: Boolean = { + // We retain the batch in order to preserve the state of active producers. There are three cases: +@@ -664,12 +686,33 @@ private[log] class Cleaner(val id: Int, + BatchRetention.DELETE_EMPTY + } + +- override def shouldRetainRecord(batch: RecordBatch, record: Record): Boolean = { ++ override def checkBatchRetention(batch: RecordBatch): BatchRetention = checkBatchRetention(batch, batch.deleteHorizonMs()) ++ ++ override def shouldRetainRecord(batch: RecordBatch, record: Record, newDeleteHorizonMs: Long): Boolean = { ++ var isRecordRetained: Boolean = true + if (discardBatchRecords) + // The batch is only retained to preserve producer sequence information; the records can be removed +- false ++ isRecordRetained = false + else +- Cleaner.this.shouldRetainRecord(map, retainDeletesAndTxnMarkers, batch, record, stats) ++ isRecordRetained = Cleaner.this.shouldRetainRecord(map, retainDeletesAndTxnMarkers, batch, record, stats, newDeleteHorizonMs, currentTime = currentTime) ++ if (isRecordRetained && !record.hasValue()) ++ containsTombstones = true ++ isRecordRetained ++ } ++ ++ override def shouldRetainRecord(batch: RecordBatch, record: Record): Boolean = { ++ shouldRetainRecord(batch, record, RecordBatch.NO_TIMESTAMP) ++ } ++ ++ override def retrieveDeleteHorizon(batch: RecordBatch) : Long = { ++ if (batch.deleteHorizonSet()) ++ return batch.deleteHorizonMs() // means that we keep the old timestamp stored ++ ++ // check that the control batch has been emptied of records ++ // if not, then we do not set a delete horizon until that is true ++ if (batch.isControlBatch() && !isControlBatchEmpty) ++ return -1L ++ return time.milliseconds() + tombstoneRetentionMs; + } + } + +@@ -709,6 +752,7 @@ private[log] class Cleaner(val id: Int, + growBuffersOrFail(sourceRecords, position, maxLogMessageSize, records) + } + restoreBuffers() ++ containsTombstones + } + + +@@ -747,20 +791,19 @@ private[log] class Cleaner(val id: Int, + private def shouldDiscardBatch(batch: RecordBatch, + transactionMetadata: CleanedTransactionMetadata, + retainTxnMarkers: Boolean): Boolean = { +- if (batch.isControlBatch) { +- val canDiscardControlBatch = transactionMetadata.onControlBatchRead(batch) +- canDiscardControlBatch && !retainTxnMarkers +- } else { +- val canDiscardBatch = transactionMetadata.onBatchRead(batch) +- canDiscardBatch +- } ++ if (batch.isControlBatch) ++ transactionMetadata.onControlBatchRead(batch) ++ else ++ transactionMetadata.onBatchRead(batch) + } + + private def shouldRetainRecord(map: kafka.log.OffsetMap, + retainDeletes: Boolean, + batch: RecordBatch, + record: Record, +- stats: CleanerStats): Boolean = { ++ stats: CleanerStats, ++ newBatchDeleteHorizonMs: Long, ++ currentTime: Long = -1L): Boolean = { + val pastLatestOffset = record.offset > map.latestOffset + if (pastLatestOffset) + return true +@@ -774,7 +817,14 @@ private[log] class Cleaner(val id: Int, + * 2) The message doesn't has value but it can't be deleted now. + */ + val latestOffsetForKey = record.offset() >= foundOffset +- val isRetainedValue = record.hasValue || retainDeletes ++ val isLatestVersion = batch.magic() >= RecordBatch.MAGIC_VALUE_V2 ++ var shouldRetainDeletes = true ++ if (isLatestVersion) ++ shouldRetainDeletes = (batch.deleteHorizonSet() && currentTime < batch.deleteHorizonMs()) || ++ (!batch.deleteHorizonSet() && currentTime < newBatchDeleteHorizonMs) ++ else ++ shouldRetainDeletes = retainDeletes ++ val isRetainedValue = record.hasValue || shouldRetainDeletes + latestOffsetForKey && isRetainedValue + } else { + stats.invalidMessage() +diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala +index 4d49e1ffa..2623e958f 100755 +--- a/core/src/main/scala/kafka/log/LogCleanerManager.scala ++++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala +@@ -181,6 +181,7 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], + case (_, log) => log.config.compact // match logs that are marked as compacted + }.filterNot { + case (topicPartition, log) => ++ + // skip any logs already in-progress and uncleanable partitions + inProgress.contains(topicPartition) || isUncleanablePartition(log, topicPartition) + }.map { +@@ -204,7 +205,35 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], + (ltc.needCompactionNow && ltc.cleanableBytes > 0) || ltc.cleanableRatio > ltc.log.config.minCleanableRatio + } + if(cleanableLogs.isEmpty) { +- None ++ // in this case, we are probably in a low throughput situation ++ // therefore, we should take advantage of this fact and remove tombstones if we can ++ val logsContainingTombstones = logs.filter { ++ case (_, log) => log.containsTombstones ++ }.filterNot { ++ case (topicPartition, log) => ++ // skip any logs already in-progress and uncleanable partitions ++ inProgress.contains(topicPartition) || isUncleanablePartition(log, topicPartition) ++ }.map { ++ case (topicPartition, log) => // create a LogToClean instance for each ++ try { ++ val lastCleanOffset = lastClean.get(topicPartition) ++ val (firstDirtyOffset, firstUncleanableDirtyOffset) = cleanableOffsets(log, lastCleanOffset, now) ++ val compactionDelayMs = maxCompactionDelay(log, firstDirtyOffset, now) ++ preCleanStats.updateMaxCompactionDelay(compactionDelayMs) ++ ++ LogToClean(topicPartition, log, firstDirtyOffset, firstUncleanableDirtyOffset, compactionDelayMs > 0) ++ } catch { ++ case e: Throwable => throw new LogCleaningException(log, ++ s"Failed to calculate log cleaning stats for partition $topicPartition", e) ++ } ++ } ++ if (!logsContainingTombstones.isEmpty) { ++ val filthiest = logsContainingTombstones.max ++ inProgress.put(filthiest.topicPartition, LogCleaningInProgress) ++ Some(filthiest) ++ } else { ++ None ++ } + } else { + preCleanStats.recordCleanablePartitions(cleanableLogs.size) + val filthiest = cleanableLogs.max +diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +index 7aea6c1fc..74244cdc8 100644 +--- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala ++++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +@@ -18,6 +18,7 @@ + package kafka.log + + import java.io.PrintWriter ++import java.util.Properties + + import com.yammer.metrics.Metrics + import com.yammer.metrics.core.{Gauge, MetricName} +@@ -178,6 +179,51 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K + s"but lastCleaned=$lastCleaned2", lastCleaned2 >= secondBlockCleanableSegmentOffset) + } + ++ @Test ++ def testTombstoneCleanWithLowThoroughput() : Unit = { ++ val tombstoneRetentionMs = 1000 // this is in milliseconds -> 1 second ++ ++ val topicPartitions = Array(new TopicPartition("log-partition", 0)) ++ val props = new Properties() ++ props.put(LogConfig.DeleteRetentionMsProp, "1000") ++ cleaner = makeCleaner(partitions = topicPartitions, propertyOverrides = props) ++ ++ val log = cleaner.logs.get(topicPartitions(0)) ++ ++ val T0 = time.milliseconds ++ writeKeyDups(numKeys = 1, numDups = 1, log, CompressionType.NONE, timestamp = T0, ++ startValue = 0, step = 1, isRecordTombstone = true) ++ ++ val activeSegAtT0 = log.activeSegment ++ ++ // roll the active segment ++ log.roll() ++ ++ cleaner.startup() ++ Thread.sleep(400) ++ ++ import JavaConverters._ ++ var containsTombstones: Boolean = false ++ for (segment <- log.logSegments; record <- segment.log.records.asScala) { ++ containsTombstones = true ++ } ++ assertTrue(containsTombstones) ++ time.sleep(tombstoneRetentionMs + 1) ++ ++ val firstBlockCleanableSegmentOffset = activeSegAtT0.baseOffset ++ ++ Thread.sleep(300) ++ ++ // the first block should get cleaned ++ cleaner.awaitCleaned(new TopicPartition("log-partition", 0), ++ firstBlockCleanableSegmentOffset, maxWaitMs = tombstoneRetentionMs * 5) ++ ++ for (segment <- log.logSegments; record <- segment.log.records.asScala) { ++ fail ("The log should not contain record " + record + ", tombstone has expired its lifetime.") ++ } ++ assertFalse(log.containsTombstones) ++ } ++ + private def readFromLog(log: Log): Iterable[(Int, Int)] = { + import JavaConverters._ + for (segment <- log.logSegments; record <- segment.log.records.asScala) yield { +@@ -187,12 +233,17 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K + } + } + +- private def writeKeyDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionType, timestamp: Long, startValue: Int, step: Int): Seq[(Int, Int)] = { ++ private def writeKeyDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionType, timestamp: Long, ++ startValue: Int, step: Int, isRecordTombstone: Boolean = false): Seq[(Int, Int)] = { + var valCounter = startValue + for (_ <- 0 until numDups; key <- 0 until numKeys) yield { + val curValue = valCounter +- log.appendAsLeader(TestUtils.singletonRecords(value = curValue.toString.getBytes, codec = codec, +- key = key.toString.getBytes, timestamp = timestamp), leaderEpoch = 0) ++ if (isRecordTombstone) ++ log.appendAsLeader(TestUtils.singletonRecords(value = null, codec = codec, ++ key = key.toString.getBytes, timestamp = timestamp), leaderEpoch = 0) ++ else ++ log.appendAsLeader(TestUtils.singletonRecords(value = curValue.toString.getBytes, codec = codec, ++ key = key.toString.getBytes, timestamp = timestamp), leaderEpoch = 0) + valCounter += step + (key, curValue) + } +diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +index bb30287bb..7d03b07fd 100755 +--- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala ++++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +@@ -347,7 +347,7 @@ class LogCleanerTest { + log.roll() + + // cannot remove the marker in this pass because there are still valid records +- var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertEquals(List(1, 3, 2), LogTest.keysInLog(log)) + assertEquals(List(0, 2, 3, 4, 5), offsetsInLog(log)) + +@@ -356,17 +356,17 @@ class LogCleanerTest { + log.roll() + + // the first cleaning preserves the commit marker (at offset 3) since there were still records for the transaction +- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) + assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) + + // delete horizon forced to 0 to verify marker is not removed early +- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = 0L)._1 ++ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = 0L)._1 + assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) + assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) + + // clean again with large delete horizon and verify the marker is removed +- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) + assertEquals(List(4, 5, 6, 7, 8), offsetsInLog(log)) + } +@@ -395,11 +395,11 @@ class LogCleanerTest { + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, isFromClient = false) + log.roll() + +- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) ++ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + assertEquals(List(2), LogTest.keysInLog(log)) + assertEquals(List(1, 3, 4), offsetsInLog(log)) + +- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) ++ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + assertEquals(List(2), LogTest.keysInLog(log)) + assertEquals(List(3, 4), offsetsInLog(log)) + } +@@ -434,14 +434,14 @@ class LogCleanerTest { + + // first time through the records are removed + // Expected State: [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}] +- var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertEquals(List(2, 3), LogTest.keysInLog(log)) + assertEquals(List(4, 5, 6, 7), offsetsInLog(log)) + assertEquals(List(1, 3, 4, 5, 6, 7), lastOffsetsPerBatchInLog(log)) + + // the empty batch remains if cleaned again because it still holds the last sequence + // Expected State: [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}] +- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertEquals(List(2, 3), LogTest.keysInLog(log)) + assertEquals(List(4, 5, 6, 7), offsetsInLog(log)) + assertEquals(List(1, 3, 4, 5, 6, 7), lastOffsetsPerBatchInLog(log)) +@@ -454,13 +454,13 @@ class LogCleanerTest { + log.roll() + + // Expected State: [{Producer1: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] +- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) + assertEquals(List(4, 5, 6, 7, 8, 9), offsetsInLog(log)) + assertEquals(List(1, 4, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log)) + + // Expected State: [{Producer1: EmptyBatch}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] +- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) + assertEquals(List(5, 6, 7, 8, 9), offsetsInLog(log)) + assertEquals(List(1, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log)) +@@ -484,14 +484,14 @@ class LogCleanerTest { + + // first time through the control batch is retained as an empty batch + // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] +- var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertEquals(List(2, 3), LogTest.keysInLog(log)) + assertEquals(List(1, 2), offsetsInLog(log)) + assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) + + // the empty control batch does not cause an exception when cleaned + // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] +- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertEquals(List(2, 3), LogTest.keysInLog(log)) + assertEquals(List(1, 2), offsetsInLog(log)) + assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) +@@ -515,7 +515,7 @@ class LogCleanerTest { + log.roll() + + // Both the record and the marker should remain after cleaning +- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) ++ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + assertEquals(List(0, 1), offsetsInLog(log)) + assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) + } +@@ -540,12 +540,12 @@ class LogCleanerTest { + // Both the batch and the marker should remain after cleaning. The batch is retained + // because it is the last entry for this producerId. The marker is retained because + // there are still batches remaining from this transaction. +- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) ++ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + assertEquals(List(1), offsetsInLog(log)) + assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) + + // The empty batch and the marker is still retained after a second cleaning. +- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) ++ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + assertEquals(List(1), offsetsInLog(log)) + assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) + } +@@ -570,12 +570,12 @@ class LogCleanerTest { + log.roll() + + // delete horizon set to 0 to verify marker is not removed early +- val dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = 0L)._1 ++ val dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = 0L)._1 + assertEquals(List(3), LogTest.keysInLog(log)) + assertEquals(List(3, 4, 5), offsetsInLog(log)) + + // clean again with large delete horizon and verify the marker is removed +- cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) ++ cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + assertEquals(List(3), LogTest.keysInLog(log)) + assertEquals(List(4, 5), offsetsInLog(log)) + } +@@ -609,12 +609,12 @@ class LogCleanerTest { + + // Both transactional batches will be cleaned. The last one will remain in the log + // as an empty batch in order to preserve the producer sequence number and epoch +- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) ++ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + assertEquals(List(1, 3, 4, 5), offsetsInLog(log)) + assertEquals(List(1, 2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) + + // On the second round of cleaning, the marker from the first transaction should be removed. +- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) ++ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + assertEquals(List(3, 4, 5), offsetsInLog(log)) + assertEquals(List(2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) + } +@@ -646,14 +646,14 @@ class LogCleanerTest { + assertAbortedTransactionIndexed() + + // first time through the records are removed +- var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertAbortedTransactionIndexed() + assertEquals(List(), LogTest.keysInLog(log)) + assertEquals(List(2), offsetsInLog(log)) // abort marker is retained + assertEquals(List(1, 2), lastOffsetsPerBatchInLog(log)) // empty batch is retained + + // the empty batch remains if cleaned again because it still holds the last sequence +- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertAbortedTransactionIndexed() + assertEquals(List(), LogTest.keysInLog(log)) + assertEquals(List(2), offsetsInLog(log)) // abort marker is still retained +@@ -663,13 +663,13 @@ class LogCleanerTest { + appendProducer(Seq(1)) + log.roll() + +- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertAbortedTransactionIndexed() + assertEquals(List(1), LogTest.keysInLog(log)) + assertEquals(List(2, 3), offsetsInLog(log)) // abort marker is not yet gone because we read the empty batch + assertEquals(List(2, 3), lastOffsetsPerBatchInLog(log)) // but we do not preserve the empty batch + +- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 ++ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + assertEquals(List(1), LogTest.keysInLog(log)) + assertEquals(List(3), offsetsInLog(log)) // abort marker is gone + assertEquals(List(3), lastOffsetsPerBatchInLog(log)) From 1587462f676afca5d35e6d5fe007c15ec515c3af Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Tue, 11 Feb 2020 19:47:44 -0800 Subject: [PATCH 09/53] Getting modified --- .../apache/kafka/common/record/MemoryRecords.java | 13 +++++++------ .../unit/kafka/log/LogCleanerIntegrationTest.scala | 6 ++---- gradle.properties | 1 - 3 files changed, 9 insertions(+), 11 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index e3ab8ddf12dbc..77c4d5d01b2e8 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -195,11 +195,12 @@ private static FilterResult filterTo(TopicPartition partition, Iterable RecordBatch.NO_TIMESTAMP; - if (writeOriginalBatch && (batch.deleteHorizonSet() || (!containsTombstonesOrMarker && !canControlBatchBeRemoved))) { + if (writeOriginalBatch && (deleteHorizonMs == RecordBatch.NO_TIMESTAMP || deleteHorizonMs == batch.deleteHorizonMs() + || (!containsTombstonesOrMarker && !canControlBatchBeRemoved))) { batch.writeTo(bufferOutputStream); filterResult.updateRetainedBatchMetadata(batch, retainedRecords.size(), false); } else { - final MemoryRecordsBuilder builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, deleteHorizonMs); + MemoryRecordsBuilder builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, deleteHorizonMs); MemoryRecords records = builder.build(); int filteredBatchSize = records.sizeInBytes(); if (filteredBatchSize > batch.sizeInBytes() && filteredBatchSize > maxRecordBatchSize) @@ -241,7 +242,7 @@ private static BatchIterationResult iterateOverBatch(RecordBatch batch, FilterResult filterResult, RecordFilter filter, byte batchMagic, - boolean recordsFiltered, + boolean writeOriginalBatch, long maxOffset, List retainedRecords) { boolean containsTombstonesOrMarker = false; @@ -254,7 +255,7 @@ private static BatchIterationResult iterateOverBatch(RecordBatch batch, // Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite // the corrupted batch with correct data. if (!record.hasMagic(batchMagic)) - recordsFiltered = false; + writeOriginalBatch = false; if (record.offset() > maxOffset) maxOffset = record.offset(); @@ -265,10 +266,10 @@ private static BatchIterationResult iterateOverBatch(RecordBatch batch, containsTombstonesOrMarker = true; } } else { - recordsFiltered = false; + writeOriginalBatch = false; } } - return new BatchIterationResult(recordsFiltered, containsTombstonesOrMarker, maxOffset); + return new BatchIterationResult(writeOriginalBatch, containsTombstonesOrMarker, maxOffset); } } diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 2c3a75a3cade2..4866e66517269 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -208,9 +208,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K // We sleep a little bit, so that log cleaner has already gone through // some iterations, ensures that delete horizons has been updated correctly - Thread.sleep(400L) - assertEquals(log.latestDeleteHorizon, T0 + tombstoneRetentionMs) - + Thread.sleep(300L) time.sleep(tombstoneRetentionMs + 1) val latestOffset: Long = log.latestEpoch match { @@ -231,10 +229,10 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K cleaner.awaitCleaned(new TopicPartition("log-partition", 0), latestOffset + 1, maxWaitMs = tombstoneRetentionMs) + assertEquals(log.latestDeleteHorizon, RecordBatch.NO_TIMESTAMP) for (segment <- log.logSegments; record <- segment.log.records.asScala) { fail ("The log should not contain record " + record + ", tombstone has expired its lifetime.") } - assertEquals(log.latestDeleteHorizon, -1L) } private def readFromLog(log: Log): Iterable[(Int, Int)] = { diff --git a/gradle.properties b/gradle.properties index 9d232a0c90d46..a063f5c5b64e8 100644 --- a/gradle.properties +++ b/gradle.properties @@ -23,5 +23,4 @@ group=org.apache.kafka version=2.5.0-SNAPSHOT scalaVersion=2.12.10 task=build -org.gradle.caching=true org.gradle.jvmargs=-Xmx1024m -Xss2m From a25187bea674f0b0fd7e276b827e053229de1b2c Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 11 Feb 2020 19:48:56 -0800 Subject: [PATCH 10/53] Delete total.out --- total.out | 1167 ----------------------------------------------------- 1 file changed, 1167 deletions(-) delete mode 100644 total.out diff --git a/total.out b/total.out deleted file mode 100644 index 87ef3a795b864..0000000000000 --- a/total.out +++ /dev/null @@ -1,1167 +0,0 @@ -diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml -index 13cfdb82b..8b28fcadb 100644 ---- a/checkstyle/checkstyle.xml -+++ b/checkstyle/checkstyle.xml -@@ -107,7 +107,7 @@ - - - -- -+ - - - -@@ -124,7 +124,7 @@ - - - -- -+ - - - -diff --git a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java -index 83637640a..cf74f24f4 100644 ---- a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java -+++ b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java -@@ -431,6 +431,16 @@ public abstract class AbstractLegacyRecordBatch extends AbstractRecordBatch impl - return record; - } - -+ @Override -+ public long deleteHorizonMs() { -+ return RecordBatch.NO_TIMESTAMP; -+ } -+ -+ @Override -+ public boolean deleteHorizonSet() { -+ return false; -+ } -+ - @Override - public boolean equals(Object o) { - if (this == o) -@@ -468,6 +478,16 @@ public abstract class AbstractLegacyRecordBatch extends AbstractRecordBatch impl - return buffer.getLong(OFFSET_OFFSET); - } - -+ @Override -+ public long deleteHorizonMs() { -+ return RecordBatch.NO_TIMESTAMP; -+ } -+ -+ @Override -+ public boolean deleteHorizonSet() { -+ return false; -+ } -+ - @Override - public LegacyRecord outerRecord() { - return record; -@@ -557,6 +577,16 @@ public abstract class AbstractLegacyRecordBatch extends AbstractRecordBatch impl - return loadFullBatch().baseOffset(); - } - -+ @Override -+ public long deleteHorizonMs() { -+ return RecordBatch.NO_TIMESTAMP; -+ } -+ -+ @Override -+ public boolean deleteHorizonSet() { -+ return false; -+ } -+ - @Override - public long lastOffset() { - return offset; -diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java -index 6d79b2685..e0d7549fe 100644 ---- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java -+++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java -@@ -89,11 +89,15 @@ import static org.apache.kafka.common.record.Records.LOG_OVERHEAD; - * by the broker and is preserved after compaction. Additionally, the MaxTimestamp of an empty batch always retains - * the previous value prior to becoming empty. - * -+ * The delete horizon flag for the sixth bit is used to determine if the first timestamp of the batch had been set to -+ * the time for which tombstones / transaction markers needs to be removed. If it is true, then the first timestamp is -+ * the delete horizon, otherwise, it is merely the first timestamp of the record batch. -+ * - * The current attributes are given below: - * -- * ------------------------------------------------------------------------------------------------- -- * | Unused (6-15) | Control (5) | Transactional (4) | Timestamp Type (3) | Compression Type (0-2) | -- * ------------------------------------------------------------------------------------------------- -+ * --------------------------------------------------------------------------------------------------------------------------- -+ * | Unused (7-15) | Delete Horizon Flag (6) | Control (5) | Transactional (4) | Timestamp Type (3) | Compression Type (0-2) | -+ * --------------------------------------------------------------------------------------------------------------------------- - */ - public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRecordBatch { - static final int BASE_OFFSET_OFFSET = 0; -@@ -128,6 +132,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe - private static final byte COMPRESSION_CODEC_MASK = 0x07; - private static final byte TRANSACTIONAL_FLAG_MASK = 0x10; - private static final int CONTROL_FLAG_MASK = 0x20; -+ private static final byte DELETE_HORIZON_FLAG_MASK = 0x40; - private static final byte TIMESTAMP_TYPE_MASK = 0x08; - - private static final int MAX_SKIP_BUFFER_SIZE = 2048; -@@ -155,10 +160,15 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe - } - - /** -- * Get the timestamp of the first record in this batch. It is always the create time of the record even if the -+ * Get the timestamp of the first record in this batch. It is usually the create time of the record even if the - * timestamp type of the batch is log append time. - * -- * @return The first timestamp or {@link RecordBatch#NO_TIMESTAMP} if the batch is empty -+ * There is the possibility that the first timestamp had been set to the delete horizon of the batch, -+ * in which case, the delete horizon will be returned instead. -+ * -+ * @return The first timestamp if the batch's delete horizon has not been set -+ * The delete horizon if the batch's delete horizon has been set -+ * {@link RecordBatch#NO_TIMESTAMP} if the batch is empty - */ - public long firstTimestamp() { - return buffer.getLong(FIRST_TIMESTAMP_OFFSET); -@@ -245,6 +255,18 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe - return (attributes() & TRANSACTIONAL_FLAG_MASK) > 0; - } - -+ @Override -+ public boolean deleteHorizonSet() { -+ return (attributes() & DELETE_HORIZON_FLAG_MASK) > 0; -+ } -+ -+ @Override -+ public long deleteHorizonMs() { -+ if (deleteHorizonSet()) -+ return firstTimestamp(); -+ return RecordBatch.NO_TIMESTAMP; -+ } -+ - @Override - public boolean isControlBatch() { - return (attributes() & CONTROL_FLAG_MASK) > 0; -@@ -360,7 +382,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe - if (timestampType() == timestampType && currentMaxTimestamp == maxTimestamp) - return; - -- byte attributes = computeAttributes(compressionType(), timestampType, isTransactional(), isControlBatch()); -+ byte attributes = computeAttributes(compressionType(), timestampType, isTransactional(), isControlBatch(), deleteHorizonSet()); - buffer.putShort(ATTRIBUTES_OFFSET, attributes); - buffer.putLong(MAX_TIMESTAMP_OFFSET, maxTimestamp); - long crc = computeChecksum(); -@@ -407,7 +429,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe - } - - private static byte computeAttributes(CompressionType type, TimestampType timestampType, -- boolean isTransactional, boolean isControl) { -+ boolean isTransactional, boolean isControl, boolean isDeleteHorizonSet) { - if (timestampType == TimestampType.NO_TIMESTAMP_TYPE) - throw new IllegalArgumentException("Timestamp type must be provided to compute attributes for message " + - "format v2 and above"); -@@ -419,6 +441,8 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe - attributes |= COMPRESSION_CODEC_MASK & type.id; - if (timestampType == TimestampType.LOG_APPEND_TIME) - attributes |= TIMESTAMP_TYPE_MASK; -+ if (isDeleteHorizonSet) -+ attributes |= DELETE_HORIZON_FLAG_MASK; - return attributes; - } - -@@ -435,9 +459,49 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe - boolean isTransactional, - boolean isControlRecord) { - int offsetDelta = (int) (lastOffset - baseOffset); -+ writeHeader(buffer, baseOffset, offsetDelta, DefaultRecordBatch.RECORD_BATCH_OVERHEAD, magic, -+ CompressionType.NONE, timestampType, RecordBatch.NO_TIMESTAMP, timestamp, producerId, -+ producerEpoch, baseSequence, isTransactional, isControlRecord, false, partitionLeaderEpoch, 0); -+ } -+ -+ public static void writeEmptyHeader(ByteBuffer buffer, -+ byte magic, -+ long producerId, -+ short producerEpoch, -+ int baseSequence, -+ long baseOffset, -+ long lastOffset, -+ int partitionLeaderEpoch, -+ TimestampType timestampType, -+ long timestamp, -+ boolean isTransactional, -+ boolean isControlRecord, -+ boolean isDeleteHorizonSet) { -+ int offsetDelta = (int) (lastOffset - baseOffset); - writeHeader(buffer, baseOffset, offsetDelta, DefaultRecordBatch.RECORD_BATCH_OVERHEAD, magic, - CompressionType.NONE, timestampType, RecordBatch.NO_TIMESTAMP, timestamp, producerId, -- producerEpoch, baseSequence, isTransactional, isControlRecord, partitionLeaderEpoch, 0); -+ producerEpoch, baseSequence, isTransactional, isControlRecord, isDeleteHorizonSet, partitionLeaderEpoch, 0); -+ } -+ -+ static void writeHeader(ByteBuffer buffer, -+ long baseOffset, -+ int lastOffsetDelta, -+ int sizeInBytes, -+ byte magic, -+ CompressionType compressionType, -+ TimestampType timestampType, -+ long firstTimestamp, -+ long maxTimestamp, -+ long producerId, -+ short epoch, -+ int sequence, -+ boolean isTransactional, -+ boolean isControlBatch, -+ int partitionLeaderEpoch, -+ int numRecords) { -+ writeHeader(buffer, baseOffset, lastOffsetDelta, sizeInBytes, magic, compressionType, -+ timestampType, firstTimestamp, maxTimestamp, producerId, epoch, sequence, -+ isTransactional, isControlBatch, false, partitionLeaderEpoch, numRecords); - } - - static void writeHeader(ByteBuffer buffer, -@@ -454,6 +518,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe - int sequence, - boolean isTransactional, - boolean isControlBatch, -+ boolean isDeleteHorizonSet, - int partitionLeaderEpoch, - int numRecords) { - if (magic < RecordBatch.CURRENT_MAGIC_VALUE) -@@ -461,7 +526,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe - if (firstTimestamp < 0 && firstTimestamp != NO_TIMESTAMP) - throw new IllegalArgumentException("Invalid message timestamp " + firstTimestamp); - -- short attributes = computeAttributes(compressionType, timestampType, isTransactional, isControlBatch); -+ short attributes = computeAttributes(compressionType, timestampType, isTransactional, isControlBatch, isDeleteHorizonSet); - - int position = buffer.position(); - buffer.putLong(position + BASE_OFFSET_OFFSET, baseOffset); -@@ -699,6 +764,18 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe - return loadBatchHeader().isTransactional(); - } - -+ @Override -+ public boolean deleteHorizonSet() { -+ return loadBatchHeader().deleteHorizonSet(); -+ } -+ -+ @Override -+ public long deleteHorizonMs() { -+ if (deleteHorizonSet()) -+ return super.loadBatchHeader().deleteHorizonMs(); -+ return RecordBatch.NO_TIMESTAMP; -+ } -+ - @Override - public boolean isControlBatch() { - return loadBatchHeader().isControlBatch(); -diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java -index 8f73565d1..7fb5fe951 100644 ---- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java -+++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java -@@ -150,15 +150,29 @@ public class MemoryRecords extends AbstractRecords { - return filterTo(partition, batches(), filter, destinationBuffer, maxRecordBatchSize, decompressionBufferSupplier); - } - -+ /** -+ * Note: This method is also used to convert the first timestamp of the batch (which is usually the timestamp of the first record) -+ * to the delete horizon of the tombstones which are present in the batch. -+ */ - private static FilterResult filterTo(TopicPartition partition, Iterable batches, - RecordFilter filter, ByteBuffer destinationBuffer, int maxRecordBatchSize, - BufferSupplier decompressionBufferSupplier) { - FilterResult filterResult = new FilterResult(destinationBuffer); - ByteBufferOutputStream bufferOutputStream = new ByteBufferOutputStream(destinationBuffer); -- - for (MutableRecordBatch batch : batches) { - long maxOffset = -1L; -- BatchRetention batchRetention = filter.checkBatchRetention(batch); -+ // we first call this method here so that the flag in LogCleaner has been set -+ // which indicates if the control batch is empty or not -+ // we do this to avoid calling CleanedTransactionMetadata#onControlBatchRead -+ // more than once since each call is relatively expensive -+ filter.isControlBatchEmpty(batch); -+ long deleteHorizonMs = filter.retrieveDeleteHorizon(batch); -+ final BatchRetention batchRetention; -+ if (!batch.deleteHorizonSet()) -+ batchRetention = filter.checkBatchRetention(batch, deleteHorizonMs); -+ else -+ batchRetention = filter.checkBatchRetention(batch); -+ - filterResult.bytesRead += batch.sizeInBytes(); - - if (batchRetention == BatchRetention.DELETE) -@@ -170,36 +184,27 @@ public class MemoryRecords extends AbstractRecords { - // recopy the messages to the destination buffer. - - byte batchMagic = batch.magic(); -+ // we want to check if the delete horizon has been set or stayed the same - boolean writeOriginalBatch = true; -+ boolean containsTombstonesOrMarker = false; - List retainedRecords = new ArrayList<>(); - -- try (final CloseableIterator iterator = batch.streamingIterator(decompressionBufferSupplier)) { -- while (iterator.hasNext()) { -- Record record = iterator.next(); -- filterResult.messagesRead += 1; -- -- if (filter.shouldRetainRecord(batch, record)) { -- // Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite -- // the corrupted batch with correct data. -- if (!record.hasMagic(batchMagic)) -- writeOriginalBatch = false; -- -- if (record.offset() > maxOffset) -- maxOffset = record.offset(); -- -- retainedRecords.add(record); -- } else { -- writeOriginalBatch = false; -- } -- } -- } -+ final BatchIterationResult iterationResult = iterateOverBatch(batch, decompressionBufferSupplier, filterResult, filter, -+ batchMagic, writeOriginalBatch, maxOffset, retainedRecords, -+ containsTombstonesOrMarker, deleteHorizonMs); -+ containsTombstonesOrMarker = iterationResult.containsTombstonesOrMarker(); -+ writeOriginalBatch = iterationResult.shouldWriteOriginalBatch(); -+ maxOffset = iterationResult.maxOffset(); - - if (!retainedRecords.isEmpty()) { -- if (writeOriginalBatch) { -+ // we check if the delete horizon should be set to a new value -+ // in which case, we need to reset the base timestamp and overwrite the timestamp deltas -+ // if the batch does not contain tombstones, then we don't need to overwrite batch -+ if (writeOriginalBatch && (deleteHorizonMs == RecordBatch.NO_TIMESTAMP || deleteHorizonMs == batch.deleteHorizonMs() || !containsTombstonesOrMarker)) { - batch.writeTo(bufferOutputStream); - filterResult.updateRetainedBatchMetadata(batch, retainedRecords.size(), false); - } else { -- MemoryRecordsBuilder builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream); -+ MemoryRecordsBuilder builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, deleteHorizonMs); - MemoryRecords records = builder.build(); - int filteredBatchSize = records.sizeInBytes(); - if (filteredBatchSize > batch.sizeInBytes() && filteredBatchSize > maxRecordBatchSize) -@@ -236,9 +241,69 @@ public class MemoryRecords extends AbstractRecords { - return filterResult; - } - -+ private static BatchIterationResult iterateOverBatch(RecordBatch batch, -+ BufferSupplier decompressionBufferSupplier, -+ FilterResult filterResult, -+ RecordFilter filter, -+ byte batchMagic, -+ boolean writeOriginalBatch, -+ long maxOffset, -+ List retainedRecords, -+ boolean containsTombstonesOrMarker, -+ long newBatchDeleteHorizonMs) { -+ try (final CloseableIterator iterator = batch.streamingIterator(decompressionBufferSupplier)) { -+ while (iterator.hasNext()) { -+ Record record = iterator.next(); -+ filterResult.messagesRead += 1; -+ -+ if (filter.shouldRetainRecord(batch, record, newBatchDeleteHorizonMs)) { -+ // Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite -+ // the corrupted batch with correct data. -+ if (!record.hasMagic(batchMagic)) -+ writeOriginalBatch = false; -+ -+ if (record.offset() > maxOffset) -+ maxOffset = record.offset(); -+ -+ retainedRecords.add(record); -+ -+ if (!record.hasValue()) { -+ containsTombstonesOrMarker = true; -+ } -+ } else { -+ writeOriginalBatch = false; -+ } -+ } -+ return new BatchIterationResult(writeOriginalBatch, containsTombstonesOrMarker, maxOffset); -+ } -+ } -+ -+ private static class BatchIterationResult { -+ private final boolean writeOriginalBatch; -+ private final boolean containsTombstonesOrMarker; -+ private final long maxOffset; -+ public BatchIterationResult(final boolean writeOriginalBatch, -+ final boolean containsTombstonesOrMarker, -+ final long maxOffset) { -+ this.writeOriginalBatch = writeOriginalBatch; -+ this.containsTombstonesOrMarker = containsTombstonesOrMarker; -+ this.maxOffset = maxOffset; -+ } -+ public boolean shouldWriteOriginalBatch() { -+ return this.writeOriginalBatch; -+ } -+ public boolean containsTombstonesOrMarker() { -+ return this.containsTombstonesOrMarker; -+ } -+ public long maxOffset() { -+ return this.maxOffset; -+ } -+ } -+ - private static MemoryRecordsBuilder buildRetainedRecordsInto(RecordBatch originalBatch, - List retainedRecords, -- ByteBufferOutputStream bufferOutputStream) { -+ ByteBufferOutputStream bufferOutputStream, -+ final long deleteHorizonMs) { - byte magic = originalBatch.magic(); - TimestampType timestampType = originalBatch.timestampType(); - long logAppendTime = timestampType == TimestampType.LOG_APPEND_TIME ? -@@ -249,7 +314,7 @@ public class MemoryRecords extends AbstractRecords { - MemoryRecordsBuilder builder = new MemoryRecordsBuilder(bufferOutputStream, magic, - originalBatch.compressionType(), timestampType, baseOffset, logAppendTime, originalBatch.producerId(), - originalBatch.producerEpoch(), originalBatch.baseSequence(), originalBatch.isTransactional(), -- originalBatch.isControlBatch(), originalBatch.partitionLeaderEpoch(), bufferOutputStream.limit()); -+ originalBatch.isControlBatch(), originalBatch.partitionLeaderEpoch(), bufferOutputStream.limit(), deleteHorizonMs); - - for (Record record : retainedRecords) - builder.append(record); -@@ -312,12 +377,34 @@ public class MemoryRecords extends AbstractRecords { - */ - protected abstract BatchRetention checkBatchRetention(RecordBatch batch); - -+ protected BatchRetention checkBatchRetention(RecordBatch batch, long newBatchDeleteHorizonMs) { -+ return checkBatchRetention(batch); -+ } -+ - /** - * Check whether a record should be retained in the log. Note that {@link #checkBatchRetention(RecordBatch)} - * is used prior to checking individual record retention. Only records from batches which were not - * explicitly discarded with {@link BatchRetention#DELETE} will be considered. - */ - protected abstract boolean shouldRetainRecord(RecordBatch recordBatch, Record record); -+ -+ protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record, long newDeleteHorizonMs) { -+ return shouldRetainRecord(recordBatch, record); -+ } -+ -+ /** -+ * Retrieves the delete horizon ms for a specific batch -+ */ -+ protected long retrieveDeleteHorizon(RecordBatch recordBatch) { -+ return -1L; -+ } -+ -+ /** -+ * Checks if the control batch (if it is one) can be removed (making sure that it is empty) -+ */ -+ protected boolean isControlBatchEmpty(RecordBatch recordBatch) { -+ return true; -+ } - } - - public static class FilterResult { -diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java -index 054fb8619..be32a8390 100644 ---- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java -+++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java -@@ -75,6 +75,7 @@ public class MemoryRecordsBuilder implements AutoCloseable { - private int numRecords = 0; - private float actualCompressionRatio = 1; - private long maxTimestamp = RecordBatch.NO_TIMESTAMP; -+ private long deleteHorizonMs; - private long offsetOfMaxTimestamp = -1; - private Long lastOffset = null; - private Long firstTimestamp = null; -@@ -94,7 +95,8 @@ public class MemoryRecordsBuilder implements AutoCloseable { - boolean isTransactional, - boolean isControlBatch, - int partitionLeaderEpoch, -- int writeLimit) { -+ int writeLimit, -+ long deleteHorizonMs) { - if (magic > RecordBatch.MAGIC_VALUE_V0 && timestampType == TimestampType.NO_TIMESTAMP_TYPE) - throw new IllegalArgumentException("TimestampType must be set for magic >= 0"); - if (magic < RecordBatch.MAGIC_VALUE_V2) { -@@ -120,6 +122,7 @@ public class MemoryRecordsBuilder implements AutoCloseable { - this.baseSequence = baseSequence; - this.isTransactional = isTransactional; - this.isControlBatch = isControlBatch; -+ this.deleteHorizonMs = deleteHorizonMs; - this.partitionLeaderEpoch = partitionLeaderEpoch; - this.writeLimit = writeLimit; - this.initialPosition = bufferStream.position(); -@@ -130,6 +133,24 @@ public class MemoryRecordsBuilder implements AutoCloseable { - this.appendStream = new DataOutputStream(compressionType.wrapForOutput(this.bufferStream, magic)); - } - -+ public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, -+ byte magic, -+ CompressionType compressionType, -+ TimestampType timestampType, -+ long baseOffset, -+ long logAppendTime, -+ long producerId, -+ short producerEpoch, -+ int baseSequence, -+ boolean isTransactional, -+ boolean isControlBatch, -+ int partitionLeaderEpoch, -+ int writeLimit) { -+ this(bufferStream, magic, compressionType, timestampType, baseOffset, logAppendTime, producerId, -+ producerEpoch, baseSequence, isTransactional, isControlBatch, partitionLeaderEpoch, writeLimit, -+ RecordBatch.NO_TIMESTAMP); -+ } -+ - /** - * Construct a new builder. - * -@@ -192,6 +213,10 @@ public class MemoryRecordsBuilder implements AutoCloseable { - return isTransactional; - } - -+ public boolean deleteHorizonSet() { -+ return deleteHorizonMs >= 0L; -+ } -+ - /** - * Close this builder and return the resulting buffer. - * @return The built log buffer -@@ -364,7 +389,7 @@ public class MemoryRecordsBuilder implements AutoCloseable { - maxTimestamp = this.maxTimestamp; - - DefaultRecordBatch.writeHeader(buffer, baseOffset, offsetDelta, size, magic, compressionType, timestampType, -- firstTimestamp, maxTimestamp, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, -+ firstTimestamp, maxTimestamp, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, deleteHorizonSet(), - partitionLeaderEpoch, numRecords); - - buffer.position(pos); -@@ -411,8 +436,12 @@ public class MemoryRecordsBuilder implements AutoCloseable { - if (magic < RecordBatch.MAGIC_VALUE_V2 && headers != null && headers.length > 0) - throw new IllegalArgumentException("Magic v" + magic + " does not support record headers"); - -- if (firstTimestamp == null) -- firstTimestamp = timestamp; -+ if (firstTimestamp == null) { -+ if (deleteHorizonSet()) -+ firstTimestamp = deleteHorizonMs; -+ else -+ firstTimestamp = timestamp; -+ } - - if (magic > RecordBatch.MAGIC_VALUE_V1) { - appendDefaultRecord(offset, timestamp, key, value, headers); -diff --git a/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java -index 65a6a95fb..45f1609e3 100644 ---- a/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java -+++ b/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java -@@ -210,6 +210,18 @@ public interface RecordBatch extends Iterable { - */ - boolean isTransactional(); - -+ /** -+ * Whether or not the base timestamp has been set to the delete horizon -+ * @return true if it is, false otherwise -+ */ -+ boolean deleteHorizonSet(); -+ -+ /** -+ * Get the delete horizon, returns -1L if the first timestamp is not the delete horizon -+ * @return timestamp of the delete horizon -+ */ -+ long deleteHorizonMs(); -+ - /** - * Get the partition leader epoch of this record batch. - * @return The leader epoch or -1 if it is unknown -diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java -index b8824d3a8..4677e40e3 100644 ---- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java -+++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java -@@ -247,6 +247,45 @@ public class MemoryRecordsTest { - } - } - -+ /** -+ * This test is used to see if the first timestamp of the batch has been successfully -+ * converted to a delete horizon for the tombstones / transaction markers of the batch. -+ */ -+ @Test -+ public void testFirstTimestampToDeleteHorizonConversion() { -+ if (magic >= RecordBatch.MAGIC_VALUE_V2) { -+ ByteBuffer buffer = ByteBuffer.allocate(2048); -+ MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, -+ 0L, RecordBatch.NO_TIMESTAMP, partitionLeaderEpoch); -+ builder.append(10L, "1".getBytes(), null); -+ -+ ByteBuffer filtered = ByteBuffer.allocate(2048); -+ final long deleteHorizon = Integer.MAX_VALUE / 2; -+ builder.build().filterTo(new TopicPartition("random", 0), new MemoryRecords.RecordFilter() { -+ @Override -+ protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { -+ return true; -+ } -+ -+ @Override -+ protected BatchRetention checkBatchRetention(RecordBatch batch) { -+ return BatchRetention.RETAIN_EMPTY; -+ } -+ -+ @Override -+ protected long retrieveDeleteHorizon(RecordBatch batch) { -+ return deleteHorizon; // arbitrary value > 1 -+ } -+ }, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); -+ filtered.flip(); -+ MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); -+ -+ List batches = TestUtils.toList(filteredRecords.batches()); -+ assertEquals(1, batches.size()); -+ assertEquals(deleteHorizon, batches.get(0).deleteHorizonMs()); -+ } -+ } -+ - @Test - public void testFilterToEmptyBatchRetention() { - if (magic >= RecordBatch.MAGIC_VALUE_V2) { -diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala -index 97fa743d5..2b1b292fb 100644 ---- a/core/src/main/scala/kafka/log/Log.scala -+++ b/core/src/main/scala/kafka/log/Log.scala -@@ -221,7 +221,8 @@ class Log(@volatile var dir: File, - val producerIdExpirationCheckIntervalMs: Int, - val topicPartition: TopicPartition, - val producerStateManager: ProducerStateManager, -- logDirFailureChannel: LogDirFailureChannel) extends Logging with KafkaMetricsGroup { -+ logDirFailureChannel: LogDirFailureChannel, -+ @volatile var containsTombstones: Boolean = false) extends Logging with KafkaMetricsGroup { - - import kafka.log.Log._ - -diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala -index bcb358631..33bea6d99 100644 ---- a/core/src/main/scala/kafka/log/LogCleaner.scala -+++ b/core/src/main/scala/kafka/log/LogCleaner.scala -@@ -334,7 +334,8 @@ class LogCleaner(initialConfig: CleanerConfig, - @throws(classOf[LogCleaningException]) - private def cleanFilthiestLog(): Boolean = { - val preCleanStats = new PreCleanStats() -- val cleaned = cleanerManager.grabFilthiestCompactedLog(time, preCleanStats) match { -+ val ltc =cleanerManager.grabFilthiestCompactedLog(time, preCleanStats) -+ val cleaned = ltc match { - case None => - false - case Some(cleanable) => -@@ -498,11 +499,10 @@ private[log] class Cleaner(val id: Int, - case None => 0L - case Some(seg) => seg.lastModified - cleanable.log.config.deleteRetentionMs - } -- -- doClean(cleanable, deleteHorizonMs) -+ doClean(cleanable, time.milliseconds(), trackedHorizon = deleteHorizonMs) - } - -- private[log] def doClean(cleanable: LogToClean, deleteHorizonMs: Long): (Long, CleanerStats) = { -+ private[log] def doClean(cleanable: LogToClean, currentTime: Long, trackedHorizon: Long = -1L): (Long, CleanerStats) = { - info("Beginning cleaning of log %s.".format(cleanable.log.name)) - - val log = cleanable.log -@@ -520,13 +520,13 @@ private[log] class Cleaner(val id: Int, - val cleanableHorizonMs = log.logSegments(0, cleanable.firstUncleanableOffset).lastOption.map(_.lastModified).getOrElse(0L) - - // group the segments and clean the groups -- info("Cleaning log %s (cleaning prior to %s, discarding tombstones prior to %s)...".format(log.name, new Date(cleanableHorizonMs), new Date(deleteHorizonMs))) -+ info("Cleaning log %s (cleaning prior to %s, discarding tombstones prior to %s)...".format(log.name, new Date(cleanableHorizonMs), new Date(trackedHorizon))) - val transactionMetadata = new CleanedTransactionMetadata - - val groupedSegments = groupSegmentsBySize(log.logSegments(0, endOffset), log.config.segmentSize, - log.config.maxIndexSize, cleanable.firstUncleanableOffset) - for (group <- groupedSegments) -- cleanSegments(log, group, offsetMap, deleteHorizonMs, stats, transactionMetadata) -+ cleanSegments(log, group, offsetMap, currentTime, stats, transactionMetadata, trackedHorizon = trackedHorizon) - - // record buffer utilization - stats.bufferUtilization = offsetMap.utilization -@@ -550,9 +550,10 @@ private[log] class Cleaner(val id: Int, - private[log] def cleanSegments(log: Log, - segments: Seq[LogSegment], - map: OffsetMap, -- deleteHorizonMs: Long, -+ currentTime: Long, - stats: CleanerStats, -- transactionMetadata: CleanedTransactionMetadata): Unit = { -+ transactionMetadata: CleanedTransactionMetadata, -+ trackedHorizon: Long = -1L): Unit = { - // create a new segment with a suffix appended to the name of the log and indexes - val cleaned = LogCleaner.createNewCleanedSegment(log, segments.head.baseOffset) - transactionMetadata.cleanedIndex = Some(cleaned.txnIndex) -@@ -572,14 +573,15 @@ private[log] class Cleaner(val id: Int, - val abortedTransactions = log.collectAbortedTransactions(startOffset, upperBoundOffset) - transactionMetadata.addAbortedTransactions(abortedTransactions) - -- val retainDeletesAndTxnMarkers = currentSegment.lastModified > deleteHorizonMs -+ val retainDeletesAndTxnMarkers = currentSegment.lastModified > trackedHorizon - info(s"Cleaning $currentSegment in log ${log.name} into ${cleaned.baseOffset} " + -- s"with deletion horizon $deleteHorizonMs, " + -+ s"with deletion horizon $trackedHorizon, " + - s"${if(retainDeletesAndTxnMarkers) "retaining" else "discarding"} deletes.") - - try { -- cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletesAndTxnMarkers, log.config.maxMessageSize, -- transactionMetadata, lastOffsetOfActiveProducers, stats) -+ val containsTombstones: Boolean = cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletesAndTxnMarkers, log.config.maxMessageSize, -+ transactionMetadata, lastOffsetOfActiveProducers, stats, log.config.deleteRetentionMs, currentTime = currentTime) -+ log.containsTombstones = containsTombstones - } catch { - case e: LogSegmentOffsetOverflowException => - // Split the current segment. It's also safest to abort the current cleaning process, so that we retry from -@@ -623,6 +625,7 @@ private[log] class Cleaner(val id: Int, - * @param retainDeletesAndTxnMarkers Should tombstones and markers be retained while cleaning this segment - * @param maxLogMessageSize The maximum message size of the corresponding topic - * @param stats Collector for cleaning statistics -+ * @param tombstoneRetentionMs Defines how long a tombstone should be kept as defined by log configuration - */ - private[log] def cleanInto(topicPartition: TopicPartition, - sourceRecords: FileRecords, -@@ -632,14 +635,33 @@ private[log] class Cleaner(val id: Int, - maxLogMessageSize: Int, - transactionMetadata: CleanedTransactionMetadata, - lastRecordsOfActiveProducers: Map[Long, LastRecord], -- stats: CleanerStats): Unit = { -+ stats: CleanerStats, -+ tombstoneRetentionMs: Long, -+ currentTime: Long = RecordBatch.NO_TIMESTAMP): Boolean = { -+ var containsTombstones: Boolean = false -+ - val logCleanerFilter: RecordFilter = new RecordFilter { - var discardBatchRecords: Boolean = _ -+ var isControlBatchEmpty: Boolean = _ - -- override def checkBatchRetention(batch: RecordBatch): BatchRetention = { -+ override def isControlBatchEmpty(batch: RecordBatch) : Boolean = { - // we piggy-back on the tombstone retention logic to delay deletion of transaction markers. - // note that we will never delete a marker until all the records from that transaction are removed. -- discardBatchRecords = shouldDiscardBatch(batch, transactionMetadata, retainTxnMarkers = retainDeletesAndTxnMarkers) -+ val canDiscardBatch = shouldDiscardBatch(batch, transactionMetadata, retainTxnMarkers = retainDeletesAndTxnMarkers) -+ isControlBatchEmpty = canDiscardBatch -+ isControlBatchEmpty -+ } -+ -+ override def checkBatchRetention(batch: RecordBatch, newBatchDeleteHorizonMs : Long): BatchRetention = { -+ val canDiscardBatch = isControlBatchEmpty -+ -+ if (batch.isControlBatch) { -+ discardBatchRecords = canDiscardBatch && -+ ((batch.deleteHorizonSet() && batch.deleteHorizonMs() < currentTime) || -+ newBatchDeleteHorizonMs != RecordBatch.NO_TIMESTAMP && newBatchDeleteHorizonMs < currentTime) -+ } else { -+ discardBatchRecords = canDiscardBatch -+ } - - def isBatchLastRecordOfProducer: Boolean = { - // We retain the batch in order to preserve the state of active producers. There are three cases: -@@ -664,12 +686,33 @@ private[log] class Cleaner(val id: Int, - BatchRetention.DELETE_EMPTY - } - -- override def shouldRetainRecord(batch: RecordBatch, record: Record): Boolean = { -+ override def checkBatchRetention(batch: RecordBatch): BatchRetention = checkBatchRetention(batch, batch.deleteHorizonMs()) -+ -+ override def shouldRetainRecord(batch: RecordBatch, record: Record, newDeleteHorizonMs: Long): Boolean = { -+ var isRecordRetained: Boolean = true - if (discardBatchRecords) - // The batch is only retained to preserve producer sequence information; the records can be removed -- false -+ isRecordRetained = false - else -- Cleaner.this.shouldRetainRecord(map, retainDeletesAndTxnMarkers, batch, record, stats) -+ isRecordRetained = Cleaner.this.shouldRetainRecord(map, retainDeletesAndTxnMarkers, batch, record, stats, newDeleteHorizonMs, currentTime = currentTime) -+ if (isRecordRetained && !record.hasValue()) -+ containsTombstones = true -+ isRecordRetained -+ } -+ -+ override def shouldRetainRecord(batch: RecordBatch, record: Record): Boolean = { -+ shouldRetainRecord(batch, record, RecordBatch.NO_TIMESTAMP) -+ } -+ -+ override def retrieveDeleteHorizon(batch: RecordBatch) : Long = { -+ if (batch.deleteHorizonSet()) -+ return batch.deleteHorizonMs() // means that we keep the old timestamp stored -+ -+ // check that the control batch has been emptied of records -+ // if not, then we do not set a delete horizon until that is true -+ if (batch.isControlBatch() && !isControlBatchEmpty) -+ return -1L -+ return time.milliseconds() + tombstoneRetentionMs; - } - } - -@@ -709,6 +752,7 @@ private[log] class Cleaner(val id: Int, - growBuffersOrFail(sourceRecords, position, maxLogMessageSize, records) - } - restoreBuffers() -+ containsTombstones - } - - -@@ -747,20 +791,19 @@ private[log] class Cleaner(val id: Int, - private def shouldDiscardBatch(batch: RecordBatch, - transactionMetadata: CleanedTransactionMetadata, - retainTxnMarkers: Boolean): Boolean = { -- if (batch.isControlBatch) { -- val canDiscardControlBatch = transactionMetadata.onControlBatchRead(batch) -- canDiscardControlBatch && !retainTxnMarkers -- } else { -- val canDiscardBatch = transactionMetadata.onBatchRead(batch) -- canDiscardBatch -- } -+ if (batch.isControlBatch) -+ transactionMetadata.onControlBatchRead(batch) -+ else -+ transactionMetadata.onBatchRead(batch) - } - - private def shouldRetainRecord(map: kafka.log.OffsetMap, - retainDeletes: Boolean, - batch: RecordBatch, - record: Record, -- stats: CleanerStats): Boolean = { -+ stats: CleanerStats, -+ newBatchDeleteHorizonMs: Long, -+ currentTime: Long = -1L): Boolean = { - val pastLatestOffset = record.offset > map.latestOffset - if (pastLatestOffset) - return true -@@ -774,7 +817,14 @@ private[log] class Cleaner(val id: Int, - * 2) The message doesn't has value but it can't be deleted now. - */ - val latestOffsetForKey = record.offset() >= foundOffset -- val isRetainedValue = record.hasValue || retainDeletes -+ val isLatestVersion = batch.magic() >= RecordBatch.MAGIC_VALUE_V2 -+ var shouldRetainDeletes = true -+ if (isLatestVersion) -+ shouldRetainDeletes = (batch.deleteHorizonSet() && currentTime < batch.deleteHorizonMs()) || -+ (!batch.deleteHorizonSet() && currentTime < newBatchDeleteHorizonMs) -+ else -+ shouldRetainDeletes = retainDeletes -+ val isRetainedValue = record.hasValue || shouldRetainDeletes - latestOffsetForKey && isRetainedValue - } else { - stats.invalidMessage() -diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala -index 4d49e1ffa..2623e958f 100755 ---- a/core/src/main/scala/kafka/log/LogCleanerManager.scala -+++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala -@@ -181,6 +181,7 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], - case (_, log) => log.config.compact // match logs that are marked as compacted - }.filterNot { - case (topicPartition, log) => -+ - // skip any logs already in-progress and uncleanable partitions - inProgress.contains(topicPartition) || isUncleanablePartition(log, topicPartition) - }.map { -@@ -204,7 +205,35 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], - (ltc.needCompactionNow && ltc.cleanableBytes > 0) || ltc.cleanableRatio > ltc.log.config.minCleanableRatio - } - if(cleanableLogs.isEmpty) { -- None -+ // in this case, we are probably in a low throughput situation -+ // therefore, we should take advantage of this fact and remove tombstones if we can -+ val logsContainingTombstones = logs.filter { -+ case (_, log) => log.containsTombstones -+ }.filterNot { -+ case (topicPartition, log) => -+ // skip any logs already in-progress and uncleanable partitions -+ inProgress.contains(topicPartition) || isUncleanablePartition(log, topicPartition) -+ }.map { -+ case (topicPartition, log) => // create a LogToClean instance for each -+ try { -+ val lastCleanOffset = lastClean.get(topicPartition) -+ val (firstDirtyOffset, firstUncleanableDirtyOffset) = cleanableOffsets(log, lastCleanOffset, now) -+ val compactionDelayMs = maxCompactionDelay(log, firstDirtyOffset, now) -+ preCleanStats.updateMaxCompactionDelay(compactionDelayMs) -+ -+ LogToClean(topicPartition, log, firstDirtyOffset, firstUncleanableDirtyOffset, compactionDelayMs > 0) -+ } catch { -+ case e: Throwable => throw new LogCleaningException(log, -+ s"Failed to calculate log cleaning stats for partition $topicPartition", e) -+ } -+ } -+ if (!logsContainingTombstones.isEmpty) { -+ val filthiest = logsContainingTombstones.max -+ inProgress.put(filthiest.topicPartition, LogCleaningInProgress) -+ Some(filthiest) -+ } else { -+ None -+ } - } else { - preCleanStats.recordCleanablePartitions(cleanableLogs.size) - val filthiest = cleanableLogs.max -diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala -index 7aea6c1fc..74244cdc8 100644 ---- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala -+++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala -@@ -18,6 +18,7 @@ - package kafka.log - - import java.io.PrintWriter -+import java.util.Properties - - import com.yammer.metrics.Metrics - import com.yammer.metrics.core.{Gauge, MetricName} -@@ -178,6 +179,51 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K - s"but lastCleaned=$lastCleaned2", lastCleaned2 >= secondBlockCleanableSegmentOffset) - } - -+ @Test -+ def testTombstoneCleanWithLowThoroughput() : Unit = { -+ val tombstoneRetentionMs = 1000 // this is in milliseconds -> 1 second -+ -+ val topicPartitions = Array(new TopicPartition("log-partition", 0)) -+ val props = new Properties() -+ props.put(LogConfig.DeleteRetentionMsProp, "1000") -+ cleaner = makeCleaner(partitions = topicPartitions, propertyOverrides = props) -+ -+ val log = cleaner.logs.get(topicPartitions(0)) -+ -+ val T0 = time.milliseconds -+ writeKeyDups(numKeys = 1, numDups = 1, log, CompressionType.NONE, timestamp = T0, -+ startValue = 0, step = 1, isRecordTombstone = true) -+ -+ val activeSegAtT0 = log.activeSegment -+ -+ // roll the active segment -+ log.roll() -+ -+ cleaner.startup() -+ Thread.sleep(400) -+ -+ import JavaConverters._ -+ var containsTombstones: Boolean = false -+ for (segment <- log.logSegments; record <- segment.log.records.asScala) { -+ containsTombstones = true -+ } -+ assertTrue(containsTombstones) -+ time.sleep(tombstoneRetentionMs + 1) -+ -+ val firstBlockCleanableSegmentOffset = activeSegAtT0.baseOffset -+ -+ Thread.sleep(300) -+ -+ // the first block should get cleaned -+ cleaner.awaitCleaned(new TopicPartition("log-partition", 0), -+ firstBlockCleanableSegmentOffset, maxWaitMs = tombstoneRetentionMs * 5) -+ -+ for (segment <- log.logSegments; record <- segment.log.records.asScala) { -+ fail ("The log should not contain record " + record + ", tombstone has expired its lifetime.") -+ } -+ assertFalse(log.containsTombstones) -+ } -+ - private def readFromLog(log: Log): Iterable[(Int, Int)] = { - import JavaConverters._ - for (segment <- log.logSegments; record <- segment.log.records.asScala) yield { -@@ -187,12 +233,17 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K - } - } - -- private def writeKeyDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionType, timestamp: Long, startValue: Int, step: Int): Seq[(Int, Int)] = { -+ private def writeKeyDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionType, timestamp: Long, -+ startValue: Int, step: Int, isRecordTombstone: Boolean = false): Seq[(Int, Int)] = { - var valCounter = startValue - for (_ <- 0 until numDups; key <- 0 until numKeys) yield { - val curValue = valCounter -- log.appendAsLeader(TestUtils.singletonRecords(value = curValue.toString.getBytes, codec = codec, -- key = key.toString.getBytes, timestamp = timestamp), leaderEpoch = 0) -+ if (isRecordTombstone) -+ log.appendAsLeader(TestUtils.singletonRecords(value = null, codec = codec, -+ key = key.toString.getBytes, timestamp = timestamp), leaderEpoch = 0) -+ else -+ log.appendAsLeader(TestUtils.singletonRecords(value = curValue.toString.getBytes, codec = codec, -+ key = key.toString.getBytes, timestamp = timestamp), leaderEpoch = 0) - valCounter += step - (key, curValue) - } -diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala -index bb30287bb..7d03b07fd 100755 ---- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala -+++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala -@@ -347,7 +347,7 @@ class LogCleanerTest { - log.roll() - - // cannot remove the marker in this pass because there are still valid records -- var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertEquals(List(1, 3, 2), LogTest.keysInLog(log)) - assertEquals(List(0, 2, 3, 4, 5), offsetsInLog(log)) - -@@ -356,17 +356,17 @@ class LogCleanerTest { - log.roll() - - // the first cleaning preserves the commit marker (at offset 3) since there were still records for the transaction -- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) - assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) - - // delete horizon forced to 0 to verify marker is not removed early -- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = 0L)._1 -+ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = 0L)._1 - assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) - assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) - - // clean again with large delete horizon and verify the marker is removed -- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) - assertEquals(List(4, 5, 6, 7, 8), offsetsInLog(log)) - } -@@ -395,11 +395,11 @@ class LogCleanerTest { - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, isFromClient = false) - log.roll() - -- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) -+ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) - assertEquals(List(2), LogTest.keysInLog(log)) - assertEquals(List(1, 3, 4), offsetsInLog(log)) - -- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) -+ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) - assertEquals(List(2), LogTest.keysInLog(log)) - assertEquals(List(3, 4), offsetsInLog(log)) - } -@@ -434,14 +434,14 @@ class LogCleanerTest { - - // first time through the records are removed - // Expected State: [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}] -- var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertEquals(List(2, 3), LogTest.keysInLog(log)) - assertEquals(List(4, 5, 6, 7), offsetsInLog(log)) - assertEquals(List(1, 3, 4, 5, 6, 7), lastOffsetsPerBatchInLog(log)) - - // the empty batch remains if cleaned again because it still holds the last sequence - // Expected State: [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}] -- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertEquals(List(2, 3), LogTest.keysInLog(log)) - assertEquals(List(4, 5, 6, 7), offsetsInLog(log)) - assertEquals(List(1, 3, 4, 5, 6, 7), lastOffsetsPerBatchInLog(log)) -@@ -454,13 +454,13 @@ class LogCleanerTest { - log.roll() - - // Expected State: [{Producer1: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] -- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) - assertEquals(List(4, 5, 6, 7, 8, 9), offsetsInLog(log)) - assertEquals(List(1, 4, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log)) - - // Expected State: [{Producer1: EmptyBatch}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] -- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) - assertEquals(List(5, 6, 7, 8, 9), offsetsInLog(log)) - assertEquals(List(1, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log)) -@@ -484,14 +484,14 @@ class LogCleanerTest { - - // first time through the control batch is retained as an empty batch - // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] -- var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertEquals(List(2, 3), LogTest.keysInLog(log)) - assertEquals(List(1, 2), offsetsInLog(log)) - assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) - - // the empty control batch does not cause an exception when cleaned - // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] -- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertEquals(List(2, 3), LogTest.keysInLog(log)) - assertEquals(List(1, 2), offsetsInLog(log)) - assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) -@@ -515,7 +515,7 @@ class LogCleanerTest { - log.roll() - - // Both the record and the marker should remain after cleaning -- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) -+ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) - assertEquals(List(0, 1), offsetsInLog(log)) - assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) - } -@@ -540,12 +540,12 @@ class LogCleanerTest { - // Both the batch and the marker should remain after cleaning. The batch is retained - // because it is the last entry for this producerId. The marker is retained because - // there are still batches remaining from this transaction. -- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) -+ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) - assertEquals(List(1), offsetsInLog(log)) - assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) - - // The empty batch and the marker is still retained after a second cleaning. -- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) -+ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) - assertEquals(List(1), offsetsInLog(log)) - assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) - } -@@ -570,12 +570,12 @@ class LogCleanerTest { - log.roll() - - // delete horizon set to 0 to verify marker is not removed early -- val dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = 0L)._1 -+ val dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = 0L)._1 - assertEquals(List(3), LogTest.keysInLog(log)) - assertEquals(List(3, 4, 5), offsetsInLog(log)) - - // clean again with large delete horizon and verify the marker is removed -- cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) -+ cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue) - assertEquals(List(3), LogTest.keysInLog(log)) - assertEquals(List(4, 5), offsetsInLog(log)) - } -@@ -609,12 +609,12 @@ class LogCleanerTest { - - // Both transactional batches will be cleaned. The last one will remain in the log - // as an empty batch in order to preserve the producer sequence number and epoch -- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) -+ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) - assertEquals(List(1, 3, 4, 5), offsetsInLog(log)) - assertEquals(List(1, 2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) - - // On the second round of cleaning, the marker from the first transaction should be removed. -- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) -+ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) - assertEquals(List(3, 4, 5), offsetsInLog(log)) - assertEquals(List(2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) - } -@@ -646,14 +646,14 @@ class LogCleanerTest { - assertAbortedTransactionIndexed() - - // first time through the records are removed -- var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertAbortedTransactionIndexed() - assertEquals(List(), LogTest.keysInLog(log)) - assertEquals(List(2), offsetsInLog(log)) // abort marker is retained - assertEquals(List(1, 2), lastOffsetsPerBatchInLog(log)) // empty batch is retained - - // the empty batch remains if cleaned again because it still holds the last sequence -- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertAbortedTransactionIndexed() - assertEquals(List(), LogTest.keysInLog(log)) - assertEquals(List(2), offsetsInLog(log)) // abort marker is still retained -@@ -663,13 +663,13 @@ class LogCleanerTest { - appendProducer(Seq(1)) - log.roll() - -- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertAbortedTransactionIndexed() - assertEquals(List(1), LogTest.keysInLog(log)) - assertEquals(List(2, 3), offsetsInLog(log)) // abort marker is not yet gone because we read the empty batch - assertEquals(List(2, 3), lastOffsetsPerBatchInLog(log)) // but we do not preserve the empty batch - -- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertEquals(List(1), LogTest.keysInLog(log)) - assertEquals(List(3), offsetsInLog(log)) // abort marker is gone - assertEquals(List(3), lastOffsetsPerBatchInLog(log)) From f469515412dc06b51a59a71defedd93626cfa80a Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 11 Feb 2020 19:49:14 -0800 Subject: [PATCH 11/53] Delete .gitignore --- tools/bin/.gitignore | 2 -- 1 file changed, 2 deletions(-) delete mode 100644 tools/bin/.gitignore diff --git a/tools/bin/.gitignore b/tools/bin/.gitignore deleted file mode 100644 index 7eed456bec8db..0000000000000 --- a/tools/bin/.gitignore +++ /dev/null @@ -1,2 +0,0 @@ -/main/ -/test/ From 15d9d8a61b5e48fc7e9d14f3e9d889ab61d5a18b Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 11 Feb 2020 19:49:35 -0800 Subject: [PATCH 12/53] Delete .gitignore --- streams/upgrade-system-tests-24/bin/.gitignore | 1 - 1 file changed, 1 deletion(-) delete mode 100644 streams/upgrade-system-tests-24/bin/.gitignore diff --git a/streams/upgrade-system-tests-24/bin/.gitignore b/streams/upgrade-system-tests-24/bin/.gitignore deleted file mode 100644 index 1933786028b34..0000000000000 --- a/streams/upgrade-system-tests-24/bin/.gitignore +++ /dev/null @@ -1 +0,0 @@ -/test/ From a3bc996811d27b66ee9377536b97f4d13c073a1c Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 11 Feb 2020 19:50:10 -0800 Subject: [PATCH 13/53] Delete LogCleaner.scala.orig --- .../scala/kafka/log/LogCleaner.scala.orig | 1199 ----------------- 1 file changed, 1199 deletions(-) delete mode 100644 core/src/main/scala/kafka/log/LogCleaner.scala.orig diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala.orig b/core/src/main/scala/kafka/log/LogCleaner.scala.orig deleted file mode 100644 index 292139868bdf3..0000000000000 --- a/core/src/main/scala/kafka/log/LogCleaner.scala.orig +++ /dev/null @@ -1,1199 +0,0 @@ -/** - * 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 kafka.log - -import java.io.{File, IOException} -import java.nio._ -import java.util.Date -import java.util.concurrent.TimeUnit - -import com.yammer.metrics.core.Gauge -import kafka.common._ -import kafka.metrics.KafkaMetricsGroup -import kafka.server.{BrokerReconfigurable, KafkaConfig, LogDirFailureChannel} -import kafka.utils._ -import org.apache.kafka.common.{KafkaException, TopicPartition} -import org.apache.kafka.common.config.ConfigException -import org.apache.kafka.common.errors.{CorruptRecordException, KafkaStorageException} -import org.apache.kafka.common.record.MemoryRecords.RecordFilter -import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention -import org.apache.kafka.common.record._ -import org.apache.kafka.common.utils.Time - -import scala.collection.JavaConverters._ -import scala.collection.{Iterable, Seq, Set, mutable} -import scala.util.control.ControlThrowable - -/** - * The cleaner is responsible for removing obsolete records from logs which have the "compact" retention strategy. - * A message with key K and offset O is obsolete if there exists a message with key K and offset O' such that O < O'. - * - * Each log can be thought of being split into two sections of segments: a "clean" section which has previously been cleaned followed by a - * "dirty" section that has not yet been cleaned. The dirty section is further divided into the "cleanable" section followed by an "uncleanable" section. - * The uncleanable section is excluded from cleaning. The active log segment is always uncleanable. If there is a - * compaction lag time set, segments whose largest message timestamp is within the compaction lag time of the cleaning operation are also uncleanable. - * - * The cleaning is carried out by a pool of background threads. Each thread chooses the dirtiest log that has the "compact" retention policy - * and cleans that. The dirtiness of the log is guessed by taking the ratio of bytes in the dirty section of the log to the total bytes in the log. - * - * To clean a log the cleaner first builds a mapping of key=>last_offset for the dirty section of the log. See kafka.log.OffsetMap for details of - * the implementation of the mapping. - * - * Once the key=>last_offset map is built, the log is cleaned by recopying each log segment but omitting any key that appears in the offset map with a - * higher offset than what is found in the segment (i.e. messages with a key that appears in the dirty section of the log). - * - * To avoid segments shrinking to very small sizes with repeated cleanings we implement a rule by which if we will merge successive segments when - * doing a cleaning if their log and index size are less than the maximum log and index size prior to the clean beginning. - * - * Cleaned segments are swapped into the log as they become available. - * - * One nuance that the cleaner must handle is log truncation. If a log is truncated while it is being cleaned the cleaning of that log is aborted. - * - * Messages with null payload are treated as deletes for the purpose of log compaction. This means that they receive special treatment by the cleaner. - * The cleaner will only retain delete records for a period of time to avoid accumulating space indefinitely. This period of time is configurable on a per-topic - * basis and is measured from the time the segment enters the clean portion of the log (at which point any prior message with that key has been removed). - * Delete markers in the clean section of the log that are older than this time will not be retained when log segments are being recopied as part of cleaning. - * - * Note that cleaning is more complicated with the idempotent/transactional producer capabilities. The following - * are the key points: - * - * 1. In order to maintain sequence number continuity for active producers, we always retain the last batch - * from each producerId, even if all the records from the batch have been removed. The batch will be removed - * once the producer either writes a new batch or is expired due to inactivity. - * 2. We do not clean beyond the last stable offset. This ensures that all records observed by the cleaner have - * been decided (i.e. committed or aborted). In particular, this allows us to use the transaction index to - * collect the aborted transactions ahead of time. - * 3. Records from aborted transactions are removed by the cleaner immediately without regard to record keys. - * 4. Transaction markers are retained until all record batches from the same transaction have been removed and - * a sufficient amount of time has passed to reasonably ensure that an active consumer wouldn't consume any - * data from the transaction prior to reaching the offset of the marker. This follows the same logic used for - * tombstone deletion. - * - * @param initialConfig Initial configuration parameters for the cleaner. Actual config may be dynamically updated. - * @param logDirs The directories where offset checkpoints reside - * @param logs The pool of logs - * @param time A way to control the passage of time - */ -class LogCleaner(initialConfig: CleanerConfig, - val logDirs: Seq[File], - val logs: Pool[TopicPartition, Log], - val logDirFailureChannel: LogDirFailureChannel, - time: Time = Time.SYSTEM) extends Logging with KafkaMetricsGroup with BrokerReconfigurable -{ - - /* Log cleaner configuration which may be dynamically updated */ - @volatile private var config = initialConfig - - /* for managing the state of partitions being cleaned. package-private to allow access in tests */ - private[log] val cleanerManager = new LogCleanerManager(logDirs, logs, logDirFailureChannel) - - /* a throttle used to limit the I/O of all the cleaner threads to a user-specified maximum rate */ - private val throttler = new Throttler(desiredRatePerSec = config.maxIoBytesPerSecond, - checkIntervalMs = 300, - throttleDown = true, - "cleaner-io", - "bytes", - time = time) - - private[log] val cleaners = mutable.ArrayBuffer[CleanerThread]() - - /* a metric to track the maximum utilization of any thread's buffer in the last cleaning */ - newGauge("max-buffer-utilization-percent", - new Gauge[Int] { - def value: Int = cleaners.map(_.lastStats).map(100 * _.bufferUtilization).max.toInt - }) - /* a metric to track the recopy rate of each thread's last cleaning */ - newGauge("cleaner-recopy-percent", - new Gauge[Int] { - def value: Int = { - val stats = cleaners.map(_.lastStats) - val recopyRate = stats.map(_.bytesWritten).sum.toDouble / math.max(stats.map(_.bytesRead).sum, 1) - (100 * recopyRate).toInt - } - }) - /* a metric to track the maximum cleaning time for the last cleaning from each thread */ - newGauge("max-clean-time-secs", - new Gauge[Int] { - def value: Int = cleaners.map(_.lastStats).map(_.elapsedSecs).max.toInt - }) - // a metric to track delay between the time when a log is required to be compacted - // as determined by max compaction lag and the time of last cleaner run. - newGauge("max-compaction-delay-secs", - new Gauge[Int] { - def value: Int = Math.max(0, (cleaners.map(_.lastPreCleanStats).map(_.maxCompactionDelayMs).max / 1000).toInt) - }) - - newGauge("DeadThreadCount", - new Gauge[Int] { - def value: Int = deadThreadCount - }) - - private[log] def deadThreadCount: Int = cleaners.count(_.isThreadFailed) - - /** - * Start the background cleaning - */ - def startup(): Unit = { - info("Starting the log cleaner") - (0 until config.numThreads).foreach { i => - val cleaner = new CleanerThread(i) - cleaners += cleaner - cleaner.start() - } - } - - /** - * Stop the background cleaning - */ - def shutdown(): Unit = { - info("Shutting down the log cleaner.") - cleaners.foreach(_.shutdown()) - cleaners.clear() - } - - override def reconfigurableConfigs: Set[String] = { - LogCleaner.ReconfigurableConfigs - } - - override def validateReconfiguration(newConfig: KafkaConfig): Unit = { - val newCleanerConfig = LogCleaner.cleanerConfig(newConfig) - val numThreads = newCleanerConfig.numThreads - val currentThreads = config.numThreads - if (numThreads < 1) - throw new ConfigException(s"Log cleaner threads should be at least 1") - if (numThreads < currentThreads / 2) - throw new ConfigException(s"Log cleaner threads cannot be reduced to less than half the current value $currentThreads") - if (numThreads > currentThreads * 2) - throw new ConfigException(s"Log cleaner threads cannot be increased to more than double the current value $currentThreads") - - } - - /** - * Reconfigure log clean config. This simply stops current log cleaners and creates new ones. - * That ensures that if any of the cleaners had failed, new cleaners are created to match the new config. - */ - override def reconfigure(oldConfig: KafkaConfig, newConfig: KafkaConfig): Unit = { - config = LogCleaner.cleanerConfig(newConfig) - shutdown() - startup() - } - - /** - * Abort the cleaning of a particular partition, if it's in progress. This call blocks until the cleaning of - * the partition is aborted. - */ - def abortCleaning(topicPartition: TopicPartition): Unit = { - cleanerManager.abortCleaning(topicPartition) - } - - /** - * Update checkpoint file, removing topics and partitions that no longer exist - */ - def updateCheckpoints(dataDir: File): Unit = { - cleanerManager.updateCheckpoints(dataDir, update=None) - } - - def alterCheckpointDir(topicPartition: TopicPartition, sourceLogDir: File, destLogDir: File): Unit = { - cleanerManager.alterCheckpointDir(topicPartition, sourceLogDir, destLogDir) - } - - def handleLogDirFailure(dir: String): Unit = { - cleanerManager.handleLogDirFailure(dir) - } - - /** - * Truncate cleaner offset checkpoint for the given partition if its checkpointed offset is larger than the given offset - */ - def maybeTruncateCheckpoint(dataDir: File, topicPartition: TopicPartition, offset: Long): Unit = { - cleanerManager.maybeTruncateCheckpoint(dataDir, topicPartition, offset) - } - - /** - * Abort the cleaning of a particular partition if it's in progress, and pause any future cleaning of this partition. - * This call blocks until the cleaning of the partition is aborted and paused. - */ - def abortAndPauseCleaning(topicPartition: TopicPartition): Unit = { - cleanerManager.abortAndPauseCleaning(topicPartition) - } - - /** - * Resume the cleaning of paused partitions. - */ - def resumeCleaning(topicPartitions: Iterable[TopicPartition]): Unit = { - cleanerManager.resumeCleaning(topicPartitions) - } - - /** - * For testing, a way to know when work has completed. This method waits until the - * cleaner has processed up to the given offset on the specified topic/partition - * - * @param topicPartition The topic and partition to be cleaned - * @param offset The first dirty offset that the cleaner doesn't have to clean - * @param maxWaitMs The maximum time in ms to wait for cleaner - * - * @return A boolean indicating whether the work has completed before timeout - */ - def awaitCleaned(topicPartition: TopicPartition, offset: Long, maxWaitMs: Long = 60000L): Boolean = { - def isCleaned = cleanerManager.allCleanerCheckpoints.get(topicPartition).fold(false)(_ >= offset) - var remainingWaitMs = maxWaitMs - while (!isCleaned && remainingWaitMs > 0) { - val sleepTime = math.min(100, remainingWaitMs) - Thread.sleep(sleepTime) - remainingWaitMs -= sleepTime - } - isCleaned - } - - /** - * To prevent race between retention and compaction, - * retention threads need to make this call to obtain: - * @return A list of log partitions that retention threads can safely work on - */ - def pauseCleaningForNonCompactedPartitions(): Iterable[(TopicPartition, Log)] = { - cleanerManager.pauseCleaningForNonCompactedPartitions() - } - - // Only for testing - private[kafka] def currentConfig: CleanerConfig = config - - // Only for testing - private[log] def cleanerCount: Int = cleaners.size - - /** - * The cleaner threads do the actual log cleaning. Each thread processes does its cleaning repeatedly by - * choosing the dirtiest log, cleaning it, and then swapping in the cleaned segments. - */ - private[log] class CleanerThread(threadId: Int) - extends ShutdownableThread(name = s"kafka-log-cleaner-thread-$threadId", isInterruptible = false) { - - protected override def loggerName = classOf[LogCleaner].getName - - if (config.dedupeBufferSize / config.numThreads > Int.MaxValue) - warn("Cannot use more than 2G of cleaner buffer space per cleaner thread, ignoring excess buffer space...") - - val cleaner = new Cleaner(id = threadId, - offsetMap = new SkimpyOffsetMap(memory = math.min(config.dedupeBufferSize / config.numThreads, Int.MaxValue).toInt, - hashAlgorithm = config.hashAlgorithm), - ioBufferSize = config.ioBufferSize / config.numThreads / 2, - maxIoBufferSize = config.maxMessageSize, - dupBufferLoadFactor = config.dedupeBufferLoadFactor, - throttler = throttler, - time = time, - checkDone = checkDone) - - @volatile var lastStats: CleanerStats = new CleanerStats() - @volatile var lastPreCleanStats: PreCleanStats = new PreCleanStats() - - private def checkDone(topicPartition: TopicPartition): Unit = { - if (!isRunning) - throw new ThreadShutdownException - cleanerManager.checkCleaningAborted(topicPartition) - } - - /** - * The main loop for the cleaner thread - * Clean a log if there is a dirty log available, otherwise sleep for a bit - */ - override def doWork(): Unit = { - val cleaned = tryCleanFilthiestLog() - if (!cleaned) - pause(config.backOffMs, TimeUnit.MILLISECONDS) - } - - /** - * Cleans a log if there is a dirty log available - * @return whether a log was cleaned - */ - private def tryCleanFilthiestLog(): Boolean = { - try { - cleanFilthiestLog() - } catch { - case e: LogCleaningException => - warn(s"Unexpected exception thrown when cleaning log ${e.log}. Marking its partition (${e.log.topicPartition}) as uncleanable", e) - cleanerManager.markPartitionUncleanable(e.log.dir.getParent, e.log.topicPartition) - - false - } - } - - @throws(classOf[LogCleaningException]) - private def cleanFilthiestLog(): Boolean = { - val preCleanStats = new PreCleanStats() - val cleaned = cleanerManager.grabFilthiestCompactedLog(time, preCleanStats) match { - case None => - false - case Some(cleanable) => - // there's a log, clean it - this.lastPreCleanStats = preCleanStats - try { - cleanLog(cleanable) - true - } catch { - case e @ (_: ThreadShutdownException | _: ControlThrowable) => throw e - case e: Exception => throw new LogCleaningException(cleanable.log, e.getMessage, e) - } - } - val deletable: Iterable[(TopicPartition, Log)] = cleanerManager.deletableLogs() - try { - deletable.foreach { case (_, log) => - try { - log.deleteOldSegments() - } catch { - case e @ (_: ThreadShutdownException | _: ControlThrowable) => throw e - case e: Exception => throw new LogCleaningException(log, e.getMessage, e) - } - } - } finally { - cleanerManager.doneDeleting(deletable.map(_._1)) - } - - cleaned - } - - private def cleanLog(cleanable: LogToClean): Unit = { - var endOffset = cleanable.firstDirtyOffset - try { - val (nextDirtyOffset, cleanerStats) = cleaner.clean(cleanable) - recordStats(cleaner.id, cleanable.log.name, cleanable.firstDirtyOffset, endOffset, cleanerStats) - endOffset = nextDirtyOffset - } catch { - case _: LogCleaningAbortedException => // task can be aborted, let it go. - case _: KafkaStorageException => // partition is already offline. let it go. - case e: IOException => - val logDirectory = cleanable.log.dir.getParent - val msg = s"Failed to clean up log for ${cleanable.topicPartition} in dir ${logDirectory} due to IOException" - logDirFailureChannel.maybeAddOfflineLogDir(logDirectory, msg, e) - } finally { - cleanerManager.doneCleaning(cleanable.topicPartition, cleanable.log.dir.getParentFile, endOffset) - } - } - - /** - * Log out statistics on a single run of the cleaner. - */ - def recordStats(id: Int, name: String, from: Long, to: Long, stats: CleanerStats): Unit = { - this.lastStats = stats - def mb(bytes: Double) = bytes / (1024*1024) - val message = - "%n\tLog cleaner thread %d cleaned log %s (dirty section = [%d, %d])%n".format(id, name, from, to) + - "\t%,.1f MB of log processed in %,.1f seconds (%,.1f MB/sec).%n".format(mb(stats.bytesRead), - stats.elapsedSecs, - mb(stats.bytesRead/stats.elapsedSecs)) + - "\tIndexed %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.mapBytesRead), - stats.elapsedIndexSecs, - mb(stats.mapBytesRead)/stats.elapsedIndexSecs, - 100 * stats.elapsedIndexSecs/stats.elapsedSecs) + - "\tBuffer utilization: %.1f%%%n".format(100 * stats.bufferUtilization) + - "\tCleaned %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.bytesRead), - stats.elapsedSecs - stats.elapsedIndexSecs, - mb(stats.bytesRead)/(stats.elapsedSecs - stats.elapsedIndexSecs), 100 * (stats.elapsedSecs - stats.elapsedIndexSecs).toDouble/stats.elapsedSecs) + - "\tStart size: %,.1f MB (%,d messages)%n".format(mb(stats.bytesRead), stats.messagesRead) + - "\tEnd size: %,.1f MB (%,d messages)%n".format(mb(stats.bytesWritten), stats.messagesWritten) + - "\t%.1f%% size reduction (%.1f%% fewer messages)%n".format(100.0 * (1.0 - stats.bytesWritten.toDouble/stats.bytesRead), - 100.0 * (1.0 - stats.messagesWritten.toDouble/stats.messagesRead)) - info(message) - if (lastPreCleanStats.delayedPartitions > 0) { - info("\tCleanable partitions: %d, Delayed partitions: %d, max delay: %d".format(lastPreCleanStats.cleanablePartitions, lastPreCleanStats.delayedPartitions, lastPreCleanStats.maxCompactionDelayMs)) - } - if (stats.invalidMessagesRead > 0) { - warn("\tFound %d invalid messages during compaction.".format(stats.invalidMessagesRead)) - } - } - - } -} - -object LogCleaner { - val ReconfigurableConfigs = Set( - KafkaConfig.LogCleanerThreadsProp, - KafkaConfig.LogCleanerDedupeBufferSizeProp, - KafkaConfig.LogCleanerDedupeBufferLoadFactorProp, - KafkaConfig.LogCleanerIoBufferSizeProp, - KafkaConfig.MessageMaxBytesProp, - KafkaConfig.LogCleanerIoMaxBytesPerSecondProp, - KafkaConfig.LogCleanerBackoffMsProp - ) - - def cleanerConfig(config: KafkaConfig): CleanerConfig = { - CleanerConfig(numThreads = config.logCleanerThreads, - dedupeBufferSize = config.logCleanerDedupeBufferSize, - dedupeBufferLoadFactor = config.logCleanerDedupeBufferLoadFactor, - ioBufferSize = config.logCleanerIoBufferSize, - maxMessageSize = config.messageMaxBytes, - maxIoBytesPerSecond = config.logCleanerIoMaxBytesPerSecond, - backOffMs = config.logCleanerBackoffMs, - enableCleaner = config.logCleanerEnable) - - } - - def createNewCleanedSegment(log: Log, baseOffset: Long): LogSegment = { - LogSegment.deleteIfExists(log.dir, baseOffset, fileSuffix = Log.CleanedFileSuffix) - LogSegment.open(log.dir, baseOffset, log.config, Time.SYSTEM, fileAlreadyExists = false, - fileSuffix = Log.CleanedFileSuffix, initFileSize = log.initFileSize, preallocate = log.config.preallocate) - } - -} - -/** - * This class holds the actual logic for cleaning a log - * @param id An identifier used for logging - * @param offsetMap The map used for deduplication - * @param ioBufferSize The size of the buffers to use. Memory usage will be 2x this number as there is a read and write buffer. - * @param maxIoBufferSize The maximum size of a message that can appear in the log - * @param dupBufferLoadFactor The maximum percent full for the deduplication buffer - * @param throttler The throttler instance to use for limiting I/O rate. - * @param time The time instance - * @param checkDone Check if the cleaning for a partition is finished or aborted. - */ -private[log] class Cleaner(val id: Int, - val offsetMap: OffsetMap, - ioBufferSize: Int, - maxIoBufferSize: Int, - dupBufferLoadFactor: Double, - throttler: Throttler, - time: Time, - checkDone: TopicPartition => Unit) extends Logging { - - protected override def loggerName = classOf[LogCleaner].getName - - this.logIdent = s"Cleaner $id: " - - /* buffer used for read i/o */ - private var readBuffer = ByteBuffer.allocate(ioBufferSize) - - /* buffer used for write i/o */ - private var writeBuffer = ByteBuffer.allocate(ioBufferSize) - - private val decompressionBufferSupplier = BufferSupplier.create(); - - require(offsetMap.slots * dupBufferLoadFactor > 1, "offset map is too small to fit in even a single message, so log cleaning will never make progress. You can increase log.cleaner.dedupe.buffer.size or decrease log.cleaner.threads") - - /** - * Clean the given log - * - * @param cleanable The log to be cleaned - * - * @return The first offset not cleaned and the statistics for this round of cleaning - */ - private[log] def clean(cleanable: LogToClean): (Long, CleanerStats) = { - // figure out the timestamp below which it is safe to remove delete tombstones - // this position is defined to be a configurable time beneath the last modified time of the last clean segment - val deleteHorizonMs = - cleanable.log.logSegments(0, cleanable.firstDirtyOffset).lastOption match { - case None => 0L - case Some(seg) => seg.lastModified - cleanable.log.config.deleteRetentionMs - } - doClean(cleanable, time.milliseconds(), trackedHorizon = deleteHorizonMs) - } - - private[log] def doClean(cleanable: LogToClean, currentTime: Long, trackedHorizon: Long = -1L): (Long, CleanerStats) = { - info("Beginning cleaning of log %s.".format(cleanable.log.name)) - - val log = cleanable.log - val stats = new CleanerStats() - - // build the offset map - info("Building offset map for %s...".format(cleanable.log.name)) - val upperBoundOffset = cleanable.firstUncleanableOffset - buildOffsetMap(log, cleanable.firstDirtyOffset, upperBoundOffset, offsetMap, stats) - val endOffset = offsetMap.latestOffset + 1 - stats.indexDone() - - // determine the timestamp up to which the log will be cleaned - // this is the lower of the last active segment and the compaction lag - val cleanableHorizonMs = log.logSegments(0, cleanable.firstUncleanableOffset).lastOption.map(_.lastModified).getOrElse(0L) - - // group the segments and clean the groups - info("Cleaning log %s (cleaning prior to %s, discarding tombstones prior to %s)...".format(log.name, new Date(cleanableHorizonMs), new Date(trackedHorizon))) - val transactionMetadata = new CleanedTransactionMetadata - - val groupedSegments = groupSegmentsBySize(log.logSegments(0, endOffset), log.config.segmentSize, - log.config.maxIndexSize, cleanable.firstUncleanableOffset) - for (group <- groupedSegments) - cleanSegments(log, group, offsetMap, currentTime, stats, transactionMetadata, trackedHorizon = trackedHorizon) - - // record buffer utilization - stats.bufferUtilization = offsetMap.utilization - - stats.allDone() - - (endOffset, stats) - } - - /** - * Clean a group of segments into a single replacement segment - * - * @param log The log being cleaned - * @param segments The group of segments being cleaned - * @param map The offset map to use for cleaning segments - * @param deleteHorizonMs The time to retain delete tombstones - * @param stats Collector for cleaning statistics - * @param transactionMetadata State of ongoing transactions which is carried between the cleaning - * of the grouped segments - */ - private[log] def cleanSegments(log: Log, - segments: Seq[LogSegment], - map: OffsetMap, - currentTime: Long, - stats: CleanerStats, - transactionMetadata: CleanedTransactionMetadata, - trackedHorizon: Long = -1L): Unit = { - // create a new segment with a suffix appended to the name of the log and indexes - val cleaned = LogCleaner.createNewCleanedSegment(log, segments.head.baseOffset) - transactionMetadata.cleanedIndex = Some(cleaned.txnIndex) - - try { - // clean segments into the new destination segment - val iter = segments.iterator - var currentSegmentOpt: Option[LogSegment] = Some(iter.next()) - val lastOffsetOfActiveProducers = log.lastRecordsOfActiveProducers - - while (currentSegmentOpt.isDefined) { - val currentSegment = currentSegmentOpt.get - val nextSegmentOpt = if (iter.hasNext) Some(iter.next()) else None - - val startOffset = currentSegment.baseOffset - val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(map.latestOffset + 1) - val abortedTransactions = log.collectAbortedTransactions(startOffset, upperBoundOffset) - transactionMetadata.addAbortedTransactions(abortedTransactions) - - val retainDeletesAndTxnMarkers = currentSegment.lastModified > trackedHorizon - info(s"Cleaning $currentSegment in log ${log.name} into ${cleaned.baseOffset} " + - s"with deletion horizon $trackedHorizon, " + - s"${if(retainDeletesAndTxnMarkers) "retaining" else "discarding"} deletes.") - - try { - val containsTombstones: Boolean = cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletesAndTxnMarkers, log.config.maxMessageSize, - transactionMetadata, lastOffsetOfActiveProducers, stats, log.config.deleteRetentionMs, currentTime = currentTime) - log.containsTombstones = containsTombstones - } catch { - case e: LogSegmentOffsetOverflowException => - // Split the current segment. It's also safest to abort the current cleaning process, so that we retry from - // scratch once the split is complete. - info(s"Caught segment overflow error during cleaning: ${e.getMessage}") - log.splitOverflowedSegment(currentSegment) - throw new LogCleaningAbortedException() - } - currentSegmentOpt = nextSegmentOpt - } - - cleaned.onBecomeInactiveSegment() - // flush new segment to disk before swap - cleaned.flush() - - // update the modification date to retain the last modified date of the original files - val modified = segments.last.lastModified - cleaned.lastModified = modified - - // swap in new segment - info(s"Swapping in cleaned segment $cleaned for segment(s) $segments in log $log") - log.replaceSegments(List(cleaned), segments) - } catch { - case e: LogCleaningAbortedException => - try cleaned.deleteIfExists() - catch { - case deleteException: Exception => - e.addSuppressed(deleteException) - } finally throw e - } - } - - /** - * Clean the given source log segment into the destination segment using the key=>offset mapping - * provided - * - * @param topicPartition The topic and partition of the log segment to clean - * @param sourceRecords The dirty log segment - * @param dest The cleaned log segment - * @param map The key=>offset mapping - * @param retainDeletesAndTxnMarkers Should tombstones and markers be retained while cleaning this segment - * @param maxLogMessageSize The maximum message size of the corresponding topic - * @param stats Collector for cleaning statistics - * @param tombstoneRetentionMs Defines how long a tombstone should be kept as defined by log configuration - */ - private[log] def cleanInto(topicPartition: TopicPartition, - sourceRecords: FileRecords, - dest: LogSegment, - map: OffsetMap, - retainDeletesAndTxnMarkers: Boolean, - maxLogMessageSize: Int, - transactionMetadata: CleanedTransactionMetadata, - lastRecordsOfActiveProducers: Map[Long, LastRecord], - stats: CleanerStats, - tombstoneRetentionMs: Long, - currentTime: Long = RecordBatch.NO_TIMESTAMP): Boolean = { - var containsTombstones: Boolean = false - - val logCleanerFilter: RecordFilter = new RecordFilter { - var discardBatchRecords: Boolean = _ - var isControlBatchEmpty: Boolean = _ - - override def isControlBatchEmpty(batch: RecordBatch) : Boolean = { - // we piggy-back on the tombstone retention logic to delay deletion of transaction markers. - // note that we will never delete a marker until all the records from that transaction are removed. - val canDiscardBatch = shouldDiscardBatch(batch, transactionMetadata, retainTxnMarkers = retainDeletesAndTxnMarkers) - isControlBatchEmpty = canDiscardBatch - isControlBatchEmpty - } - - override def checkBatchRetention(batch: RecordBatch, newBatchDeleteHorizonMs : Long): BatchRetention = { - val canDiscardBatch = isControlBatchEmpty - - if (batch.isControlBatch) { - discardBatchRecords = canDiscardBatch && - ((batch.deleteHorizonSet() && batch.deleteHorizonMs() < currentTime) || - newBatchDeleteHorizonMs != RecordBatch.NO_TIMESTAMP && newBatchDeleteHorizonMs < currentTime) - } else { - discardBatchRecords = canDiscardBatch - } - - def isBatchLastRecordOfProducer: Boolean = { - // We retain the batch in order to preserve the state of active producers. There are three cases: - // 1) The producer is no longer active, which means we can delete all records for that producer. - // 2) The producer is still active and has a last data offset. We retain the batch that contains - // this offset since it also contains the last sequence number for this producer. - // 3) The last entry in the log is a transaction marker. We retain this marker since it has the - // last producer epoch, which is needed to ensure fencing. - lastRecordsOfActiveProducers.get(batch.producerId).exists { lastRecord => - lastRecord.lastDataOffset match { - case Some(offset) => batch.lastOffset == offset - case None => batch.isControlBatch && batch.producerEpoch == lastRecord.producerEpoch - } - } - } - - if (batch.hasProducerId && isBatchLastRecordOfProducer) - BatchRetention.RETAIN_EMPTY - else if (discardBatchRecords) - BatchRetention.DELETE - else - BatchRetention.DELETE_EMPTY - } - - override def checkBatchRetention(batch: RecordBatch): BatchRetention = checkBatchRetention(batch, batch.deleteHorizonMs()) - - override def shouldRetainRecord(batch: RecordBatch, record: Record, newDeleteHorizonMs: Long): Boolean = { - var isRecordRetained: Boolean = true - if (discardBatchRecords) - // The batch is only retained to preserve producer sequence information; the records can be removed - isRecordRetained = false - else - isRecordRetained = Cleaner.this.shouldRetainRecord(map, retainDeletesAndTxnMarkers, batch, record, stats, newDeleteHorizonMs, currentTime = currentTime) - if (isRecordRetained && !record.hasValue()) - containsTombstones = true - isRecordRetained - } - - override def shouldRetainRecord(batch: RecordBatch, record: Record): Boolean = { - shouldRetainRecord(batch, record, RecordBatch.NO_TIMESTAMP) - } - - override def retrieveDeleteHorizon(batch: RecordBatch) : Long = { - if (batch.deleteHorizonSet()) - return batch.deleteHorizonMs() // means that we keep the old timestamp stored - - // check that the control batch has been emptied of records - // if not, then we do not set a delete horizon until that is true - if (batch.isControlBatch() && !isControlBatchEmpty) - return -1L - return time.milliseconds() + tombstoneRetentionMs; - } - } - - var position = 0 - while (position < sourceRecords.sizeInBytes) { - checkDone(topicPartition) - // read a chunk of messages and copy any that are to be retained to the write buffer to be written out - readBuffer.clear() - writeBuffer.clear() - - sourceRecords.readInto(readBuffer, position) - val records = MemoryRecords.readableRecords(readBuffer) - throttler.maybeThrottle(records.sizeInBytes) - val result = records.filterTo(topicPartition, logCleanerFilter, writeBuffer, maxLogMessageSize, decompressionBufferSupplier) - stats.readMessages(result.messagesRead, result.bytesRead) - stats.recopyMessages(result.messagesRetained, result.bytesRetained) - - position += result.bytesRead - - // if any messages are to be retained, write them out - val outputBuffer = result.outputBuffer - if (outputBuffer.position() > 0) { - outputBuffer.flip() - val retained = MemoryRecords.readableRecords(outputBuffer) - // it's OK not to hold the Log's lock in this case, because this segment is only accessed by other threads - // after `Log.replaceSegments` (which acquires the lock) is called - dest.append(largestOffset = result.maxOffset, - largestTimestamp = result.maxTimestamp, - shallowOffsetOfMaxTimestamp = result.shallowOffsetOfMaxTimestamp, - records = retained) - throttler.maybeThrottle(outputBuffer.limit()) - } - - // if we read bytes but didn't get even one complete batch, our I/O buffer is too small, grow it and try again - // `result.bytesRead` contains bytes from `messagesRead` and any discarded batches. - if (readBuffer.limit() > 0 && result.bytesRead == 0) - growBuffersOrFail(sourceRecords, position, maxLogMessageSize, records) - } - restoreBuffers() - containsTombstones - } - - - /** - * Grow buffers to process next batch of records from `sourceRecords.` Buffers are doubled in size - * up to a maximum of `maxLogMessageSize`. In some scenarios, a record could be bigger than the - * current maximum size configured for the log. For example: - * 1. A compacted topic using compression may contain a message set slightly larger than max.message.bytes - * 2. max.message.bytes of a topic could have been reduced after writing larger messages - * In these cases, grow the buffer to hold the next batch. - */ - private def growBuffersOrFail(sourceRecords: FileRecords, - position: Int, - maxLogMessageSize: Int, - memoryRecords: MemoryRecords): Unit = { - - val maxSize = if (readBuffer.capacity >= maxLogMessageSize) { - val nextBatchSize = memoryRecords.firstBatchSize - val logDesc = s"log segment ${sourceRecords.file} at position $position" - if (nextBatchSize == null) - throw new IllegalStateException(s"Could not determine next batch size for $logDesc") - if (nextBatchSize <= 0) - throw new IllegalStateException(s"Invalid batch size $nextBatchSize for $logDesc") - if (nextBatchSize <= readBuffer.capacity) - throw new IllegalStateException(s"Batch size $nextBatchSize < buffer size ${readBuffer.capacity}, but not processed for $logDesc") - val bytesLeft = sourceRecords.channel.size - position - if (nextBatchSize > bytesLeft) - throw new CorruptRecordException(s"Log segment may be corrupt, batch size $nextBatchSize > $bytesLeft bytes left in segment for $logDesc") - nextBatchSize.intValue - } else - maxLogMessageSize - - growBuffers(maxSize) - } - - private def shouldDiscardBatch(batch: RecordBatch, - transactionMetadata: CleanedTransactionMetadata, - retainTxnMarkers: Boolean): Boolean = { - if (batch.isControlBatch) - transactionMetadata.onControlBatchRead(batch) - else - transactionMetadata.onBatchRead(batch) - } - - private def shouldRetainRecord(map: kafka.log.OffsetMap, - retainDeletes: Boolean, - batch: RecordBatch, - record: Record, - stats: CleanerStats, - newBatchDeleteHorizonMs: Long, - currentTime: Long = -1L): Boolean = { - val pastLatestOffset = record.offset > map.latestOffset - if (pastLatestOffset) - return true - - if (record.hasKey) { - val key = record.key - val foundOffset = map.get(key) - /* First,the message must have the latest offset for the key - * then there are two cases in which we can retain a message: - * 1) The message has value - * 2) The message doesn't has value but it can't be deleted now. - */ - val latestOffsetForKey = record.offset() >= foundOffset - val isLatestVersion = batch.magic() >= RecordBatch.MAGIC_VALUE_V2 - var shouldRetainDeletes = true - if (isLatestVersion) - shouldRetainDeletes = (batch.deleteHorizonSet() && currentTime < batch.deleteHorizonMs()) || - (!batch.deleteHorizonSet() && currentTime < newBatchDeleteHorizonMs) - else - shouldRetainDeletes = retainDeletes - val isRetainedValue = record.hasValue || shouldRetainDeletes - - latestOffsetForKey && isRetainedValue - } else { - stats.invalidMessage() - false - } - } - - /** - * Double the I/O buffer capacity - */ - def growBuffers(maxLogMessageSize: Int): Unit = { - val maxBufferSize = math.max(maxLogMessageSize, maxIoBufferSize) - if(readBuffer.capacity >= maxBufferSize || writeBuffer.capacity >= maxBufferSize) - throw new IllegalStateException("This log contains a message larger than maximum allowable size of %s.".format(maxBufferSize)) - val newSize = math.min(this.readBuffer.capacity * 2, maxBufferSize) - info(s"Growing cleaner I/O buffers from ${readBuffer.capacity} bytes to $newSize bytes.") - this.readBuffer = ByteBuffer.allocate(newSize) - this.writeBuffer = ByteBuffer.allocate(newSize) - } - - /** - * Restore the I/O buffer capacity to its original size - */ - def restoreBuffers(): Unit = { - if(this.readBuffer.capacity > this.ioBufferSize) - this.readBuffer = ByteBuffer.allocate(this.ioBufferSize) - if(this.writeBuffer.capacity > this.ioBufferSize) - this.writeBuffer = ByteBuffer.allocate(this.ioBufferSize) - } - - /** - * Group the segments in a log into groups totaling less than a given size. the size is enforced separately for the log data and the index data. - * We collect a group of such segments together into a single - * destination segment. This prevents segment sizes from shrinking too much. - * - * @param segments The log segments to group - * @param maxSize the maximum size in bytes for the total of all log data in a group - * @param maxIndexSize the maximum size in bytes for the total of all index data in a group - * - * @return A list of grouped segments - */ - private[log] def groupSegmentsBySize(segments: Iterable[LogSegment], maxSize: Int, maxIndexSize: Int, firstUncleanableOffset: Long): List[Seq[LogSegment]] = { - var grouped = List[List[LogSegment]]() - var segs = segments.toList - while(segs.nonEmpty) { - var group = List(segs.head) - var logSize = segs.head.size.toLong - var indexSize = segs.head.offsetIndex.sizeInBytes.toLong - var timeIndexSize = segs.head.timeIndex.sizeInBytes.toLong - segs = segs.tail - while(segs.nonEmpty && - logSize + segs.head.size <= maxSize && - indexSize + segs.head.offsetIndex.sizeInBytes <= maxIndexSize && - timeIndexSize + segs.head.timeIndex.sizeInBytes <= maxIndexSize && - lastOffsetForFirstSegment(segs, firstUncleanableOffset) - group.last.baseOffset <= Int.MaxValue) { - group = segs.head :: group - logSize += segs.head.size - indexSize += segs.head.offsetIndex.sizeInBytes - timeIndexSize += segs.head.timeIndex.sizeInBytes - segs = segs.tail - } - grouped ::= group.reverse - } - grouped.reverse - } - - /** - * We want to get the last offset in the first log segment in segs. - * LogSegment.nextOffset() gives the exact last offset in a segment, but can be expensive since it requires - * scanning the segment from the last index entry. - * Therefore, we estimate the last offset of the first log segment by using - * the base offset of the next segment in the list. - * If the next segment doesn't exist, first Uncleanable Offset will be used. - * - * @param segs - remaining segments to group. - * @return The estimated last offset for the first segment in segs - */ - private def lastOffsetForFirstSegment(segs: List[LogSegment], firstUncleanableOffset: Long): Long = { - if (segs.size > 1) { - /* if there is a next segment, use its base offset as the bounding offset to guarantee we know - * the worst case offset */ - segs(1).baseOffset - 1 - } else { - //for the last segment in the list, use the first uncleanable offset. - firstUncleanableOffset - 1 - } - } - - /** - * Build a map of key_hash => offset for the keys in the cleanable dirty portion of the log to use in cleaning. - * @param log The log to use - * @param start The offset at which dirty messages begin - * @param end The ending offset for the map that is being built - * @param map The map in which to store the mappings - * @param stats Collector for cleaning statistics - */ - private[log] def buildOffsetMap(log: Log, - start: Long, - end: Long, - map: OffsetMap, - stats: CleanerStats): Unit = { - map.clear() - val dirty = log.logSegments(start, end).toBuffer - info("Building offset map for log %s for %d segments in offset range [%d, %d).".format(log.name, dirty.size, start, end)) - - val transactionMetadata = new CleanedTransactionMetadata - val abortedTransactions = log.collectAbortedTransactions(start, end) - transactionMetadata.addAbortedTransactions(abortedTransactions) - - // Add all the cleanable dirty segments. We must take at least map.slots * load_factor, - // but we may be able to fit more (if there is lots of duplication in the dirty section of the log) - var full = false - for (segment <- dirty if !full) { - checkDone(log.topicPartition) - - full = buildOffsetMapForSegment(log.topicPartition, segment, map, start, log.config.maxMessageSize, - transactionMetadata, stats) - if (full) - debug("Offset map is full, %d segments fully mapped, segment with base offset %d is partially mapped".format(dirty.indexOf(segment), segment.baseOffset)) - } - info("Offset map for log %s complete.".format(log.name)) - } - - /** - * Add the messages in the given segment to the offset map - * - * @param segment The segment to index - * @param map The map in which to store the key=>offset mapping - * @param stats Collector for cleaning statistics - * - * @return If the map was filled whilst loading from this segment - */ - private def buildOffsetMapForSegment(topicPartition: TopicPartition, - segment: LogSegment, - map: OffsetMap, - startOffset: Long, - maxLogMessageSize: Int, - transactionMetadata: CleanedTransactionMetadata, - stats: CleanerStats): Boolean = { - var position = segment.offsetIndex.lookup(startOffset).position - val maxDesiredMapSize = (map.slots * this.dupBufferLoadFactor).toInt - while (position < segment.log.sizeInBytes) { - checkDone(topicPartition) - readBuffer.clear() - try { - segment.log.readInto(readBuffer, position) - } catch { - case e: Exception => - throw new KafkaException(s"Failed to read from segment $segment of partition $topicPartition " + - "while loading offset map", e) - } - val records = MemoryRecords.readableRecords(readBuffer) - throttler.maybeThrottle(records.sizeInBytes) - - val startPosition = position - for (batch <- records.batches.asScala) { - if (batch.isControlBatch) { - transactionMetadata.onControlBatchRead(batch) - stats.indexMessagesRead(1) - } else { - val isAborted = transactionMetadata.onBatchRead(batch) - if (isAborted) { - // If the batch is aborted, do not bother populating the offset map. - // Note that abort markers are supported in v2 and above, which means count is defined. - stats.indexMessagesRead(batch.countOrNull) - } else { - for (record <- batch.asScala) { - if (record.hasKey && record.offset >= startOffset) { - if (map.size < maxDesiredMapSize) - map.put(record.key, record.offset) - else - return true - } - stats.indexMessagesRead(1) - } - } - } - - if (batch.lastOffset >= startOffset) - map.updateLatestOffset(batch.lastOffset) - } - val bytesRead = records.validBytes - position += bytesRead - stats.indexBytesRead(bytesRead) - - // if we didn't read even one complete message, our read buffer may be too small - if(position == startPosition) - growBuffersOrFail(segment.log, position, maxLogMessageSize, records) - } - restoreBuffers() - false - } -} - -/** - * A simple struct for collecting pre-clean stats - */ -private class PreCleanStats() { - var maxCompactionDelayMs = 0L - var delayedPartitions = 0 - var cleanablePartitions = 0 - - def updateMaxCompactionDelay(delayMs: Long): Unit = { - maxCompactionDelayMs = Math.max(maxCompactionDelayMs, delayMs) - if (delayMs > 0) { - delayedPartitions += 1 - } - } - def recordCleanablePartitions(numOfCleanables: Int): Unit = { - cleanablePartitions = numOfCleanables - } -} - -/** - * A simple struct for collecting stats about log cleaning - */ -private class CleanerStats(time: Time = Time.SYSTEM) { - val startTime = time.milliseconds - var mapCompleteTime = -1L - var endTime = -1L - var bytesRead = 0L - var bytesWritten = 0L - var mapBytesRead = 0L - var mapMessagesRead = 0L - var messagesRead = 0L - var invalidMessagesRead = 0L - var messagesWritten = 0L - var bufferUtilization = 0.0d - - def readMessages(messagesRead: Int, bytesRead: Int): Unit = { - this.messagesRead += messagesRead - this.bytesRead += bytesRead - } - - def invalidMessage(): Unit = { - invalidMessagesRead += 1 - } - - def recopyMessages(messagesWritten: Int, bytesWritten: Int): Unit = { - this.messagesWritten += messagesWritten - this.bytesWritten += bytesWritten - } - - def indexMessagesRead(size: Int): Unit = { - mapMessagesRead += size - } - - def indexBytesRead(size: Int): Unit = { - mapBytesRead += size - } - - def indexDone(): Unit = { - mapCompleteTime = time.milliseconds - } - - def allDone(): Unit = { - endTime = time.milliseconds - } - - def elapsedSecs = (endTime - startTime)/1000.0 - - def elapsedIndexSecs = (mapCompleteTime - startTime)/1000.0 - -} - -/** - * Helper class for a log, its topic/partition, the first cleanable position, the first uncleanable dirty position, - * and whether it needs compaction immediately. - */ -private case class LogToClean(topicPartition: TopicPartition, - log: Log, - firstDirtyOffset: Long, - uncleanableOffset: Long, - needCompactionNow: Boolean = false) extends Ordered[LogToClean] { - val cleanBytes = log.logSegments(-1, firstDirtyOffset).map(_.size.toLong).sum - val (firstUncleanableOffset, cleanableBytes) = LogCleanerManager.calculateCleanableBytes(log, firstDirtyOffset, uncleanableOffset) - val totalBytes = cleanBytes + cleanableBytes - val cleanableRatio = cleanableBytes / totalBytes.toDouble - override def compare(that: LogToClean): Int = math.signum(this.cleanableRatio - that.cleanableRatio).toInt -} - -/** - * This is a helper class to facilitate tracking transaction state while cleaning the log. It maintains a set - * of the ongoing aborted and committed transactions as the cleaner is working its way through the log. This - * class is responsible for deciding when transaction markers can be removed and is therefore also responsible - * for updating the cleaned transaction index accordingly. - */ -private[log] class CleanedTransactionMetadata { - private val ongoingCommittedTxns = mutable.Set.empty[Long] - private val ongoingAbortedTxns = mutable.Map.empty[Long, AbortedTransactionMetadata] - // Minheap of aborted transactions sorted by the transaction first offset - private val abortedTransactions = mutable.PriorityQueue.empty[AbortedTxn](new Ordering[AbortedTxn] { - override def compare(x: AbortedTxn, y: AbortedTxn): Int = x.firstOffset compare y.firstOffset - }.reverse) - - // Output cleaned index to write retained aborted transactions - var cleanedIndex: Option[TransactionIndex] = None - - def addAbortedTransactions(abortedTransactions: List[AbortedTxn]): Unit = { - this.abortedTransactions ++= abortedTransactions - } - - /** - * Update the cleaned transaction state with a control batch that has just been traversed by the cleaner. - * Return true if the control batch can be discarded. - */ - def onControlBatchRead(controlBatch: RecordBatch): Boolean = { - consumeAbortedTxnsUpTo(controlBatch.lastOffset) - - val controlRecordIterator = controlBatch.iterator - if (controlRecordIterator.hasNext) { - val controlRecord = controlRecordIterator.next() - val controlType = ControlRecordType.parse(controlRecord.key) - val producerId = controlBatch.producerId - controlType match { - case ControlRecordType.ABORT => - ongoingAbortedTxns.remove(producerId) match { - // Retain the marker until all batches from the transaction have been removed. - // We may retain a record from an aborted transaction if it is the last entry - // written by a given producerId. - case Some(abortedTxnMetadata) if abortedTxnMetadata.lastObservedBatchOffset.isDefined => - cleanedIndex.foreach(_.append(abortedTxnMetadata.abortedTxn)) - false - case _ => true - } - - case ControlRecordType.COMMIT => - // This marker is eligible for deletion if we didn't traverse any batches from the transaction - !ongoingCommittedTxns.remove(producerId) - - case _ => false - } - } else { - // An empty control batch was already cleaned, so it's safe to discard - true - } - } - - private def consumeAbortedTxnsUpTo(offset: Long): Unit = { - while (abortedTransactions.headOption.exists(_.firstOffset <= offset)) { - val abortedTxn = abortedTransactions.dequeue() - ongoingAbortedTxns.getOrElseUpdate(abortedTxn.producerId, new AbortedTransactionMetadata(abortedTxn)) - } - } - - /** - * Update the transactional state for the incoming non-control batch. If the batch is part of - * an aborted transaction, return true to indicate that it is safe to discard. - */ - def onBatchRead(batch: RecordBatch): Boolean = { - consumeAbortedTxnsUpTo(batch.lastOffset) - if (batch.isTransactional) { - ongoingAbortedTxns.get(batch.producerId) match { - case Some(abortedTransactionMetadata) => - abortedTransactionMetadata.lastObservedBatchOffset = Some(batch.lastOffset) - true - case None => - ongoingCommittedTxns += batch.producerId - false - } - } else { - false - } - } - -} - -private class AbortedTransactionMetadata(val abortedTxn: AbortedTxn) { - var lastObservedBatchOffset: Option[Long] = None - - override def toString: String = s"(txn: $abortedTxn, lastOffset: $lastObservedBatchOffset)" -} From e00f37cd68292bfad1e142df9491bce6986486c6 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 11 Feb 2020 19:50:39 -0800 Subject: [PATCH 14/53] Delete diff.out --- diff.out | 1155 ------------------------------------------------------ 1 file changed, 1155 deletions(-) delete mode 100644 diff.out diff --git a/diff.out b/diff.out deleted file mode 100644 index 9b932c690de3b..0000000000000 --- a/diff.out +++ /dev/null @@ -1,1155 +0,0 @@ -diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml -index 13cfdb82bd0..8b28fcadb17 100644 ---- a/checkstyle/checkstyle.xml -+++ b/checkstyle/checkstyle.xml -@@ -107,7 +107,7 @@ - - - -- -+ - - - -@@ -124,7 +124,7 @@ - - - -- -+ - - - -diff --git a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java -index 83637640af4..cf74f24f490 100644 ---- a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java -+++ b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java -@@ -431,6 +431,16 @@ public LegacyRecord outerRecord() { - return record; - } - -+ @Override -+ public long deleteHorizonMs() { -+ return RecordBatch.NO_TIMESTAMP; -+ } -+ -+ @Override -+ public boolean deleteHorizonSet() { -+ return false; -+ } -+ - @Override - public boolean equals(Object o) { - if (this == o) -@@ -468,6 +478,16 @@ public long offset() { - return buffer.getLong(OFFSET_OFFSET); - } - -+ @Override -+ public long deleteHorizonMs() { -+ return RecordBatch.NO_TIMESTAMP; -+ } -+ -+ @Override -+ public boolean deleteHorizonSet() { -+ return false; -+ } -+ - @Override - public LegacyRecord outerRecord() { - return record; -@@ -557,6 +577,16 @@ public long baseOffset() { - return loadFullBatch().baseOffset(); - } - -+ @Override -+ public long deleteHorizonMs() { -+ return RecordBatch.NO_TIMESTAMP; -+ } -+ -+ @Override -+ public boolean deleteHorizonSet() { -+ return false; -+ } -+ - @Override - public long lastOffset() { - return offset; -diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java -index 6d79b268575..e0d7549fede 100644 ---- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java -+++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java -@@ -89,11 +89,15 @@ - * by the broker and is preserved after compaction. Additionally, the MaxTimestamp of an empty batch always retains - * the previous value prior to becoming empty. - * -+ * The delete horizon flag for the sixth bit is used to determine if the first timestamp of the batch had been set to -+ * the time for which tombstones / transaction markers needs to be removed. If it is true, then the first timestamp is -+ * the delete horizon, otherwise, it is merely the first timestamp of the record batch. -+ * - * The current attributes are given below: - * -- * ------------------------------------------------------------------------------------------------- -- * | Unused (6-15) | Control (5) | Transactional (4) | Timestamp Type (3) | Compression Type (0-2) | -- * ------------------------------------------------------------------------------------------------- -+ * --------------------------------------------------------------------------------------------------------------------------- -+ * | Unused (7-15) | Delete Horizon Flag (6) | Control (5) | Transactional (4) | Timestamp Type (3) | Compression Type (0-2) | -+ * --------------------------------------------------------------------------------------------------------------------------- - */ - public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRecordBatch { - static final int BASE_OFFSET_OFFSET = 0; -@@ -128,6 +132,7 @@ - private static final byte COMPRESSION_CODEC_MASK = 0x07; - private static final byte TRANSACTIONAL_FLAG_MASK = 0x10; - private static final int CONTROL_FLAG_MASK = 0x20; -+ private static final byte DELETE_HORIZON_FLAG_MASK = 0x40; - private static final byte TIMESTAMP_TYPE_MASK = 0x08; - - private static final int MAX_SKIP_BUFFER_SIZE = 2048; -@@ -155,10 +160,15 @@ public void ensureValid() { - } - - /** -- * Get the timestamp of the first record in this batch. It is always the create time of the record even if the -+ * Get the timestamp of the first record in this batch. It is usually the create time of the record even if the - * timestamp type of the batch is log append time. - * -- * @return The first timestamp or {@link RecordBatch#NO_TIMESTAMP} if the batch is empty -+ * There is the possibility that the first timestamp had been set to the delete horizon of the batch, -+ * in which case, the delete horizon will be returned instead. -+ * -+ * @return The first timestamp if the batch's delete horizon has not been set -+ * The delete horizon if the batch's delete horizon has been set -+ * {@link RecordBatch#NO_TIMESTAMP} if the batch is empty - */ - public long firstTimestamp() { - return buffer.getLong(FIRST_TIMESTAMP_OFFSET); -@@ -245,6 +255,18 @@ public boolean isTransactional() { - return (attributes() & TRANSACTIONAL_FLAG_MASK) > 0; - } - -+ @Override -+ public boolean deleteHorizonSet() { -+ return (attributes() & DELETE_HORIZON_FLAG_MASK) > 0; -+ } -+ -+ @Override -+ public long deleteHorizonMs() { -+ if (deleteHorizonSet()) -+ return firstTimestamp(); -+ return RecordBatch.NO_TIMESTAMP; -+ } -+ - @Override - public boolean isControlBatch() { - return (attributes() & CONTROL_FLAG_MASK) > 0; -@@ -360,7 +382,7 @@ public void setMaxTimestamp(TimestampType timestampType, long maxTimestamp) { - if (timestampType() == timestampType && currentMaxTimestamp == maxTimestamp) - return; - -- byte attributes = computeAttributes(compressionType(), timestampType, isTransactional(), isControlBatch()); -+ byte attributes = computeAttributes(compressionType(), timestampType, isTransactional(), isControlBatch(), deleteHorizonSet()); - buffer.putShort(ATTRIBUTES_OFFSET, attributes); - buffer.putLong(MAX_TIMESTAMP_OFFSET, maxTimestamp); - long crc = computeChecksum(); -@@ -407,7 +429,7 @@ public int hashCode() { - } - - private static byte computeAttributes(CompressionType type, TimestampType timestampType, -- boolean isTransactional, boolean isControl) { -+ boolean isTransactional, boolean isControl, boolean isDeleteHorizonSet) { - if (timestampType == TimestampType.NO_TIMESTAMP_TYPE) - throw new IllegalArgumentException("Timestamp type must be provided to compute attributes for message " + - "format v2 and above"); -@@ -419,6 +441,8 @@ private static byte computeAttributes(CompressionType type, TimestampType timest - attributes |= COMPRESSION_CODEC_MASK & type.id; - if (timestampType == TimestampType.LOG_APPEND_TIME) - attributes |= TIMESTAMP_TYPE_MASK; -+ if (isDeleteHorizonSet) -+ attributes |= DELETE_HORIZON_FLAG_MASK; - return attributes; - } - -@@ -435,9 +459,49 @@ public static void writeEmptyHeader(ByteBuffer buffer, - boolean isTransactional, - boolean isControlRecord) { - int offsetDelta = (int) (lastOffset - baseOffset); -+ writeHeader(buffer, baseOffset, offsetDelta, DefaultRecordBatch.RECORD_BATCH_OVERHEAD, magic, -+ CompressionType.NONE, timestampType, RecordBatch.NO_TIMESTAMP, timestamp, producerId, -+ producerEpoch, baseSequence, isTransactional, isControlRecord, false, partitionLeaderEpoch, 0); -+ } -+ -+ public static void writeEmptyHeader(ByteBuffer buffer, -+ byte magic, -+ long producerId, -+ short producerEpoch, -+ int baseSequence, -+ long baseOffset, -+ long lastOffset, -+ int partitionLeaderEpoch, -+ TimestampType timestampType, -+ long timestamp, -+ boolean isTransactional, -+ boolean isControlRecord, -+ boolean isDeleteHorizonSet) { -+ int offsetDelta = (int) (lastOffset - baseOffset); - writeHeader(buffer, baseOffset, offsetDelta, DefaultRecordBatch.RECORD_BATCH_OVERHEAD, magic, - CompressionType.NONE, timestampType, RecordBatch.NO_TIMESTAMP, timestamp, producerId, -- producerEpoch, baseSequence, isTransactional, isControlRecord, partitionLeaderEpoch, 0); -+ producerEpoch, baseSequence, isTransactional, isControlRecord, isDeleteHorizonSet, partitionLeaderEpoch, 0); -+ } -+ -+ static void writeHeader(ByteBuffer buffer, -+ long baseOffset, -+ int lastOffsetDelta, -+ int sizeInBytes, -+ byte magic, -+ CompressionType compressionType, -+ TimestampType timestampType, -+ long firstTimestamp, -+ long maxTimestamp, -+ long producerId, -+ short epoch, -+ int sequence, -+ boolean isTransactional, -+ boolean isControlBatch, -+ int partitionLeaderEpoch, -+ int numRecords) { -+ writeHeader(buffer, baseOffset, lastOffsetDelta, sizeInBytes, magic, compressionType, -+ timestampType, firstTimestamp, maxTimestamp, producerId, epoch, sequence, -+ isTransactional, isControlBatch, false, partitionLeaderEpoch, numRecords); - } - - static void writeHeader(ByteBuffer buffer, -@@ -454,6 +518,7 @@ static void writeHeader(ByteBuffer buffer, - int sequence, - boolean isTransactional, - boolean isControlBatch, -+ boolean isDeleteHorizonSet, - int partitionLeaderEpoch, - int numRecords) { - if (magic < RecordBatch.CURRENT_MAGIC_VALUE) -@@ -461,7 +526,7 @@ static void writeHeader(ByteBuffer buffer, - if (firstTimestamp < 0 && firstTimestamp != NO_TIMESTAMP) - throw new IllegalArgumentException("Invalid message timestamp " + firstTimestamp); - -- short attributes = computeAttributes(compressionType, timestampType, isTransactional, isControlBatch); -+ short attributes = computeAttributes(compressionType, timestampType, isTransactional, isControlBatch, isDeleteHorizonSet); - - int position = buffer.position(); - buffer.putLong(position + BASE_OFFSET_OFFSET, baseOffset); -@@ -699,6 +764,18 @@ public boolean isTransactional() { - return loadBatchHeader().isTransactional(); - } - -+ @Override -+ public boolean deleteHorizonSet() { -+ return loadBatchHeader().deleteHorizonSet(); -+ } -+ -+ @Override -+ public long deleteHorizonMs() { -+ if (deleteHorizonSet()) -+ return super.loadBatchHeader().deleteHorizonMs(); -+ return RecordBatch.NO_TIMESTAMP; -+ } -+ - @Override - public boolean isControlBatch() { - return loadBatchHeader().isControlBatch(); -diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java -index 8f73565d1b4..7fb5fe951b9 100644 ---- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java -+++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java -@@ -150,15 +150,29 @@ public FilterResult filterTo(TopicPartition partition, RecordFilter filter, Byte - return filterTo(partition, batches(), filter, destinationBuffer, maxRecordBatchSize, decompressionBufferSupplier); - } - -+ /** -+ * Note: This method is also used to convert the first timestamp of the batch (which is usually the timestamp of the first record) -+ * to the delete horizon of the tombstones which are present in the batch. -+ */ - private static FilterResult filterTo(TopicPartition partition, Iterable batches, - RecordFilter filter, ByteBuffer destinationBuffer, int maxRecordBatchSize, - BufferSupplier decompressionBufferSupplier) { - FilterResult filterResult = new FilterResult(destinationBuffer); - ByteBufferOutputStream bufferOutputStream = new ByteBufferOutputStream(destinationBuffer); -- - for (MutableRecordBatch batch : batches) { - long maxOffset = -1L; -- BatchRetention batchRetention = filter.checkBatchRetention(batch); -+ // we first call this method here so that the flag in LogCleaner has been set -+ // which indicates if the control batch is empty or not -+ // we do this to avoid calling CleanedTransactionMetadata#onControlBatchRead -+ // more than once since each call is relatively expensive -+ filter.isControlBatchEmpty(batch); -+ long deleteHorizonMs = filter.retrieveDeleteHorizon(batch); -+ final BatchRetention batchRetention; -+ if (!batch.deleteHorizonSet()) -+ batchRetention = filter.checkBatchRetention(batch, deleteHorizonMs); -+ else -+ batchRetention = filter.checkBatchRetention(batch); -+ - filterResult.bytesRead += batch.sizeInBytes(); - - if (batchRetention == BatchRetention.DELETE) -@@ -170,36 +184,27 @@ private static FilterResult filterTo(TopicPartition partition, Iterable retainedRecords = new ArrayList<>(); - -- try (final CloseableIterator iterator = batch.streamingIterator(decompressionBufferSupplier)) { -- while (iterator.hasNext()) { -- Record record = iterator.next(); -- filterResult.messagesRead += 1; -- -- if (filter.shouldRetainRecord(batch, record)) { -- // Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite -- // the corrupted batch with correct data. -- if (!record.hasMagic(batchMagic)) -- writeOriginalBatch = false; -- -- if (record.offset() > maxOffset) -- maxOffset = record.offset(); -- -- retainedRecords.add(record); -- } else { -- writeOriginalBatch = false; -- } -- } -- } -+ final BatchIterationResult iterationResult = iterateOverBatch(batch, decompressionBufferSupplier, filterResult, filter, -+ batchMagic, writeOriginalBatch, maxOffset, retainedRecords, -+ containsTombstonesOrMarker, deleteHorizonMs); -+ containsTombstonesOrMarker = iterationResult.containsTombstonesOrMarker(); -+ writeOriginalBatch = iterationResult.shouldWriteOriginalBatch(); -+ maxOffset = iterationResult.maxOffset(); - - if (!retainedRecords.isEmpty()) { -- if (writeOriginalBatch) { -+ // we check if the delete horizon should be set to a new value -+ // in which case, we need to reset the base timestamp and overwrite the timestamp deltas -+ // if the batch does not contain tombstones, then we don't need to overwrite batch -+ if (writeOriginalBatch && (deleteHorizonMs == RecordBatch.NO_TIMESTAMP || deleteHorizonMs == batch.deleteHorizonMs() || !containsTombstonesOrMarker)) { - batch.writeTo(bufferOutputStream); - filterResult.updateRetainedBatchMetadata(batch, retainedRecords.size(), false); - } else { -- MemoryRecordsBuilder builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream); -+ MemoryRecordsBuilder builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, deleteHorizonMs); - MemoryRecords records = builder.build(); - int filteredBatchSize = records.sizeInBytes(); - if (filteredBatchSize > batch.sizeInBytes() && filteredBatchSize > maxRecordBatchSize) -@@ -236,9 +241,69 @@ private static FilterResult filterTo(TopicPartition partition, Iterable retainedRecords, -+ boolean containsTombstonesOrMarker, -+ long newBatchDeleteHorizonMs) { -+ try (final CloseableIterator iterator = batch.streamingIterator(decompressionBufferSupplier)) { -+ while (iterator.hasNext()) { -+ Record record = iterator.next(); -+ filterResult.messagesRead += 1; -+ -+ if (filter.shouldRetainRecord(batch, record, newBatchDeleteHorizonMs)) { -+ // Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite -+ // the corrupted batch with correct data. -+ if (!record.hasMagic(batchMagic)) -+ writeOriginalBatch = false; -+ -+ if (record.offset() > maxOffset) -+ maxOffset = record.offset(); -+ -+ retainedRecords.add(record); -+ -+ if (!record.hasValue()) { -+ containsTombstonesOrMarker = true; -+ } -+ } else { -+ writeOriginalBatch = false; -+ } -+ } -+ return new BatchIterationResult(writeOriginalBatch, containsTombstonesOrMarker, maxOffset); -+ } -+ } -+ -+ private static class BatchIterationResult { -+ private final boolean writeOriginalBatch; -+ private final boolean containsTombstonesOrMarker; -+ private final long maxOffset; -+ public BatchIterationResult(final boolean writeOriginalBatch, -+ final boolean containsTombstonesOrMarker, -+ final long maxOffset) { -+ this.writeOriginalBatch = writeOriginalBatch; -+ this.containsTombstonesOrMarker = containsTombstonesOrMarker; -+ this.maxOffset = maxOffset; -+ } -+ public boolean shouldWriteOriginalBatch() { -+ return this.writeOriginalBatch; -+ } -+ public boolean containsTombstonesOrMarker() { -+ return this.containsTombstonesOrMarker; -+ } -+ public long maxOffset() { -+ return this.maxOffset; -+ } -+ } -+ - private static MemoryRecordsBuilder buildRetainedRecordsInto(RecordBatch originalBatch, - List retainedRecords, -- ByteBufferOutputStream bufferOutputStream) { -+ ByteBufferOutputStream bufferOutputStream, -+ final long deleteHorizonMs) { - byte magic = originalBatch.magic(); - TimestampType timestampType = originalBatch.timestampType(); - long logAppendTime = timestampType == TimestampType.LOG_APPEND_TIME ? -@@ -249,7 +314,7 @@ private static MemoryRecordsBuilder buildRetainedRecordsInto(RecordBatch origina - MemoryRecordsBuilder builder = new MemoryRecordsBuilder(bufferOutputStream, magic, - originalBatch.compressionType(), timestampType, baseOffset, logAppendTime, originalBatch.producerId(), - originalBatch.producerEpoch(), originalBatch.baseSequence(), originalBatch.isTransactional(), -- originalBatch.isControlBatch(), originalBatch.partitionLeaderEpoch(), bufferOutputStream.limit()); -+ originalBatch.isControlBatch(), originalBatch.partitionLeaderEpoch(), bufferOutputStream.limit(), deleteHorizonMs); - - for (Record record : retainedRecords) - builder.append(record); -@@ -312,12 +377,34 @@ public int hashCode() { - */ - protected abstract BatchRetention checkBatchRetention(RecordBatch batch); - -+ protected BatchRetention checkBatchRetention(RecordBatch batch, long newBatchDeleteHorizonMs) { -+ return checkBatchRetention(batch); -+ } -+ - /** - * Check whether a record should be retained in the log. Note that {@link #checkBatchRetention(RecordBatch)} - * is used prior to checking individual record retention. Only records from batches which were not - * explicitly discarded with {@link BatchRetention#DELETE} will be considered. - */ - protected abstract boolean shouldRetainRecord(RecordBatch recordBatch, Record record); -+ -+ protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record, long newDeleteHorizonMs) { -+ return shouldRetainRecord(recordBatch, record); -+ } -+ -+ /** -+ * Retrieves the delete horizon ms for a specific batch -+ */ -+ protected long retrieveDeleteHorizon(RecordBatch recordBatch) { -+ return -1L; -+ } -+ -+ /** -+ * Checks if the control batch (if it is one) can be removed (making sure that it is empty) -+ */ -+ protected boolean isControlBatchEmpty(RecordBatch recordBatch) { -+ return true; -+ } - } - - public static class FilterResult { -diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java -index 054fb861998..be32a839059 100644 ---- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java -+++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java -@@ -75,6 +75,7 @@ public void write(int b) { - private int numRecords = 0; - private float actualCompressionRatio = 1; - private long maxTimestamp = RecordBatch.NO_TIMESTAMP; -+ private long deleteHorizonMs; - private long offsetOfMaxTimestamp = -1; - private Long lastOffset = null; - private Long firstTimestamp = null; -@@ -94,7 +95,8 @@ public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, - boolean isTransactional, - boolean isControlBatch, - int partitionLeaderEpoch, -- int writeLimit) { -+ int writeLimit, -+ long deleteHorizonMs) { - if (magic > RecordBatch.MAGIC_VALUE_V0 && timestampType == TimestampType.NO_TIMESTAMP_TYPE) - throw new IllegalArgumentException("TimestampType must be set for magic >= 0"); - if (magic < RecordBatch.MAGIC_VALUE_V2) { -@@ -120,6 +122,7 @@ public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, - this.baseSequence = baseSequence; - this.isTransactional = isTransactional; - this.isControlBatch = isControlBatch; -+ this.deleteHorizonMs = deleteHorizonMs; - this.partitionLeaderEpoch = partitionLeaderEpoch; - this.writeLimit = writeLimit; - this.initialPosition = bufferStream.position(); -@@ -130,6 +133,24 @@ public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, - this.appendStream = new DataOutputStream(compressionType.wrapForOutput(this.bufferStream, magic)); - } - -+ public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, -+ byte magic, -+ CompressionType compressionType, -+ TimestampType timestampType, -+ long baseOffset, -+ long logAppendTime, -+ long producerId, -+ short producerEpoch, -+ int baseSequence, -+ boolean isTransactional, -+ boolean isControlBatch, -+ int partitionLeaderEpoch, -+ int writeLimit) { -+ this(bufferStream, magic, compressionType, timestampType, baseOffset, logAppendTime, producerId, -+ producerEpoch, baseSequence, isTransactional, isControlBatch, partitionLeaderEpoch, writeLimit, -+ RecordBatch.NO_TIMESTAMP); -+ } -+ - /** - * Construct a new builder. - * -@@ -192,6 +213,10 @@ public boolean isTransactional() { - return isTransactional; - } - -+ public boolean deleteHorizonSet() { -+ return deleteHorizonMs >= 0L; -+ } -+ - /** - * Close this builder and return the resulting buffer. - * @return The built log buffer -@@ -364,7 +389,7 @@ private int writeDefaultBatchHeader() { - maxTimestamp = this.maxTimestamp; - - DefaultRecordBatch.writeHeader(buffer, baseOffset, offsetDelta, size, magic, compressionType, timestampType, -- firstTimestamp, maxTimestamp, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, -+ firstTimestamp, maxTimestamp, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, deleteHorizonSet(), - partitionLeaderEpoch, numRecords); - - buffer.position(pos); -@@ -411,8 +436,12 @@ private Long appendWithOffset(long offset, boolean isControlRecord, long timesta - if (magic < RecordBatch.MAGIC_VALUE_V2 && headers != null && headers.length > 0) - throw new IllegalArgumentException("Magic v" + magic + " does not support record headers"); - -- if (firstTimestamp == null) -- firstTimestamp = timestamp; -+ if (firstTimestamp == null) { -+ if (deleteHorizonSet()) -+ firstTimestamp = deleteHorizonMs; -+ else -+ firstTimestamp = timestamp; -+ } - - if (magic > RecordBatch.MAGIC_VALUE_V1) { - appendDefaultRecord(offset, timestamp, key, value, headers); -diff --git a/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java -index 65a6a95fbe4..45f1609e3bc 100644 ---- a/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java -+++ b/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java -@@ -210,6 +210,18 @@ - */ - boolean isTransactional(); - -+ /** -+ * Whether or not the base timestamp has been set to the delete horizon -+ * @return true if it is, false otherwise -+ */ -+ boolean deleteHorizonSet(); -+ -+ /** -+ * Get the delete horizon, returns -1L if the first timestamp is not the delete horizon -+ * @return timestamp of the delete horizon -+ */ -+ long deleteHorizonMs(); -+ - /** - * Get the partition leader epoch of this record batch. - * @return The leader epoch or -1 if it is unknown -diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java -index b8824d3a827..4677e40e388 100644 ---- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java -+++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java -@@ -247,6 +247,45 @@ public void testFilterToPreservesPartitionLeaderEpoch() { - } - } - -+ /** -+ * This test is used to see if the first timestamp of the batch has been successfully -+ * converted to a delete horizon for the tombstones / transaction markers of the batch. -+ */ -+ @Test -+ public void testFirstTimestampToDeleteHorizonConversion() { -+ if (magic >= RecordBatch.MAGIC_VALUE_V2) { -+ ByteBuffer buffer = ByteBuffer.allocate(2048); -+ MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, -+ 0L, RecordBatch.NO_TIMESTAMP, partitionLeaderEpoch); -+ builder.append(10L, "1".getBytes(), null); -+ -+ ByteBuffer filtered = ByteBuffer.allocate(2048); -+ final long deleteHorizon = Integer.MAX_VALUE / 2; -+ builder.build().filterTo(new TopicPartition("random", 0), new MemoryRecords.RecordFilter() { -+ @Override -+ protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { -+ return true; -+ } -+ -+ @Override -+ protected BatchRetention checkBatchRetention(RecordBatch batch) { -+ return BatchRetention.RETAIN_EMPTY; -+ } -+ -+ @Override -+ protected long retrieveDeleteHorizon(RecordBatch batch) { -+ return deleteHorizon; // arbitrary value > 1 -+ } -+ }, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); -+ filtered.flip(); -+ MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); -+ -+ List batches = TestUtils.toList(filteredRecords.batches()); -+ assertEquals(1, batches.size()); -+ assertEquals(deleteHorizon, batches.get(0).deleteHorizonMs()); -+ } -+ } -+ - @Test - public void testFilterToEmptyBatchRetention() { - if (magic >= RecordBatch.MAGIC_VALUE_V2) { -diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala -index 5e04c3cb55c..9e221ffb19a 100644 ---- a/core/src/main/scala/kafka/log/Log.scala -+++ b/core/src/main/scala/kafka/log/Log.scala -@@ -221,7 +221,8 @@ class Log(@volatile var dir: File, - val producerIdExpirationCheckIntervalMs: Int, - val topicPartition: TopicPartition, - val producerStateManager: ProducerStateManager, -- logDirFailureChannel: LogDirFailureChannel) extends Logging with KafkaMetricsGroup { -+ logDirFailureChannel: LogDirFailureChannel, -+ @volatile var containsTombstones: Boolean = false) extends Logging with KafkaMetricsGroup { - - import kafka.log.Log._ - -diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala -index 219f49716f0..a59dfadb147 100644 ---- a/core/src/main/scala/kafka/log/LogCleaner.scala -+++ b/core/src/main/scala/kafka/log/LogCleaner.scala -@@ -509,11 +509,10 @@ private[log] class Cleaner(val id: Int, - case None => 0L - case Some(seg) => seg.lastModified - cleanable.log.config.deleteRetentionMs - } -- -- doClean(cleanable, deleteHorizonMs) -+ doClean(cleanable, time.milliseconds(), trackedHorizon = deleteHorizonMs) - } - -- private[log] def doClean(cleanable: LogToClean, deleteHorizonMs: Long): (Long, CleanerStats) = { -+ private[log] def doClean(cleanable: LogToClean, currentTime: Long, trackedHorizon: Long = -1L): (Long, CleanerStats) = { - info("Beginning cleaning of log %s.".format(cleanable.log.name)) - - val log = cleanable.log -@@ -531,13 +530,13 @@ private[log] class Cleaner(val id: Int, - val cleanableHorizonMs = log.logSegments(0, cleanable.firstUncleanableOffset).lastOption.map(_.lastModified).getOrElse(0L) - - // group the segments and clean the groups -- info("Cleaning log %s (cleaning prior to %s, discarding tombstones prior to %s)...".format(log.name, new Date(cleanableHorizonMs), new Date(deleteHorizonMs))) -+ info("Cleaning log %s (cleaning prior to %s, discarding tombstones prior to %s)...".format(log.name, new Date(cleanableHorizonMs), new Date(trackedHorizon))) - val transactionMetadata = new CleanedTransactionMetadata - - val groupedSegments = groupSegmentsBySize(log.logSegments(0, endOffset), log.config.segmentSize, - log.config.maxIndexSize, cleanable.firstUncleanableOffset) - for (group <- groupedSegments) -- cleanSegments(log, group, offsetMap, deleteHorizonMs, stats, transactionMetadata) -+ cleanSegments(log, group, offsetMap, currentTime, stats, transactionMetadata, trackedHorizon = trackedHorizon) - - // record buffer utilization - stats.bufferUtilization = offsetMap.utilization -@@ -561,9 +560,10 @@ private[log] class Cleaner(val id: Int, - private[log] def cleanSegments(log: Log, - segments: Seq[LogSegment], - map: OffsetMap, -- deleteHorizonMs: Long, -+ currentTime: Long, - stats: CleanerStats, -- transactionMetadata: CleanedTransactionMetadata): Unit = { -+ transactionMetadata: CleanedTransactionMetadata, -+ trackedHorizon: Long = -1L): Unit = { - // create a new segment with a suffix appended to the name of the log and indexes - val cleaned = LogCleaner.createNewCleanedSegment(log, segments.head.baseOffset) - transactionMetadata.cleanedIndex = Some(cleaned.txnIndex) -@@ -583,14 +583,15 @@ private[log] class Cleaner(val id: Int, - val abortedTransactions = log.collectAbortedTransactions(startOffset, upperBoundOffset) - transactionMetadata.addAbortedTransactions(abortedTransactions) - -- val retainDeletesAndTxnMarkers = currentSegment.lastModified > deleteHorizonMs -+ val retainDeletesAndTxnMarkers = currentSegment.lastModified > trackedHorizon - info(s"Cleaning $currentSegment in log ${log.name} into ${cleaned.baseOffset} " + -- s"with deletion horizon $deleteHorizonMs, " + -+ s"with deletion horizon $trackedHorizon, " + - s"${if(retainDeletesAndTxnMarkers) "retaining" else "discarding"} deletes.") - - try { -- cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletesAndTxnMarkers, log.config.maxMessageSize, -- transactionMetadata, lastOffsetOfActiveProducers, stats) -+ val containsTombstones: Boolean = cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletesAndTxnMarkers, log.config.maxMessageSize, -+ transactionMetadata, lastOffsetOfActiveProducers, stats, log.config.deleteRetentionMs, currentTime = currentTime) -+ log.containsTombstones = containsTombstones - } catch { - case e: LogSegmentOffsetOverflowException => - // Split the current segment. It's also safest to abort the current cleaning process, so that we retry from -@@ -634,6 +635,7 @@ private[log] class Cleaner(val id: Int, - * @param retainDeletesAndTxnMarkers Should tombstones and markers be retained while cleaning this segment - * @param maxLogMessageSize The maximum message size of the corresponding topic - * @param stats Collector for cleaning statistics -+ * @param tombstoneRetentionMs Defines how long a tombstone should be kept as defined by log configuration - */ - private[log] def cleanInto(topicPartition: TopicPartition, - sourceRecords: FileRecords, -@@ -643,14 +645,33 @@ private[log] class Cleaner(val id: Int, - maxLogMessageSize: Int, - transactionMetadata: CleanedTransactionMetadata, - lastRecordsOfActiveProducers: Map[Long, LastRecord], -- stats: CleanerStats): Unit = { -+ stats: CleanerStats, -+ tombstoneRetentionMs: Long, -+ currentTime: Long = RecordBatch.NO_TIMESTAMP): Boolean = { -+ var containsTombstones: Boolean = false -+ - val logCleanerFilter: RecordFilter = new RecordFilter { - var discardBatchRecords: Boolean = _ -+ var isControlBatchEmpty: Boolean = _ - -- override def checkBatchRetention(batch: RecordBatch): BatchRetention = { -+ override def isControlBatchEmpty(batch: RecordBatch) : Boolean = { - // we piggy-back on the tombstone retention logic to delay deletion of transaction markers. - // note that we will never delete a marker until all the records from that transaction are removed. -- discardBatchRecords = shouldDiscardBatch(batch, transactionMetadata, retainTxnMarkers = retainDeletesAndTxnMarkers) -+ val canDiscardBatch = shouldDiscardBatch(batch, transactionMetadata, retainTxnMarkers = retainDeletesAndTxnMarkers) -+ isControlBatchEmpty = canDiscardBatch -+ isControlBatchEmpty -+ } -+ -+ override def checkBatchRetention(batch: RecordBatch, newBatchDeleteHorizonMs : Long): BatchRetention = { -+ val canDiscardBatch = isControlBatchEmpty -+ -+ if (batch.isControlBatch) { -+ discardBatchRecords = canDiscardBatch && -+ ((batch.deleteHorizonSet() && batch.deleteHorizonMs() < currentTime) || -+ newBatchDeleteHorizonMs != RecordBatch.NO_TIMESTAMP && newBatchDeleteHorizonMs < currentTime) -+ } else { -+ discardBatchRecords = canDiscardBatch -+ } - - def isBatchLastRecordOfProducer: Boolean = { - // We retain the batch in order to preserve the state of active producers. There are three cases: -@@ -675,12 +696,33 @@ private[log] class Cleaner(val id: Int, - BatchRetention.DELETE_EMPTY - } - -- override def shouldRetainRecord(batch: RecordBatch, record: Record): Boolean = { -+ override def checkBatchRetention(batch: RecordBatch): BatchRetention = checkBatchRetention(batch, batch.deleteHorizonMs()) -+ -+ override def shouldRetainRecord(batch: RecordBatch, record: Record, newDeleteHorizonMs: Long): Boolean = { -+ var isRecordRetained: Boolean = true - if (discardBatchRecords) - // The batch is only retained to preserve producer sequence information; the records can be removed -- false -+ isRecordRetained = false - else -- Cleaner.this.shouldRetainRecord(map, retainDeletesAndTxnMarkers, batch, record, stats) -+ isRecordRetained = Cleaner.this.shouldRetainRecord(map, retainDeletesAndTxnMarkers, batch, record, stats, newDeleteHorizonMs, currentTime = currentTime) -+ if (isRecordRetained && !record.hasValue()) -+ containsTombstones = true -+ isRecordRetained -+ } -+ -+ override def shouldRetainRecord(batch: RecordBatch, record: Record): Boolean = { -+ shouldRetainRecord(batch, record, RecordBatch.NO_TIMESTAMP) -+ } -+ -+ override def retrieveDeleteHorizon(batch: RecordBatch) : Long = { -+ if (batch.deleteHorizonSet()) -+ return batch.deleteHorizonMs() // means that we keep the old timestamp stored -+ -+ // check that the control batch has been emptied of records -+ // if not, then we do not set a delete horizon until that is true -+ if (batch.isControlBatch() && !isControlBatchEmpty) -+ return -1L -+ return time.milliseconds() + tombstoneRetentionMs; - } - } - -@@ -720,6 +762,7 @@ private[log] class Cleaner(val id: Int, - growBuffersOrFail(sourceRecords, position, maxLogMessageSize, records) - } - restoreBuffers() -+ containsTombstones - } - - -@@ -758,20 +801,19 @@ private[log] class Cleaner(val id: Int, - private def shouldDiscardBatch(batch: RecordBatch, - transactionMetadata: CleanedTransactionMetadata, - retainTxnMarkers: Boolean): Boolean = { -- if (batch.isControlBatch) { -- val canDiscardControlBatch = transactionMetadata.onControlBatchRead(batch) -- canDiscardControlBatch && !retainTxnMarkers -- } else { -- val canDiscardBatch = transactionMetadata.onBatchRead(batch) -- canDiscardBatch -- } -+ if (batch.isControlBatch) -+ transactionMetadata.onControlBatchRead(batch) -+ else -+ transactionMetadata.onBatchRead(batch) - } - - private def shouldRetainRecord(map: kafka.log.OffsetMap, - retainDeletes: Boolean, - batch: RecordBatch, - record: Record, -- stats: CleanerStats): Boolean = { -+ stats: CleanerStats, -+ newBatchDeleteHorizonMs: Long, -+ currentTime: Long = -1L): Boolean = { - val pastLatestOffset = record.offset > map.latestOffset - if (pastLatestOffset) - return true -@@ -785,7 +827,15 @@ private[log] class Cleaner(val id: Int, - * 2) The message doesn't has value but it can't be deleted now. - */ - val latestOffsetForKey = record.offset() >= foundOffset -- val isRetainedValue = record.hasValue || retainDeletes -+ val isLatestVersion = batch.magic() >= RecordBatch.MAGIC_VALUE_V2 -+ var shouldRetainDeletes = true -+ if (isLatestVersion) -+ shouldRetainDeletes = (batch.deleteHorizonSet() && currentTime < batch.deleteHorizonMs()) || -+ (!batch.deleteHorizonSet() && currentTime < newBatchDeleteHorizonMs) -+ else -+ shouldRetainDeletes = retainDeletes -+ val isRetainedValue = record.hasValue || shouldRetainDeletes -+ - latestOffsetForKey && isRetainedValue - } else { - stats.invalidMessage() -diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala -index a5cfed5c094..6edd73530a9 100755 ---- a/core/src/main/scala/kafka/log/LogCleanerManager.scala -+++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala -@@ -180,6 +180,7 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], - case (_, log) => log.config.compact // match logs that are marked as compacted - }.filterNot { - case (topicPartition, log) => -+ - // skip any logs already in-progress and uncleanable partitions - inProgress.contains(topicPartition) || isUncleanablePartition(log, topicPartition) - }.map { -@@ -202,7 +203,34 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], - (ltc.needCompactionNow && ltc.cleanableBytes > 0) || ltc.cleanableRatio > ltc.log.config.minCleanableRatio - } - if(cleanableLogs.isEmpty) { -- None -+ // in this case, we are probably in a low throughput situation -+ // therefore, we should take advantage of this fact and remove tombstones if we can -+ val logsContainingTombstones = logs.filter { -+ case (_, log) => log.containsTombstones -+ }.filterNot { -+ case (topicPartition, log) => -+ // skip any logs already in-progress and uncleanable partitions -+ inProgress.contains(topicPartition) || isUncleanablePartition(log, topicPartition) -+ }.map { -+ case (topicPartition, log) => // create a LogToClean instance for each -+ try { -+ val (firstDirtyOffset, firstUncleanableDirtyOffset) = cleanableOffsets(log, topicPartition, lastClean, now) -+ val compactionDelayMs = maxCompactionDelay(log, firstDirtyOffset, now) -+ preCleanStats.updateMaxCompactionDelay(compactionDelayMs) -+ -+ LogToClean(topicPartition, log, firstDirtyOffset, firstUncleanableDirtyOffset, compactionDelayMs > 0) -+ } catch { -+ case e: Throwable => throw new LogCleaningException(log, -+ s"Failed to calculate log cleaning stats for partition $topicPartition", e) -+ } -+ } -+ if (!logsContainingTombstones.isEmpty) { -+ val filthiest = logsContainingTombstones.max -+ inProgress.put(filthiest.topicPartition, LogCleaningInProgress) -+ Some(filthiest) -+ } else { -+ None -+ } - } else { - preCleanStats.recordCleanablePartitions(cleanableLogs.size) - val filthiest = cleanableLogs.max -diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala -index d148c3f8959..aa908c4c787 100644 ---- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala -+++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala -@@ -18,6 +18,7 @@ - package kafka.log - - import java.io.PrintWriter -+import java.util.Properties - - import com.yammer.metrics.Metrics - import com.yammer.metrics.core.{Gauge, MetricName} -@@ -178,6 +179,49 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K - s"but lastCleaned=$lastCleaned2", lastCleaned2 >= secondBlockCleanableSegmentOffset) - } - -+ @Test -+ def testTombstoneCleanWithLowThoroughput() : Unit = { -+ val tombstoneRetentionMs = 1000 // this is in milliseconds -> 1 second -+ -+ val topicPartitions = Array(new TopicPartition("log-partition", 0)) -+ val props = new Properties() -+ props.put(LogConfig.DeleteRetentionMsProp, "1000") -+ cleaner = makeCleaner(partitions = topicPartitions, propertyOverrides = props) -+ -+ val log = cleaner.logs.get(topicPartitions(0)) -+ -+ val T0 = time.milliseconds -+ writeKeyDups(numKeys = 1, numDups = 1, log, CompressionType.NONE, timestamp = T0, -+ startValue = 0, step = 1, isRecordTombstone = true) -+ -+ val activeSegAtT0 = log.activeSegment -+ -+ // roll the active segment -+ log.roll() -+ -+ cleaner.startup() -+ Thread.sleep(100) -+ -+ import JavaConverters._ -+ var containsTombstones: Boolean = false -+ for (segment <- log.logSegments; record <- segment.log.records.asScala) { -+ containsTombstones = true -+ } -+ assertTrue(containsTombstones) -+ time.sleep(tombstoneRetentionMs + 1) -+ -+ val firstBlockCleanableSegmentOffset = activeSegAtT0.baseOffset -+ -+ // the first block should get cleaned -+ cleaner.awaitCleaned(new TopicPartition("log-partition", 0), -+ firstBlockCleanableSegmentOffset, maxWaitMs = tombstoneRetentionMs * 3) -+ -+ for (segment <- log.logSegments; record <- segment.log.records.asScala) { -+ fail ("The log should not contain record " + record + ", tombstone has expired its lifetime.") -+ } -+ assertFalse(log.containsTombstones) -+ } -+ - private def readFromLog(log: Log): Iterable[(Int, Int)] = { - import JavaConverters._ - for (segment <- log.logSegments; record <- segment.log.records.asScala) yield { -@@ -187,12 +231,17 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K - } - } - -- private def writeKeyDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionType, timestamp: Long, startValue: Int, step: Int): Seq[(Int, Int)] = { -+ private def writeKeyDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionType, timestamp: Long, -+ startValue: Int, step: Int, isRecordTombstone: Boolean = false): Seq[(Int, Int)] = { - var valCounter = startValue - for (_ <- 0 until numDups; key <- 0 until numKeys) yield { - val curValue = valCounter -- log.appendAsLeader(TestUtils.singletonRecords(value = curValue.toString.getBytes, codec = codec, -- key = key.toString.getBytes, timestamp = timestamp), leaderEpoch = 0) -+ if (isRecordTombstone) -+ log.appendAsLeader(TestUtils.singletonRecords(value = null, codec = codec, -+ key = key.toString.getBytes, timestamp = timestamp), leaderEpoch = 0) -+ else -+ log.appendAsLeader(TestUtils.singletonRecords(value = curValue.toString.getBytes, codec = codec, -+ key = key.toString.getBytes, timestamp = timestamp), leaderEpoch = 0) - valCounter += step - (key, curValue) - } -diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala -index bb30287bb12..7d03b07fd70 100755 ---- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala -+++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala -@@ -347,7 +347,7 @@ class LogCleanerTest { - log.roll() - - // cannot remove the marker in this pass because there are still valid records -- var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertEquals(List(1, 3, 2), LogTest.keysInLog(log)) - assertEquals(List(0, 2, 3, 4, 5), offsetsInLog(log)) - -@@ -356,17 +356,17 @@ class LogCleanerTest { - log.roll() - - // the first cleaning preserves the commit marker (at offset 3) since there were still records for the transaction -- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) - assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) - - // delete horizon forced to 0 to verify marker is not removed early -- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = 0L)._1 -+ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = 0L)._1 - assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) - assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) - - // clean again with large delete horizon and verify the marker is removed -- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) - assertEquals(List(4, 5, 6, 7, 8), offsetsInLog(log)) - } -@@ -395,11 +395,11 @@ class LogCleanerTest { - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, isFromClient = false) - log.roll() - -- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) -+ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) - assertEquals(List(2), LogTest.keysInLog(log)) - assertEquals(List(1, 3, 4), offsetsInLog(log)) - -- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) -+ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) - assertEquals(List(2), LogTest.keysInLog(log)) - assertEquals(List(3, 4), offsetsInLog(log)) - } -@@ -434,14 +434,14 @@ class LogCleanerTest { - - // first time through the records are removed - // Expected State: [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}] -- var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertEquals(List(2, 3), LogTest.keysInLog(log)) - assertEquals(List(4, 5, 6, 7), offsetsInLog(log)) - assertEquals(List(1, 3, 4, 5, 6, 7), lastOffsetsPerBatchInLog(log)) - - // the empty batch remains if cleaned again because it still holds the last sequence - // Expected State: [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}] -- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertEquals(List(2, 3), LogTest.keysInLog(log)) - assertEquals(List(4, 5, 6, 7), offsetsInLog(log)) - assertEquals(List(1, 3, 4, 5, 6, 7), lastOffsetsPerBatchInLog(log)) -@@ -454,13 +454,13 @@ class LogCleanerTest { - log.roll() - - // Expected State: [{Producer1: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] -- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) - assertEquals(List(4, 5, 6, 7, 8, 9), offsetsInLog(log)) - assertEquals(List(1, 4, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log)) - - // Expected State: [{Producer1: EmptyBatch}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] -- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) - assertEquals(List(5, 6, 7, 8, 9), offsetsInLog(log)) - assertEquals(List(1, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log)) -@@ -484,14 +484,14 @@ class LogCleanerTest { - - // first time through the control batch is retained as an empty batch - // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] -- var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertEquals(List(2, 3), LogTest.keysInLog(log)) - assertEquals(List(1, 2), offsetsInLog(log)) - assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) - - // the empty control batch does not cause an exception when cleaned - // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] -- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertEquals(List(2, 3), LogTest.keysInLog(log)) - assertEquals(List(1, 2), offsetsInLog(log)) - assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) -@@ -515,7 +515,7 @@ class LogCleanerTest { - log.roll() - - // Both the record and the marker should remain after cleaning -- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) -+ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) - assertEquals(List(0, 1), offsetsInLog(log)) - assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) - } -@@ -540,12 +540,12 @@ class LogCleanerTest { - // Both the batch and the marker should remain after cleaning. The batch is retained - // because it is the last entry for this producerId. The marker is retained because - // there are still batches remaining from this transaction. -- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) -+ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) - assertEquals(List(1), offsetsInLog(log)) - assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) - - // The empty batch and the marker is still retained after a second cleaning. -- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) -+ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) - assertEquals(List(1), offsetsInLog(log)) - assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) - } -@@ -570,12 +570,12 @@ class LogCleanerTest { - log.roll() - - // delete horizon set to 0 to verify marker is not removed early -- val dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = 0L)._1 -+ val dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = 0L)._1 - assertEquals(List(3), LogTest.keysInLog(log)) - assertEquals(List(3, 4, 5), offsetsInLog(log)) - - // clean again with large delete horizon and verify the marker is removed -- cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) -+ cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue) - assertEquals(List(3), LogTest.keysInLog(log)) - assertEquals(List(4, 5), offsetsInLog(log)) - } -@@ -609,12 +609,12 @@ class LogCleanerTest { - - // Both transactional batches will be cleaned. The last one will remain in the log - // as an empty batch in order to preserve the producer sequence number and epoch -- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) -+ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) - assertEquals(List(1, 3, 4, 5), offsetsInLog(log)) - assertEquals(List(1, 2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) - - // On the second round of cleaning, the marker from the first transaction should be removed. -- cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) -+ cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) - assertEquals(List(3, 4, 5), offsetsInLog(log)) - assertEquals(List(2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) - } -@@ -646,14 +646,14 @@ class LogCleanerTest { - assertAbortedTransactionIndexed() - - // first time through the records are removed -- var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertAbortedTransactionIndexed() - assertEquals(List(), LogTest.keysInLog(log)) - assertEquals(List(2), offsetsInLog(log)) // abort marker is retained - assertEquals(List(1, 2), lastOffsetsPerBatchInLog(log)) // empty batch is retained - - // the empty batch remains if cleaned again because it still holds the last sequence -- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertAbortedTransactionIndexed() - assertEquals(List(), LogTest.keysInLog(log)) - assertEquals(List(2), offsetsInLog(log)) // abort marker is still retained -@@ -663,13 +663,13 @@ class LogCleanerTest { - appendProducer(Seq(1)) - log.roll() - -- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertAbortedTransactionIndexed() - assertEquals(List(1), LogTest.keysInLog(log)) - assertEquals(List(2, 3), offsetsInLog(log)) // abort marker is not yet gone because we read the empty batch - assertEquals(List(2, 3), lastOffsetsPerBatchInLog(log)) // but we do not preserve the empty batch - -- dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 -+ dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 - assertEquals(List(1), LogTest.keysInLog(log)) - assertEquals(List(3), offsetsInLog(log)) // abort marker is gone - assertEquals(List(3), lastOffsetsPerBatchInLog(log)) From 331ebada7bbe7e033fae67047458222010c871bd Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 11 Feb 2020 19:51:07 -0800 Subject: [PATCH 15/53] Delete LogCleanerManager.scala.orig --- .../kafka/log/LogCleanerManager.scala.orig | 626 ------------------ 1 file changed, 626 deletions(-) delete mode 100755 core/src/main/scala/kafka/log/LogCleanerManager.scala.orig diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala.orig b/core/src/main/scala/kafka/log/LogCleanerManager.scala.orig deleted file mode 100755 index 561a6437d91ae..0000000000000 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala.orig +++ /dev/null @@ -1,626 +0,0 @@ -/** - * 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 kafka.log - -import java.io.File -import java.util.concurrent.TimeUnit -import java.util.concurrent.locks.ReentrantLock - -import com.yammer.metrics.core.Gauge -import kafka.common.{KafkaException, LogCleaningAbortedException} -import kafka.metrics.KafkaMetricsGroup -import kafka.server.LogDirFailureChannel -import kafka.server.checkpoints.OffsetCheckpointFile -import kafka.utils.CoreUtils._ -import kafka.utils.{Logging, Pool} -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.utils.Time -import org.apache.kafka.common.errors.KafkaStorageException - -import scala.collection.{Iterable, Seq, mutable} - -private[log] sealed trait LogCleaningState -private[log] case object LogCleaningInProgress extends LogCleaningState -private[log] case object LogCleaningAborted extends LogCleaningState -private[log] case class LogCleaningPaused(pausedCount: Int) extends LogCleaningState - -private[log] class LogCleaningException(val log: Log, - private val message: String, - private val cause: Throwable) extends KafkaException(message, cause) - -/** - * This class manages the state of each partition being cleaned. - * LogCleaningState defines the cleaning states that a TopicPartition can be in. - * 1. None : No cleaning state in a TopicPartition. In this state, it can become LogCleaningInProgress - * or LogCleaningPaused(1). Valid previous state are LogCleaningInProgress and LogCleaningPaused(1) - * 2. LogCleaningInProgress : The cleaning is currently in progress. In this state, it can become None when log cleaning is finished - * or become LogCleaningAborted. Valid previous state is None. - * 3. LogCleaningAborted : The cleaning abort is requested. In this state, it can become LogCleaningPaused(1). - * Valid previous state is LogCleaningInProgress. - * 4-a. LogCleaningPaused(1) : The cleaning is paused once. No log cleaning can be done in this state. - * In this state, it can become None or LogCleaningPaused(2). - * Valid previous state is None, LogCleaningAborted or LogCleaningPaused(2). - * 4-b. LogCleaningPaused(i) : The cleaning is paused i times where i>= 2. No log cleaning can be done in this state. - * In this state, it can become LogCleaningPaused(i-1) or LogCleaningPaused(i+1). - * Valid previous state is LogCleaningPaused(i-1) or LogCleaningPaused(i+1). - */ -private[log] class LogCleanerManager(val logDirs: Seq[File], - val logs: Pool[TopicPartition, Log], - val logDirFailureChannel: LogDirFailureChannel) extends Logging with KafkaMetricsGroup { - import LogCleanerManager._ - - - protected override def loggerName = classOf[LogCleaner].getName - - // package-private for testing - private[log] val offsetCheckpointFile = "cleaner-offset-checkpoint" - - /* the offset checkpoints holding the last cleaned point for each log */ - @volatile private var checkpoints = logDirs.map(dir => - (dir, new OffsetCheckpointFile(new File(dir, offsetCheckpointFile), logDirFailureChannel))).toMap - - /* the set of logs currently being cleaned */ - private val inProgress = mutable.HashMap[TopicPartition, LogCleaningState]() - - /* the set of uncleanable partitions (partitions that have raised an unexpected error during cleaning) - * for each log directory */ - private val uncleanablePartitions = mutable.HashMap[String, mutable.Set[TopicPartition]]() - - /* a global lock used to control all access to the in-progress set and the offset checkpoints */ - private val lock = new ReentrantLock - - /* for coordinating the pausing and the cleaning of a partition */ - private val pausedCleaningCond = lock.newCondition() - - /* gauges for tracking the number of partitions marked as uncleanable for each log directory */ - for (dir <- logDirs) { - newGauge( - "uncleanable-partitions-count", - new Gauge[Int] { def value = inLock(lock) { uncleanablePartitions.get(dir.getAbsolutePath).map(_.size).getOrElse(0) } }, - Map("logDirectory" -> dir.getAbsolutePath) - ) - } - - /* gauges for tracking the number of uncleanable bytes from uncleanable partitions for each log directory */ - for (dir <- logDirs) { - newGauge( - "uncleanable-bytes", - new Gauge[Long] { - def value = { - inLock(lock) { - uncleanablePartitions.get(dir.getAbsolutePath) match { - case Some(partitions) => { - val lastClean = allCleanerCheckpoints - val now = Time.SYSTEM.milliseconds - partitions.map { tp => - val log = logs.get(tp) - val lastCleanOffset = lastClean.get(tp) - val (firstDirtyOffset, firstUncleanableDirtyOffset) = cleanableOffsets(log, lastCleanOffset, now) - val (_, uncleanableBytes) = calculateCleanableBytes(log, firstDirtyOffset, firstUncleanableDirtyOffset) - uncleanableBytes - }.sum - } - case _ => 0 - } - } - } - }, - Map("logDirectory" -> dir.getAbsolutePath) - ) - } - - /* a gauge for tracking the cleanable ratio of the dirtiest log */ - @volatile private var dirtiestLogCleanableRatio = 0.0 - newGauge("max-dirty-percent", new Gauge[Int] { def value = (100 * dirtiestLogCleanableRatio).toInt }) - - /* a gauge for tracking the time since the last log cleaner run, in milli seconds */ - @volatile private var timeOfLastRun : Long = Time.SYSTEM.milliseconds - newGauge("time-since-last-run-ms", new Gauge[Long] { def value = Time.SYSTEM.milliseconds - timeOfLastRun }) - - /** - * @return the position processed for all logs. - */ - def allCleanerCheckpoints: Map[TopicPartition, Long] = { - inLock(lock) { - checkpoints.values.flatMap(checkpoint => { - try { - checkpoint.read() - } catch { - case e: KafkaStorageException => - error(s"Failed to access checkpoint file ${checkpoint.file.getName} in dir ${checkpoint.file.getParentFile.getAbsolutePath}", e) - Map.empty[TopicPartition, Long] - } - }).toMap - } - } - - /** - * Package private for unit test. Get the cleaning state of the partition. - */ - private[log] def cleaningState(tp: TopicPartition): Option[LogCleaningState] = { - inLock(lock) { - inProgress.get(tp) - } - } - - /** - * Package private for unit test. Set the cleaning state of the partition. - */ - private[log] def setCleaningState(tp: TopicPartition, state: LogCleaningState): Unit = { - inLock(lock) { - inProgress.put(tp, state) - } - } - - /** - * Choose the log to clean next and add it to the in-progress set. We recompute this - * each time from the full set of logs to allow logs to be dynamically added to the pool of logs - * the log manager maintains. - */ - def grabFilthiestCompactedLog(time: Time, preCleanStats: PreCleanStats = new PreCleanStats()): Option[LogToClean] = { - inLock(lock) { - val now = time.milliseconds - this.timeOfLastRun = now - val lastClean = allCleanerCheckpoints - val dirtyLogs = logs.filter { - case (_, log) => log.config.compact // match logs that are marked as compacted - }.filterNot { - case (topicPartition, log) => - - // skip any logs already in-progress and uncleanable partitions - inProgress.contains(topicPartition) || isUncleanablePartition(log, topicPartition) - }.map { - case (topicPartition, log) => // create a LogToClean instance for each - try { - val lastCleanOffset = lastClean.get(topicPartition) - val (firstDirtyOffset, firstUncleanableDirtyOffset) = cleanableOffsets(log, lastCleanOffset, now) - val compactionDelayMs = maxCompactionDelay(log, firstDirtyOffset, now) - preCleanStats.updateMaxCompactionDelay(compactionDelayMs) - - LogToClean(topicPartition, log, firstDirtyOffset, firstUncleanableDirtyOffset, compactionDelayMs > 0) - } catch { - case e: Throwable => throw new LogCleaningException(log, - s"Failed to calculate log cleaning stats for partition $topicPartition", e) - } - }.filter(ltc => ltc.totalBytes > 0) // skip any empty logs - - this.dirtiestLogCleanableRatio = if (dirtyLogs.nonEmpty) dirtyLogs.max.cleanableRatio else 0 - // and must meet the minimum threshold for dirty byte ratio or have some bytes required to be compacted - val cleanableLogs = dirtyLogs.filter { ltc => - (ltc.needCompactionNow && ltc.cleanableBytes > 0) || ltc.cleanableRatio > ltc.log.config.minCleanableRatio - } - if(cleanableLogs.isEmpty) { - // in this case, we are probably in a low throughput situation - // therefore, we should take advantage of this fact and remove tombstones if we can - val logsContainingTombstones = logs.filter { - case (_, log) => log.containsTombstones - }.filterNot { - case (topicPartition, log) => - // skip any logs already in-progress and uncleanable partitions - inProgress.contains(topicPartition) || isUncleanablePartition(log, topicPartition) - }.map { - case (topicPartition, log) => // create a LogToClean instance for each - try { - val (firstDirtyOffset, firstUncleanableDirtyOffset) = cleanableOffsets(log, topicPartition, lastClean, now) - val compactionDelayMs = maxCompactionDelay(log, firstDirtyOffset, now) - preCleanStats.updateMaxCompactionDelay(compactionDelayMs) - - LogToClean(topicPartition, log, firstDirtyOffset, firstUncleanableDirtyOffset, compactionDelayMs > 0) - } catch { - case e: Throwable => throw new LogCleaningException(log, - s"Failed to calculate log cleaning stats for partition $topicPartition", e) - } - } - if (!logsContainingTombstones.isEmpty) { - val filthiest = logsContainingTombstones.max - inProgress.put(filthiest.topicPartition, LogCleaningInProgress) - Some(filthiest) - } else { - None - } - } else { - preCleanStats.recordCleanablePartitions(cleanableLogs.size) - val filthiest = cleanableLogs.max - inProgress.put(filthiest.topicPartition, LogCleaningInProgress) - Some(filthiest) - } - } - } - - /** - * Pause logs cleaning for logs that do not have compaction enabled - * and do not have other deletion or compaction in progress. - * This is to handle potential race between retention and cleaner threads when users - * switch topic configuration between compacted and non-compacted topic. - * @return retention logs that have log cleaning successfully paused - */ - def pauseCleaningForNonCompactedPartitions(): Iterable[(TopicPartition, Log)] = { - inLock(lock) { - val deletableLogs = logs.filter { - case (_, log) => !log.config.compact // pick non-compacted logs - }.filterNot { - case (topicPartition, _) => inProgress.contains(topicPartition) // skip any logs already in-progress - } - - deletableLogs.foreach { - case (topicPartition, _) => inProgress.put(topicPartition, LogCleaningPaused(1)) - } - deletableLogs - } - } - - /** - * Find any logs that have compaction enabled. Mark them as being cleaned - * Include logs without delete enabled, as they may have segments - * that precede the start offset. - */ - def deletableLogs(): Iterable[(TopicPartition, Log)] = { - inLock(lock) { - val toClean = logs.filter { case (topicPartition, log) => - !inProgress.contains(topicPartition) && log.config.compact && - !isUncleanablePartition(log, topicPartition) - } - toClean.foreach { case (tp, _) => inProgress.put(tp, LogCleaningInProgress) } - toClean - } - - } - - /** - * Abort the cleaning of a particular partition, if it's in progress. This call blocks until the cleaning of - * the partition is aborted. - * This is implemented by first abortAndPausing and then resuming the cleaning of the partition. - */ - def abortCleaning(topicPartition: TopicPartition): Unit = { - inLock(lock) { - abortAndPauseCleaning(topicPartition) - resumeCleaning(Seq(topicPartition)) - } - info(s"The cleaning for partition $topicPartition is aborted") - } - - /** - * Abort the cleaning of a particular partition if it's in progress, and pause any future cleaning of this partition. - * This call blocks until the cleaning of the partition is aborted and paused. - * 1. If the partition is not in progress, mark it as paused. - * 2. Otherwise, first mark the state of the partition as aborted. - * 3. The cleaner thread checks the state periodically and if it sees the state of the partition is aborted, it - * throws a LogCleaningAbortedException to stop the cleaning task. - * 4. When the cleaning task is stopped, doneCleaning() is called, which sets the state of the partition as paused. - * 5. abortAndPauseCleaning() waits until the state of the partition is changed to paused. - * 6. If the partition is already paused, a new call to this function - * will increase the paused count by one. - */ - def abortAndPauseCleaning(topicPartition: TopicPartition): Unit = { - inLock(lock) { - inProgress.get(topicPartition) match { - case None => - inProgress.put(topicPartition, LogCleaningPaused(1)) - case Some(LogCleaningInProgress) => - inProgress.put(topicPartition, LogCleaningAborted) - case Some(LogCleaningPaused(count)) => - inProgress.put(topicPartition, LogCleaningPaused(count + 1)) - case Some(s) => - throw new IllegalStateException(s"Compaction for partition $topicPartition cannot be aborted and paused since it is in $s state.") - } - - while(!isCleaningInStatePaused(topicPartition)) - pausedCleaningCond.await(100, TimeUnit.MILLISECONDS) - } - info(s"The cleaning for partition $topicPartition is aborted and paused") - } - - /** - * Resume the cleaning of paused partitions. - * Each call of this function will undo one pause. - */ - def resumeCleaning(topicPartitions: Iterable[TopicPartition]): Unit = { - inLock(lock) { - topicPartitions.foreach { - topicPartition => - inProgress.get(topicPartition) match { - case None => - throw new IllegalStateException(s"Compaction for partition $topicPartition cannot be resumed since it is not paused.") - case Some(state) => - state match { - case LogCleaningPaused(count) if count == 1 => - inProgress.remove(topicPartition) - case LogCleaningPaused(count) if count > 1 => - inProgress.put(topicPartition, LogCleaningPaused(count - 1)) - case s => - throw new IllegalStateException(s"Compaction for partition $topicPartition cannot be resumed since it is in $s state.") - } - } - } - } - } - - /** - * Check if the cleaning for a partition is in a particular state. The caller is expected to hold lock while making the call. - */ - private def isCleaningInState(topicPartition: TopicPartition, expectedState: LogCleaningState): Boolean = { - inProgress.get(topicPartition) match { - case None => false - case Some(state) => - if (state == expectedState) - true - else - false - } - } - - /** - * Check if the cleaning for a partition is paused. The caller is expected to hold lock while making the call. - */ - private def isCleaningInStatePaused(topicPartition: TopicPartition): Boolean = { - inProgress.get(topicPartition) match { - case None => false - case Some(state) => - state match { - case _: LogCleaningPaused => - true - case _ => - false - } - } - } - - /** - * Check if the cleaning for a partition is aborted. If so, throw an exception. - */ - def checkCleaningAborted(topicPartition: TopicPartition): Unit = { - inLock(lock) { - if (isCleaningInState(topicPartition, LogCleaningAborted)) - throw new LogCleaningAbortedException() - } - } - - def updateCheckpoints(dataDir: File, update: Option[(TopicPartition,Long)]): Unit = { - inLock(lock) { - val checkpoint = checkpoints(dataDir) - if (checkpoint != null) { - try { - val existing = checkpoint.read().filter { case (k, _) => logs.keys.contains(k) } ++ update - checkpoint.write(existing) - } catch { - case e: KafkaStorageException => - error(s"Failed to access checkpoint file ${checkpoint.file.getName} in dir ${checkpoint.file.getParentFile.getAbsolutePath}", e) - } - } - } - } - - def alterCheckpointDir(topicPartition: TopicPartition, sourceLogDir: File, destLogDir: File): Unit = { - inLock(lock) { - try { - checkpoints.get(sourceLogDir).flatMap(_.read().get(topicPartition)) match { - case Some(offset) => - // Remove this partition from the checkpoint file in the source log directory - updateCheckpoints(sourceLogDir, None) - // Add offset for this partition to the checkpoint file in the source log directory - updateCheckpoints(destLogDir, Option(topicPartition, offset)) - case None => - } - } catch { - case e: KafkaStorageException => - error(s"Failed to access checkpoint file in dir ${sourceLogDir.getAbsolutePath}", e) - } - - val logUncleanablePartitions = uncleanablePartitions.getOrElse(sourceLogDir.toString, mutable.Set[TopicPartition]()) - if (logUncleanablePartitions.contains(topicPartition)) { - logUncleanablePartitions.remove(topicPartition) - markPartitionUncleanable(destLogDir.toString, topicPartition) - } - } - } - - def handleLogDirFailure(dir: String): Unit = { - warn(s"Stopping cleaning logs in dir $dir") - inLock(lock) { - checkpoints = checkpoints.filter { case (k, _) => k.getAbsolutePath != dir } - } - } - - def maybeTruncateCheckpoint(dataDir: File, topicPartition: TopicPartition, offset: Long): Unit = { - inLock(lock) { - if (logs.get(topicPartition).config.compact) { - val checkpoint = checkpoints(dataDir) - if (checkpoint != null) { - val existing = checkpoint.read() - if (existing.getOrElse(topicPartition, 0L) > offset) - checkpoint.write(existing + (topicPartition -> offset)) - } - } - } - } - - /** - * Save out the endOffset and remove the given log from the in-progress set, if not aborted. - */ - def doneCleaning(topicPartition: TopicPartition, dataDir: File, endOffset: Long): Unit = { - inLock(lock) { - inProgress.get(topicPartition) match { - case Some(LogCleaningInProgress) => - updateCheckpoints(dataDir, Option(topicPartition, endOffset)) - inProgress.remove(topicPartition) - case Some(LogCleaningAborted) => - inProgress.put(topicPartition, LogCleaningPaused(1)) - pausedCleaningCond.signalAll() - case None => - throw new IllegalStateException(s"State for partition $topicPartition should exist.") - case s => - throw new IllegalStateException(s"In-progress partition $topicPartition cannot be in $s state.") - } - } - } - - def doneDeleting(topicPartitions: Iterable[TopicPartition]): Unit = { - inLock(lock) { - topicPartitions.foreach { - topicPartition => - inProgress.get(topicPartition) match { - case Some(LogCleaningInProgress) => - inProgress.remove(topicPartition) - case Some(LogCleaningAborted) => - inProgress.put(topicPartition, LogCleaningPaused(1)) - pausedCleaningCond.signalAll() - case None => - throw new IllegalStateException(s"State for partition $topicPartition should exist.") - case s => - throw new IllegalStateException(s"In-progress partition $topicPartition cannot be in $s state.") - } - } - } - } - - /** - * Returns an immutable set of the uncleanable partitions for a given log directory - * Only used for testing - */ - private[log] def uncleanablePartitions(logDir: String): Set[TopicPartition] = { - var partitions: Set[TopicPartition] = Set() - inLock(lock) { partitions ++= uncleanablePartitions.getOrElse(logDir, partitions) } - partitions - } - - def markPartitionUncleanable(logDir: String, partition: TopicPartition): Unit = { - inLock(lock) { - uncleanablePartitions.get(logDir) match { - case Some(partitions) => - partitions.add(partition) - case None => - uncleanablePartitions.put(logDir, mutable.Set(partition)) - } - } - } - - private def isUncleanablePartition(log: Log, topicPartition: TopicPartition): Boolean = { - inLock(lock) { - uncleanablePartitions.get(log.dir.getParent).exists(partitions => partitions.contains(topicPartition)) - } - } -} - -private[log] object LogCleanerManager extends Logging { - - def isCompactAndDelete(log: Log): Boolean = { - log.config.compact && log.config.delete - } - - /** - * get max delay between the time when log is required to be compacted as determined - * by maxCompactionLagMs and the current time. - */ - def maxCompactionDelay(log: Log, firstDirtyOffset: Long, now: Long) : Long = { - val dirtyNonActiveSegments = log.nonActiveLogSegmentsFrom(firstDirtyOffset) - val firstBatchTimestamps = log.getFirstBatchTimestampForSegments(dirtyNonActiveSegments).filter(_ > 0) - - val earliestDirtySegmentTimestamp = { - if (firstBatchTimestamps.nonEmpty) - firstBatchTimestamps.min - else Long.MaxValue - } - - val maxCompactionLagMs = math.max(log.config.maxCompactionLagMs, 0L) - val cleanUntilTime = now - maxCompactionLagMs - - if (earliestDirtySegmentTimestamp < cleanUntilTime) - cleanUntilTime - earliestDirtySegmentTimestamp - else - 0L - } - - /** - * Returns the range of dirty offsets that can be cleaned. - * - * @param log the log - * @param lastCleanOffset the last checkpointed offset - * @param now the current time in milliseconds of the cleaning operation - * @return the lower (inclusive) and upper (exclusive) offsets - */ - def cleanableOffsets(log: Log, lastCleanOffset: Option[Long], now: Long): (Long, Long) = { - // If the log segments are abnormally truncated and hence the checkpointed offset is no longer valid; - // reset to the log starting offset and log the error - val firstDirtyOffset = { - val logStartOffset = log.logStartOffset - val checkpointDirtyOffset = lastCleanOffset.getOrElse(logStartOffset) - - if (checkpointDirtyOffset < logStartOffset) { - // Don't bother with the warning if compact and delete are enabled. - if (!isCompactAndDelete(log)) - warn(s"Resetting first dirty offset of ${log.name} to log start offset $logStartOffset " + - s"since the checkpointed offset $checkpointDirtyOffset is invalid.") - logStartOffset - } else if (checkpointDirtyOffset > log.logEndOffset) { - // The dirty offset has gotten ahead of the log end offset. This could happen if there was data - // corruption at the end of the log. We conservatively assume that the full log needs cleaning. - warn(s"The last checkpoint dirty offset for partition ${log.name} is $checkpointDirtyOffset, " + - s"which is larger than the log end offset ${log.logEndOffset}. Resetting to the log start offset $logStartOffset.") - logStartOffset - } else { - checkpointDirtyOffset - } - } - - val minCompactionLagMs = math.max(log.config.compactionLagMs, 0L) - - // find first segment that cannot be cleaned - // neither the active segment, nor segments with any messages closer to the head of the log than the minimum compaction lag time - // may be cleaned - val firstUncleanableDirtyOffset: Long = Seq( - - // we do not clean beyond the first unstable offset - log.firstUnstableOffset, - - // the active segment is always uncleanable - Option(log.activeSegment.baseOffset), - - // the first segment whose largest message timestamp is within a minimum time lag from now - if (minCompactionLagMs > 0) { - // dirty log segments - val dirtyNonActiveSegments = log.nonActiveLogSegmentsFrom(firstDirtyOffset) - dirtyNonActiveSegments.find { s => - val isUncleanable = s.largestTimestamp > now - minCompactionLagMs - debug(s"Checking if log segment may be cleaned: log='${log.name}' segment.baseOffset=${s.baseOffset} " + - s"segment.largestTimestamp=${s.largestTimestamp}; now - compactionLag=${now - minCompactionLagMs}; " + - s"is uncleanable=$isUncleanable") - isUncleanable - }.map(_.baseOffset) - } else None - ).flatten.min - - debug(s"Finding range of cleanable offsets for log=${log.name}. Last clean offset=$lastCleanOffset " + - s"now=$now => firstDirtyOffset=$firstDirtyOffset firstUncleanableOffset=$firstUncleanableDirtyOffset " + - s"activeSegment.baseOffset=${log.activeSegment.baseOffset}") - - (firstDirtyOffset, math.max(firstDirtyOffset, firstUncleanableDirtyOffset)) - } - - /** - * Given the first dirty offset and an uncleanable offset, calculates the total cleanable bytes for this log - * @return the biggest uncleanable offset and the total amount of cleanable bytes - */ - def calculateCleanableBytes(log: Log, firstDirtyOffset: Long, uncleanableOffset: Long): (Long, Long) = { - val firstUncleanableSegment = log.nonActiveLogSegmentsFrom(uncleanableOffset).headOption.getOrElse(log.activeSegment) - val firstUncleanableOffset = firstUncleanableSegment.baseOffset - val cleanableBytes = log.logSegments(firstDirtyOffset, math.max(firstDirtyOffset, firstUncleanableOffset)).map(_.size.toLong).sum - - (firstUncleanableOffset, cleanableBytes) - } - -} From 4b12ebb227c5bcf8fb6b214b7dc494457e09c988 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 11 Feb 2020 19:51:33 -0800 Subject: [PATCH 16/53] Delete .gitignore --- connect/mirror/bin/.gitignore | 2 -- 1 file changed, 2 deletions(-) delete mode 100644 connect/mirror/bin/.gitignore diff --git a/connect/mirror/bin/.gitignore b/connect/mirror/bin/.gitignore deleted file mode 100644 index 7eed456bec8db..0000000000000 --- a/connect/mirror/bin/.gitignore +++ /dev/null @@ -1,2 +0,0 @@ -/main/ -/test/ From 2fc90d305da9ed55cc6b8a35f3155eacc4c47a2b Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 11 Feb 2020 19:52:19 -0800 Subject: [PATCH 17/53] Delete .gitignore --- streams/upgrade-system-tests-23/bin/.gitignore | 1 - 1 file changed, 1 deletion(-) delete mode 100644 streams/upgrade-system-tests-23/bin/.gitignore diff --git a/streams/upgrade-system-tests-23/bin/.gitignore b/streams/upgrade-system-tests-23/bin/.gitignore deleted file mode 100644 index 1933786028b34..0000000000000 --- a/streams/upgrade-system-tests-23/bin/.gitignore +++ /dev/null @@ -1 +0,0 @@ -/test/ From 041e86727f0a9976a0d3e61eb56f9f7015d44e66 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 11 Feb 2020 19:52:36 -0800 Subject: [PATCH 18/53] Delete .gitignore --- generator/bin/.gitignore | 2 -- 1 file changed, 2 deletions(-) delete mode 100644 generator/bin/.gitignore diff --git a/generator/bin/.gitignore b/generator/bin/.gitignore deleted file mode 100644 index 7eed456bec8db..0000000000000 --- a/generator/bin/.gitignore +++ /dev/null @@ -1,2 +0,0 @@ -/main/ -/test/ From 9df0e70ab625df44930d390155230e4420aaa4a5 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 11 Feb 2020 19:52:51 -0800 Subject: [PATCH 19/53] Delete .gitignore --- streams/upgrade-system-tests-22/bin/.gitignore | 1 - 1 file changed, 1 deletion(-) delete mode 100644 streams/upgrade-system-tests-22/bin/.gitignore diff --git a/streams/upgrade-system-tests-22/bin/.gitignore b/streams/upgrade-system-tests-22/bin/.gitignore deleted file mode 100644 index 1933786028b34..0000000000000 --- a/streams/upgrade-system-tests-22/bin/.gitignore +++ /dev/null @@ -1 +0,0 @@ -/test/ From 613d17d2ddcbcffdced6c95f0dfdf2bdeea18018 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 11 Feb 2020 19:53:07 -0800 Subject: [PATCH 20/53] Delete .gitignore --- streams/upgrade-system-tests-21/bin/.gitignore | 1 - 1 file changed, 1 deletion(-) delete mode 100644 streams/upgrade-system-tests-21/bin/.gitignore diff --git a/streams/upgrade-system-tests-21/bin/.gitignore b/streams/upgrade-system-tests-21/bin/.gitignore deleted file mode 100644 index 1933786028b34..0000000000000 --- a/streams/upgrade-system-tests-21/bin/.gitignore +++ /dev/null @@ -1 +0,0 @@ -/test/ From 69b524044457b4e8d819fea2aa3135d1b1aa848f Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 11 Feb 2020 19:53:41 -0800 Subject: [PATCH 21/53] Delete .gitignore --- streams/upgrade-system-tests-0100/bin/.gitignore | 1 - 1 file changed, 1 deletion(-) delete mode 100644 streams/upgrade-system-tests-0100/bin/.gitignore diff --git a/streams/upgrade-system-tests-0100/bin/.gitignore b/streams/upgrade-system-tests-0100/bin/.gitignore deleted file mode 100644 index 1933786028b34..0000000000000 --- a/streams/upgrade-system-tests-0100/bin/.gitignore +++ /dev/null @@ -1 +0,0 @@ -/test/ From ad4ff10622f0fb44d823ea78c79c752076ac9857 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 11 Feb 2020 19:54:00 -0800 Subject: [PATCH 22/53] Delete .gitignore --- streams/upgrade-system-tests-0101/bin/.gitignore | 1 - 1 file changed, 1 deletion(-) delete mode 100644 streams/upgrade-system-tests-0101/bin/.gitignore diff --git a/streams/upgrade-system-tests-0101/bin/.gitignore b/streams/upgrade-system-tests-0101/bin/.gitignore deleted file mode 100644 index 1933786028b34..0000000000000 --- a/streams/upgrade-system-tests-0101/bin/.gitignore +++ /dev/null @@ -1 +0,0 @@ -/test/ From 8c9b50dcc02ff696d4a0a21b3c2c3f2c934ffb7a Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 11 Feb 2020 19:54:14 -0800 Subject: [PATCH 23/53] Delete .gitignore --- streams/upgrade-system-tests-20/bin/.gitignore | 1 - 1 file changed, 1 deletion(-) delete mode 100644 streams/upgrade-system-tests-20/bin/.gitignore diff --git a/streams/upgrade-system-tests-20/bin/.gitignore b/streams/upgrade-system-tests-20/bin/.gitignore deleted file mode 100644 index 1933786028b34..0000000000000 --- a/streams/upgrade-system-tests-20/bin/.gitignore +++ /dev/null @@ -1 +0,0 @@ -/test/ From de5d0a1b3c64f44b293a7447242933acd1de6fe2 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 11 Feb 2020 19:54:29 -0800 Subject: [PATCH 24/53] Delete .gitignore --- streams/upgrade-system-tests-0102/bin/.gitignore | 1 - 1 file changed, 1 deletion(-) delete mode 100644 streams/upgrade-system-tests-0102/bin/.gitignore diff --git a/streams/upgrade-system-tests-0102/bin/.gitignore b/streams/upgrade-system-tests-0102/bin/.gitignore deleted file mode 100644 index 1933786028b34..0000000000000 --- a/streams/upgrade-system-tests-0102/bin/.gitignore +++ /dev/null @@ -1 +0,0 @@ -/test/ From 5b43e43720539ec28174ab9cc92154d09b015c77 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 11 Feb 2020 19:54:45 -0800 Subject: [PATCH 25/53] Delete .gitignore --- streams/upgrade-system-tests-0110/bin/.gitignore | 1 - 1 file changed, 1 deletion(-) delete mode 100644 streams/upgrade-system-tests-0110/bin/.gitignore diff --git a/streams/upgrade-system-tests-0110/bin/.gitignore b/streams/upgrade-system-tests-0110/bin/.gitignore deleted file mode 100644 index 1933786028b34..0000000000000 --- a/streams/upgrade-system-tests-0110/bin/.gitignore +++ /dev/null @@ -1 +0,0 @@ -/test/ From f6652752c65e2b4a9e8589e7219c62aa3169cd13 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 11 Feb 2020 19:55:00 -0800 Subject: [PATCH 26/53] Delete .gitignore --- streams/upgrade-system-tests-10/bin/.gitignore | 1 - 1 file changed, 1 deletion(-) delete mode 100644 streams/upgrade-system-tests-10/bin/.gitignore diff --git a/streams/upgrade-system-tests-10/bin/.gitignore b/streams/upgrade-system-tests-10/bin/.gitignore deleted file mode 100644 index 1933786028b34..0000000000000 --- a/streams/upgrade-system-tests-10/bin/.gitignore +++ /dev/null @@ -1 +0,0 @@ -/test/ From e570f6ca1573e284f1f7b368130a396925957d79 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 11 Feb 2020 19:55:15 -0800 Subject: [PATCH 27/53] Delete .gitignore --- streams/upgrade-system-tests-11/bin/.gitignore | 1 - 1 file changed, 1 deletion(-) delete mode 100644 streams/upgrade-system-tests-11/bin/.gitignore diff --git a/streams/upgrade-system-tests-11/bin/.gitignore b/streams/upgrade-system-tests-11/bin/.gitignore deleted file mode 100644 index 1933786028b34..0000000000000 --- a/streams/upgrade-system-tests-11/bin/.gitignore +++ /dev/null @@ -1 +0,0 @@ -/test/ From 3c96d552d848c2e1367b764071cbc8d4063bdf85 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 11 Feb 2020 19:55:45 -0800 Subject: [PATCH 28/53] Delete .gitignore --- connect/runtime/bin/.gitignore | 2 -- 1 file changed, 2 deletions(-) delete mode 100644 connect/runtime/bin/.gitignore diff --git a/connect/runtime/bin/.gitignore b/connect/runtime/bin/.gitignore deleted file mode 100644 index 7eed456bec8db..0000000000000 --- a/connect/runtime/bin/.gitignore +++ /dev/null @@ -1,2 +0,0 @@ -/main/ -/test/ From a78c563f65bc7fd893dcdf09b179ab86b0197a53 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Tue, 11 Feb 2020 19:58:18 -0800 Subject: [PATCH 29/53] Delete org.apache.kafka.connect.rest.ConnectRestExtension --- ...pache.kafka.connect.rest.ConnectRestExtension | 16 ---------------- 1 file changed, 16 deletions(-) delete mode 100644 connect/basic-auth-extension/bin/main/META-INF/services/org.apache.kafka.connect.rest.ConnectRestExtension diff --git a/connect/basic-auth-extension/bin/main/META-INF/services/org.apache.kafka.connect.rest.ConnectRestExtension b/connect/basic-auth-extension/bin/main/META-INF/services/org.apache.kafka.connect.rest.ConnectRestExtension deleted file mode 100644 index ba7ae5b580d80..0000000000000 --- a/connect/basic-auth-extension/bin/main/META-INF/services/org.apache.kafka.connect.rest.ConnectRestExtension +++ /dev/null @@ -1,16 +0,0 @@ - # 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. - -org.apache.kafka.connect.rest.basic.auth.extension.BasicAuthSecurityRestExtension \ No newline at end of file From e287b49d4d8429cd4381276305efa154e1a56be6 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Tue, 11 Feb 2020 20:03:00 -0800 Subject: [PATCH 30/53] Getting modified --- .../java/org/apache/kafka/common/record/MemoryRecords.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index 77c4d5d01b2e8..5674556218f60 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -195,8 +195,7 @@ private static FilterResult filterTo(TopicPartition partition, Iterable RecordBatch.NO_TIMESTAMP; - if (writeOriginalBatch && (deleteHorizonMs == RecordBatch.NO_TIMESTAMP || deleteHorizonMs == batch.deleteHorizonMs() - || (!containsTombstonesOrMarker && !canControlBatchBeRemoved))) { + if (writeOriginalBatch && (deleteHorizonMs == batch.deleteHorizonMs() || (!containsTombstonesOrMarker && !canControlBatchBeRemoved))) { batch.writeTo(bufferOutputStream); filterResult.updateRetainedBatchMetadata(batch, retainedRecords.size(), false); } else { From ee672476c976fd27a8806759a0db0cb85232b949 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Tue, 11 Feb 2020 20:11:04 -0800 Subject: [PATCH 31/53] Last renaming --- .../org/apache/kafka/common/record/MemoryRecords.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index 5674556218f60..f5295fda1c8a7 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -241,7 +241,7 @@ private static BatchIterationResult iterateOverBatch(RecordBatch batch, FilterResult filterResult, RecordFilter filter, byte batchMagic, - boolean writeOriginalBatch, + boolean recordsFiltered, long maxOffset, List retainedRecords) { boolean containsTombstonesOrMarker = false; @@ -254,7 +254,7 @@ private static BatchIterationResult iterateOverBatch(RecordBatch batch, // Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite // the corrupted batch with correct data. if (!record.hasMagic(batchMagic)) - writeOriginalBatch = false; + recordsFiltered = false; if (record.offset() > maxOffset) maxOffset = record.offset(); @@ -265,10 +265,10 @@ private static BatchIterationResult iterateOverBatch(RecordBatch batch, containsTombstonesOrMarker = true; } } else { - writeOriginalBatch = false; + recordsFiltered = false; } } - return new BatchIterationResult(writeOriginalBatch, containsTombstonesOrMarker, maxOffset); + return new BatchIterationResult(recordsFiltered, containsTombstonesOrMarker, maxOffset); } } From 5bedf9c978630f6e33552c264962f922325a8d47 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Tue, 18 Feb 2020 16:06:19 -0800 Subject: [PATCH 32/53] Addressing some other comments --- .../kafka/common/record/MemoryRecords.java | 18 +++++++++++------- .../kafka/common/record/MemoryRecordsTest.java | 2 +- core/src/main/scala/kafka/log/LogCleaner.scala | 9 +++++++-- 3 files changed, 19 insertions(+), 10 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index f5295fda1c8a7..63704a29e7c68 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -177,7 +177,6 @@ private static FilterResult filterTo(TopicPartition partition, Iterable RecordBatch.NO_TIMESTAMP; - if (writeOriginalBatch && (deleteHorizonMs == batch.deleteHorizonMs() || (!containsTombstonesOrMarker && !canControlBatchBeRemoved))) { + boolean needToSetDeleteHorizon = batch.magic() >= 2 && containsTombstonesOrMarker && !batch.deleteHorizonSet(); + if (writeOriginalBatch && (!needToSetDeleteHorizon || deleteHorizonMs == RecordBatch.NO_TIMESTAMP)) { batch.writeTo(bufferOutputStream); filterResult.updateRetainedBatchMetadata(batch, retainedRecords.size(), false); } else { - MemoryRecordsBuilder builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, deleteHorizonMs); + final MemoryRecordsBuilder builder; + if (containsTombstonesOrMarker) + builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, deleteHorizonMs); + else + builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, RecordBatch.NO_TIMESTAMP); + MemoryRecords records = builder.build(); int filteredBatchSize = records.sizeInBytes(); if (filteredBatchSize > batch.sizeInBytes() && filteredBatchSize > maxRecordBatchSize) @@ -244,7 +248,7 @@ private static BatchIterationResult iterateOverBatch(RecordBatch batch, boolean recordsFiltered, long maxOffset, List retainedRecords) { - boolean containsTombstonesOrMarker = false; + boolean containsTombstonesOrMarker = batch.isControlBatch(); try (final CloseableIterator iterator = batch.streamingIterator(decompressionBufferSupplier)) { while (iterator.hasNext()) { Record record = iterator.next(); @@ -379,10 +383,10 @@ public enum BatchRetention { protected abstract boolean shouldRetainRecord(RecordBatch recordBatch, Record record); /** - * Retrieves the delete horizon ms for a specific batch + * Retrieves the latest delete horizon for given batch */ protected long retrieveDeleteHorizon(RecordBatch recordBatch) { - return -1L; + return RecordBatch.NO_TIMESTAMP; } } diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java index 4677e40e388e1..f0164eb499f6d 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java @@ -274,7 +274,7 @@ protected BatchRetention checkBatchRetention(RecordBatch batch) { @Override protected long retrieveDeleteHorizon(RecordBatch batch) { - return deleteHorizon; // arbitrary value > 1 + return deleteHorizon; // arbitrary value > 0 } }, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); filtered.flip(); diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index da19006017b8b..7bbb194b65ba6 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -501,6 +501,11 @@ private[log] class Cleaner(val id: Int, doClean(cleanable, time.milliseconds(), trackedHorizon = deleteHorizonMs) } + private[log] def doTwoPassClean(cleanable: LogToClean, currentTime: Long, trackedHorizon: Long = -1L) : (Long, CleanerStats) = { + val firstResult = doClean(cleanable, currentTime, trackedHorizon) + doClean(cleanable, currentTime, trackedHorizon) + } + private[log] def doClean(cleanable: LogToClean, currentTime: Long, trackedHorizon: Long = -1L): (Long, CleanerStats) = { info("Beginning cleaning of log %s.".format(cleanable.log.name)) @@ -642,7 +647,7 @@ private[log] class Cleaner(val id: Int, transactionMetadata: CleanedTransactionMetadata, lastRecordsOfActiveProducers: Map[Long, LastRecord], stats: CleanerStats, - currentTime: Long = RecordBatch.NO_TIMESTAMP): Long = { + currentTime: Long): Long = { var latestDeleteHorizon: Long = RecordBatch.NO_TIMESTAMP val logCleanerFilter: RecordFilter = new RecordFilter { @@ -805,7 +810,7 @@ private[log] class Cleaner(val id: Int, batch: RecordBatch, record: Record, stats: CleanerStats, - currentTime: Long = -1L): Boolean = { + currentTime: Long): Boolean = { val pastLatestOffset = record.offset > map.latestOffset if (pastLatestOffset) return true From bd3e18f77e0782dcffc7478a2635cb461e3f9c45 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Tue, 18 Feb 2020 16:08:21 -0800 Subject: [PATCH 33/53] Removing dead code --- core/src/main/scala/kafka/log/LogCleaner.scala | 5 ----- 1 file changed, 5 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 7bbb194b65ba6..39022a9243ea0 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -501,11 +501,6 @@ private[log] class Cleaner(val id: Int, doClean(cleanable, time.milliseconds(), trackedHorizon = deleteHorizonMs) } - private[log] def doTwoPassClean(cleanable: LogToClean, currentTime: Long, trackedHorizon: Long = -1L) : (Long, CleanerStats) = { - val firstResult = doClean(cleanable, currentTime, trackedHorizon) - doClean(cleanable, currentTime, trackedHorizon) - } - private[log] def doClean(cleanable: LogToClean, currentTime: Long, trackedHorizon: Long = -1L): (Long, CleanerStats) = { info("Beginning cleaning of log %s.".format(cleanable.log.name)) From 4515e7d3781074b5dc6e291025eeb0b4539c6f66 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Wed, 19 Feb 2020 10:14:18 -0800 Subject: [PATCH 34/53] Resolving last comments --- .../kafka/common/record/MemoryRecords.java | 53 ++++++++++++--- .../src/main/scala/kafka/log/LogCleaner.scala | 36 +++++----- .../scala/unit/kafka/log/LogCleanerTest.scala | 66 +++++++++---------- 3 files changed, 91 insertions(+), 64 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index 63704a29e7c68..ead920d4876e1 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -165,7 +165,7 @@ private static FilterResult filterTo(TopicPartition partition, Iterable retainedRecords = new ArrayList<>(); final BatchIterationResult iterationResult = iterateOverBatch(batch, decompressionBufferSupplier, filterResult, filter, batchMagic, writeOriginalBatch, maxOffset, retainedRecords); - containsTombstonesOrMarker = iterationResult.containsTombstonesOrMarker(); + containsTombstones = iterationResult.containsTombstonesOrMarker(); writeOriginalBatch = iterationResult.shouldWriteOriginalBatch(); maxOffset = iterationResult.maxOffset(); @@ -193,15 +193,20 @@ private static FilterResult filterTo(TopicPartition partition, Iterable= 2 && containsTombstonesOrMarker && !batch.deleteHorizonSet(); - if (writeOriginalBatch && (!needToSetDeleteHorizon || deleteHorizonMs == RecordBatch.NO_TIMESTAMP)) { + boolean needToSetDeleteHorizon = batch.magic() >= 2 && (containsTombstones || containsEmptyMarker) + && !batch.deleteHorizonSet(); + if (writeOriginalBatch && (!needToSetDeleteHorizon)) { batch.writeTo(bufferOutputStream); filterResult.updateRetainedBatchMetadata(batch, retainedRecords.size(), false); } else { final MemoryRecordsBuilder builder; - if (containsTombstonesOrMarker) + if (needToSetDeleteHorizon) { + long deleteHorizonMs = retrieveDeleteHorizon(batch, filter, containsEmptyMarker); builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, deleteHorizonMs); - else + if (deleteHorizonMs > filterResult.latestDeleteHorizon()) { + filterResult.updateLatestDeleteHorizon(deleteHorizonMs); + } + } else builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, RecordBatch.NO_TIMESTAMP); MemoryRecords records = builder.build(); @@ -240,6 +245,15 @@ private static FilterResult filterTo(TopicPartition partition, Iterable retainedRecords) { - boolean containsTombstonesOrMarker = batch.isControlBatch(); + boolean containsTombstones = batch.isControlBatch(); try (final CloseableIterator iterator = batch.streamingIterator(decompressionBufferSupplier)) { while (iterator.hasNext()) { Record record = iterator.next(); @@ -266,13 +280,13 @@ private static BatchIterationResult iterateOverBatch(RecordBatch batch, retainedRecords.add(record); if (!record.hasValue()) { - containsTombstonesOrMarker = true; + containsTombstones = true; } } else { recordsFiltered = false; } } - return new BatchIterationResult(recordsFiltered, containsTombstonesOrMarker, maxOffset); + return new BatchIterationResult(recordsFiltered, containsTombstones, maxOffset); } } @@ -363,6 +377,9 @@ public int hashCode() { } public static abstract class RecordFilter { + public long currentTime = RecordBatch.NO_TIMESTAMP; + public long tombstoneRetentionMs = RecordBatch.NO_TIMESTAMP; + public enum BatchRetention { DELETE, // Delete the batch without inspecting records RETAIN_EMPTY, // Retain the batch even if it is empty @@ -388,6 +405,13 @@ public enum BatchRetention { protected long retrieveDeleteHorizon(RecordBatch recordBatch) { return RecordBatch.NO_TIMESTAMP; } + + /** + * Checks whether or not the batch can be discarded + */ + protected boolean containsEmptyMarker(RecordBatch batch) { + return false; + } } public static class FilterResult { @@ -401,11 +425,20 @@ public static class FilterResult { private long maxOffset = -1L; private long maxTimestamp = RecordBatch.NO_TIMESTAMP; private long shallowOffsetOfMaxTimestamp = -1L; + private long latestDeleteHorizonMs = RecordBatch.NO_TIMESTAMP; private FilterResult(ByteBuffer outputBuffer) { this.outputBuffer = outputBuffer; } + public void updateLatestDeleteHorizon(long deleteHorizon) { + this.latestDeleteHorizonMs = deleteHorizon; + } + + public long latestDeleteHorizon() { + return latestDeleteHorizonMs; + } + private void updateRetainedBatchMetadata(MutableRecordBatch retainedBatch, int numMessagesInBatch, boolean headerOnly) { int bytesRetained = headerOnly ? DefaultRecordBatch.RECORD_BATCH_OVERHEAD : retainedBatch.sizeInBytes(); updateRetainedBatchMetadata(retainedBatch.maxTimestamp(), retainedBatch.lastOffset(), diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 39022a9243ea0..10e68c1a6daae 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -501,6 +501,12 @@ private[log] class Cleaner(val id: Int, doClean(cleanable, time.milliseconds(), trackedHorizon = deleteHorizonMs) } + private[log] def doTwoPassClean(cleanable: LogToClean, currentTime: Long, trackedHorizon: Long = -1L, + tombstoneRetentionMs: Long = 86400000): (Long, CleanerStats) = { + val firstClean = doClean(cleanable, currentTime, trackedHorizon) + doClean(cleanable, currentTime + tombstoneRetentionMs + 1, trackedHorizon) + } + private[log] def doClean(cleanable: LogToClean, currentTime: Long, trackedHorizon: Long = -1L): (Long, CleanerStats) = { info("Beginning cleaning of log %s.".format(cleanable.log.name)) @@ -657,7 +663,7 @@ private[log] class Cleaner(val id: Int, discardBatchRecords = canDiscardBatch && !retainDeletesAndTxnMarkers } else { discardBatchRecords = canDiscardBatch && - batch.deleteHorizonSet() && batch.deleteHorizonMs() < currentTime + batch.deleteHorizonSet() && batch.deleteHorizonMs() <= currentTime } } else { discardBatchRecords = canDiscardBatch @@ -696,29 +702,13 @@ private[log] class Cleaner(val id: Int, isRecordRetained } - override def retrieveDeleteHorizon(batch: RecordBatch) : Long = { + override def containsEmptyMarker(batch: RecordBatch) : Boolean = { isBatchDiscardable = shouldDiscardBatch(batch, transactionMetadata) - - if (batch.deleteHorizonSet()) { - if (batch.deleteHorizonMs() > latestDeleteHorizon) { - latestDeleteHorizon = batch.deleteHorizonMs() - } - return batch.deleteHorizonMs() // means that we keep the old timestamp stored - } - - // check that the control batch has been emptied of records - // if not, then we do not set a delete horizon until that is true - if (batch.isControlBatch() && !isBatchDiscardable) { - return -1L - } - - val newDeleteHorizon: Long = time.milliseconds() + tombstoneRetentionMs - if (newDeleteHorizon > latestDeleteHorizon) { - latestDeleteHorizon = newDeleteHorizon - } - newDeleteHorizon + isBatchDiscardable } } + logCleanerFilter.currentTime = currentTime + logCleanerFilter.tombstoneRetentionMs = tombstoneRetentionMs var position = 0 while (position < sourceRecords.sizeInBytes) { @@ -731,6 +721,10 @@ private[log] class Cleaner(val id: Int, val records = MemoryRecords.readableRecords(readBuffer) throttler.maybeThrottle(records.sizeInBytes) val result = records.filterTo(topicPartition, logCleanerFilter, writeBuffer, maxLogMessageSize, decompressionBufferSupplier) + if (result.latestDeleteHorizon() > latestDeleteHorizon) { + latestDeleteHorizon = result.latestDeleteHorizon(); + } + stats.readMessages(result.messagesRead, result.bytesRead) stats.recopyMessages(result.messagesRetained, result.bytesRetained) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index d4cfcabadfb86..7f77d222c4db7 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -52,6 +52,8 @@ class LogCleanerTest { val logConfig = LogConfig(logProps) val time = new MockTime() val throttler = new Throttler(desiredRatePerSec = Double.MaxValue, checkIntervalMs = Long.MaxValue, time = time) + val tombstoneRetentionMs = 86400000 + val largeDeleteHorizon = Long.MaxValue - tombstoneRetentionMs - 1 @After def teardown(): Unit = { @@ -78,7 +80,7 @@ class LogCleanerTest { // pretend we have the following keys val keys = immutable.ListSet(1L, 3L, 5L, 7L, 9L) val map = new FakeOffsetMap(Int.MaxValue) - keys.foreach(k => map.put(key(k), Long.MaxValue)) + keys.foreach(k => map.put(key(k), largeDeleteHorizon)) // clean the log val segments = log.logSegments.take(3).toSeq @@ -347,7 +349,7 @@ class LogCleanerTest { log.roll() // cannot remove the marker in this pass because there are still valid records - var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 assertEquals(List(1, 3, 2), LogTest.keysInLog(log)) assertEquals(List(0, 2, 3, 4, 5), offsetsInLog(log)) @@ -356,7 +358,7 @@ class LogCleanerTest { log.roll() // the first cleaning preserves the commit marker (at offset 3) since there were still records for the transaction - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) @@ -366,7 +368,7 @@ class LogCleanerTest { assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) // clean again with large delete horizon and verify the marker is removed - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) assertEquals(List(4, 5, 6, 7, 8), offsetsInLog(log)) } @@ -395,11 +397,11 @@ class LogCleanerTest { log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, isFromClient = false) log.roll() - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) assertEquals(List(2), LogTest.keysInLog(log)) assertEquals(List(1, 3, 4), offsetsInLog(log)) - runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) assertEquals(List(2), LogTest.keysInLog(log)) assertEquals(List(3, 4), offsetsInLog(log)) } @@ -434,14 +436,14 @@ class LogCleanerTest { // first time through the records are removed // Expected State: [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}] - var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 assertEquals(List(2, 3), LogTest.keysInLog(log)) assertEquals(List(4, 5, 6, 7), offsetsInLog(log)) assertEquals(List(1, 3, 4, 5, 6, 7), lastOffsetsPerBatchInLog(log)) // the empty batch remains if cleaned again because it still holds the last sequence // Expected State: [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}] - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 assertEquals(List(2, 3), LogTest.keysInLog(log)) assertEquals(List(4, 5, 6, 7), offsetsInLog(log)) assertEquals(List(1, 3, 4, 5, 6, 7), lastOffsetsPerBatchInLog(log)) @@ -454,13 +456,13 @@ class LogCleanerTest { log.roll() // Expected State: [{Producer1: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) assertEquals(List(4, 5, 6, 7, 8, 9), offsetsInLog(log)) assertEquals(List(1, 4, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log)) // Expected State: [{Producer1: EmptyBatch}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] - dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) assertEquals(List(5, 6, 7, 8, 9), offsetsInLog(log)) assertEquals(List(1, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log)) @@ -484,14 +486,14 @@ class LogCleanerTest { // first time through the control batch is retained as an empty batch // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] - var dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + var dirtyOffset = cleaner.doTwoPassClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 assertEquals(List(2, 3), LogTest.keysInLog(log)) assertEquals(List(1, 2), offsetsInLog(log)) assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) // the empty control batch does not cause an exception when cleaned // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] - dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + dirtyOffset = cleaner.doTwoPassClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 assertEquals(List(2, 3), LogTest.keysInLog(log)) assertEquals(List(1, 2), offsetsInLog(log)) assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) @@ -515,7 +517,7 @@ class LogCleanerTest { log.roll() // Both the record and the marker should remain after cleaning - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) assertEquals(List(0, 1), offsetsInLog(log)) assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) } @@ -540,12 +542,12 @@ class LogCleanerTest { // Both the batch and the marker should remain after cleaning. The batch is retained // because it is the last entry for this producerId. The marker is retained because // there are still batches remaining from this transaction. - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) assertEquals(List(1), offsetsInLog(log)) assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) // The empty batch and the marker is still retained after a second cleaning. - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) assertEquals(List(1), offsetsInLog(log)) assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) } @@ -575,7 +577,7 @@ class LogCleanerTest { assertEquals(List(3, 4, 5), offsetsInLog(log)) // clean again with large delete horizon and verify the marker is removed - runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) assertEquals(List(3), LogTest.keysInLog(log)) assertEquals(List(4, 5), offsetsInLog(log)) } @@ -609,12 +611,12 @@ class LogCleanerTest { // Both transactional batches will be cleaned. The last one will remain in the log // as an empty batch in order to preserve the producer sequence number and epoch - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) assertEquals(List(1, 3, 4, 5), offsetsInLog(log)) assertEquals(List(1, 2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) // On the second round of cleaning, the marker from the first transaction should be removed. - runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) assertEquals(List(3, 4, 5), offsetsInLog(log)) assertEquals(List(2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) } @@ -646,14 +648,14 @@ class LogCleanerTest { assertAbortedTransactionIndexed() // first time through the records are removed - var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + var dirtyOffset = cleaner.doTwoPassClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 assertAbortedTransactionIndexed() assertEquals(List(), LogTest.keysInLog(log)) assertEquals(List(2), offsetsInLog(log)) // abort marker is retained assertEquals(List(1, 2), lastOffsetsPerBatchInLog(log)) // empty batch is retained // the empty batch remains if cleaned again because it still holds the last sequence - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + dirtyOffset = cleaner.doTwoPassClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 assertAbortedTransactionIndexed() assertEquals(List(), LogTest.keysInLog(log)) assertEquals(List(2), offsetsInLog(log)) // abort marker is still retained @@ -663,13 +665,13 @@ class LogCleanerTest { appendProducer(Seq(1)) log.roll() - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 assertAbortedTransactionIndexed() assertEquals(List(1), LogTest.keysInLog(log)) assertEquals(List(2, 3), offsetsInLog(log)) // abort marker is not yet gone because we read the empty batch assertEquals(List(2, 3), lastOffsetsPerBatchInLog(log)) // but we do not preserve the empty batch - dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + dirtyOffset = cleaner.doTwoPassClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 assertEquals(List(1), LogTest.keysInLog(log)) assertEquals(List(3), offsetsInLog(log)) // abort marker is gone assertEquals(List(3), lastOffsetsPerBatchInLog(log)) @@ -700,7 +702,7 @@ class LogCleanerTest { // pretend we have the following keys val keys = immutable.ListSet(1L, 3L, 5L, 7L, 9L) val map = new FakeOffsetMap(Int.MaxValue) - keys.foreach(k => map.put(key(k), Long.MaxValue)) + keys.foreach(k => map.put(key(k), largeDeleteHorizon)) // clean the log val stats = new CleanerStats() @@ -776,7 +778,7 @@ class LogCleanerTest { // pretend we have the following keys val keys = immutable.ListSet(1, 3, 5, 7, 9) val map = new FakeOffsetMap(Int.MaxValue) - keys.foreach(k => map.put(key(k), Long.MaxValue)) + keys.foreach(k => map.put(key(k), largeDeleteHorizon)) (log, map) } @@ -1082,7 +1084,7 @@ class LogCleanerTest { val keys = LogTest.keysInLog(log) val map = new FakeOffsetMap(Int.MaxValue) - keys.foreach(k => map.put(key(k), Long.MaxValue)) + keys.foreach(k => map.put(key(k), largeDeleteHorizon)) intercept[LogCleaningAbortedException] { cleaner.cleanSegments(log, log.logSegments.take(3).toSeq, map, 0L, new CleanerStats(), new CleanedTransactionMetadata) @@ -1276,7 +1278,7 @@ class LogCleanerTest { LogTest.initializeLogDirWithOverflowedSegment(dir) - val log = makeLog(config = config, recoveryPoint = Long.MaxValue) + val log = makeLog(config = config, recoveryPoint = largeDeleteHorizon) val segmentWithOverflow = LogTest.firstOverflowSegment(log).getOrElse { fail("Failed to create log with a segment which has overflowed offsets") } @@ -1289,7 +1291,7 @@ class LogCleanerTest { val offsetMap = new FakeOffsetMap(Int.MaxValue) for (k <- 1 until allKeys.size by 2) { expectedKeysAfterCleaning += allKeys(k - 1) - offsetMap.put(key(allKeys(k)), Long.MaxValue) + offsetMap.put(key(allKeys(k)), largeDeleteHorizon) } // Try to clean segment with offset overflow. This will trigger log split and the cleaning itself must abort. @@ -1341,7 +1343,7 @@ class LogCleanerTest { // pretend we have odd-numbered keys val offsetMap = new FakeOffsetMap(Int.MaxValue) for (k <- 1 until messageCount by 2) - offsetMap.put(key(k), Long.MaxValue) + offsetMap.put(key(k), largeDeleteHorizon) // clean the log cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats(), @@ -1381,7 +1383,7 @@ class LogCleanerTest { messageCount += 1 } for (k <- 1 until messageCount by 2) - offsetMap.put(key(k), Long.MaxValue) + offsetMap.put(key(k), largeDeleteHorizon) cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats(), new CleanedTransactionMetadata) // clear scheduler so that async deletes don't run @@ -1399,7 +1401,7 @@ class LogCleanerTest { messageCount += 1 } for (k <- 1 until messageCount by 2) - offsetMap.put(key(k), Long.MaxValue) + offsetMap.put(key(k), largeDeleteHorizon) cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats(), new CleanedTransactionMetadata) // clear scheduler so that async deletes don't run @@ -1677,9 +1679,7 @@ class LogCleanerTest { } private def runTwoPassClean(cleaner: Cleaner, logToClean: LogToClean, currentTime: Long) : Long = { - var offsetReturned: Long = cleaner.doClean(logToClean, currentTime = currentTime)._1 - offsetReturned = cleaner.doClean(logToClean, currentTime = currentTime)._1 - return offsetReturned + cleaner.doTwoPassClean(logToClean, currentTime = currentTime)._1 } } From 60d72d031fea88b7b0d653f1e5bb64dd4ba89771 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Wed, 19 Feb 2020 10:20:59 -0800 Subject: [PATCH 35/53] One liner --- .../main/java/org/apache/kafka/common/record/MemoryRecords.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index ead920d4876e1..d13b327a73e2e 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -262,7 +262,7 @@ private static BatchIterationResult iterateOverBatch(RecordBatch batch, boolean recordsFiltered, long maxOffset, List retainedRecords) { - boolean containsTombstones = batch.isControlBatch(); + boolean containsTombstones = false; try (final CloseableIterator iterator = batch.streamingIterator(decompressionBufferSupplier)) { while (iterator.hasNext()) { Record record = iterator.next(); From 92530fe44a91e828ae67bd9f4c51c080d997518a Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Wed, 19 Feb 2020 10:33:00 -0800 Subject: [PATCH 36/53] Fixing broken test --- .../apache/kafka/common/record/MemoryRecords.java | 7 ------- .../kafka/common/record/MemoryRecordsTest.java | 12 ++++++++---- 2 files changed, 8 insertions(+), 11 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index d13b327a73e2e..ed774e1d380b7 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -399,13 +399,6 @@ public enum BatchRetention { */ protected abstract boolean shouldRetainRecord(RecordBatch recordBatch, Record record); - /** - * Retrieves the latest delete horizon for given batch - */ - protected long retrieveDeleteHorizon(RecordBatch recordBatch) { - return RecordBatch.NO_TIMESTAMP; - } - /** * Checks whether or not the batch can be discarded */ diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java index f0164eb499f6d..7cb4a586ff4f4 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.header.internals.RecordHeaders; +import org.apache.kafka.common.record.MemoryRecords.RecordFilter; import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.test.TestUtils; @@ -261,7 +262,7 @@ public void testFirstTimestampToDeleteHorizonConversion() { ByteBuffer filtered = ByteBuffer.allocate(2048); final long deleteHorizon = Integer.MAX_VALUE / 2; - builder.build().filterTo(new TopicPartition("random", 0), new MemoryRecords.RecordFilter() { + final RecordFilter recordFilter = new MemoryRecords.RecordFilter() { @Override protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { return true; @@ -273,10 +274,13 @@ protected BatchRetention checkBatchRetention(RecordBatch batch) { } @Override - protected long retrieveDeleteHorizon(RecordBatch batch) { - return deleteHorizon; // arbitrary value > 0 + protected boolean containsEmptyMarker(RecordBatch batch) { + return true; } - }, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); + }; + recordFilter.currentTime = deleteHorizon; + recordFilter.tombstoneRetentionMs = 0L; + builder.build().filterTo(new TopicPartition("random", 0), recordFilter, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); From 8baa41690df37927cdaf74c1fa58375a9e6f9774 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Fri, 21 Feb 2020 14:02:10 -0800 Subject: [PATCH 37/53] Adding changed files --- .../kafka/common/record/MemoryRecords.java | 49 +++++++++++-------- .../consumer/internals/FetcherTest.java | 6 +-- .../common/record/MemoryRecordsTest.java | 47 +++++++++--------- .../src/main/scala/kafka/log/LogCleaner.scala | 27 +++++----- .../test/scala/unit/kafka/log/LogTest.scala | 15 +++--- 5 files changed, 73 insertions(+), 71 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index ed774e1d380b7..b7d5aa2d955e6 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention; +import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetentionAndEmptyMarker; import org.apache.kafka.common.utils.AbstractIterator; import org.apache.kafka.common.utils.ByteBufferOutputStream; import org.apache.kafka.common.utils.CloseableIterator; @@ -165,8 +166,9 @@ private static FilterResult filterTo(TopicPartition partition, Iterable data() { } private static class RetainNonNullKeysFilter extends MemoryRecords.RecordFilter { + public RetainNonNullKeysFilter() { + super(0, 0); + } + @Override - protected BatchRetention checkBatchRetention(RecordBatch batch) { - return BatchRetention.DELETE_EMPTY; + protected BatchRetentionAndEmptyMarker checkBatchRetention(RecordBatch batch) { + return new BatchRetentionAndEmptyMarker(BatchRetention.DELETE_EMPTY, false); } @Override diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 10e68c1a6daae..f96ebdc961518 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -563,6 +563,7 @@ private[log] class Cleaner(val id: Int, // create a new segment with a suffix appended to the name of the log and indexes val cleaned = LogCleaner.createNewCleanedSegment(log, segments.head.baseOffset) transactionMetadata.cleanedIndex = Some(cleaned.txnIndex) + log.latestDeleteHorizon = RecordBatch.NO_TIMESTAMP try { // clean segments into the new destination segment @@ -589,8 +590,6 @@ private[log] class Cleaner(val id: Int, log.config.maxMessageSize, transactionMetadata, lastOffsetOfActiveProducers, stats, currentTime = currentTime) if (log.latestDeleteHorizon < latestDeleteHorizon) { log.latestDeleteHorizon = latestDeleteHorizon - } else if (log.latestDeleteHorizon < currentTime) { - log.latestDeleteHorizon = RecordBatch.NO_TIMESTAMP } } catch { case e: LogSegmentOffsetOverflowException => @@ -651,12 +650,13 @@ private[log] class Cleaner(val id: Int, currentTime: Long): Long = { var latestDeleteHorizon: Long = RecordBatch.NO_TIMESTAMP - val logCleanerFilter: RecordFilter = new RecordFilter { + val logCleanerFilter: RecordFilter = new RecordFilter (currentTime, tombstoneRetentionMs) { var discardBatchRecords: Boolean = _ - var isBatchDiscardable: Boolean = _ - override def checkBatchRetention(batch: RecordBatch): BatchRetention = { - val canDiscardBatch = isBatchDiscardable + override def checkBatchRetention(batch: RecordBatch): BatchRetentionAndEmptyMarker = { + // we piggy-back on the tombstone retention logic to delay deletion of transaction markers. + // note that we will never delete a marker until all the records from that transaction are removed. + val canDiscardBatch = shouldDiscardBatch(batch, transactionMetadata) if (batch.isControlBatch) { if (batch.magic() < 2) { @@ -684,12 +684,14 @@ private[log] class Cleaner(val id: Int, } } + var batchRetention: BatchRetention = BatchRetention.RETAIN_EMPTY if (batch.hasProducerId && isBatchLastRecordOfProducer) - BatchRetention.RETAIN_EMPTY + batchRetention = BatchRetention.RETAIN_EMPTY else if (discardBatchRecords) - BatchRetention.DELETE + batchRetention = BatchRetention.DELETE else - BatchRetention.DELETE_EMPTY + batchRetention = BatchRetention.DELETE_EMPTY + new BatchRetentionAndEmptyMarker(batchRetention, canDiscardBatch) } override def shouldRetainRecord(batch: RecordBatch, record: Record): Boolean = { @@ -701,14 +703,7 @@ private[log] class Cleaner(val id: Int, isRecordRetained = Cleaner.this.shouldRetainRecord(map, retainDeletesAndTxnMarkers, batch, record, stats, currentTime = currentTime) isRecordRetained } - - override def containsEmptyMarker(batch: RecordBatch) : Boolean = { - isBatchDiscardable = shouldDiscardBatch(batch, transactionMetadata) - isBatchDiscardable - } } - logCleanerFilter.currentTime = currentTime - logCleanerFilter.tombstoneRetentionMs = tombstoneRetentionMs var position = 0 while (position < sourceRecords.sizeInBytes) { diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index b2c486b862cdc..817f243d03db2 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -966,8 +966,9 @@ class LogTest { records.batches.asScala.foreach(_.setPartitionLeaderEpoch(0)) val filtered = ByteBuffer.allocate(2048) - records.filterTo(new TopicPartition("foo", 0), new RecordFilter { - override def checkBatchRetention(batch: RecordBatch): BatchRetention = RecordFilter.BatchRetention.DELETE_EMPTY + records.filterTo(new TopicPartition("foo", 0), new RecordFilter(0, 0) { + override def checkBatchRetention(batch: RecordBatch): BatchRetentionAndEmptyMarker = + new BatchRetentionAndEmptyMarker(RecordFilter.BatchRetention.DELETE_EMPTY, false) override def shouldRetainRecord(recordBatch: RecordBatch, record: Record): Boolean = !record.hasKey }, filtered, Int.MaxValue, BufferSupplier.NO_CACHING) filtered.flip() @@ -1007,8 +1008,9 @@ class LogTest { records.batches.asScala.foreach(_.setPartitionLeaderEpoch(0)) val filtered = ByteBuffer.allocate(2048) - records.filterTo(new TopicPartition("foo", 0), new RecordFilter { - override def checkBatchRetention(batch: RecordBatch): BatchRetention = RecordFilter.BatchRetention.RETAIN_EMPTY + records.filterTo(new TopicPartition("foo", 0), new RecordFilter(0, 0) { + override def checkBatchRetention(batch: RecordBatch): BatchRetentionAndEmptyMarker = + new BatchRetentionAndEmptyMarker(RecordFilter.BatchRetention.RETAIN_EMPTY, true) override def shouldRetainRecord(recordBatch: RecordBatch, record: Record): Boolean = false }, filtered, Int.MaxValue, BufferSupplier.NO_CACHING) filtered.flip() @@ -1050,8 +1052,9 @@ class LogTest { records.batches.asScala.foreach(_.setPartitionLeaderEpoch(0)) val filtered = ByteBuffer.allocate(2048) - records.filterTo(new TopicPartition("foo", 0), new RecordFilter { - override def checkBatchRetention(batch: RecordBatch): BatchRetention = RecordFilter.BatchRetention.DELETE_EMPTY + records.filterTo(new TopicPartition("foo", 0), new RecordFilter(0, 0) { + override def checkBatchRetention(batch: RecordBatch): BatchRetentionAndEmptyMarker = + new BatchRetentionAndEmptyMarker(RecordFilter.BatchRetention.DELETE_EMPTY, false) override def shouldRetainRecord(recordBatch: RecordBatch, record: Record): Boolean = !record.hasKey }, filtered, Int.MaxValue, BufferSupplier.NO_CACHING) filtered.flip() From 6d011edc303f09c321091b806ea87bde96fc3704 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Tue, 25 Feb 2020 20:43:18 -0800 Subject: [PATCH 38/53] Cleaning up messy code --- .../kafka/common/record/MemoryRecords.java | 38 +++++------- .../consumer/internals/FetcherTest.java | 4 +- .../common/record/MemoryRecordsTest.java | 26 ++++----- .../src/main/scala/kafka/log/LogCleaner.scala | 58 +++++++++---------- .../scala/unit/kafka/log/LogCleanerTest.scala | 16 ++--- .../test/scala/unit/kafka/log/LogTest.scala | 13 ++--- 6 files changed, 71 insertions(+), 84 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index b7d5aa2d955e6..d546955e4529c 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention; -import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetentionAndEmptyMarker; +import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetentionResult; import org.apache.kafka.common.utils.AbstractIterator; import org.apache.kafka.common.utils.ByteBufferOutputStream; import org.apache.kafka.common.utils.CloseableIterator; @@ -153,7 +153,7 @@ public FilterResult filterTo(TopicPartition partition, RecordFilter filter, Byte /** * Note: This method is also used to convert the first timestamp of the batch (which is usually the timestamp of the first record) - * to the delete horizon of the tombstones which are present in the batch. + * to the delete horizon of the tombstones or txn markers which are present in the batch. */ private static FilterResult filterTo(TopicPartition partition, Iterable batches, RecordFilter filter, ByteBuffer destinationBuffer, int maxRecordBatchSize, @@ -162,12 +162,9 @@ private static FilterResult filterTo(TopicPartition partition, Iterable retainedRecords = new ArrayList<>(); @@ -195,15 +191,17 @@ private static FilterResult filterTo(TopicPartition partition, Iterable= 2 && (containsTombstones || containsEmptyMarker) + boolean needToSetDeleteHorizon = batch.magic() >= 2 && (containsTombstones || containsMarkerForEmptyTxn) && !batch.deleteHorizonSet(); if (writeOriginalBatch && (!needToSetDeleteHorizon)) { + if (batch.deleteHorizonMs() > filterResult.latestDeleteHorizon()) + filterResult.updateLatestDeleteHorizon(batch.deleteHorizonMs()); batch.writeTo(bufferOutputStream); filterResult.updateRetainedBatchMetadata(batch, retainedRecords.size(), false); } else { final MemoryRecordsBuilder builder; if (needToSetDeleteHorizon) { - long deleteHorizonMs = retrieveDeleteHorizon(batch, filter, containsEmptyMarker); + long deleteHorizonMs = filter.currentTime + filter.tombstoneRetentionMs; builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, deleteHorizonMs); if (deleteHorizonMs > filterResult.latestDeleteHorizon()) { filterResult.updateLatestDeleteHorizon(deleteHorizonMs); @@ -247,12 +245,6 @@ private static FilterResult filterTo(TopicPartition partition, Iterable 0L case Some(seg) => seg.lastModified - cleanable.log.config.deleteRetentionMs } - doClean(cleanable, time.milliseconds(), trackedHorizon = deleteHorizonMs) + doClean(cleanable, time.milliseconds(), legacyDeleteHorizonMs = deleteHorizonMs) } - private[log] def doTwoPassClean(cleanable: LogToClean, currentTime: Long, trackedHorizon: Long = -1L, - tombstoneRetentionMs: Long = 86400000): (Long, CleanerStats) = { - val firstClean = doClean(cleanable, currentTime, trackedHorizon) - doClean(cleanable, currentTime + tombstoneRetentionMs + 1, trackedHorizon) - } - - private[log] def doClean(cleanable: LogToClean, currentTime: Long, trackedHorizon: Long = -1L): (Long, CleanerStats) = { + private[log] def doClean(cleanable: LogToClean, currentTime: Long, legacyDeleteHorizonMs: Long = -1L): (Long, CleanerStats) = { info("Beginning cleaning of log %s.".format(cleanable.log.name)) val log = cleanable.log + log.latestDeleteHorizon = RecordBatch.NO_TIMESTAMP + val stats = new CleanerStats() // build the offset map @@ -525,13 +521,13 @@ private[log] class Cleaner(val id: Int, val cleanableHorizonMs = log.logSegments(0, cleanable.firstUncleanableOffset).lastOption.map(_.lastModified).getOrElse(0L) // group the segments and clean the groups - info("Cleaning log %s (cleaning prior to %s, discarding tombstones prior to %s)...".format(log.name, new Date(cleanableHorizonMs), new Date(trackedHorizon))) + info("Cleaning log %s (cleaning prior to %s, discarding legacy tombstones prior to %s)...".format(log.name, new Date(cleanableHorizonMs), new Date(legacyDeleteHorizonMs))) val transactionMetadata = new CleanedTransactionMetadata val groupedSegments = groupSegmentsBySize(log.logSegments(0, endOffset), log.config.segmentSize, log.config.maxIndexSize, cleanable.firstUncleanableOffset) for (group <- groupedSegments) - cleanSegments(log, group, offsetMap, currentTime, stats, transactionMetadata, trackedHorizon = trackedHorizon) + cleanSegments(log, group, offsetMap, currentTime, stats, transactionMetadata, legacyDeleteHorizonMs = legacyDeleteHorizonMs) // record buffer utilization stats.bufferUtilization = offsetMap.utilization @@ -551,7 +547,7 @@ private[log] class Cleaner(val id: Int, * @param stats Collector for cleaning statistics * @param transactionMetadata State of ongoing transactions which is carried between the cleaning * of the grouped segments - * @param trackedHorizon The delete horizon used for tombstones whose version is less than 2 + * @param legacyDeleteHorizonMs The delete horizon used for tombstones whose version is less than 2 */ private[log] def cleanSegments(log: Log, segments: Seq[LogSegment], @@ -559,11 +555,10 @@ private[log] class Cleaner(val id: Int, currentTime: Long, stats: CleanerStats, transactionMetadata: CleanedTransactionMetadata, - trackedHorizon: Long = -1L): Unit = { + legacyDeleteHorizonMs: Long = -1L): Unit = { // create a new segment with a suffix appended to the name of the log and indexes val cleaned = LogCleaner.createNewCleanedSegment(log, segments.head.baseOffset) transactionMetadata.cleanedIndex = Some(cleaned.txnIndex) - log.latestDeleteHorizon = RecordBatch.NO_TIMESTAMP try { // clean segments into the new destination segment @@ -580,9 +575,9 @@ private[log] class Cleaner(val id: Int, val abortedTransactions = log.collectAbortedTransactions(startOffset, upperBoundOffset) transactionMetadata.addAbortedTransactions(abortedTransactions) - val retainDeletesAndTxnMarkers = currentSegment.lastModified > trackedHorizon + val retainDeletesAndTxnMarkers = currentSegment.lastModified > legacyDeleteHorizonMs info(s"Cleaning $currentSegment in log ${log.name} into ${cleaned.baseOffset} " + - s"with deletion horizon $trackedHorizon, " + + s"with legacy deletion horizon $legacyDeleteHorizonMs, " + s"${if(retainDeletesAndTxnMarkers) "retaining" else "discarding"} deletes.") try { @@ -650,10 +645,10 @@ private[log] class Cleaner(val id: Int, currentTime: Long): Long = { var latestDeleteHorizon: Long = RecordBatch.NO_TIMESTAMP - val logCleanerFilter: RecordFilter = new RecordFilter (currentTime, tombstoneRetentionMs) { + val logCleanerFilter: RecordFilter = new RecordFilter(currentTime, tombstoneRetentionMs) { var discardBatchRecords: Boolean = _ - override def checkBatchRetention(batch: RecordBatch): BatchRetentionAndEmptyMarker = { + override def checkBatchRetention(batch: RecordBatch): BatchRetentionResult = { // we piggy-back on the tombstone retention logic to delay deletion of transaction markers. // note that we will never delete a marker until all the records from that transaction are removed. val canDiscardBatch = shouldDiscardBatch(batch, transactionMetadata) @@ -684,14 +679,14 @@ private[log] class Cleaner(val id: Int, } } - var batchRetention: BatchRetention = BatchRetention.RETAIN_EMPTY - if (batch.hasProducerId && isBatchLastRecordOfProducer) - batchRetention = BatchRetention.RETAIN_EMPTY - else if (discardBatchRecords) - batchRetention = BatchRetention.DELETE - else - batchRetention = BatchRetention.DELETE_EMPTY - new BatchRetentionAndEmptyMarker(batchRetention, canDiscardBatch) + val batchRetention: BatchRetention = + if (batch.hasProducerId && isBatchLastRecordOfProducer) + BatchRetention.RETAIN_EMPTY + else if (discardBatchRecords) + BatchRetention.DELETE + else + BatchRetention.DELETE_EMPTY + new BatchRetentionResult(batchRetention, canDiscardBatch) } override def shouldRetainRecord(batch: RecordBatch, record: Record): Boolean = { @@ -790,7 +785,7 @@ private[log] class Cleaner(val id: Int, } private def shouldRetainRecord(map: kafka.log.OffsetMap, - retainDeletes: Boolean, + retainDeletesForLegacyRecords: Boolean, batch: RecordBatch, record: Record, stats: CleanerStats, @@ -809,12 +804,11 @@ private[log] class Cleaner(val id: Int, */ val latestOffsetForKey = record.offset() >= foundOffset val supportDeleteHorizon = batch.magic() >= RecordBatch.MAGIC_VALUE_V2 - var shouldRetainDeletes = true - if (supportDeleteHorizon) - shouldRetainDeletes = (batch.deleteHorizonSet() && currentTime < batch.deleteHorizonMs()) || - !batch.deleteHorizonSet() - else - shouldRetainDeletes = retainDeletes + val shouldRetainDeletes = + if (supportDeleteHorizon) + !batch.deleteHorizonSet() || currentTime < batch.deleteHorizonMs() + else + retainDeletesForLegacyRecords val isRetainedValue = record.hasValue || shouldRetainDeletes latestOffsetForKey && isRetainedValue } else { diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 7f77d222c4db7..006669e06d98d 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -486,14 +486,14 @@ class LogCleanerTest { // first time through the control batch is retained as an empty batch // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] - var dirtyOffset = cleaner.doTwoPassClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 + var dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) assertEquals(List(2, 3), LogTest.keysInLog(log)) assertEquals(List(1, 2), offsetsInLog(log)) assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) // the empty control batch does not cause an exception when cleaned // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] - dirtyOffset = cleaner.doTwoPassClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 + dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue) assertEquals(List(2, 3), LogTest.keysInLog(log)) assertEquals(List(1, 2), offsetsInLog(log)) assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) @@ -648,14 +648,14 @@ class LogCleanerTest { assertAbortedTransactionIndexed() // first time through the records are removed - var dirtyOffset = cleaner.doTwoPassClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 + var dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) assertAbortedTransactionIndexed() assertEquals(List(), LogTest.keysInLog(log)) assertEquals(List(2), offsetsInLog(log)) // abort marker is retained assertEquals(List(1, 2), lastOffsetsPerBatchInLog(log)) // empty batch is retained // the empty batch remains if cleaned again because it still holds the last sequence - dirtyOffset = cleaner.doTwoPassClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 + dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) assertAbortedTransactionIndexed() assertEquals(List(), LogTest.keysInLog(log)) assertEquals(List(2), offsetsInLog(log)) // abort marker is still retained @@ -671,7 +671,7 @@ class LogCleanerTest { assertEquals(List(2, 3), offsetsInLog(log)) // abort marker is not yet gone because we read the empty batch assertEquals(List(2, 3), lastOffsetsPerBatchInLog(log)) // but we do not preserve the empty batch - dirtyOffset = cleaner.doTwoPassClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 + dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) assertEquals(List(1), LogTest.keysInLog(log)) assertEquals(List(3), offsetsInLog(log)) // abort marker is gone assertEquals(List(3), lastOffsetsPerBatchInLog(log)) @@ -1678,8 +1678,10 @@ class LogCleanerTest { LogTest.recoverAndCheck(dir, config, expectedKeys, new BrokerTopicStats(), time, time.scheduler) } - private def runTwoPassClean(cleaner: Cleaner, logToClean: LogToClean, currentTime: Long) : Long = { - cleaner.doTwoPassClean(logToClean, currentTime = currentTime)._1 + private def runTwoPassClean(cleaner: Cleaner, logToClean: LogToClean, currentTime: Long, + legacyDeleteHorizonMs: Long = -1L, tombstoneRetentionMs: Long = 86400000) : Long = { + cleaner.doClean(logToClean, currentTime, legacyDeleteHorizonMs) + cleaner.doClean(logToClean, currentTime + tombstoneRetentionMs + 1, legacyDeleteHorizonMs)._1 } } diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 817f243d03db2..16e337fe96de9 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -35,7 +35,6 @@ import org.apache.kafka.common.{InvalidRecordException, KafkaException, TopicPar import org.apache.kafka.common.errors._ import org.apache.kafka.common.record.FileRecords.TimestampAndOffset import org.apache.kafka.common.record.MemoryRecords.RecordFilter -import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction import org.apache.kafka.common.requests.{ListOffsetRequest, ListOffsetResponse} @@ -967,8 +966,8 @@ class LogTest { val filtered = ByteBuffer.allocate(2048) records.filterTo(new TopicPartition("foo", 0), new RecordFilter(0, 0) { - override def checkBatchRetention(batch: RecordBatch): BatchRetentionAndEmptyMarker = - new BatchRetentionAndEmptyMarker(RecordFilter.BatchRetention.DELETE_EMPTY, false) + override def checkBatchRetention(batch: RecordBatch): BatchRetentionResult = + new BatchRetentionResult(RecordFilter.BatchRetention.DELETE_EMPTY, false) override def shouldRetainRecord(recordBatch: RecordBatch, record: Record): Boolean = !record.hasKey }, filtered, Int.MaxValue, BufferSupplier.NO_CACHING) filtered.flip() @@ -1009,8 +1008,8 @@ class LogTest { val filtered = ByteBuffer.allocate(2048) records.filterTo(new TopicPartition("foo", 0), new RecordFilter(0, 0) { - override def checkBatchRetention(batch: RecordBatch): BatchRetentionAndEmptyMarker = - new BatchRetentionAndEmptyMarker(RecordFilter.BatchRetention.RETAIN_EMPTY, true) + override def checkBatchRetention(batch: RecordBatch): BatchRetentionResult = + new BatchRetentionResult(RecordFilter.BatchRetention.RETAIN_EMPTY, true) override def shouldRetainRecord(recordBatch: RecordBatch, record: Record): Boolean = false }, filtered, Int.MaxValue, BufferSupplier.NO_CACHING) filtered.flip() @@ -1053,8 +1052,8 @@ class LogTest { val filtered = ByteBuffer.allocate(2048) records.filterTo(new TopicPartition("foo", 0), new RecordFilter(0, 0) { - override def checkBatchRetention(batch: RecordBatch): BatchRetentionAndEmptyMarker = - new BatchRetentionAndEmptyMarker(RecordFilter.BatchRetention.DELETE_EMPTY, false) + override def checkBatchRetention(batch: RecordBatch): BatchRetentionResult = + new BatchRetentionResult(RecordFilter.BatchRetention.DELETE_EMPTY, false) override def shouldRetainRecord(recordBatch: RecordBatch, record: Record): Boolean = !record.hasKey }, filtered, Int.MaxValue, BufferSupplier.NO_CACHING) filtered.flip() From 6cc19fe94709442989fc9eae180777b0cca75ffc Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Wed, 26 Feb 2020 16:58:43 -0800 Subject: [PATCH 39/53] Adding some test modifications --- .../kafka/common/record/MemoryRecords.java | 13 +++++--- .../common/record/MemoryRecordsTest.java | 6 ++-- .../src/main/scala/kafka/log/LogCleaner.scala | 14 ++++---- .../scala/kafka/log/LogCleanerManager.scala | 3 +- .../kafka/log/LogCleanerIntegrationTest.scala | 32 ++++--------------- .../scala/unit/kafka/log/LogCleanerTest.scala | 11 +++++-- 6 files changed, 36 insertions(+), 43 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index d546955e4529c..89a9d8fef31d1 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -163,9 +163,9 @@ private static FilterResult filterTo(TopicPartition partition, Iterable filterResult.latestDeleteHorizon()) { filterResult.updateLatestDeleteHorizon(deleteHorizonMs); } - } else - builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, RecordBatch.NO_TIMESTAMP); + } else { + builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, batch.deleteHorizonMs()); + if (batch.deleteHorizonMs() > filterResult.latestDeleteHorizon()) + filterResult.updateLatestDeleteHorizon(batch.deleteHorizonMs()); + } MemoryRecords records = builder.build(); int filteredBatchSize = records.sizeInBytes(); diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java index bb312e2e29cd6..db6cb67e1d777 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java @@ -309,7 +309,7 @@ public void testFilterToEmptyBatchRetention() { @Override protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { // retain all batches - return new BatchRetentionResult(BatchRetention.RETAIN_EMPTY, true); + return new BatchRetentionResult(BatchRetention.RETAIN_EMPTY, false); } @Override @@ -420,7 +420,7 @@ public void testEmptyBatchDeletion() { new MemoryRecords.RecordFilter(0, 0) { @Override protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { - return new BatchRetentionResult(deleteRetention, true); + return new BatchRetentionResult(deleteRetention, false); } @Override @@ -511,7 +511,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { // discard the second and fourth batches if (batch.lastOffset() == 2L || batch.lastOffset() == 6L) return new BatchRetentionResult(BatchRetention.DELETE, false); - return new BatchRetentionResult(BatchRetention.DELETE_EMPTY, true); + return new BatchRetentionResult(BatchRetention.DELETE_EMPTY, false); } @Override diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 450d9fa9284a1..f81ad78d10169 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -575,13 +575,13 @@ private[log] class Cleaner(val id: Int, val abortedTransactions = log.collectAbortedTransactions(startOffset, upperBoundOffset) transactionMetadata.addAbortedTransactions(abortedTransactions) - val retainDeletesAndTxnMarkers = currentSegment.lastModified > legacyDeleteHorizonMs + val retainLegacyDeletesAndTxnMarkers = currentSegment.lastModified > legacyDeleteHorizonMs info(s"Cleaning $currentSegment in log ${log.name} into ${cleaned.baseOffset} " + s"with legacy deletion horizon $legacyDeleteHorizonMs, " + - s"${if(retainDeletesAndTxnMarkers) "retaining" else "discarding"} deletes.") + s"${if(retainLegacyDeletesAndTxnMarkers) "retaining" else "discarding"} deletes.") try { - val latestDeleteHorizon: Long = cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletesAndTxnMarkers, log.config.deleteRetentionMs, + val latestDeleteHorizon: Long = cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainLegacyDeletesAndTxnMarkers, log.config.deleteRetentionMs, log.config.maxMessageSize, transactionMetadata, lastOffsetOfActiveProducers, stats, currentTime = currentTime) if (log.latestDeleteHorizon < latestDeleteHorizon) { log.latestDeleteHorizon = latestDeleteHorizon @@ -626,7 +626,7 @@ private[log] class Cleaner(val id: Int, * @param sourceRecords The dirty log segment * @param dest The cleaned log segment * @param map The key=>offset mapping - * @param retainDeletesAndTxnMarkers Should tombstones (lower than version 2) and markers be retained while cleaning this segment + * @param retainLegacyDeletesAndTxnMarkers Should tombstones (lower than version 2) and markers be retained while cleaning this segment * @param tombstoneRetentionMs Defines how long a tombstone should be kept as defined by log configuration * @param maxLogMessageSize The maximum message size of the corresponding topic * @param stats Collector for cleaning statistics @@ -636,7 +636,7 @@ private[log] class Cleaner(val id: Int, sourceRecords: FileRecords, dest: LogSegment, map: OffsetMap, - retainDeletesAndTxnMarkers: Boolean, + retainLegacyDeletesAndTxnMarkers: Boolean, tombstoneRetentionMs: Long, maxLogMessageSize: Int, transactionMetadata: CleanedTransactionMetadata, @@ -655,7 +655,7 @@ private[log] class Cleaner(val id: Int, if (batch.isControlBatch) { if (batch.magic() < 2) { - discardBatchRecords = canDiscardBatch && !retainDeletesAndTxnMarkers + discardBatchRecords = canDiscardBatch && !retainLegacyDeletesAndTxnMarkers } else { discardBatchRecords = canDiscardBatch && batch.deleteHorizonSet() && batch.deleteHorizonMs() <= currentTime @@ -695,7 +695,7 @@ private[log] class Cleaner(val id: Int, // The batch is only retained to preserve producer sequence information; the records can be removed isRecordRetained = false else - isRecordRetained = Cleaner.this.shouldRetainRecord(map, retainDeletesAndTxnMarkers, batch, record, stats, currentTime = currentTime) + isRecordRetained = Cleaner.this.shouldRetainRecord(map, retainLegacyDeletesAndTxnMarkers, batch, record, stats, currentTime = currentTime) isRecordRetained } } diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala index b0c3bbccf032f..12fdb0893fb55 100755 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -31,6 +31,7 @@ import kafka.utils.{Logging, Pool} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.utils.Time import org.apache.kafka.common.errors.KafkaStorageException +import org.apache.kafka.common.record.RecordBatch import scala.collection.{Iterable, Seq, mutable} @@ -211,7 +212,7 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], // therefore, we should take advantage of this fact and remove tombstones if we can // under the condition that the log's latest delete horizon is less than the current time // tracked - ltc.log.latestDeleteHorizon != -1L && ltc.log.latestDeleteHorizon <= time.milliseconds() + ltc.log.latestDeleteHorizon != RecordBatch.NO_TIMESTAMP && ltc.log.latestDeleteHorizon <= time.milliseconds() } if (!logsWithTombstonesExpired.isEmpty) { val filthiest = logsWithTombstonesExpired.max diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 4866e66517269..fda3a59dc0a02 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -180,7 +180,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K } @Test - def testTombstoneCleanWithLowThoroughput() : Unit = { + def testTombstoneCleanWithLowThroughput() : Unit = { val tombstoneRetentionMs = 1000 // this is in milliseconds -> 1 second val topicPartitions = Array(new TopicPartition("log-partition", 0)) @@ -199,40 +199,22 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K cleaner.startup() - import JavaConverters._ - var containsTombstones: Boolean = false - for (segment <- log.logSegments; record <- segment.log.records.asScala) { - containsTombstones = true - } - assertTrue(containsTombstones) + Thread.sleep(200) + assertEquals(log.latestDeleteHorizon, T0 + tombstoneRetentionMs) - // We sleep a little bit, so that log cleaner has already gone through - // some iterations, ensures that delete horizons has been updated correctly - Thread.sleep(300L) time.sleep(tombstoneRetentionMs + 1) - val latestOffset: Long = log.latestEpoch match { - case None => - fail("There should be epoch defined.") - RecordBatch.NO_TIMESTAMP - case Some(epoch) => - log.endOffsetForEpoch(epoch) match { - case None => - fail("Offset should have been found.") - RecordBatch.NO_TIMESTAMP - case Some(offsetAndEpoch) => - offsetAndEpoch.offset - } - } + val latestOffset: Long = log.logEndOffset // the first block should get cleaned cleaner.awaitCleaned(new TopicPartition("log-partition", 0), - latestOffset + 1, maxWaitMs = tombstoneRetentionMs) + latestOffset + 1, maxWaitMs = 5000) - assertEquals(log.latestDeleteHorizon, RecordBatch.NO_TIMESTAMP) + import JavaConverters._ for (segment <- log.logSegments; record <- segment.log.records.asScala) { fail ("The log should not contain record " + record + ", tombstone has expired its lifetime.") } + assertEquals(log.latestDeleteHorizon, RecordBatch.NO_TIMESTAMP) } private def readFromLog(log: Log): Iterable[(Int, Int)] = { diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 006669e06d98d..980600ed3fb03 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -80,14 +80,15 @@ class LogCleanerTest { // pretend we have the following keys val keys = immutable.ListSet(1L, 3L, 5L, 7L, 9L) val map = new FakeOffsetMap(Int.MaxValue) - keys.foreach(k => map.put(key(k), largeDeleteHorizon)) + keys.foreach(k => map.put(key(k), Long.MaxValue)) // clean the log val segments = log.logSegments.take(3).toSeq val stats = new CleanerStats() val expectedBytesRead = segments.map(_.size).sum - cleaner.cleanSegments(log, segments, map, 0L, stats, new CleanedTransactionMetadata) val shouldRemain = LogTest.keysInLog(log).filter(!keys.contains(_)) + + cleaner.cleanSegments(log, segments, map, 0L, stats, new CleanedTransactionMetadata) assertEquals(shouldRemain, LogTest.keysInLog(log)) assertEquals(expectedBytesRead, stats.bytesRead) } @@ -1678,6 +1679,12 @@ class LogCleanerTest { LogTest.recoverAndCheck(dir, config, expectedKeys, new BrokerTopicStats(), time, time.scheduler) } + /** + * We need to run a two pass clean to perform the following steps to stimulate a proper clean: + * 1. On the first run, set the delete horizon stored in the first timestamp of the batches in the log. + * 2. For the second pass, we will advance the current time by tombstoneRetentionMs, which will cause the + * tombstones to expire, leading to their prompt removal from the log. + */ private def runTwoPassClean(cleaner: Cleaner, logToClean: LogToClean, currentTime: Long, legacyDeleteHorizonMs: Long = -1L, tombstoneRetentionMs: Long = 86400000) : Long = { cleaner.doClean(logToClean, currentTime, legacyDeleteHorizonMs) From 8e6f9a277abaa8fe51a603c699f9759af719d8c8 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Thu, 27 Feb 2020 19:40:06 -0800 Subject: [PATCH 40/53] Adding some further test modifications --- .../kafka/log/LogCleanerIntegrationTest.scala | 14 +++----- .../scala/unit/kafka/log/LogCleanerTest.scala | 32 +++++++++---------- 2 files changed, 21 insertions(+), 25 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index fda3a59dc0a02..c504a3028e145 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -199,22 +199,18 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K cleaner.startup() - Thread.sleep(200) + val latestOffset: Long = log.logEndOffset + + cleaner.awaitCleaned(new TopicPartition("log-partition", 0), + latestOffset + 1, maxWaitMs = 5000) assertEquals(log.latestDeleteHorizon, T0 + tombstoneRetentionMs) time.sleep(tombstoneRetentionMs + 1) - val latestOffset: Long = log.logEndOffset - // the first block should get cleaned cleaner.awaitCleaned(new TopicPartition("log-partition", 0), latestOffset + 1, maxWaitMs = 5000) - - import JavaConverters._ - for (segment <- log.logSegments; record <- segment.log.records.asScala) { - fail ("The log should not contain record " + record + ", tombstone has expired its lifetime.") - } - assertEquals(log.latestDeleteHorizon, RecordBatch.NO_TIMESTAMP) + TestUtils.waitUntilTrue(() => log.size == 0, "Log should be empty", tombstoneRetentionMs, 100) } private def readFromLog(log: Log): Iterable[(Int, Int)] = { diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 980600ed3fb03..b469fc3799cd8 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -363,13 +363,13 @@ class LogCleanerTest { assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) - // delete horizon forced to 0 to verify marker is not removed early - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = 0L)._1 + // delete horizon forced to smaller timestamp to verify marker is not removed early + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) // clean again with large delete horizon and verify the marker is removed - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) assertEquals(List(4, 5, 6, 7, 8), offsetsInLog(log)) } @@ -494,7 +494,7 @@ class LogCleanerTest { // the empty control batch does not cause an exception when cleaned // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] - dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue) + dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) assertEquals(List(2, 3), LogTest.keysInLog(log)) assertEquals(List(1, 2), offsetsInLog(log)) assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) @@ -518,7 +518,7 @@ class LogCleanerTest { log.roll() // Both the record and the marker should remain after cleaning - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) + runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) assertEquals(List(0, 1), offsetsInLog(log)) assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) } @@ -548,7 +548,7 @@ class LogCleanerTest { assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) // The empty batch and the marker is still retained after a second cleaning. - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) + cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue - 1) assertEquals(List(1), offsetsInLog(log)) assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) } @@ -572,8 +572,8 @@ class LogCleanerTest { log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, isFromClient = false) log.roll() - // delete horizon set to 0 to verify marker is not removed early - val dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = 0L)._1 + // delete horizon set to some smaller timestamp to verify marker is not removed early + val dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 assertEquals(List(3), LogTest.keysInLog(log)) assertEquals(List(3, 4, 5), offsetsInLog(log)) @@ -703,7 +703,7 @@ class LogCleanerTest { // pretend we have the following keys val keys = immutable.ListSet(1L, 3L, 5L, 7L, 9L) val map = new FakeOffsetMap(Int.MaxValue) - keys.foreach(k => map.put(key(k), largeDeleteHorizon)) + keys.foreach(k => map.put(key(k), Long.MaxValue)) // clean the log val stats = new CleanerStats() @@ -779,7 +779,7 @@ class LogCleanerTest { // pretend we have the following keys val keys = immutable.ListSet(1, 3, 5, 7, 9) val map = new FakeOffsetMap(Int.MaxValue) - keys.foreach(k => map.put(key(k), largeDeleteHorizon)) + keys.foreach(k => map.put(key(k), Long.MaxValue)) (log, map) } @@ -1085,7 +1085,7 @@ class LogCleanerTest { val keys = LogTest.keysInLog(log) val map = new FakeOffsetMap(Int.MaxValue) - keys.foreach(k => map.put(key(k), largeDeleteHorizon)) + keys.foreach(k => map.put(key(k), Long.MaxValue)) intercept[LogCleaningAbortedException] { cleaner.cleanSegments(log, log.logSegments.take(3).toSeq, map, 0L, new CleanerStats(), new CleanedTransactionMetadata) @@ -1279,7 +1279,7 @@ class LogCleanerTest { LogTest.initializeLogDirWithOverflowedSegment(dir) - val log = makeLog(config = config, recoveryPoint = largeDeleteHorizon) + val log = makeLog(config = config, recoveryPoint = Long.MaxValue) val segmentWithOverflow = LogTest.firstOverflowSegment(log).getOrElse { fail("Failed to create log with a segment which has overflowed offsets") } @@ -1292,7 +1292,7 @@ class LogCleanerTest { val offsetMap = new FakeOffsetMap(Int.MaxValue) for (k <- 1 until allKeys.size by 2) { expectedKeysAfterCleaning += allKeys(k - 1) - offsetMap.put(key(allKeys(k)), largeDeleteHorizon) + offsetMap.put(key(allKeys(k)), Long.MaxValue) } // Try to clean segment with offset overflow. This will trigger log split and the cleaning itself must abort. @@ -1344,7 +1344,7 @@ class LogCleanerTest { // pretend we have odd-numbered keys val offsetMap = new FakeOffsetMap(Int.MaxValue) for (k <- 1 until messageCount by 2) - offsetMap.put(key(k), largeDeleteHorizon) + offsetMap.put(key(k), Long.MaxValue) // clean the log cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats(), @@ -1384,7 +1384,7 @@ class LogCleanerTest { messageCount += 1 } for (k <- 1 until messageCount by 2) - offsetMap.put(key(k), largeDeleteHorizon) + offsetMap.put(key(k), Long.MaxValue) cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats(), new CleanedTransactionMetadata) // clear scheduler so that async deletes don't run @@ -1681,7 +1681,7 @@ class LogCleanerTest { /** * We need to run a two pass clean to perform the following steps to stimulate a proper clean: - * 1. On the first run, set the delete horizon stored in the first timestamp of the batches in the log. + * 1. On the first run, set the delete horizon in the batches with tombstone or markers with empty txn records. * 2. For the second pass, we will advance the current time by tombstoneRetentionMs, which will cause the * tombstones to expire, leading to their prompt removal from the log. */ From d6dd028d63f536fb627d41e0a3416fde44795d6d Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Sun, 1 Mar 2020 09:37:07 -0800 Subject: [PATCH 41/53] Making some changes to test --- .../kafka/log/LogCleanerIntegrationTest.scala | 10 ++-- .../scala/unit/kafka/log/LogCleanerTest.scala | 48 +++++++++---------- 2 files changed, 27 insertions(+), 31 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index c504a3028e145..6e33fb4163a7b 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -201,16 +201,12 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K val latestOffset: Long = log.logEndOffset - cleaner.awaitCleaned(new TopicPartition("log-partition", 0), - latestOffset + 1, maxWaitMs = 5000) + assertTrue(cleaner.awaitCleaned(new TopicPartition("log-partition", 0), + latestOffset, maxWaitMs = 5000)) assertEquals(log.latestDeleteHorizon, T0 + tombstoneRetentionMs) time.sleep(tombstoneRetentionMs + 1) - - // the first block should get cleaned - cleaner.awaitCleaned(new TopicPartition("log-partition", 0), - latestOffset + 1, maxWaitMs = 5000) - TestUtils.waitUntilTrue(() => log.size == 0, "Log should be empty", tombstoneRetentionMs, 100) + TestUtils.waitUntilTrue(() => log.size == 0, "Log should be empty") } private def readFromLog(log: Log): Iterable[(Int, Int)] = { diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index b469fc3799cd8..b95ba2dcb6270 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -53,7 +53,7 @@ class LogCleanerTest { val time = new MockTime() val throttler = new Throttler(desiredRatePerSec = Double.MaxValue, checkIntervalMs = Long.MaxValue, time = time) val tombstoneRetentionMs = 86400000 - val largeDeleteHorizon = Long.MaxValue - tombstoneRetentionMs - 1 + val largeTimestamp = Long.MaxValue - tombstoneRetentionMs - 1 @After def teardown(): Unit = { @@ -350,7 +350,7 @@ class LogCleanerTest { log.roll() // cannot remove the marker in this pass because there are still valid records - var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 + var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp)._1 assertEquals(List(1, 3, 2), LogTest.keysInLog(log)) assertEquals(List(0, 2, 3, 4, 5), offsetsInLog(log)) @@ -359,12 +359,12 @@ class LogCleanerTest { log.roll() // the first cleaning preserves the commit marker (at offset 3) since there were still records for the transaction - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeTimestamp)._1 assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) - // delete horizon forced to smaller timestamp to verify marker is not removed early - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 + // current time is still before delete horizon + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeTimestamp)._1 assertEquals(List(2, 1, 3), LogTest.keysInLog(log)) assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) @@ -398,11 +398,11 @@ class LogCleanerTest { log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, isFromClient = false) log.roll() - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) + cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp) assertEquals(List(2), LogTest.keysInLog(log)) assertEquals(List(1, 3, 4), offsetsInLog(log)) - runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) + runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp) assertEquals(List(2), LogTest.keysInLog(log)) assertEquals(List(3, 4), offsetsInLog(log)) } @@ -437,14 +437,14 @@ class LogCleanerTest { // first time through the records are removed // Expected State: [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}] - var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 + var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp)._1 assertEquals(List(2, 3), LogTest.keysInLog(log)) assertEquals(List(4, 5, 6, 7), offsetsInLog(log)) assertEquals(List(1, 3, 4, 5, 6, 7), lastOffsetsPerBatchInLog(log)) // the empty batch remains if cleaned again because it still holds the last sequence // Expected State: [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}] - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeTimestamp)._1 assertEquals(List(2, 3), LogTest.keysInLog(log)) assertEquals(List(4, 5, 6, 7), offsetsInLog(log)) assertEquals(List(1, 3, 4, 5, 6, 7), lastOffsetsPerBatchInLog(log)) @@ -457,13 +457,13 @@ class LogCleanerTest { log.roll() // Expected State: [{Producer1: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeTimestamp)._1 assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) assertEquals(List(4, 5, 6, 7, 8, 9), offsetsInLog(log)) assertEquals(List(1, 4, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log)) // Expected State: [{Producer1: EmptyBatch}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] - dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) + dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeTimestamp) assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) assertEquals(List(5, 6, 7, 8, 9), offsetsInLog(log)) assertEquals(List(1, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log)) @@ -487,14 +487,14 @@ class LogCleanerTest { // first time through the control batch is retained as an empty batch // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] - var dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) + var dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp) assertEquals(List(2, 3), LogTest.keysInLog(log)) assertEquals(List(1, 2), offsetsInLog(log)) assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) // the empty control batch does not cause an exception when cleaned // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] - dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) + dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeTimestamp) assertEquals(List(2, 3), LogTest.keysInLog(log)) assertEquals(List(1, 2), offsetsInLog(log)) assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) @@ -518,7 +518,7 @@ class LogCleanerTest { log.roll() // Both the record and the marker should remain after cleaning - runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) + runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp) assertEquals(List(0, 1), offsetsInLog(log)) assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) } @@ -543,7 +543,7 @@ class LogCleanerTest { // Both the batch and the marker should remain after cleaning. The batch is retained // because it is the last entry for this producerId. The marker is retained because // there are still batches remaining from this transaction. - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) + cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp) assertEquals(List(1), offsetsInLog(log)) assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) @@ -573,12 +573,12 @@ class LogCleanerTest { log.roll() // delete horizon set to some smaller timestamp to verify marker is not removed early - val dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 + val dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp)._1 assertEquals(List(3), LogTest.keysInLog(log)) assertEquals(List(3, 4, 5), offsetsInLog(log)) // clean again with large delete horizon and verify the marker is removed - runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) + runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeTimestamp) assertEquals(List(3), LogTest.keysInLog(log)) assertEquals(List(4, 5), offsetsInLog(log)) } @@ -612,12 +612,12 @@ class LogCleanerTest { // Both transactional batches will be cleaned. The last one will remain in the log // as an empty batch in order to preserve the producer sequence number and epoch - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) + cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp) assertEquals(List(1, 3, 4, 5), offsetsInLog(log)) assertEquals(List(1, 2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) // On the second round of cleaning, the marker from the first transaction should be removed. - runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) + runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp) assertEquals(List(3, 4, 5), offsetsInLog(log)) assertEquals(List(2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) } @@ -649,14 +649,14 @@ class LogCleanerTest { assertAbortedTransactionIndexed() // first time through the records are removed - var dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) + var dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp) assertAbortedTransactionIndexed() assertEquals(List(), LogTest.keysInLog(log)) assertEquals(List(2), offsetsInLog(log)) // abort marker is retained assertEquals(List(1, 2), lastOffsetsPerBatchInLog(log)) // empty batch is retained // the empty batch remains if cleaned again because it still holds the last sequence - dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) + dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeTimestamp) assertAbortedTransactionIndexed() assertEquals(List(), LogTest.keysInLog(log)) assertEquals(List(2), offsetsInLog(log)) // abort marker is still retained @@ -666,13 +666,13 @@ class LogCleanerTest { appendProducer(Seq(1)) log.roll() - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon)._1 + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeTimestamp)._1 assertAbortedTransactionIndexed() assertEquals(List(1), LogTest.keysInLog(log)) assertEquals(List(2, 3), offsetsInLog(log)) // abort marker is not yet gone because we read the empty batch assertEquals(List(2, 3), lastOffsetsPerBatchInLog(log)) // but we do not preserve the empty batch - dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeDeleteHorizon) + dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeTimestamp) assertEquals(List(1), LogTest.keysInLog(log)) assertEquals(List(3), offsetsInLog(log)) // abort marker is gone assertEquals(List(3), lastOffsetsPerBatchInLog(log)) @@ -1402,7 +1402,7 @@ class LogCleanerTest { messageCount += 1 } for (k <- 1 until messageCount by 2) - offsetMap.put(key(k), largeDeleteHorizon) + offsetMap.put(key(k), largeTimestamp) cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats(), new CleanedTransactionMetadata) // clear scheduler so that async deletes don't run From 9335c36fe9598cac92fe843c0bfa830061e3743a Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Sun, 1 Mar 2020 09:38:28 -0800 Subject: [PATCH 42/53] Making one liner --- core/src/test/scala/unit/kafka/log/LogCleanerTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index b95ba2dcb6270..2f4c9493bc3fb 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -1402,7 +1402,7 @@ class LogCleanerTest { messageCount += 1 } for (k <- 1 until messageCount by 2) - offsetMap.put(key(k), largeTimestamp) + offsetMap.put(key(k), Long.MaxValue) cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats(), new CleanedTransactionMetadata) // clear scheduler so that async deletes don't run From 3541ea2311d46b6cf0ce6a3df65a084bc4f7c0c2 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Fri, 6 Mar 2020 16:40:35 -0800 Subject: [PATCH 43/53] Adding last comments --- .../scala/unit/kafka/log/LogCleanerTest.scala | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 2f4c9493bc3fb..bc708f573a297 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -402,6 +402,7 @@ class LogCleanerTest { assertEquals(List(2), LogTest.keysInLog(log)) assertEquals(List(1, 3, 4), offsetsInLog(log)) + // In the first pass, the deleteHorizon for {Producer2: Commit} is set. In the second pass, it's removed. runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp) assertEquals(List(2), LogTest.keysInLog(log)) assertEquals(List(3, 4), offsetsInLog(log)) @@ -457,12 +458,14 @@ class LogCleanerTest { log.roll() // Expected State: [{Producer1: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] + // The deleteHorizon for {Producer2: Commit} is still not set yet. dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeTimestamp)._1 assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) assertEquals(List(4, 5, 6, 7, 8, 9), offsetsInLog(log)) assertEquals(List(1, 4, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log)) // Expected State: [{Producer1: EmptyBatch}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] + // In the first pass, the deleteHorizon for {Producer2: Commit} is set. In the second pass, it's removed. dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeTimestamp) assertEquals(List(2, 3, 1), LogTest.keysInLog(log)) assertEquals(List(5, 6, 7, 8, 9), offsetsInLog(log)) @@ -487,6 +490,8 @@ class LogCleanerTest { // first time through the control batch is retained as an empty batch // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] + // In the first pass, the deleteHorizon for the commit marker is set. In the second pass, the commit marker is removed + // but the empty batch is retained for preserving the producer epoch. var dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp) assertEquals(List(2, 3), LogTest.keysInLog(log)) assertEquals(List(1, 2), offsetsInLog(log)) @@ -494,7 +499,7 @@ class LogCleanerTest { // the empty control batch does not cause an exception when cleaned // Expected State: [{Producer1: EmptyBatch}], [{2}, {3}] - dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeTimestamp) + dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 assertEquals(List(2, 3), LogTest.keysInLog(log)) assertEquals(List(1, 2), offsetsInLog(log)) assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log)) @@ -572,12 +577,12 @@ class LogCleanerTest { log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, isFromClient = false) log.roll() - // delete horizon set to some smaller timestamp to verify marker is not removed early + // Aborted records are removed, but the abort marker is still preserved. val dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp)._1 assertEquals(List(3), LogTest.keysInLog(log)) assertEquals(List(3, 4, 5), offsetsInLog(log)) - // clean again with large delete horizon and verify the marker is removed + // In the first pass, the delete horizon for the abort marker is set. In the second pass, the abort marker is removed. runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeTimestamp) assertEquals(List(3), LogTest.keysInLog(log)) assertEquals(List(4, 5), offsetsInLog(log)) @@ -616,7 +621,7 @@ class LogCleanerTest { assertEquals(List(1, 3, 4, 5), offsetsInLog(log)) assertEquals(List(1, 2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) - // On the second round of cleaning, the marker from the first transaction should be removed. + // In the first pass, the delete horizon for the first marker is set. In the second pass, the first marker is removed. runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp) assertEquals(List(3, 4, 5), offsetsInLog(log)) assertEquals(List(2, 3, 4, 5), lastOffsetsPerBatchInLog(log)) @@ -649,7 +654,7 @@ class LogCleanerTest { assertAbortedTransactionIndexed() // first time through the records are removed - var dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp) + var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp)._1 assertAbortedTransactionIndexed() assertEquals(List(), LogTest.keysInLog(log)) assertEquals(List(2), offsetsInLog(log)) // abort marker is retained @@ -672,6 +677,7 @@ class LogCleanerTest { assertEquals(List(2, 3), offsetsInLog(log)) // abort marker is not yet gone because we read the empty batch assertEquals(List(2, 3), lastOffsetsPerBatchInLog(log)) // but we do not preserve the empty batch + // In the first pass, the delete horizon for the abort marker is set. In the second pass, the abort marker is removed. dirtyOffset = runTwoPassClean(cleaner, LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeTimestamp) assertEquals(List(1), LogTest.keysInLog(log)) assertEquals(List(3), offsetsInLog(log)) // abort marker is gone From a88dc201bcb50b7626552b77f240846c6eaa4435 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Fri, 6 Mar 2020 20:30:17 -0800 Subject: [PATCH 44/53] Bumping checkstyle --- checkstyle/checkstyle.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml index 8b28fcadb17c3..5f8554e45560b 100644 --- a/checkstyle/checkstyle.xml +++ b/checkstyle/checkstyle.xml @@ -124,7 +124,7 @@ - + From bc3f867549ca278941ac40da10091ca46d9302a7 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Sun, 8 Mar 2020 09:58:57 -0700 Subject: [PATCH 45/53] Adding some spotbug fixes --- .../apache/kafka/common/record/MemoryRecords.java | 2 +- core/src/main/scala/kafka/log/LogCleaner.scala | 4 ++-- core/src/test/scala/unit/kafka/log/LogTest.scala | 12 ++++++------ 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index 89a9d8fef31d1..ad50476b6a92e 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -379,7 +379,7 @@ public RecordFilter(final long currentTime, final long tombstoneRetentionMs) { this.tombstoneRetentionMs = tombstoneRetentionMs; } - public class BatchRetentionResult { + public static class BatchRetentionResult { public final BatchRetention batchRetention; public final boolean containsMarkerForEmptyTxn; public BatchRetentionResult(final BatchRetention batchRetention, diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index f81ad78d10169..7c947954aee89 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -648,7 +648,7 @@ private[log] class Cleaner(val id: Int, val logCleanerFilter: RecordFilter = new RecordFilter(currentTime, tombstoneRetentionMs) { var discardBatchRecords: Boolean = _ - override def checkBatchRetention(batch: RecordBatch): BatchRetentionResult = { + override def checkBatchRetention(batch: RecordBatch): RecordFilter.BatchRetentionResult = { // we piggy-back on the tombstone retention logic to delay deletion of transaction markers. // note that we will never delete a marker until all the records from that transaction are removed. val canDiscardBatch = shouldDiscardBatch(batch, transactionMetadata) @@ -686,7 +686,7 @@ private[log] class Cleaner(val id: Int, BatchRetention.DELETE else BatchRetention.DELETE_EMPTY - new BatchRetentionResult(batchRetention, canDiscardBatch) + new RecordFilter.BatchRetentionResult(batchRetention, canDiscardBatch) } override def shouldRetainRecord(batch: RecordBatch, record: Record): Boolean = { diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 16e337fe96de9..fd65be43880c6 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -966,8 +966,8 @@ class LogTest { val filtered = ByteBuffer.allocate(2048) records.filterTo(new TopicPartition("foo", 0), new RecordFilter(0, 0) { - override def checkBatchRetention(batch: RecordBatch): BatchRetentionResult = - new BatchRetentionResult(RecordFilter.BatchRetention.DELETE_EMPTY, false) + override def checkBatchRetention(batch: RecordBatch): RecordFilter.BatchRetentionResult = + new RecordFilter.BatchRetentionResult(RecordFilter.BatchRetention.DELETE_EMPTY, false) override def shouldRetainRecord(recordBatch: RecordBatch, record: Record): Boolean = !record.hasKey }, filtered, Int.MaxValue, BufferSupplier.NO_CACHING) filtered.flip() @@ -1008,8 +1008,8 @@ class LogTest { val filtered = ByteBuffer.allocate(2048) records.filterTo(new TopicPartition("foo", 0), new RecordFilter(0, 0) { - override def checkBatchRetention(batch: RecordBatch): BatchRetentionResult = - new BatchRetentionResult(RecordFilter.BatchRetention.RETAIN_EMPTY, true) + override def checkBatchRetention(batch: RecordBatch): RecordFilter.BatchRetentionResult = + new RecordFilter.BatchRetentionResult(RecordFilter.BatchRetention.RETAIN_EMPTY, true) override def shouldRetainRecord(recordBatch: RecordBatch, record: Record): Boolean = false }, filtered, Int.MaxValue, BufferSupplier.NO_CACHING) filtered.flip() @@ -1052,8 +1052,8 @@ class LogTest { val filtered = ByteBuffer.allocate(2048) records.filterTo(new TopicPartition("foo", 0), new RecordFilter(0, 0) { - override def checkBatchRetention(batch: RecordBatch): BatchRetentionResult = - new BatchRetentionResult(RecordFilter.BatchRetention.DELETE_EMPTY, false) + override def checkBatchRetention(batch: RecordBatch): RecordFilter.BatchRetentionResult = + new RecordFilter.BatchRetentionResult(RecordFilter.BatchRetention.DELETE_EMPTY, false) override def shouldRetainRecord(recordBatch: RecordBatch, record: Record): Boolean = !record.hasKey }, filtered, Int.MaxValue, BufferSupplier.NO_CACHING) filtered.flip() From 6a1b3da8395abdd52e4368ba70b1ef4992bb0513 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Mon, 9 Mar 2020 16:21:09 -0700 Subject: [PATCH 46/53] Adding some temporary log statements --- .../java/org/apache/kafka/common/record/MemoryRecords.java | 3 +++ .../java/org/apache/kafka/connect/util/KafkaBasedLogTest.java | 1 + 2 files changed, 4 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index ad50476b6a92e..3c7b31c23e23b 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -164,6 +164,7 @@ private static FilterResult filterTo(TopicPartition partition, Iterable callback1 = EasyMock.newCapture(); EasyMock.expect(producer.send(EasyMock.eq(tp1Record), EasyMock.capture(callback1))).andReturn(tp1Future); + System.out.println("Executing next"); // Producer flushes when read to log end is called producer.flush(); PowerMock.expectLastCall(); From d7491c14b0e1d62b2ad61e1d6b270e1993ef67d7 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Tue, 10 Mar 2020 14:15:22 -0700 Subject: [PATCH 47/53] Rolling back changes --- .../java/org/apache/kafka/common/record/MemoryRecords.java | 3 --- .../java/org/apache/kafka/connect/util/KafkaBasedLogTest.java | 1 - 2 files changed, 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index 3c7b31c23e23b..ad50476b6a92e 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -164,7 +164,6 @@ private static FilterResult filterTo(TopicPartition partition, Iterable callback1 = EasyMock.newCapture(); EasyMock.expect(producer.send(EasyMock.eq(tp1Record), EasyMock.capture(callback1))).andReturn(tp1Future); - System.out.println("Executing next"); // Producer flushes when read to log end is called producer.flush(); PowerMock.expectLastCall(); From 06d9ff52a0888230159b9cdd73e07e820ac3ef61 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Tue, 17 Mar 2020 11:38:02 -0700 Subject: [PATCH 48/53] Addressing most comments --- .../record/AbstractLegacyRecordBatch.java | 20 ++++++------- .../kafka/common/record/MemoryRecords.java | 30 +++++++++---------- .../common/record/MemoryRecordsBuilder.java | 14 ++++----- 3 files changed, 31 insertions(+), 33 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java index cf74f24f490f0..3e1fd2564ff02 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java @@ -217,6 +217,16 @@ public boolean isControlBatch() { return false; } + @Override + public long deleteHorizonMs() { + return RecordBatch.NO_TIMESTAMP; + } + + @Override + public boolean deleteHorizonSet() { + return false; + } + /** * Get an iterator for the nested entries contained within this batch. Note that * if the batch is not compressed, then this method will return an iterator over the @@ -431,16 +441,6 @@ public LegacyRecord outerRecord() { return record; } - @Override - public long deleteHorizonMs() { - return RecordBatch.NO_TIMESTAMP; - } - - @Override - public boolean deleteHorizonSet() { - return false; - } - @Override public boolean equals(Object o) { if (this == o) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index ad50476b6a92e..b0bda1666884b 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -177,14 +177,12 @@ private static FilterResult filterTo(TopicPartition partition, Iterable retainedRecords = new ArrayList<>(); - final BatchIterationResult iterationResult = iterateOverBatch(batch, decompressionBufferSupplier, filterResult, filter, - batchMagic, writeOriginalBatch, maxOffset, retainedRecords); - containsTombstones = iterationResult.containsTombstones(); - writeOriginalBatch = iterationResult.shouldWriteOriginalBatch(); + final BatchFilterResult iterationResult = filterBatch(batch, decompressionBufferSupplier, filterResult, filter, + batchMagic, true, maxOffset, retainedRecords); + boolean containsTombstones = iterationResult.containsTombstones(); + boolean writeOriginalBatch = iterationResult.shouldWriteOriginalBatch(); maxOffset = iterationResult.maxOffset(); if (!retainedRecords.isEmpty()) { @@ -193,7 +191,7 @@ private static FilterResult filterTo(TopicPartition partition, Iterable= 2 && (containsTombstones || containsMarkerForEmptyTxn) && !batch.deleteHorizonSet(); - if (writeOriginalBatch && (!needToSetDeleteHorizon)) { + if (writeOriginalBatch && !needToSetDeleteHorizon) { if (batch.deleteHorizonMs() > filterResult.latestDeleteHorizon()) filterResult.updateLatestDeleteHorizon(batch.deleteHorizonMs()); batch.writeTo(bufferOutputStream); @@ -248,12 +246,12 @@ private static FilterResult filterTo(TopicPartition partition, Iterable retainedRecords) { boolean containsTombstones = false; @@ -266,7 +264,7 @@ private static BatchIterationResult iterateOverBatch(RecordBatch batch, // Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite // the corrupted batch with correct data. if (!record.hasMagic(batchMagic)) - recordsFiltered = false; + writeOriginalBatch = false; if (record.offset() > maxOffset) maxOffset = record.offset(); @@ -277,20 +275,20 @@ private static BatchIterationResult iterateOverBatch(RecordBatch batch, containsTombstones = true; } } else { - recordsFiltered = false; + writeOriginalBatch = false; } } - return new BatchIterationResult(recordsFiltered, containsTombstones, maxOffset); + return new BatchFilterResult(writeOriginalBatch, containsTombstones, maxOffset); } } - private static class BatchIterationResult { + private static class BatchFilterResult { private final boolean writeOriginalBatch; private final boolean containsTombstones; private final long maxOffset; - public BatchIterationResult(final boolean writeOriginalBatch, - final boolean containsTombstones, - final long maxOffset) { + public BatchFilterResult(final boolean writeOriginalBatch, + final boolean containsTombstones, + final long maxOffset) { this.writeOriginalBatch = writeOriginalBatch; this.containsTombstones = containsTombstones; this.maxOffset = maxOffset; diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java index be32a83905935..14e9e7d1331fe 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java @@ -131,6 +131,10 @@ public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, bufferStream.position(initialPosition + batchHeaderSizeInBytes); this.bufferStream = bufferStream; this.appendStream = new DataOutputStream(compressionType.wrapForOutput(this.bufferStream, magic)); + + if (deleteHorizonSet()) { + this.firstTimestamp = deleteHorizonMs; + } } public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, @@ -214,7 +218,7 @@ public boolean isTransactional() { } public boolean deleteHorizonSet() { - return deleteHorizonMs >= 0L; + return magic >= RecordBatch.MAGIC_VALUE_V2 && deleteHorizonMs >= 0L; } /** @@ -436,12 +440,8 @@ private Long appendWithOffset(long offset, boolean isControlRecord, long timesta if (magic < RecordBatch.MAGIC_VALUE_V2 && headers != null && headers.length > 0) throw new IllegalArgumentException("Magic v" + magic + " does not support record headers"); - if (firstTimestamp == null) { - if (deleteHorizonSet()) - firstTimestamp = deleteHorizonMs; - else - firstTimestamp = timestamp; - } + if (firstTimestamp == null) + firstTimestamp = timestamp; if (magic > RecordBatch.MAGIC_VALUE_V1) { appendDefaultRecord(offset, timestamp, key, value, headers); From bed40ab8713d3ec4026ac8a6737adfbacbb5cee3 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Thu, 19 Mar 2020 14:50:26 -0700 Subject: [PATCH 49/53] Addressing last comments --- .../record/AbstractLegacyRecordBatch.java | 6 +-- .../common/record/DefaultRecordBatch.java | 38 ++++++++++++------- .../kafka/common/record/MemoryRecords.java | 12 +++--- .../common/record/MemoryRecordsBuilder.java | 8 ++-- .../kafka/common/record/RecordBatch.java | 2 +- .../src/main/scala/kafka/log/LogCleaner.scala | 10 ++--- 6 files changed, 43 insertions(+), 33 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java index 3e1fd2564ff02..ea6c2b23bbad1 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java @@ -223,7 +223,7 @@ public long deleteHorizonMs() { } @Override - public boolean deleteHorizonSet() { + public boolean hasDeleteHorizonMs() { return false; } @@ -484,7 +484,7 @@ public long deleteHorizonMs() { } @Override - public boolean deleteHorizonSet() { + public boolean hasDeleteHorizonMs() { return false; } @@ -583,7 +583,7 @@ public long deleteHorizonMs() { } @Override - public boolean deleteHorizonSet() { + public boolean hasDeleteHorizonMs() { return false; } diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java index 0d2157ae13da8..0e74f9bd4e166 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java @@ -159,19 +159,28 @@ public void ensureValid() { + ", computed crc = " + computeChecksum() + ")"); } + /** + * Gets the base timestamp of the batch which is used to calculate the timestamp deltas. + * + * @return The base timestamp or + * {@link RecordBatch#NO_TIMESTAMP} if the batch is empty + */ + public long baseTimestamp() { + return buffer.getLong(FIRST_TIMESTAMP_OFFSET); + } + /** * Get the timestamp of the first record in this batch. It is usually the create time of the record even if the * timestamp type of the batch is log append time. - * - * There is the possibility that the first timestamp had been set to the delete horizon of the batch, - * in which case, the delete horizon will be returned instead. * - * @return The first timestamp if the batch's delete horizon has not been set - * The delete horizon if the batch's delete horizon has been set - * {@link RecordBatch#NO_TIMESTAMP} if the batch is empty + * @return The first timestamp if a record has been appended, unless the delete horizon has been set + * {@link RecordBatch#NO_TIMESTAMP} if the batch is empty or if the delete horizon is set */ public long firstTimestamp() { - return buffer.getLong(FIRST_TIMESTAMP_OFFSET); + final long baseTimestamp = baseTimestamp(); + if (hasDeleteHorizonMs()) + return RecordBatch.NO_TIMESTAMP; + return baseTimestamp; } @Override @@ -256,14 +265,15 @@ public boolean isTransactional() { } @Override - public boolean deleteHorizonSet() { + public boolean hasDeleteHorizonMs() { return (attributes() & DELETE_HORIZON_FLAG_MASK) > 0; } @Override public long deleteHorizonMs() { - if (deleteHorizonSet()) - return firstTimestamp(); + final long baseTimestamp = baseTimestamp(); + if (hasDeleteHorizonMs()) + return baseTimestamp; return RecordBatch.NO_TIMESTAMP; } @@ -382,7 +392,7 @@ public void setMaxTimestamp(TimestampType timestampType, long maxTimestamp) { if (timestampType() == timestampType && currentMaxTimestamp == maxTimestamp) return; - byte attributes = computeAttributes(compressionType(), timestampType, isTransactional(), isControlBatch(), deleteHorizonSet()); + byte attributes = computeAttributes(compressionType(), timestampType, isTransactional(), isControlBatch(), hasDeleteHorizonMs()); buffer.putShort(ATTRIBUTES_OFFSET, attributes); buffer.putLong(MAX_TIMESTAMP_OFFSET, maxTimestamp); long crc = computeChecksum(); @@ -725,13 +735,13 @@ public boolean isTransactional() { } @Override - public boolean deleteHorizonSet() { - return loadBatchHeader().deleteHorizonSet(); + public boolean hasDeleteHorizonMs() { + return loadBatchHeader().hasDeleteHorizonMs(); } @Override public long deleteHorizonMs() { - if (deleteHorizonSet()) + if (hasDeleteHorizonMs()) return super.loadBatchHeader().deleteHorizonMs(); return RecordBatch.NO_TIMESTAMP; } diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index b0bda1666884b..53d38318c431a 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -190,7 +190,7 @@ private static FilterResult filterTo(TopicPartition partition, Iterable= 2 && (containsTombstones || containsMarkerForEmptyTxn) - && !batch.deleteHorizonSet(); + && !batch.hasDeleteHorizonMs(); if (writeOriginalBatch && !needToSetDeleteHorizon) { if (batch.deleteHorizonMs() > filterResult.latestDeleteHorizon()) filterResult.updateLatestDeleteHorizon(batch.deleteHorizonMs()); @@ -199,7 +199,7 @@ private static FilterResult filterTo(TopicPartition partition, Iterable filterResult.latestDeleteHorizon()) { filterResult.updateLatestDeleteHorizon(deleteHorizonMs); @@ -370,18 +370,18 @@ public int hashCode() { public static abstract class RecordFilter { public final long currentTime; - public final long tombstoneRetentionMs; + public final long deleteRetentionMs; - public RecordFilter(final long currentTime, final long tombstoneRetentionMs) { + public RecordFilter(final long currentTime, final long deleteRetentionMs) { this.currentTime = currentTime; - this.tombstoneRetentionMs = tombstoneRetentionMs; + this.deleteRetentionMs = deleteRetentionMs; } public static class BatchRetentionResult { public final BatchRetention batchRetention; public final boolean containsMarkerForEmptyTxn; public BatchRetentionResult(final BatchRetention batchRetention, - final boolean containsMarkerForEmptyTxn) { + final boolean containsMarkerForEmptyTxn) { this.batchRetention = batchRetention; this.containsMarkerForEmptyTxn = containsMarkerForEmptyTxn; } diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java index 14e9e7d1331fe..66dc654f0426d 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java @@ -132,7 +132,7 @@ public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, this.bufferStream = bufferStream; this.appendStream = new DataOutputStream(compressionType.wrapForOutput(this.bufferStream, magic)); - if (deleteHorizonSet()) { + if (hasDeleteHorizonMs()) { this.firstTimestamp = deleteHorizonMs; } } @@ -217,7 +217,7 @@ public boolean isTransactional() { return isTransactional; } - public boolean deleteHorizonSet() { + public boolean hasDeleteHorizonMs() { return magic >= RecordBatch.MAGIC_VALUE_V2 && deleteHorizonMs >= 0L; } @@ -393,8 +393,8 @@ private int writeDefaultBatchHeader() { maxTimestamp = this.maxTimestamp; DefaultRecordBatch.writeHeader(buffer, baseOffset, offsetDelta, size, magic, compressionType, timestampType, - firstTimestamp, maxTimestamp, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, deleteHorizonSet(), - partitionLeaderEpoch, numRecords); + firstTimestamp, maxTimestamp, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, + hasDeleteHorizonMs(), partitionLeaderEpoch, numRecords); buffer.position(pos); return writtenCompressed; diff --git a/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java index 45f1609e3bc29..af65ebaf9ad18 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java @@ -214,7 +214,7 @@ public interface RecordBatch extends Iterable { * Whether or not the base timestamp has been set to the delete horizon * @return true if it is, false otherwise */ - boolean deleteHorizonSet(); + boolean hasDeleteHorizonMs(); /** * Get the delete horizon, returns -1L if the first timestamp is not the delete horizon diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 7c947954aee89..4dd71bc000da6 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -627,7 +627,7 @@ private[log] class Cleaner(val id: Int, * @param dest The cleaned log segment * @param map The key=>offset mapping * @param retainLegacyDeletesAndTxnMarkers Should tombstones (lower than version 2) and markers be retained while cleaning this segment - * @param tombstoneRetentionMs Defines how long a tombstone should be kept as defined by log configuration + * @param deleteRetentionMs Defines how long a tombstone should be kept as defined by log configuration * @param maxLogMessageSize The maximum message size of the corresponding topic * @param stats Collector for cleaning statistics * @param currentTime The time at which the clean was initiated @@ -637,7 +637,7 @@ private[log] class Cleaner(val id: Int, dest: LogSegment, map: OffsetMap, retainLegacyDeletesAndTxnMarkers: Boolean, - tombstoneRetentionMs: Long, + deleteRetentionMs: Long, maxLogMessageSize: Int, transactionMetadata: CleanedTransactionMetadata, lastRecordsOfActiveProducers: Map[Long, LastRecord], @@ -645,7 +645,7 @@ private[log] class Cleaner(val id: Int, currentTime: Long): Long = { var latestDeleteHorizon: Long = RecordBatch.NO_TIMESTAMP - val logCleanerFilter: RecordFilter = new RecordFilter(currentTime, tombstoneRetentionMs) { + val logCleanerFilter: RecordFilter = new RecordFilter(currentTime, deleteRetentionMs) { var discardBatchRecords: Boolean = _ override def checkBatchRetention(batch: RecordBatch): RecordFilter.BatchRetentionResult = { @@ -658,7 +658,7 @@ private[log] class Cleaner(val id: Int, discardBatchRecords = canDiscardBatch && !retainLegacyDeletesAndTxnMarkers } else { discardBatchRecords = canDiscardBatch && - batch.deleteHorizonSet() && batch.deleteHorizonMs() <= currentTime + batch.hasDeleteHorizonMs() && batch.deleteHorizonMs() <= currentTime } } else { discardBatchRecords = canDiscardBatch @@ -806,7 +806,7 @@ private[log] class Cleaner(val id: Int, val supportDeleteHorizon = batch.magic() >= RecordBatch.MAGIC_VALUE_V2 val shouldRetainDeletes = if (supportDeleteHorizon) - !batch.deleteHorizonSet() || currentTime < batch.deleteHorizonMs() + !batch.hasDeleteHorizonMs() || currentTime < batch.deleteHorizonMs() else retainDeletesForLegacyRecords val isRetainedValue = record.hasValue || shouldRetainDeletes From 54cb56bbd15d82dda17b21c543ef1ab011536d12 Mon Sep 17 00:00:00 2001 From: Richard Yu Date: Mon, 20 Apr 2020 12:51:27 -0700 Subject: [PATCH 50/53] Create .asf.yml --- .asf.yml | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) create mode 100644 .asf.yml diff --git a/.asf.yml b/.asf.yml new file mode 100644 index 0000000000000..cc88717175563 --- /dev/null +++ b/.asf.yml @@ -0,0 +1,25 @@ +# 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. + +notifications: + commits: commits@kafka.apache.org + issues: jira@kafka.apache.org + pullrequests: jira@kafka.apache.org + jira_options: link label +jenkins: + github_whitelist: + - ConcurrencyPractitioner From cadee721e14ff3afbbf8eee211e2555cab036f7c Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Mon, 20 Apr 2020 12:56:55 -0700 Subject: [PATCH 51/53] Adding github whitelist --- .asf.yaml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/.asf.yaml b/.asf.yaml index ae0dede4e04d0..0a13850add337 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -20,3 +20,7 @@ notifications: issues: jira@kafka.apache.org pullrequests: jira@kafka.apache.org jira_options: link label + +jenkins: + github_whitelist: + - ConcurrencyPractitioner From e694f13ba09215d2bdd2f97b894bb396b82eea49 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Mon, 20 Apr 2020 12:57:39 -0700 Subject: [PATCH 52/53] Removing .yml file --- .asf.yml | 25 ------------------------- 1 file changed, 25 deletions(-) delete mode 100644 .asf.yml diff --git a/.asf.yml b/.asf.yml deleted file mode 100644 index cc88717175563..0000000000000 --- a/.asf.yml +++ /dev/null @@ -1,25 +0,0 @@ -# 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. - -notifications: - commits: commits@kafka.apache.org - issues: jira@kafka.apache.org - pullrequests: jira@kafka.apache.org - jira_options: link label -jenkins: - github_whitelist: - - ConcurrencyPractitioner From a9316ad85d7fc5cfcbfd30118fac8e35635c59e4 Mon Sep 17 00:00:00 2001 From: ConcurrencyPractitioner Date: Mon, 20 Apr 2020 14:27:02 -0700 Subject: [PATCH 53/53] Reverting unnecessary change --- .asf.yaml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/.asf.yaml b/.asf.yaml index 0a13850add337..ae0dede4e04d0 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -20,7 +20,3 @@ notifications: issues: jira@kafka.apache.org pullrequests: jira@kafka.apache.org jira_options: link label - -jenkins: - github_whitelist: - - ConcurrencyPractitioner