From 96d128a630ed40cca812354032f1f3afd7c38da9 Mon Sep 17 00:00:00 2001 From: tedyu Date: Fri, 15 Jan 2021 13:58:52 -0800 Subject: [PATCH 01/20] [KAKFA-8522] Streamline tombstone and transaction marker removal --- checkstyle/checkstyle.xml | 4 +- .../record/AbstractLegacyRecordBatch.java | 30 ++++ .../common/record/DefaultRecordBatch.java | 71 ++++++-- .../kafka/common/record/MemoryRecords.java | 154 ++++++++++++++---- .../common/record/MemoryRecordsBuilder.java | 33 +++- .../kafka/common/record/RecordBatch.java | 12 ++ .../consumer/internals/FetcherTest.java | 6 +- .../common/record/MemoryRecordsTest.java | 71 ++++++-- core/src/main/scala/kafka/log/Log.scala | 3 +- .../src/main/scala/kafka/log/LogCleaner.scala | 113 ++++++++----- .../scala/kafka/log/LogCleanerManager.scala | 22 ++- .../kafka/log/LogCleanerIntegrationTest.scala | 42 ++++- .../scala/unit/kafka/log/LogCleanerTest.scala | 78 +++++---- .../test/scala/unit/kafka/log/LogTest.scala | 16 +- gradlew | 2 +- .../kafka/raft/internals/BatchBuilder.java | 1 + 16 files changed, 513 insertions(+), 145 deletions(-) diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml index 91045adc60856..6fce70ea30f9a 100644 --- a/checkstyle/checkstyle.xml +++ b/checkstyle/checkstyle.xml @@ -115,7 +115,7 @@ - + @@ -132,7 +132,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..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 @@ -217,6 +217,16 @@ public boolean isControlBatch() { return false; } + @Override + public long deleteHorizonMs() { + return RecordBatch.NO_TIMESTAMP; + } + + @Override + public boolean hasDeleteHorizonMs() { + 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 @@ -468,6 +478,16 @@ public long offset() { return buffer.getLong(OFFSET_OFFSET); } + @Override + public long deleteHorizonMs() { + return RecordBatch.NO_TIMESTAMP; + } + + @Override + public boolean hasDeleteHorizonMs() { + 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 hasDeleteHorizonMs() { + 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 33709c0387963..24d1efdf20847 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,13 +160,27 @@ 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 + * 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. - * - * @return The first timestamp or {@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 @@ -245,6 +264,19 @@ public boolean isTransactional() { return (attributes() & TRANSACTIONAL_FLAG_MASK) > 0; } + @Override + public boolean hasDeleteHorizonMs() { + return (attributes() & DELETE_HORIZON_FLAG_MASK) > 0; + } + + @Override + public long deleteHorizonMs() { + final long baseTimestamp = baseTimestamp(); + if (hasDeleteHorizonMs()) + return baseTimestamp; + return RecordBatch.NO_TIMESTAMP; + } + @Override public boolean isControlBatch() { return (attributes() & CONTROL_FLAG_MASK) > 0; @@ -363,7 +395,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(), hasDeleteHorizonMs()); buffer.putShort(ATTRIBUTES_OFFSET, attributes); buffer.putLong(MAX_TIMESTAMP_OFFSET, maxTimestamp); long crc = computeChecksum(); @@ -410,7 +442,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"); @@ -422,6 +454,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; } @@ -439,8 +473,8 @@ public static void writeEmptyHeader(ByteBuffer buffer, 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, partitionLeaderEpoch, 0); + CompressionType.NONE, timestampType, RecordBatch.NO_TIMESTAMP, timestamp, producerId, + producerEpoch, baseSequence, isTransactional, isControlRecord, false, partitionLeaderEpoch, 0); } public static void writeHeader(ByteBuffer buffer, @@ -457,6 +491,7 @@ public static void writeHeader(ByteBuffer buffer, int sequence, boolean isTransactional, boolean isControlBatch, + boolean isDeleteHorizonSet, int partitionLeaderEpoch, int numRecords) { if (magic < RecordBatch.CURRENT_MAGIC_VALUE) @@ -464,7 +499,7 @@ public 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); @@ -704,6 +739,18 @@ public boolean isTransactional() { return loadBatchHeader().isTransactional(); } + @Override + public boolean hasDeleteHorizonMs() { + return loadBatchHeader().hasDeleteHorizonMs(); + } + + @Override + public long deleteHorizonMs() { + if (hasDeleteHorizonMs()) + 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 ebc5dc4233a14..55e8e771391fb 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 @@ -21,6 +21,7 @@ import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.network.TransferableChannel; import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention; +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,15 +154,22 @@ 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 or txn markers 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); + + final BatchRetentionResult batchRetentionResult = filter.checkBatchRetention(batch); + final boolean containsMarkerForEmptyTxn = batchRetentionResult.containsMarkerForEmptyTxn; + final BatchRetention batchRetention = batchRetentionResult.batchRetention; + filterResult.bytesRead += batch.sizeInBytes(); if (batchRetention == BatchRetention.DELETE) @@ -171,38 +179,40 @@ 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 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()) { - 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 + boolean needToSetDeleteHorizon = batch.magic() >= 2 && (containsTombstones || containsMarkerForEmptyTxn) + && !batch.hasDeleteHorizonMs(); + if (writeOriginalBatch && !needToSetDeleteHorizon) { + if (batch.deleteHorizonMs() > filterResult.latestDeleteHorizon()) + filterResult.updateLatestDeleteHorizon(batch.deleteHorizonMs()); batch.writeTo(bufferOutputStream); filterResult.updateRetainedBatchMetadata(batch, retainedRecords.size(), false); } else { - MemoryRecordsBuilder builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream); + final MemoryRecordsBuilder builder; + if (needToSetDeleteHorizon) { + long deleteHorizonMs = filter.currentTime + filter.deleteRetentionMs; + builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, deleteHorizonMs); + if (deleteHorizonMs > filterResult.latestDeleteHorizon()) { + filterResult.updateLatestDeleteHorizon(deleteHorizonMs); + } + } 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(); if (filteredBatchSize > batch.sizeInBytes() && filteredBatchSize > maxRecordBatchSize) @@ -239,9 +249,68 @@ private static FilterResult filterTo(TopicPartition partition, Iterable retainedRecords) { + boolean containsTombstones = false; + 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); + + if (!record.hasValue()) { + containsTombstones = true; + } + } else { + writeOriginalBatch = false; + } + } + return new BatchFilterResult(writeOriginalBatch, containsTombstones, maxOffset); + } + } + + private static class BatchFilterResult { + private final boolean writeOriginalBatch; + private final boolean containsTombstones; + private final long maxOffset; + public BatchFilterResult(final boolean writeOriginalBatch, + final boolean containsTombstones, + final long maxOffset) { + this.writeOriginalBatch = writeOriginalBatch; + this.containsTombstones = containsTombstones; + this.maxOffset = maxOffset; + } + public boolean shouldWriteOriginalBatch() { + return this.writeOriginalBatch; + } + public boolean containsTombstones() { + return this.containsTombstones; + } + 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 ? @@ -252,7 +321,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); @@ -303,6 +372,24 @@ public int hashCode() { } public static abstract class RecordFilter { + public final long currentTime; + public final long deleteRetentionMs; + + public RecordFilter(final long currentTime, final long deleteRetentionMs) { + this.currentTime = currentTime; + this.deleteRetentionMs = deleteRetentionMs; + } + + public static class BatchRetentionResult { + public final BatchRetention batchRetention; + public final boolean containsMarkerForEmptyTxn; + public BatchRetentionResult(final BatchRetention batchRetention, + final boolean containsMarkerForEmptyTxn) { + this.batchRetention = batchRetention; + this.containsMarkerForEmptyTxn = containsMarkerForEmptyTxn; + } + } + public enum BatchRetention { DELETE, // Delete the batch without inspecting records RETAIN_EMPTY, // Retain the batch even if it is empty @@ -313,7 +400,7 @@ public enum BatchRetention { * Check whether the full batch can be discarded (i.e. whether we even need to * check the records individually). */ - protected abstract BatchRetention checkBatchRetention(RecordBatch batch); + protected abstract BatchRetentionResult checkBatchRetention(RecordBatch batch); /** * Check whether a record should be retained in the log. Note that {@link #checkBatchRetention(RecordBatch)} @@ -334,11 +421,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/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java index 480f4dc7a343b..bca76d281cf5d 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 @@ -79,6 +79,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; @@ -98,7 +99,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) { @@ -124,6 +126,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(); @@ -132,6 +135,28 @@ public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, bufferStream.position(initialPosition + batchHeaderSizeInBytes); this.bufferStream = bufferStream; this.appendStream = new DataOutputStream(compressionType.wrapForOutput(this.bufferStream, magic)); + + if (hasDeleteHorizonMs()) { + this.firstTimestamp = deleteHorizonMs; + } + } + + 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); } /** @@ -196,6 +221,10 @@ public boolean isTransactional() { return isTransactional; } + public boolean hasDeleteHorizonMs() { + return magic >= RecordBatch.MAGIC_VALUE_V2 && deleteHorizonMs >= 0L; + } + /** * Close this builder and return the resulting buffer. * @return The built log buffer @@ -369,7 +398,7 @@ private int writeDefaultBatchHeader() { DefaultRecordBatch.writeHeader(buffer, baseOffset, offsetDelta, size, magic, compressionType, timestampType, firstTimestamp, maxTimestamp, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, - partitionLeaderEpoch, numRecords); + 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 65a6a95fbe41f..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 @@ -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 hasDeleteHorizonMs(); + + /** + * 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/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 8dab0f03dbf83..353c18b2ebef8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -2999,10 +2999,10 @@ public void testUpdatePositionWithLastRecordMissingFromBatch() { new SimpleRecord(null, "value".getBytes())); // Remove the last record to simulate compaction - MemoryRecords.FilterResult result = records.filterTo(tp0, new MemoryRecords.RecordFilter() { + MemoryRecords.FilterResult result = records.filterTo(tp0, new MemoryRecords.RecordFilter(0, 0) { @Override - protected BatchRetention checkBatchRetention(RecordBatch batch) { - return BatchRetention.DELETE_EMPTY; + protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { + return new BatchRetentionResult(BatchRetention.DELETE_EMPTY, false); } @Override 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 cab667a8b4d93..4ccaa8f4e784a 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 @@ -21,6 +21,7 @@ import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.message.LeaderChangeMessage.Voter; +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; @@ -313,11 +314,11 @@ public void testFilterToEmptyBatchRetention(Args args) { ByteBuffer filtered = ByteBuffer.allocate(2048); MemoryRecords.FilterResult filterResult = records.filterTo(new TopicPartition("foo", 0), - new MemoryRecords.RecordFilter() { + new MemoryRecords.RecordFilter(0, 0) { @Override - protected BatchRetention checkBatchRetention(RecordBatch batch) { + protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { // retain all batches - return BatchRetention.RETAIN_EMPTY; + return new BatchRetentionResult(BatchRetention.RETAIN_EMPTY, false); } @Override @@ -376,11 +377,11 @@ public void testEmptyBatchRetention(Args args) { ByteBuffer filtered = ByteBuffer.allocate(2048); MemoryRecords records = MemoryRecords.readableRecords(buffer); MemoryRecords.FilterResult filterResult = records.filterTo(new TopicPartition("foo", 0), - new MemoryRecords.RecordFilter() { + new MemoryRecords.RecordFilter(0, 0) { @Override - protected BatchRetention checkBatchRetention(RecordBatch batch) { + protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { // retain all batches - return BatchRetention.RETAIN_EMPTY; + return new BatchRetentionResult(BatchRetention.RETAIN_EMPTY, false); } @Override @@ -427,10 +428,10 @@ public void testEmptyBatchDeletion(Args args) { ByteBuffer filtered = ByteBuffer.allocate(2048); MemoryRecords records = MemoryRecords.readableRecords(buffer); MemoryRecords.FilterResult filterResult = records.filterTo(new TopicPartition("foo", 0), - new MemoryRecords.RecordFilter() { + new MemoryRecords.RecordFilter(0, 0) { @Override - protected BatchRetention checkBatchRetention(RecordBatch batch) { - return deleteRetention; + protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { + return new BatchRetentionResult(deleteRetention, false); } @Override @@ -488,6 +489,42 @@ public void testBuildEndTxnMarker(Args args) { } } + /** + * 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. + */ + @ParameterizedTest + @ArgumentsSource(MemoryRecordsArgumentsProvider.class) + public void testFirstTimestampToDeleteHorizonConversion(Args args) { + if (args.magic >= RecordBatch.MAGIC_VALUE_V2) { + ByteBuffer buffer = ByteBuffer.allocate(2048); + MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, args.magic, args.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; + final RecordFilter recordFilter = new MemoryRecords.RecordFilter(deleteHorizon - 1, 1) { + @Override + protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { + return true; + } + + @Override + protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { + return new BatchRetentionResult(BatchRetention.RETAIN_EMPTY, true); + } + }; + builder.build().filterTo(new TopicPartition("random", 0), recordFilter, 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()); + } + } + @ParameterizedTest @ArgumentsSource(MemoryRecordsArgumentsProvider.class) public void testBuildLeaderChangeMessage(Args args) { @@ -558,13 +595,13 @@ public void testFilterToBatchDiscard(Args args) { buffer.flip(); ByteBuffer filtered = ByteBuffer.allocate(2048); - MemoryRecords.readableRecords(buffer).filterTo(new TopicPartition("foo", 0), new MemoryRecords.RecordFilter() { + MemoryRecords.readableRecords(buffer).filterTo(new TopicPartition("foo", 0), new MemoryRecords.RecordFilter(0, 0) { @Override - protected BatchRetention checkBatchRetention(RecordBatch batch) { + protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { // discard the second and fourth batches if (batch.lastOffset() == 2L || batch.lastOffset() == 6L) - return BatchRetention.DELETE; - return BatchRetention.DELETE_EMPTY; + return new BatchRetentionResult(BatchRetention.DELETE, false); + return new BatchRetentionResult(BatchRetention.DELETE_EMPTY, false); } @Override @@ -1009,9 +1046,13 @@ private void assumeAtLeastV2OrNotZstd(Args args) { } private static class RetainNonNullKeysFilter extends MemoryRecords.RecordFilter { + public RetainNonNullKeysFilter() { + super(0, 0); + } + @Override - protected BatchRetention checkBatchRetention(RecordBatch batch) { - return BatchRetention.DELETE_EMPTY; + protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { + return new BatchRetentionResult(BatchRetention.DELETE_EMPTY, false); } @Override diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 0df66d80a13ee..ea02904c7705d 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -249,7 +249,8 @@ class Log(@volatile private var _dir: File, val topicPartition: TopicPartition, val producerStateManager: ProducerStateManager, logDirFailureChannel: LogDirFailureChannel, - private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup { + private val hadCleanShutdown: Boolean = true, + @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 3225d9d1b314f..a333e6b77e6fb 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -508,14 +508,15 @@ 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(), legacyDeleteHorizonMs = deleteHorizonMs) } - private[log] def doClean(cleanable: LogToClean, deleteHorizonMs: Long): (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 @@ -530,13 +531,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 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, deleteHorizonMs, stats, transactionMetadata) + cleanSegments(log, group, offsetMap, currentTime, stats, transactionMetadata, legacyDeleteHorizonMs = legacyDeleteHorizonMs) // record buffer utilization stats.bufferUtilization = offsetMap.utilization @@ -552,17 +553,19 @@ 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 deleteHorizonMs The time to retain delete tombstones + * @param currentTime The current time in milliseconds * @param stats Collector for cleaning statistics * @param transactionMetadata State of ongoing transactions which is carried between the cleaning * of the grouped segments + * @param legacyDeleteHorizonMs The delete horizon used for tombstones whose version is less than 2 */ private[log] def cleanSegments(log: Log, segments: Seq[LogSegment], map: OffsetMap, - deleteHorizonMs: Long, + currentTime: Long, stats: CleanerStats, - transactionMetadata: CleanedTransactionMetadata): Unit = { + transactionMetadata: CleanedTransactionMetadata, + 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) @@ -582,14 +585,17 @@ private[log] class Cleaner(val id: Int, val abortedTransactions = log.collectAbortedTransactions(startOffset, upperBoundOffset) transactionMetadata.addAbortedTransactions(abortedTransactions) - val retainDeletesAndTxnMarkers = currentSegment.lastModified > deleteHorizonMs + val retainLegacyDeletesAndTxnMarkers = currentSegment.lastModified > legacyDeleteHorizonMs info(s"Cleaning $currentSegment in log ${log.name} into ${cleaned.baseOffset} " + - s"with deletion horizon $deleteHorizonMs, " + - s"${if(retainDeletesAndTxnMarkers) "retaining" else "discarding"} deletes.") + s"with legacy deletion horizon $legacyDeleteHorizonMs, " + + s"${if(retainLegacyDeletesAndTxnMarkers) "retaining" else "discarding"} deletes.") try { - cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainDeletesAndTxnMarkers, log.config.maxMessageSize, - transactionMetadata, lastOffsetOfActiveProducers, stats) + 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 + } } catch { case e: LogSegmentOffsetOverflowException => // Split the current segment. It's also safest to abort the current cleaning process, so that we retry from @@ -630,26 +636,43 @@ 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 retainLegacyDeletesAndTxnMarkers Should tombstones (lower than version 2) and markers be retained while cleaning this segment + * @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 */ private[log] def cleanInto(topicPartition: TopicPartition, sourceRecords: FileRecords, dest: LogSegment, map: OffsetMap, - retainDeletesAndTxnMarkers: Boolean, + retainLegacyDeletesAndTxnMarkers: Boolean, + deleteRetentionMs: Long, maxLogMessageSize: Int, transactionMetadata: CleanedTransactionMetadata, lastRecordsOfActiveProducers: Map[Long, LastRecord], - stats: CleanerStats): Unit = { - val logCleanerFilter: RecordFilter = new RecordFilter { + stats: CleanerStats, + currentTime: Long): Long = { + var latestDeleteHorizon: Long = RecordBatch.NO_TIMESTAMP + + val logCleanerFilter: RecordFilter = new RecordFilter(currentTime, deleteRetentionMs) { var discardBatchRecords: Boolean = _ - override def checkBatchRetention(batch: RecordBatch): BatchRetention = { + 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. - discardBatchRecords = shouldDiscardBatch(batch, transactionMetadata, retainTxnMarkers = retainDeletesAndTxnMarkers) + val canDiscardBatch = shouldDiscardBatch(batch, transactionMetadata) + + if (batch.isControlBatch) { + if (batch.magic() < 2) { + discardBatchRecords = canDiscardBatch && !retainLegacyDeletesAndTxnMarkers + } else { + discardBatchRecords = canDiscardBatch && + batch.hasDeleteHorizonMs() && batch.deleteHorizonMs() <= currentTime + } + } else { + discardBatchRecords = canDiscardBatch + } def isBatchLastRecordOfProducer: Boolean = { // We retain the batch in order to preserve the state of active producers. There are three cases: @@ -666,20 +689,24 @@ private[log] class Cleaner(val id: Int, } } - if (batch.hasProducerId && isBatchLastRecordOfProducer) - BatchRetention.RETAIN_EMPTY - else if (discardBatchRecords) - BatchRetention.DELETE - else - BatchRetention.DELETE_EMPTY + val batchRetention: BatchRetention = + if (batch.hasProducerId && isBatchLastRecordOfProducer) + BatchRetention.RETAIN_EMPTY + else if (discardBatchRecords) + BatchRetention.DELETE + else + BatchRetention.DELETE_EMPTY + new RecordFilter.BatchRetentionResult(batchRetention, canDiscardBatch) } 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 - false + isRecordRetained = false else - Cleaner.this.shouldRetainRecord(map, retainDeletesAndTxnMarkers, batch, record, stats) + isRecordRetained = Cleaner.this.shouldRetainRecord(map, retainLegacyDeletesAndTxnMarkers, batch, record, stats, currentTime = currentTime) + isRecordRetained } } @@ -694,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) @@ -719,6 +750,7 @@ private[log] class Cleaner(val id: Int, growBuffersOrFail(sourceRecords, position, maxLogMessageSize, records) } restoreBuffers() + latestDeleteHorizon } @@ -755,22 +787,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 - } + transactionMetadata: CleanedTransactionMetadata): Boolean = { + if (batch.isControlBatch) + transactionMetadata.onControlBatchRead(batch) + else + transactionMetadata.onBatchRead(batch) } private def shouldRetainRecord(map: kafka.log.OffsetMap, - retainDeletes: Boolean, + retainDeletesForLegacyRecords: Boolean, batch: RecordBatch, record: Record, - stats: CleanerStats): Boolean = { + stats: CleanerStats, + currentTime: Long): Boolean = { val pastLatestOffset = record.offset > map.latestOffset if (pastLatestOffset) return true @@ -784,7 +813,13 @@ 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 supportDeleteHorizon = batch.magic() >= RecordBatch.MAGIC_VALUE_V2 + val shouldRetainDeletes = + if (supportDeleteHorizon) + !batch.hasDeleteHorizonMs() || currentTime < batch.deleteHorizonMs() + else + retainDeletesForLegacyRecords + 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 eea889a7218e5..886270ace948d 100755 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -30,6 +30,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} @@ -169,11 +170,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 @@ -198,8 +199,23 @@ 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) { - None + 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 != RecordBatch.NO_TIMESTAMP && ltc.log.latestDeleteHorizon <= time.milliseconds() + } + if (!logsWithTombstonesExpired.isEmpty) { + val filthiest = logsWithTombstonesExpired.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 1ac9e62a7b1b0..7ab8573cab348 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.core.{Gauge, MetricName} import kafka.metrics.{KafkaMetricsGroup, KafkaYammerMetrics} @@ -177,6 +178,36 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K s"but lastCleaned=$lastCleaned2", lastCleaned2 >= secondBlockCleanableSegmentOffset) } + @Test + def testTombstoneCleanWithLowThroughput() : 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, backOffMs = 100L) + + 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) + + // roll the active segment + log.roll() + + cleaner.startup() + + val latestOffset: Long = log.logEndOffset + + assertTrue(cleaner.awaitCleaned(new TopicPartition("log-partition", 0), + latestOffset, maxWaitMs = 5000)) + assertEquals(log.latestDeleteHorizon, T0 + tombstoneRetentionMs) + + time.sleep(tombstoneRetentionMs + 1) + TestUtils.waitUntilTrue(() => log.size == 0, "Log should be empty") + } + private def readFromLog(log: Log): Iterable[(Int, Int)] = { for (segment <- log.logSegments; record <- segment.log.records.asScala) yield { val key = TestUtils.readString(record.key).toInt @@ -185,12 +216,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 505aa9a8f0e82..9f7ed6903f713 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 largeTimestamp = Long.MaxValue - tombstoneRetentionMs - 1 @After def teardown(): Unit = { @@ -84,8 +86,9 @@ class LogCleanerTest { 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) } @@ -347,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), deleteHorizonMs = Long.MaxValue)._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)) @@ -356,17 +359,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 = largeTimestamp)._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 + // 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)) // 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 +398,12 @@ class LogCleanerTest { log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) log.roll() - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) + cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp) 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) + // 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)) } @@ -436,14 +440,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 = 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), deleteHorizonMs = Long.MaxValue)._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 +461,15 @@ 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 + // 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}] - dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 + // 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)) assertEquals(List(1, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log)) @@ -488,14 +494,16 @@ 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 + // 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)) 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)) @@ -519,7 +527,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) + runTwoPassClean(cleaner, LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp) assertEquals(List(0, 1), offsetsInLog(log)) assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) } @@ -544,12 +552,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 = largeTimestamp) 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 - 1) assertEquals(List(1), offsetsInLog(log)) assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) } @@ -573,13 +581,13 @@ class LogCleanerTest { log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) 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 + // 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 - cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) + // 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)) } @@ -615,12 +623,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 = 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. - cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue) + // 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)) } @@ -652,14 +660,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 = largeTimestamp)._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 = 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 @@ -669,13 +677,14 @@ 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 = 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 = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), deleteHorizonMs = Long.MaxValue)._1 + // 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 assertEquals(List(3), lastOffsetsPerBatchInLog(log)) @@ -1549,6 +1558,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. @@ -1744,6 +1754,18 @@ class LogCleanerTest { private def recoverAndCheck(config: LogConfig, expectedKeys: Iterable[Long]): Log = { 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 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. + */ + 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 + } } class FakeOffsetMap(val slots: Int) extends OffsetMap { diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index a7794c87e6c86..579ac5dee78e5 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -36,7 +36,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.{ListOffsetsRequest, ListOffsetsResponse} @@ -1199,8 +1198,9 @@ class LogTest { records.batches.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): 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() @@ -1240,8 +1240,9 @@ class LogTest { records.batches.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): 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() @@ -1283,8 +1284,9 @@ class LogTest { records.batches.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): 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() diff --git a/gradlew b/gradlew index d38a482c9177b..68e8dfec805af 100755 --- a/gradlew +++ b/gradlew @@ -83,7 +83,7 @@ esac # Loop in case we encounter an error. for attempt in 1 2 3; do - if [ ! -e "$APP_HOME"/gradle/wrapper/gradle-wrapper.jar ]; then + if [ ! -e $APP_HOME/gradle/wrapper/gradle-wrapper.jar ]; then if ! curl -s -S --retry 3 -L -o "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" "https://raw.githubusercontent.com/gradle/gradle/v6.7.1/gradle/wrapper/gradle-wrapper.jar"; then rm -f "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" # Pause for a bit before looping in case the server throttled us. diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/BatchBuilder.java b/raft/src/main/java/org/apache/kafka/raft/internals/BatchBuilder.java index a264f7bfeb7ca..acd824644d5bd 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/BatchBuilder.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/BatchBuilder.java @@ -253,6 +253,7 @@ private void writeDefaultBatchHeader() { RecordBatch.NO_SEQUENCE, false, isControlBatch, + false, leaderEpoch, numRecords() ); From e179f9264b2e53b9af598d46265556348c636b15 Mon Sep 17 00:00:00 2001 From: Matthew Wong Date: Mon, 21 Jun 2021 18:17:25 -0700 Subject: [PATCH 02/20] missed a couple fixes on the merge --- .../org/apache/kafka/common/record/MemoryRecordsTest.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) 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 0e1944c0ba8c3..7b0ad7c61150f 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 @@ -23,6 +23,7 @@ import org.apache.kafka.common.message.LeaderChangeMessage.Voter; import org.apache.kafka.common.record.MemoryRecords.RecordFilter; import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention; +import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetentionResult; import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.test.TestUtils; @@ -321,7 +322,7 @@ public void testFilterToEmptyBatchRetention(Args args) { @Override protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { // retain all batches - return BatchRetentionResult(BatchRetention.RETAIN_EMPTY, false); + return new BatchRetentionResult(BatchRetention.RETAIN_EMPTY, false); } @Override @@ -383,7 +384,7 @@ public void testEmptyBatchRetention() { @Override protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { // retain all batches - return BatchRetentionResult(BatchRetention.RETAIN_EMPTY, false); + return new BatchRetentionResult(BatchRetention.RETAIN_EMPTY, false); } @Override @@ -430,7 +431,7 @@ public void testEmptyBatchDeletion() { new MemoryRecords.RecordFilter(0, 0) { @Override protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { - return BatchRetentionResult(deleteRetention, false); + return new BatchRetentionResult(deleteRetention, false); } @Override @@ -491,6 +492,7 @@ public void testBuildEndTxnMarker() { @ParameterizedTest @ArgumentsSource(MemoryRecordsArgumentsProvider.class) public void testFirstTimestampToDeleteHorizonConversion(Args args) { + int partitionLeaderEpoch = 998; if (args.magic >= RecordBatch.MAGIC_VALUE_V2) { ByteBuffer buffer = ByteBuffer.allocate(2048); MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, args.magic, args.compression, TimestampType.CREATE_TIME, From 8403960617a719c78205a0e4cfd3adbacdacc857 Mon Sep 17 00:00:00 2001 From: Matthew Wong Date: Thu, 8 Jul 2021 10:53:21 -0700 Subject: [PATCH 03/20] fix nit, remove firstTimestamp(), revert checkstyle --- checkstyle/checkstyle.xml | 4 +-- .../common/record/DefaultRecordBatch.java | 35 ++++++------------- .../common/record/DefaultRecordBatchTest.java | 2 +- .../unit/kafka/log/LogValidatorTest.scala | 2 +- 4 files changed, 15 insertions(+), 28 deletions(-) diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml index 8c6a5ab0fa50f..7f912dc428a15 100644 --- a/checkstyle/checkstyle.xml +++ b/checkstyle/checkstyle.xml @@ -115,7 +115,7 @@ - + @@ -133,7 +133,7 @@ - + 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 2b939128c0b13..509583b39d12f 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 @@ -91,7 +91,7 @@ * 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 time for which tombstones / transaction markers need 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: @@ -167,21 +167,9 @@ public void ensureValid() { * {@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. - * - * @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() { - final long baseTimestamp = baseTimestamp(); if (hasDeleteHorizonMs()) return RecordBatch.NO_TIMESTAMP; - return baseTimestamp; + return buffer.getLong(FIRST_TIMESTAMP_OFFSET); } @Override @@ -272,9 +260,8 @@ public boolean hasDeleteHorizonMs() { @Override public long deleteHorizonMs() { - final long baseTimestamp = baseTimestamp(); if (hasDeleteHorizonMs()) - return baseTimestamp; + return buffer.getLong(FIRST_TIMESTAMP_OFFSET); return RecordBatch.NO_TIMESTAMP; } @@ -322,9 +309,9 @@ private CloseableIterator uncompressedIterator() { buffer.position(RECORDS_OFFSET); return new RecordIterator() { @Override - protected Record readNext(long baseOffset, long firstTimestamp, int baseSequence, Long logAppendTime) { + protected Record readNext(long baseOffset, long baseTimestamp, int baseSequence, Long logAppendTime) { try { - return DefaultRecord.readFrom(buffer, baseOffset, firstTimestamp, baseSequence, logAppendTime); + return DefaultRecord.readFrom(buffer, baseOffset, baseTimestamp, baseSequence, logAppendTime); } catch (BufferUnderflowException e) { throw new InvalidRecordException("Incorrect declared batch size, premature EOF reached"); } @@ -590,7 +577,7 @@ public static int decrementSequence(int sequence, int decrement) { private abstract class RecordIterator implements CloseableIterator { private final Long logAppendTime; private final long baseOffset; - private final long firstTimestamp; + private final long baseTimestamp; private final int baseSequence; private final int numRecords; private int readRecords = 0; @@ -598,7 +585,7 @@ private abstract class RecordIterator implements CloseableIterator { RecordIterator() { this.logAppendTime = timestampType() == TimestampType.LOG_APPEND_TIME ? maxTimestamp() : null; this.baseOffset = baseOffset(); - this.firstTimestamp = firstTimestamp(); + this.baseTimestamp = baseTimestamp(); this.baseSequence = baseSequence(); int numRecords = count(); if (numRecords < 0) @@ -618,7 +605,7 @@ public Record next() { throw new NoSuchElementException(); readRecords++; - Record rec = readNext(baseOffset, firstTimestamp, baseSequence, logAppendTime); + Record rec = readNext(baseOffset, baseTimestamp, baseSequence, logAppendTime); if (readRecords == numRecords) { // Validate that the actual size of the batch is equal to declared size // by checking that after reading declared number of items, there no items left @@ -629,7 +616,7 @@ public Record next() { return rec; } - protected abstract Record readNext(long baseOffset, long firstTimestamp, int baseSequence, Long logAppendTime); + protected abstract Record readNext(long baseOffset, long baseTimestamp, int baseSequence, Long logAppendTime); protected abstract boolean ensureNoneRemaining(); @@ -651,9 +638,9 @@ private abstract class StreamRecordIterator extends RecordIterator { abstract Record doReadRecord(long baseOffset, long firstTimestamp, int baseSequence, Long logAppendTime) throws IOException; @Override - protected Record readNext(long baseOffset, long firstTimestamp, int baseSequence, Long logAppendTime) { + protected Record readNext(long baseOffset, long baseTimestamp, int baseSequence, Long logAppendTime) { try { - return doReadRecord(baseOffset, firstTimestamp, baseSequence, logAppendTime); + return doReadRecord(baseOffset, baseTimestamp, baseSequence, logAppendTime); } catch (EOFException e) { throw new InvalidRecordException("Incorrect declared batch size, premature EOF reached"); } catch (IOException e) { diff --git a/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java b/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java index 065f1b51b39aa..0864a2dd9f347 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java @@ -67,7 +67,7 @@ public void testWriteEmptyHeader() { assertEquals(isTransactional, batch.isTransactional()); assertEquals(timestampType, batch.timestampType()); assertEquals(timestamp, batch.maxTimestamp()); - assertEquals(RecordBatch.NO_TIMESTAMP, batch.firstTimestamp()); + assertEquals(RecordBatch.NO_TIMESTAMP, batch.baseTimestamp()); assertEquals(isControlBatch, batch.isControlBatch()); } } diff --git a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala index af585bfd49605..4275684230736 100644 --- a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala @@ -1521,7 +1521,7 @@ class LogValidatorTest { def maybeCheckBaseTimestamp(expected: Long, batch: RecordBatch): Unit = { batch match { case b: DefaultRecordBatch => - assertEquals(expected, b.firstTimestamp, s"Unexpected base timestamp of batch $batch") + assertEquals(expected, b.baseTimestamp, s"Unexpected base timestamp of batch $batch") case _ => // no-op } } From 6731c4dff7c63d4716ca22a3d0173d80f3d294d2 Mon Sep 17 00:00:00 2001 From: Matthew Wong Date: Mon, 12 Jul 2021 15:04:58 -0700 Subject: [PATCH 04/20] suppress checkstyle --- checkstyle/suppressions.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 538feca238cb0..7f78a76783a67 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -57,7 +57,7 @@ + files="MemoryRecordsBuilder.java"/> @@ -68,7 +68,7 @@ files="(Utils|Topic|KafkaLZ4BlockOutputStream|AclData|JoinGroupRequest).java"/> + files="(ConsumerCoordinator|Fetcher|KafkaProducer|ConfigDef|KerberosLogin|AbstractRequest|AbstractResponse|Selector|SslFactory|SslTransportLayer|SaslClientAuthenticator|SaslClientCallbackHandler|SaslServerAuthenticator|AbstractCoordinator|TransactionManager|AbstractStickyAssignor|DefaultSslEngineFactory|Authorizer|RecordAccumulator|MemoryRecords).java"/> From bb47b1603a85870d699ab9c39f68e73d69758ded Mon Sep 17 00:00:00 2001 From: Matthew Wong Date: Mon, 12 Jul 2021 15:05:43 -0700 Subject: [PATCH 05/20] address comments - use OptionalLong, rename constants --- .../record/AbstractLegacyRecordBatch.java | 28 ++++----------- .../common/record/DefaultRecordBatch.java | 34 +++++++------------ .../kafka/common/record/MemoryRecords.java | 26 +++++++------- .../kafka/common/record/RecordBatch.java | 15 ++++---- .../common/record/MemoryRecordsTest.java | 2 +- .../src/main/scala/kafka/log/LogCleaner.scala | 4 +-- 6 files changed, 44 insertions(+), 65 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 37c1032e6c521..0f2ccde2cd7c8 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 @@ -35,6 +35,7 @@ import java.util.Iterator; import java.util.NoSuchElementException; import java.util.Objects; +import java.util.OptionalLong; import static org.apache.kafka.common.record.Records.LOG_OVERHEAD; import static org.apache.kafka.common.record.Records.OFFSET_OFFSET; @@ -214,13 +215,8 @@ public boolean isControlBatch() { } @Override - public long deleteHorizonMs() { - return RecordBatch.NO_TIMESTAMP; - } - - @Override - public boolean hasDeleteHorizonMs() { - return false; + public OptionalLong deleteHorizonMs() { + return OptionalLong.empty(); } /** @@ -475,13 +471,8 @@ public long offset() { } @Override - public long deleteHorizonMs() { - return RecordBatch.NO_TIMESTAMP; - } - - @Override - public boolean hasDeleteHorizonMs() { - return false; + public OptionalLong deleteHorizonMs() { + return OptionalLong.empty(); } @Override @@ -574,13 +565,8 @@ public long baseOffset() { } @Override - public long deleteHorizonMs() { - return RecordBatch.NO_TIMESTAMP; - } - - @Override - public boolean hasDeleteHorizonMs() { - return false; + public OptionalLong deleteHorizonMs() { + return OptionalLong.empty(); } @Override 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 509583b39d12f..3d8c115b31533 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 @@ -37,6 +37,7 @@ import java.util.List; import java.util.NoSuchElementException; import java.util.Objects; +import java.util.OptionalLong; import static org.apache.kafka.common.record.Records.LOG_OVERHEAD; @@ -115,9 +116,9 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe static final int ATTRIBUTE_LENGTH = 2; public static final int LAST_OFFSET_DELTA_OFFSET = ATTRIBUTES_OFFSET + ATTRIBUTE_LENGTH; static final int LAST_OFFSET_DELTA_LENGTH = 4; - static final int FIRST_TIMESTAMP_OFFSET = LAST_OFFSET_DELTA_OFFSET + LAST_OFFSET_DELTA_LENGTH; - static final int FIRST_TIMESTAMP_LENGTH = 8; - static final int MAX_TIMESTAMP_OFFSET = FIRST_TIMESTAMP_OFFSET + FIRST_TIMESTAMP_LENGTH; + static final int BASE_TIMESTAMP_OFFSET = LAST_OFFSET_DELTA_OFFSET + LAST_OFFSET_DELTA_LENGTH; + static final int BASE_TIMESTAMP_LENGTH = 8; + static final int MAX_TIMESTAMP_OFFSET = BASE_TIMESTAMP_OFFSET + BASE_TIMESTAMP_LENGTH; static final int MAX_TIMESTAMP_LENGTH = 8; static final int PRODUCER_ID_OFFSET = MAX_TIMESTAMP_OFFSET + MAX_TIMESTAMP_LENGTH; static final int PRODUCER_ID_LENGTH = 8; @@ -167,9 +168,7 @@ public void ensureValid() { * {@link RecordBatch#NO_TIMESTAMP} if the batch is empty */ public long baseTimestamp() { - if (hasDeleteHorizonMs()) - return RecordBatch.NO_TIMESTAMP; - return buffer.getLong(FIRST_TIMESTAMP_OFFSET); + return buffer.getLong(BASE_TIMESTAMP_OFFSET); } @Override @@ -253,16 +252,16 @@ public boolean isTransactional() { return (attributes() & TRANSACTIONAL_FLAG_MASK) > 0; } - @Override - public boolean hasDeleteHorizonMs() { + private boolean hasDeleteHorizonMs() { return (attributes() & DELETE_HORIZON_FLAG_MASK) > 0; } @Override - public long deleteHorizonMs() { + public OptionalLong deleteHorizonMs() { if (hasDeleteHorizonMs()) - return buffer.getLong(FIRST_TIMESTAMP_OFFSET); - return RecordBatch.NO_TIMESTAMP; + return OptionalLong.of(buffer.getLong(BASE_TIMESTAMP_OFFSET)); + else + return OptionalLong.empty(); } @Override @@ -495,7 +494,7 @@ public static void writeHeader(ByteBuffer buffer, buffer.putInt(position + PARTITION_LEADER_EPOCH_OFFSET, partitionLeaderEpoch); buffer.put(position + MAGIC_OFFSET, magic); buffer.putShort(position + ATTRIBUTES_OFFSET, attributes); - buffer.putLong(position + FIRST_TIMESTAMP_OFFSET, firstTimestamp); + buffer.putLong(position + BASE_TIMESTAMP_OFFSET, firstTimestamp); buffer.putLong(position + MAX_TIMESTAMP_OFFSET, maxTimestamp); buffer.putInt(position + LAST_OFFSET_DELTA_OFFSET, lastOffsetDelta); buffer.putLong(position + PRODUCER_ID_OFFSET, producerId); @@ -728,15 +727,8 @@ public boolean isTransactional() { } @Override - public boolean hasDeleteHorizonMs() { - return loadBatchHeader().hasDeleteHorizonMs(); - } - - @Override - public long deleteHorizonMs() { - if (hasDeleteHorizonMs()) - return super.loadBatchHeader().deleteHorizonMs(); - return RecordBatch.NO_TIMESTAMP; + public OptionalLong deleteHorizonMs() { + return loadBatchHeader().deleteHorizonMs(); } @Override 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 3eb8f94dcf4bf..d63516f711278 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 @@ -193,10 +193,10 @@ private static FilterResult filterTo(TopicPartition partition, Iterable= 2 && (containsTombstones || containsMarkerForEmptyTxn) - && !batch.hasDeleteHorizonMs(); + && !batch.deleteHorizonMs().isPresent(); if (writeOriginalBatch && !needToSetDeleteHorizon) { - if (batch.deleteHorizonMs() > filterResult.latestDeleteHorizon()) - filterResult.updateLatestDeleteHorizon(batch.deleteHorizonMs()); + if (batch.deleteHorizonMs().orElse(RecordBatch.NO_TIMESTAMP) > filterResult.latestDeleteHorizon()) + filterResult.updateLatestDeleteHorizon(batch.deleteHorizonMs().getAsLong()); batch.writeTo(bufferOutputStream); filterResult.updateRetainedBatchMetadata(batch, retainedRecords.size(), false); } else { @@ -208,9 +208,9 @@ private static FilterResult filterTo(TopicPartition partition, Iterable filterResult.latestDeleteHorizon()) - filterResult.updateLatestDeleteHorizon(batch.deleteHorizonMs()); + builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, batch.deleteHorizonMs().orElse(RecordBatch.NO_TIMESTAMP)); + if (batch.deleteHorizonMs().orElse(RecordBatch.NO_TIMESTAMP) > filterResult.latestDeleteHorizon()) + filterResult.updateLatestDeleteHorizon(batch.deleteHorizonMs().getAsLong()); } MemoryRecords records = builder.build(); @@ -250,13 +250,13 @@ private static FilterResult filterTo(TopicPartition partition, Iterable retainedRecords) { + BufferSupplier decompressionBufferSupplier, + FilterResult filterResult, + RecordFilter filter, + byte batchMagic, + boolean writeOriginalBatch, + long maxOffset, + List retainedRecords) { boolean containsTombstones = false; try (final CloseableIterator iterator = batch.streamingIterator(decompressionBufferSupplier)) { while (iterator.hasNext()) { 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 2266fba0a2e5a..6e19140429eb6 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 @@ -21,6 +21,7 @@ import java.nio.ByteBuffer; import java.util.Iterator; +import java.util.OptionalLong; /** * A record batch is a container for records. In old versions of the record format (versions 0 and 1), @@ -211,17 +212,17 @@ 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 hasDeleteHorizonMs(); +// /** +// * Whether or not the base timestamp has been set to the delete horizon +// * @return true if it is, false otherwise +// */ +// boolean hasDeleteHorizonMs(); /** - * Get the delete horizon, returns -1L if the first timestamp is not the delete horizon + * Get the delete horizon, returns None if the first timestamp is not the delete horizon * @return timestamp of the delete horizon */ - long deleteHorizonMs(); + OptionalLong deleteHorizonMs(); /** * Get the partition leader epoch of this record batch. 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 7b0ad7c61150f..118f1b2698f92 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 @@ -518,7 +518,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { List batches = TestUtils.toList(filteredRecords.batches()); assertEquals(1, batches.size()); - assertEquals(deleteHorizon, batches.get(0).deleteHorizonMs()); + assertEquals(deleteHorizon, batches.get(0).deleteHorizonMs().getAsLong()); } } diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index fb75391e755e3..3fd64785092a5 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -666,7 +666,7 @@ private[log] class Cleaner(val id: Int, discardBatchRecords = canDiscardBatch && !retainLegacyDeletesAndTxnMarkers } else { discardBatchRecords = canDiscardBatch && - batch.hasDeleteHorizonMs() && batch.deleteHorizonMs() <= currentTime + batch.deleteHorizonMs().isPresent && batch.deleteHorizonMs().getAsLong <= currentTime } } else { discardBatchRecords = canDiscardBatch @@ -814,7 +814,7 @@ private[log] class Cleaner(val id: Int, val supportDeleteHorizon = batch.magic() >= RecordBatch.MAGIC_VALUE_V2 val shouldRetainDeletes = if (supportDeleteHorizon) - !batch.hasDeleteHorizonMs() || currentTime < batch.deleteHorizonMs() + !batch.deleteHorizonMs().isPresent || currentTime < batch.deleteHorizonMs().getAsLong else retainDeletesForLegacyRecords val isRetainedValue = record.hasValue || shouldRetainDeletes From c2933929bd50563953e00162b583864d3ce457f2 Mon Sep 17 00:00:00 2001 From: Matthew Wong Date: Mon, 12 Jul 2021 15:32:15 -0700 Subject: [PATCH 06/20] improve test for correct timestamps as delta for delete horizon --- .../org/apache/kafka/common/record/RecordBatch.java | 6 ------ .../apache/kafka/common/record/MemoryRecordsTest.java | 11 +++++++++-- 2 files changed, 9 insertions(+), 8 deletions(-) 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 6e19140429eb6..ced49383b797d 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 @@ -212,12 +212,6 @@ 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 hasDeleteHorizonMs(); - /** * Get the delete horizon, returns None if the first timestamp is not the delete horizon * @return timestamp of the delete horizon 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 118f1b2698f92..70569d9dc6c2d 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 @@ -25,6 +25,7 @@ import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention; import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetentionResult; import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.common.utils.CloseableIterator; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Test; @@ -486,12 +487,13 @@ public void testBuildEndTxnMarker() { } /** - * This test is used to see if the first timestamp of the batch has been successfully + * This test is used to see if the base timestamp of the batch has been successfully * converted to a delete horizon for the tombstones / transaction markers of the batch. + * It also verifies that the record timestamps remain correct as a delta relative to the delete horizon. */ @ParameterizedTest @ArgumentsSource(MemoryRecordsArgumentsProvider.class) - public void testFirstTimestampToDeleteHorizonConversion(Args args) { + public void testBaseTimestampToDeleteHorizonConversion(Args args) { int partitionLeaderEpoch = 998; if (args.magic >= RecordBatch.MAGIC_VALUE_V2) { ByteBuffer buffer = ByteBuffer.allocate(2048); @@ -519,6 +521,11 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { List batches = TestUtils.toList(filteredRecords.batches()); assertEquals(1, batches.size()); assertEquals(deleteHorizon, batches.get(0).deleteHorizonMs().getAsLong()); + + CloseableIterator recordIterator = batches.get(0).streamingIterator(BufferSupplier.create()); + Record record = recordIterator.next(); + assertEquals(10L, record.timestamp()); + recordIterator.close(); } } From 798a5f7af1c7f266302cb98ce600a94d3d92aa71 Mon Sep 17 00:00:00 2001 From: Matthew Wong Date: Thu, 29 Jul 2021 11:21:32 -0700 Subject: [PATCH 07/20] address comments --- .../org/apache/kafka/common/record/MemoryRecords.java | 10 ++++------ core/src/main/scala/kafka/log/LogCleaner.scala | 2 +- 2 files changed, 5 insertions(+), 7 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 d63516f711278..2ec564cd1289c 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,8 +164,6 @@ private static FilterResult filterTo(TopicPartition partition, Iterable retainedRecords = new ArrayList<>(); final BatchFilterResult iterationResult = filterBatch(batch, decompressionBufferSupplier, filterResult, filter, - batchMagic, true, maxOffset, retainedRecords); + batchMagic, true, retainedRecords); boolean containsTombstones = iterationResult.containsTombstones(); boolean writeOriginalBatch = iterationResult.shouldWriteOriginalBatch(); - maxOffset = iterationResult.maxOffset(); + long maxOffset = iterationResult.maxOffset(); if (!retainedRecords.isEmpty()) { // 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 - boolean needToSetDeleteHorizon = batch.magic() >= 2 && (containsTombstones || containsMarkerForEmptyTxn) + boolean needToSetDeleteHorizon = batch.magic() >= RecordBatch.MAGIC_VALUE_V2 && (containsTombstones || containsMarkerForEmptyTxn) && !batch.deleteHorizonMs().isPresent(); if (writeOriginalBatch && !needToSetDeleteHorizon) { if (batch.deleteHorizonMs().orElse(RecordBatch.NO_TIMESTAMP) > filterResult.latestDeleteHorizon()) @@ -255,8 +253,8 @@ private static BatchFilterResult filterBatch(RecordBatch batch, RecordFilter filter, byte batchMagic, boolean writeOriginalBatch, - long maxOffset, List retainedRecords) { + long maxOffset = -1; boolean containsTombstones = false; try (final CloseableIterator iterator = batch.streamingIterator(decompressionBufferSupplier)) { while (iterator.hasNext()) { diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 3fd64785092a5..06bfef04701a0 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -662,7 +662,7 @@ private[log] class Cleaner(val id: Int, val canDiscardBatch = shouldDiscardBatch(batch, transactionMetadata) if (batch.isControlBatch) { - if (batch.magic() < 2) { + if (batch.magic() < RecordBatch.MAGIC_VALUE_V2) { discardBatchRecords = canDiscardBatch && !retainLegacyDeletesAndTxnMarkers } else { discardBatchRecords = canDiscardBatch && From 3f4dc8791740421f6130bf26c90202a1b04e297d Mon Sep 17 00:00:00 2001 From: Matthew Wong Date: Tue, 3 Aug 2021 13:19:47 -0700 Subject: [PATCH 08/20] wip addressing comments --- .../kafka/common/record/MemoryRecords.java | 19 +++++----------- .../common/record/MemoryRecordsBuilder.java | 22 ++++++++++--------- .../common/record/MemoryRecordsTest.java | 2 +- core/src/main/scala/kafka/log/Log.scala | 3 +-- .../src/main/scala/kafka/log/LogCleaner.scala | 21 +++--------------- .../kafka/log/LogCleanerIntegrationTest.scala | 2 +- 6 files changed, 23 insertions(+), 46 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 2ec564cd1289c..8d927b9bcdbb3 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 @@ -181,10 +181,10 @@ private static FilterResult filterTo(TopicPartition partition, Iterable retainedRecords = new ArrayList<>(); final BatchFilterResult iterationResult = filterBatch(batch, decompressionBufferSupplier, filterResult, filter, - batchMagic, true, retainedRecords); - boolean containsTombstones = iterationResult.containsTombstones(); - boolean writeOriginalBatch = iterationResult.shouldWriteOriginalBatch(); - long maxOffset = iterationResult.maxOffset(); + batchMagic, true, retainedRecords); + boolean containsTombstones = iterationResult.containsTombstones; + boolean writeOriginalBatch = iterationResult.writeOriginalBatch; + long maxOffset = iterationResult.maxOffset; if (!retainedRecords.isEmpty()) { // we check if the delete horizon should be set to a new value @@ -287,22 +287,13 @@ private static class BatchFilterResult { private final boolean writeOriginalBatch; private final boolean containsTombstones; private final long maxOffset; - public BatchFilterResult(final boolean writeOriginalBatch, + private BatchFilterResult(final boolean writeOriginalBatch, final boolean containsTombstones, final long maxOffset) { this.writeOriginalBatch = writeOriginalBatch; this.containsTombstones = containsTombstones; this.maxOffset = maxOffset; } - public boolean shouldWriteOriginalBatch() { - return this.writeOriginalBatch; - } - public boolean containsTombstones() { - return this.containsTombstones; - } - public long maxOffset() { - return this.maxOffset; - } } private static MemoryRecordsBuilder buildRetainedRecordsInto(RecordBatch originalBatch, 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 8c58882d69508..b825a937e084b 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 @@ -83,7 +83,7 @@ public void write(int b) { private long deleteHorizonMs; private long offsetOfMaxTimestamp = -1; private Long lastOffset = null; - private Long firstTimestamp = null; + private Long baseTimestamp = null; private MemoryRecords builtRecords; private boolean aborted = false; @@ -111,6 +111,8 @@ public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, throw new IllegalArgumentException("Control records are not supported for magic " + magic); if (compressionType == CompressionType.ZSTD) throw new IllegalArgumentException("ZStandard compression is not supported for magic " + magic); + if (deleteHorizonMs != RecordBatch.NO_TIMESTAMP) + throw new IllegalArgumentException("Delete horizon timestamp is not supported for magic " + magic); } this.magic = magic; @@ -138,7 +140,7 @@ public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, this.appendStream = new DataOutputStream(compressionType.wrapForOutput(this.bufferStream, magic)); if (hasDeleteHorizonMs()) { - this.firstTimestamp = deleteHorizonMs; + this.baseTimestamp = deleteHorizonMs; } } @@ -398,7 +400,7 @@ private int writeDefaultBatchHeader() { maxTimestamp = this.maxTimestamp; DefaultRecordBatch.writeHeader(buffer, baseOffset, offsetDelta, size, magic, compressionType, timestampType, - firstTimestamp, maxTimestamp, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, + baseTimestamp, maxTimestamp, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, hasDeleteHorizonMs(), partitionLeaderEpoch, numRecords); buffer.position(pos); @@ -445,8 +447,8 @@ private void 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 (baseTimestamp == null) + baseTimestamp = timestamp; if (magic > RecordBatch.MAGIC_VALUE_V1) { appendDefaultRecord(offset, timestamp, key, value, headers); @@ -653,12 +655,12 @@ public void appendUncheckedWithOffset(long offset, SimpleRecord record) throws I if (magic >= RecordBatch.MAGIC_VALUE_V2) { int offsetDelta = (int) (offset - baseOffset); long timestamp = record.timestamp(); - if (firstTimestamp == null) - firstTimestamp = timestamp; + if (baseTimestamp == null) + baseTimestamp = timestamp; int sizeInBytes = DefaultRecord.writeTo(appendStream, offsetDelta, - timestamp - firstTimestamp, + timestamp - baseTimestamp, record.key(), record.value(), record.headers()); @@ -712,7 +714,7 @@ private void appendDefaultRecord(long offset, long timestamp, ByteBuffer key, By Header[] headers) throws IOException { ensureOpenForRecordAppend(); int offsetDelta = (int) (offset - baseOffset); - long timestampDelta = timestamp - firstTimestamp; + long timestampDelta = timestamp - baseTimestamp; int sizeInBytes = DefaultRecord.writeTo(appendStream, offsetDelta, timestampDelta, key, value, headers); recordWritten(offset, timestamp, sizeInBytes); } @@ -817,7 +819,7 @@ public boolean hasRoomFor(long timestamp, ByteBuffer key, ByteBuffer value, Head recordSize = Records.LOG_OVERHEAD + LegacyRecord.recordSize(magic, key, value); } else { int nextOffsetDelta = lastOffset == null ? 0 : (int) (lastOffset - baseOffset + 1); - long timestampDelta = firstTimestamp == null ? 0 : timestamp - firstTimestamp; + long timestampDelta = baseTimestamp == null ? 0 : timestamp - baseTimestamp; recordSize = DefaultRecord.sizeInBytes(nextOffsetDelta, timestampDelta, key, value, headers); } 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 70569d9dc6c2d..78b4d5baf49ef 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 @@ -520,7 +520,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { List batches = TestUtils.toList(filteredRecords.batches()); assertEquals(1, batches.size()); - assertEquals(deleteHorizon, batches.get(0).deleteHorizonMs().getAsLong()); + assertEquals(OptionalLong.of(deleteHorizon), batches.get(0).deleteHorizonMs()); CloseableIterator recordIterator = batches.get(0).streamingIterator(BufferSupplier.create()); Record record = recordIterator.next(); diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index d56e3b8decd86..13f7598431e91 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -270,8 +270,7 @@ class Log(@volatile private var _dir: File, val producerStateManager: ProducerStateManager, logDirFailureChannel: LogDirFailureChannel, @volatile private var _topicId: Option[Uuid], - val keepPartitionMetadataFile: Boolean, - @volatile var latestDeleteHorizon: Long = RecordBatch.NO_TIMESTAMP) extends Logging with KafkaMetricsGroup { + val keepPartitionMetadataFile: Boolean) 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 06bfef04701a0..3dbeb12b3cee7 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -513,7 +513,6 @@ private[log] class Cleaner(val id: Int, info("Beginning cleaning of log %s.".format(cleanable.log.name)) val log = cleanable.log - log.latestDeleteHorizon = RecordBatch.NO_TIMESTAMP val stats = new CleanerStats() @@ -589,11 +588,8 @@ private[log] class Cleaner(val id: Int, s"${if(retainLegacyDeletesAndTxnMarkers) "retaining" else "discarding"} deletes.") try { - val latestDeleteHorizon: Long = cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainLegacyDeletesAndTxnMarkers, log.config.deleteRetentionMs, + 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 - } } catch { case e: LogSegmentOffsetOverflowException => // Split the current segment. It's also safest to abort the current cleaning process, so that we retry from @@ -650,9 +646,7 @@ private[log] class Cleaner(val id: Int, transactionMetadata: CleanedTransactionMetadata, lastRecordsOfActiveProducers: Map[Long, LastRecord], stats: CleanerStats, - currentTime: Long): Long = { - var latestDeleteHorizon: Long = RecordBatch.NO_TIMESTAMP - + currentTime: Long): Unit = { val logCleanerFilter: RecordFilter = new RecordFilter(currentTime, deleteRetentionMs) { var discardBatchRecords: Boolean = _ @@ -662,12 +656,7 @@ private[log] class Cleaner(val id: Int, val canDiscardBatch = shouldDiscardBatch(batch, transactionMetadata) if (batch.isControlBatch) { - if (batch.magic() < RecordBatch.MAGIC_VALUE_V2) { - discardBatchRecords = canDiscardBatch && !retainLegacyDeletesAndTxnMarkers - } else { - discardBatchRecords = canDiscardBatch && - batch.deleteHorizonMs().isPresent && batch.deleteHorizonMs().getAsLong <= currentTime - } + discardBatchRecords = canDiscardBatch && batch.deleteHorizonMs().isPresent && batch.deleteHorizonMs().getAsLong <= currentTime } else { discardBatchRecords = canDiscardBatch } @@ -719,9 +708,6 @@ 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) @@ -748,7 +734,6 @@ private[log] class Cleaner(val id: Int, growBuffersOrFail(sourceRecords, position, maxLogMessageSize, records) } restoreBuffers() - latestDeleteHorizon } diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 2b6cc7bced460..1b652a1e59544 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -204,7 +204,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K assertTrue(cleaner.awaitCleaned(new TopicPartition("log-partition", 0), latestOffset, maxWaitMs = 5000)) - assertEquals(log.latestDeleteHorizon, T0 + tombstoneRetentionMs) +// assertEquals(log.latestDeleteHorizon, T0 + tombstoneRetentionMs) time.sleep(tombstoneRetentionMs + 1) TestUtils.waitUntilTrue(() => log.size == 0, "Log should be empty") From 7a13fc11769d2da93231dafacf23101187801612 Mon Sep 17 00:00:00 2001 From: Matthew Wong Date: Tue, 3 Aug 2021 16:20:18 -0700 Subject: [PATCH 09/20] Add tests to MemoryRecordsBuilder for direct deleteHorizon value set --- .../record/MemoryRecordsBuilderTest.java | 46 +++++++++++ .../common/record/MemoryRecordsTest.java | 77 +++++++++++-------- 2 files changed, 93 insertions(+), 30 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java index 5dedf66f36d32..4f3f03c3f2d21 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java @@ -19,6 +19,9 @@ import org.apache.kafka.common.errors.UnsupportedCompressionTypeException; import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.message.LeaderChangeMessage.Voter; +import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.common.utils.ByteBufferOutputStream; +import org.apache.kafka.common.utils.CloseableIterator; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.test.TestUtils; @@ -35,6 +38,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.OptionalLong; import java.util.Random; import java.util.function.BiFunction; import java.util.function.Supplier; @@ -755,6 +759,48 @@ else if (iterations > 2 && memUsed < (iterations - 2) * 1024) assertTrue(iterations < 100, "Memory usage too high: " + memUsed); } + @ParameterizedTest + @ArgumentsSource(V2MemoryRecordsBuilderArgumentsProvider.class) + public void testRecordTimestampsWithDeleteHorizon(Args args) { + long deleteHorizon = 100; + int payloadLen = 1024 * 1024; + ByteBuffer buffer = ByteBuffer.allocate(payloadLen * 2); + ByteBufferOutputStream byteBufferOutputStream = new ByteBufferOutputStream(buffer); + MemoryRecordsBuilder builder = new MemoryRecordsBuilder(byteBufferOutputStream, args.magic, args.compressionType, + TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID, + RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false, false, + RecordBatch.NO_PARTITION_LEADER_EPOCH, 0, deleteHorizon); + + builder.append(50L, "0".getBytes(), "0".getBytes()); + builder.append(100L, "1".getBytes(), null); + builder.append(150L, "2".getBytes(), "2".getBytes()); + + MemoryRecords records = builder.build(); + List batches = TestUtils.toList(records.batches()); + assertEquals(OptionalLong.of(deleteHorizon), batches.get(0).deleteHorizonMs()); + + CloseableIterator recordIterator = batches.get(0).streamingIterator(BufferSupplier.create()); + Record record = recordIterator.next(); + assertEquals(50L, record.timestamp()); + record = recordIterator.next(); + assertEquals(100L, record.timestamp()); + record = recordIterator.next(); + assertEquals(150L, record.timestamp()); + recordIterator.close(); + } + + private static class V2MemoryRecordsBuilderArgumentsProvider implements ArgumentsProvider { + @Override + public Stream provideArguments(ExtensionContext context) { + List values = new ArrayList<>(); + for (int bufferOffset : Arrays.asList(0, 15)) + for (CompressionType type: CompressionType.values()) { + values.add(Arguments.of(new Args(bufferOffset, type, MAGIC_VALUE_V2))); + } + return values.stream(); + } + } + private void verifyRecordsProcessingStats(CompressionType compressionType, RecordConversionStats processingStats, int numRecords, int numRecordsConverted, long finalBytes, long preConvertedBytes) { 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 78b4d5baf49ef..596694e88a441 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 @@ -40,6 +40,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.OptionalLong; import java.util.function.BiFunction; import java.util.function.Supplier; import java.util.stream.Stream; @@ -104,6 +105,18 @@ public Stream provideArguments(ExtensionContext context) { } } + private static class V2MemoryRecordsArgumentsProvider implements ArgumentsProvider { + @Override + public Stream provideArguments(ExtensionContext context) { + List arguments = new ArrayList<>(); + for (long firstOffset : asList(0L, 57L)) + for (CompressionType type: CompressionType.values()) { + arguments.add(Arguments.of(new Args(RecordBatch.MAGIC_VALUE_V2, firstOffset, type))); + } + return arguments.stream(); + } + } + private final long logAppendTime = System.currentTimeMillis(); @ParameterizedTest @@ -492,41 +505,45 @@ public void testBuildEndTxnMarker() { * It also verifies that the record timestamps remain correct as a delta relative to the delete horizon. */ @ParameterizedTest - @ArgumentsSource(MemoryRecordsArgumentsProvider.class) + @ArgumentsSource(V2MemoryRecordsArgumentsProvider.class) public void testBaseTimestampToDeleteHorizonConversion(Args args) { int partitionLeaderEpoch = 998; - if (args.magic >= RecordBatch.MAGIC_VALUE_V2) { - ByteBuffer buffer = ByteBuffer.allocate(2048); - MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, args.magic, args.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; - final RecordFilter recordFilter = new MemoryRecords.RecordFilter(deleteHorizon - 1, 1) { - @Override - protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { - return true; - } + ByteBuffer buffer = ByteBuffer.allocate(2048); + MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, args.magic, args.compression, TimestampType.CREATE_TIME, + 0L, RecordBatch.NO_TIMESTAMP, partitionLeaderEpoch); + builder.append(5L, "0".getBytes(), "0".getBytes()); + builder.append(10L, "1".getBytes(), null); + builder.append(15L, "2".getBytes(), "2".getBytes()); - @Override - protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { - return new BatchRetentionResult(BatchRetention.RETAIN_EMPTY, true); - } - }; - builder.build().filterTo(new TopicPartition("random", 0), recordFilter, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); - filtered.flip(); - MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); + ByteBuffer filtered = ByteBuffer.allocate(2048); + final long deleteHorizon = Integer.MAX_VALUE / 2; + final RecordFilter recordFilter = new MemoryRecords.RecordFilter(deleteHorizon - 1, 1) { + @Override + protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { + return true; + } - List batches = TestUtils.toList(filteredRecords.batches()); - assertEquals(1, batches.size()); - assertEquals(OptionalLong.of(deleteHorizon), batches.get(0).deleteHorizonMs()); + @Override + protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { + return new BatchRetentionResult(BatchRetention.RETAIN_EMPTY, true); + } + }; + builder.build().filterTo(new TopicPartition("random", 0), recordFilter, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); + filtered.flip(); + MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); - CloseableIterator recordIterator = batches.get(0).streamingIterator(BufferSupplier.create()); - Record record = recordIterator.next(); - assertEquals(10L, record.timestamp()); - recordIterator.close(); - } + List batches = TestUtils.toList(filteredRecords.batches()); + assertEquals(1, batches.size()); + assertEquals(OptionalLong.of(deleteHorizon), batches.get(0).deleteHorizonMs()); + + CloseableIterator recordIterator = batches.get(0).streamingIterator(BufferSupplier.create()); + Record record = recordIterator.next(); + assertEquals(5L, record.timestamp()); + record = recordIterator.next(); + assertEquals(10L, record.timestamp()); + record = recordIterator.next(); + assertEquals(15L, record.timestamp()); + recordIterator.close(); } @Test From 9bc5cb615215bec1beffc1884cfbe873d9e96711 Mon Sep 17 00:00:00 2001 From: Matthew Wong Date: Tue, 3 Aug 2021 18:03:46 -0700 Subject: [PATCH 10/20] add in latestDeleteHorizon again for review --- core/src/main/scala/kafka/log/Log.scala | 3 ++- core/src/main/scala/kafka/log/LogCleaner.scala | 17 ++++++++++++++--- .../kafka/log/LogCleanerIntegrationTest.scala | 2 +- 3 files changed, 17 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 13f7598431e91..d56e3b8decd86 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -270,7 +270,8 @@ class Log(@volatile private var _dir: File, val producerStateManager: ProducerStateManager, logDirFailureChannel: LogDirFailureChannel, @volatile private var _topicId: Option[Uuid], - val keepPartitionMetadataFile: Boolean) extends Logging with KafkaMetricsGroup { + val keepPartitionMetadataFile: Boolean, + @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 3dbeb12b3cee7..09f734cefa61a 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -513,6 +513,7 @@ private[log] class Cleaner(val id: Int, info("Beginning cleaning of log %s.".format(cleanable.log.name)) val log = cleanable.log + log.latestDeleteHorizon = RecordBatch.NO_TIMESTAMP val stats = new CleanerStats() @@ -588,8 +589,11 @@ private[log] class Cleaner(val id: Int, s"${if(retainLegacyDeletesAndTxnMarkers) "retaining" else "discarding"} deletes.") try { - cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainLegacyDeletesAndTxnMarkers, log.config.deleteRetentionMs, - log.config.maxMessageSize, transactionMetadata, lastOffsetOfActiveProducers, stats, currentTime = currentTime) + 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 + } } catch { case e: LogSegmentOffsetOverflowException => // Split the current segment. It's also safest to abort the current cleaning process, so that we retry from @@ -635,6 +639,8 @@ private[log] class Cleaner(val id: Int, * @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 + * + * @return the latestDeleteHorizon that is found from the FilterResult of the cleaning */ private[log] def cleanInto(topicPartition: TopicPartition, sourceRecords: FileRecords, @@ -646,7 +652,7 @@ private[log] class Cleaner(val id: Int, transactionMetadata: CleanedTransactionMetadata, lastRecordsOfActiveProducers: Map[Long, LastRecord], stats: CleanerStats, - currentTime: Long): Unit = { + currentTime: Long): Long = { val logCleanerFilter: RecordFilter = new RecordFilter(currentTime, deleteRetentionMs) { var discardBatchRecords: Boolean = _ @@ -697,6 +703,7 @@ private[log] class Cleaner(val id: Int, } } + var latestDeleteHorizon: Long = RecordBatch.NO_TIMESTAMP var position = 0 while (position < sourceRecords.sizeInBytes) { checkDone(topicPartition) @@ -708,6 +715,9 @@ 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) @@ -734,6 +744,7 @@ private[log] class Cleaner(val id: Int, growBuffersOrFail(sourceRecords, position, maxLogMessageSize, records) } restoreBuffers() + latestDeleteHorizon } diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 1b652a1e59544..2b6cc7bced460 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -204,7 +204,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K assertTrue(cleaner.awaitCleaned(new TopicPartition("log-partition", 0), latestOffset, maxWaitMs = 5000)) -// assertEquals(log.latestDeleteHorizon, T0 + tombstoneRetentionMs) + assertEquals(log.latestDeleteHorizon, T0 + tombstoneRetentionMs) time.sleep(tombstoneRetentionMs + 1) TestUtils.waitUntilTrue(() => log.size == 0, "Log should be empty") From cd77df933b03f2cf955455753410d906ae2ecb1c Mon Sep 17 00:00:00 2001 From: Matthew Wong Date: Wed, 18 Aug 2021 16:44:53 -0700 Subject: [PATCH 11/20] Trigger build From 43698e0ccd93f61a23c919209d02e4c6f324ba58 Mon Sep 17 00:00:00 2001 From: Matthew Wong Date: Wed, 1 Sep 2021 18:43:05 -0700 Subject: [PATCH 12/20] address Jun's comments, log reason for compaction --- .../kafka/common/record/MemoryRecords.java | 19 ++++++-------- .../kafka/common/record/RecordBatch.java | 2 +- .../common/record/MemoryRecordsTest.java | 3 +-- .../src/main/scala/kafka/log/LogCleaner.scala | 25 +++++++++--------- .../scala/kafka/log/LogCleanerManager.scala | 13 +++++++--- .../src/main/scala/kafka/log/UnifiedLog.scala | 9 +++++-- .../kafka/log/LogCleanerManagerTest.scala | 26 +++++++++---------- .../scala/unit/kafka/log/LogCleanerTest.scala | 7 ++--- 8 files changed, 56 insertions(+), 48 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 10f9843005ebc..768b16410cf61 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 @@ -199,17 +199,14 @@ private static FilterResult filterTo(TopicPartition partition, Iterable filterResult.latestDeleteHorizon()) { - filterResult.updateLatestDeleteHorizon(deleteHorizonMs); - } - } else { - builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, batch.deleteHorizonMs().orElse(RecordBatch.NO_TIMESTAMP)); - if (batch.deleteHorizonMs().orElse(RecordBatch.NO_TIMESTAMP) > filterResult.latestDeleteHorizon()) - filterResult.updateLatestDeleteHorizon(batch.deleteHorizonMs().getAsLong()); - } + long deleteHorizonMs; + if (needToSetDeleteHorizon) + deleteHorizonMs = filter.currentTime + filter.deleteRetentionMs; + else + deleteHorizonMs = batch.deleteHorizonMs().orElse(RecordBatch.NO_TIMESTAMP); + builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream, deleteHorizonMs); + if (deleteHorizonMs > filterResult.latestDeleteHorizon()) + filterResult.updateLatestDeleteHorizon(deleteHorizonMs); MemoryRecords records = builder.build(); int filteredBatchSize = records.sizeInBytes(); 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 ced49383b797d..7d231c1774367 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 @@ -213,7 +213,7 @@ public interface RecordBatch extends Iterable { boolean isTransactional(); /** - * Get the delete horizon, returns None if the first timestamp is not the delete horizon + * Get the delete horizon, returns OptionalLong.EMPTY if the first timestamp is not the delete horizon * @return timestamp of the delete horizon */ OptionalLong deleteHorizonMs(); 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 596694e88a441..3f0195bf5d149 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 @@ -23,7 +23,6 @@ import org.apache.kafka.common.message.LeaderChangeMessage.Voter; import org.apache.kafka.common.record.MemoryRecords.RecordFilter; import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention; -import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetentionResult; import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.common.utils.CloseableIterator; import org.apache.kafka.common.utils.Utils; @@ -525,7 +524,7 @@ protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { @Override protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { - return new BatchRetentionResult(BatchRetention.RETAIN_EMPTY, true); + return new BatchRetentionResult(BatchRetention.RETAIN_EMPTY, false); } }; builder.build().filterTo(new TopicPartition("random", 0), recordFilter, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index e972e85141d18..fdbfe19081d3d 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -342,14 +342,15 @@ class LogCleaner(initialConfig: CleanerConfig, @throws(classOf[LogCleaningException]) private def cleanFilthiestLog(): Boolean = { val preCleanStats = new PreCleanStats() - val cleaned = cleanerManager.grabFilthiestCompactedLog(time, preCleanStats) match { + val (ltc, reason) = cleanerManager.grabFilthiestCompactedLog(time, preCleanStats) + val cleaned = ltc match { case None => false case Some(cleanable) => // there's a log, clean it this.lastPreCleanStats = preCleanStats try { - cleanLog(cleanable) + cleanLog(cleanable, reason) true } catch { case e @ (_: ThreadShutdownException | _: ControlThrowable) => throw e @@ -373,11 +374,11 @@ class LogCleaner(initialConfig: CleanerConfig, cleaned } - private def cleanLog(cleanable: LogToClean): Unit = { + private def cleanLog(cleanable: LogToClean, reason: LogCleaningReason): Unit = { val startOffset = cleanable.firstDirtyOffset var endOffset = startOffset try { - val (nextDirtyOffset, cleanerStats) = cleaner.clean(cleanable) + val (nextDirtyOffset, cleanerStats) = cleaner.clean(cleanable, reason) endOffset = nextDirtyOffset recordStats(cleaner.id, cleanable.log.name, startOffset, endOffset, cleanerStats) } catch { @@ -489,10 +490,11 @@ private[log] class Cleaner(val id: Int, * Clean the given log * * @param cleanable The log to be cleaned + * @param reason The reason the log is being cleaned, for logging purposes * * @return The first offset not cleaned and the statistics for this round of cleaning */ - private[log] def clean(cleanable: LogToClean): (Long, CleanerStats) = { + private[log] def clean(cleanable: LogToClean, reason: LogCleaningReason = DirtyRatio): (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 = @@ -500,11 +502,11 @@ private[log] class Cleaner(val id: Int, case None => 0L case Some(seg) => seg.lastModified - cleanable.log.config.deleteRetentionMs } - doClean(cleanable, time.milliseconds(), legacyDeleteHorizonMs = deleteHorizonMs) + doClean(cleanable, time.milliseconds(), legacyDeleteHorizonMs = deleteHorizonMs, logCleaningReason = reason) } - private[log] def doClean(cleanable: LogToClean, currentTime: Long, legacyDeleteHorizonMs: Long = -1L): (Long, CleanerStats) = { - info("Beginning cleaning of log %s.".format(cleanable.log.name)) + private[log] def doClean(cleanable: LogToClean, currentTime: Long, legacyDeleteHorizonMs: Long = -1L, logCleaningReason: LogCleaningReason = DirtyRatio): (Long, CleanerStats) = { + info("Beginning cleaning of log %s due to %s.".format(cleanable.log.name, logCleaningReason.getClass.getName)) val log = cleanable.log log.latestDeleteHorizon = RecordBatch.NO_TIMESTAMP @@ -585,9 +587,8 @@ private[log] class Cleaner(val id: Int, try { 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) { + 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 @@ -801,9 +802,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 supportDeleteHorizon = batch.magic() >= RecordBatch.MAGIC_VALUE_V2 + val legacyRecord = batch.magic() < RecordBatch.MAGIC_VALUE_V2 val shouldRetainDeletes = - if (supportDeleteHorizon) + if (!legacyRecord) !batch.deleteHorizonMs().isPresent || currentTime < batch.deleteHorizonMs().getAsLong else retainDeletesForLegacyRecords diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala index 1202c2551c863..3749c43e62c6f 100755 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -39,6 +39,10 @@ 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] sealed trait LogCleaningReason +private[log] case object DirtyRatio extends LogCleaningReason +private[log] case object DeleteHorizon extends LogCleaningReason + private[log] class LogCleaningException(val log: UnifiedLog, private val message: String, private val cause: Throwable) extends KafkaException(message, cause) @@ -164,8 +168,9 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], * 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. + * Returns a tuple of an Option of the log selected to be cleaned and the reason it was selected. */ - def grabFilthiestCompactedLog(time: Time, preCleanStats: PreCleanStats = new PreCleanStats()): Option[LogToClean] = { + def grabFilthiestCompactedLog(time: Time, preCleanStats: PreCleanStats = new PreCleanStats()): (Option[LogToClean], LogCleaningReason) = { inLock(lock) { val now = time.milliseconds this.timeOfLastRun = now @@ -212,15 +217,15 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], if (!logsWithTombstonesExpired.isEmpty) { val filthiest = logsWithTombstonesExpired.max inProgress.put(filthiest.topicPartition, LogCleaningInProgress) - Some(filthiest) + (Some(filthiest), DeleteHorizon) } else { - None + (None, DirtyRatio) } } else { preCleanStats.recordCleanablePartitions(cleanableLogs.size) val filthiest = cleanableLogs.max inProgress.put(filthiest.topicPartition, LogCleaningInProgress) - Some(filthiest) + (Some(filthiest), DirtyRatio) } } } diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index ecd14cac3acb6..a816416d39de5 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -257,8 +257,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, @volatile var leaderEpochCache: Option[LeaderEpochFileCache], val producerStateManager: ProducerStateManager, @volatile private var _topicId: Option[Uuid], - val keepPartitionMetadataFile: Boolean, - @volatile var latestDeleteHorizon: Long = RecordBatch.NO_TIMESTAMP) extends Logging with KafkaMetricsGroup { + val keepPartitionMetadataFile: Boolean) extends Logging with KafkaMetricsGroup { import kafka.log.UnifiedLog._ @@ -289,6 +288,12 @@ class UnifiedLog(@volatile var logStartOffset: Long, @volatile var partitionMetadataFile : PartitionMetadataFile = null + /* Keep track of the largest deleteHorizonMs that is present from records in the log. This value helps streamline the + * deletion of tombstones in compacted logs. The LogCleanerManager can use this value to determine if the log can + * be cleaned to remove eligible tombstones without waiting for the log's dirty section to grow large enough. + */ + @volatile var latestDeleteHorizon: Long = RecordBatch.NO_TIMESTAMP + locally { initializePartitionMetadata() updateLogStartOffset(logStartOffset) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index e20e661431e52..adb2c32cc4542 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -139,7 +139,7 @@ class LogCleanerManagerTest extends Logging { val cleanerManager = createCleanerManagerMock(logsPool) cleanerCheckpoints.put(tp, 1) - val thrownException = assertThrows(classOf[LogCleaningException], () => cleanerManager.grabFilthiestCompactedLog(time).get) + val thrownException = assertThrows(classOf[LogCleaningException], () => cleanerManager.grabFilthiestCompactedLog(time)._1.get) assertEquals(log, thrownException.log) assertTrue(thrownException.getCause.isInstanceOf[IllegalStateException]) } @@ -156,7 +156,7 @@ class LogCleanerManagerTest extends Logging { val cleanerManager = createCleanerManagerMock(logs) partitions.foreach(partition => cleanerCheckpoints.put(partition, 20)) - val filthiestLog: LogToClean = cleanerManager.grabFilthiestCompactedLog(time).get + val filthiestLog: LogToClean = cleanerManager.grabFilthiestCompactedLog(time)._1.get assertEquals(tp2, filthiestLog.topicPartition) assertEquals(tp2, filthiestLog.log.topicPartition) } @@ -175,7 +175,7 @@ class LogCleanerManagerTest extends Logging { cleanerManager.markPartitionUncleanable(logs.get(tp2).dir.getParent, tp2) - val filthiestLog: LogToClean = cleanerManager.grabFilthiestCompactedLog(time).get + val filthiestLog: LogToClean = cleanerManager.grabFilthiestCompactedLog(time)._1.get assertEquals(tp1, filthiestLog.topicPartition) assertEquals(tp1, filthiestLog.log.topicPartition) } @@ -194,7 +194,7 @@ class LogCleanerManagerTest extends Logging { cleanerManager.setCleaningState(tp2, LogCleaningInProgress) - val filthiestLog: LogToClean = cleanerManager.grabFilthiestCompactedLog(time).get + val filthiestLog: LogToClean = cleanerManager.grabFilthiestCompactedLog(time)._1.get assertEquals(tp1, filthiestLog.topicPartition) assertEquals(tp1, filthiestLog.log.topicPartition) @@ -215,7 +215,7 @@ class LogCleanerManagerTest extends Logging { cleanerManager.setCleaningState(tp2, LogCleaningInProgress) cleanerManager.markPartitionUncleanable(logs.get(tp1).dir.getParent, tp1) - val filthiestLog: Option[LogToClean] = cleanerManager.grabFilthiestCompactedLog(time) + val filthiestLog: Option[LogToClean] = cleanerManager.grabFilthiestCompactedLog(time)._1 assertEquals(None, filthiestLog) } @@ -226,7 +226,7 @@ class LogCleanerManagerTest extends Logging { val cleanerManager = createCleanerManagerMock(logs) cleanerCheckpoints.put(tp, 200) - val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time).get + val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time)._1.get assertEquals(0L, filthiestLog.firstDirtyOffset) } @@ -240,7 +240,7 @@ class LogCleanerManagerTest extends Logging { val cleanerManager = createCleanerManagerMock(logs) cleanerCheckpoints.put(tp, 0L) - val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time).get + val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time)._1.get assertEquals(10L, filthiestLog.firstDirtyOffset) } @@ -264,7 +264,7 @@ class LogCleanerManagerTest extends Logging { cleanerCheckpoints.put(tp, 0L) // The active segment is uncleanable and hence not filthy from the POV of the CleanerManager. - val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time) + val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time)._1 assertEquals(None, filthiestLog) } @@ -290,7 +290,7 @@ class LogCleanerManagerTest extends Logging { cleanerCheckpoints.put(tp, 3L) // These segments are uncleanable and hence not filthy - val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time) + val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time)._1 assertEquals(None, filthiestLog) } @@ -364,12 +364,12 @@ class LogCleanerManagerTest extends Logging { log.updateConfig(config) // log cleanup inprogress, the log is not available for compaction - val cleanable = cleanerManager.grabFilthiestCompactedLog(time) + val cleanable = cleanerManager.grabFilthiestCompactedLog(time)._1 assertEquals(0, cleanable.size, "should have 0 logs ready to be compacted") // log cleanup finished, and log can be picked up for compaction cleanerManager.resumeCleaning(deletableLog.map(_._1)) - val cleanable2 = cleanerManager.grabFilthiestCompactedLog(time) + val cleanable2 = cleanerManager.grabFilthiestCompactedLog(time)._1 assertEquals(1, cleanable2.size, "should have 1 logs ready to be compacted") // update cleanup policy to delete @@ -753,7 +753,7 @@ class LogCleanerManagerTest extends Logging { val cleanerManager = createCleanerManagerMock(logs) cleanerCheckpoints.put(tp, 15L) - val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time) + val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time)._1 assertEquals(None, filthiestLog, "Log should not be selected for cleaning") assertEquals(20L, cleanerCheckpoints.get(tp).get, "Unselected log should have checkpoint offset updated") } @@ -775,7 +775,7 @@ class LogCleanerManagerTest extends Logging { cleanerCheckpoints.put(tp0, 10L) cleanerCheckpoints.put(tp1, 5L) - val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time).get + val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time)._1.get assertEquals(tp1, filthiestLog.topicPartition, "Dirtier log should be selected") assertEquals(15L, cleanerCheckpoints.get(tp0).get, "Unselected log should have checkpoint offset updated") } diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 39e85389a91b7..a2f6b36f0ee3e 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -380,12 +380,12 @@ class LogCleanerTest { assertEquals(List(2, 1, 3), LogTestUtils.keysInLog(log)) assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) - // delete horizon forced to 0 to verify marker is not removed early + // clean again with same timestamp to verify marker is not removed early dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = largeTimestamp)._1 assertEquals(List(2, 1, 3), LogTestUtils.keysInLog(log)) assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) - // clean again with large delete horizon and verify the marker is removed + // clean again with max timestamp to verify the marker is removed dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, log.activeSegment.baseOffset), currentTime = Long.MaxValue)._1 assertEquals(List(2, 1, 3), LogTestUtils.keysInLog(log)) assertEquals(List(4, 5, 6, 7, 8), offsetsInLog(log)) @@ -574,7 +574,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 = Long.MaxValue - 1) + cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = Long.MaxValue) assertEquals(List(1), offsetsInLog(log)) assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log)) } @@ -1869,6 +1869,7 @@ class LogCleanerTest { * 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. + * Returns the first dirty offset in the log as a result of the second cleaning. */ private def runTwoPassClean(cleaner: Cleaner, logToClean: LogToClean, currentTime: Long, legacyDeleteHorizonMs: Long = -1L, tombstoneRetentionMs: Long = 86400000) : Long = { From 05fea4287b47c6e2540b9c1fce893bfe25273b59 Mon Sep 17 00:00:00 2001 From: Matthew Wong Date: Thu, 9 Sep 2021 14:43:49 -0700 Subject: [PATCH 13/20] addressing comments --- .../common/record/DefaultRecordBatch.java | 5 +- .../src/main/scala/kafka/log/LogCleaner.scala | 35 ++++++------- .../scala/kafka/log/LogCleanerManager.scala | 18 +++---- .../kafka/log/LogCleanerManagerTest.scala | 50 +++++++++++++------ 4 files changed, 65 insertions(+), 43 deletions(-) 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 3d8c115b31533..ec3c7204fe67b 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 @@ -162,10 +162,9 @@ public void ensureValid() { } /** - * Gets the base timestamp of the batch which is used to calculate the timestamp deltas. + * Gets the base timestamp of the batch which is used to calculate the record timestamps from the deltas. * - * @return The base timestamp or - * {@link RecordBatch#NO_TIMESTAMP} if the batch is empty + * @return The base timestamp */ public long baseTimestamp() { return buffer.getLong(BASE_TIMESTAMP_OFFSET); diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index fdbfe19081d3d..4231c611d86b8 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -342,7 +342,7 @@ class LogCleaner(initialConfig: CleanerConfig, @throws(classOf[LogCleaningException]) private def cleanFilthiestLog(): Boolean = { val preCleanStats = new PreCleanStats() - val (ltc, reason) = cleanerManager.grabFilthiestCompactedLog(time, preCleanStats) + val ltc = cleanerManager.grabFilthiestCompactedLog(time, preCleanStats) val cleaned = ltc match { case None => false @@ -350,7 +350,7 @@ class LogCleaner(initialConfig: CleanerConfig, // there's a log, clean it this.lastPreCleanStats = preCleanStats try { - cleanLog(cleanable, reason) + cleanLog(cleanable) true } catch { case e @ (_: ThreadShutdownException | _: ControlThrowable) => throw e @@ -374,11 +374,11 @@ class LogCleaner(initialConfig: CleanerConfig, cleaned } - private def cleanLog(cleanable: LogToClean, reason: LogCleaningReason): Unit = { + private def cleanLog(cleanable: LogToClean): Unit = { val startOffset = cleanable.firstDirtyOffset var endOffset = startOffset try { - val (nextDirtyOffset, cleanerStats) = cleaner.clean(cleanable, reason) + val (nextDirtyOffset, cleanerStats) = cleaner.clean(cleanable) endOffset = nextDirtyOffset recordStats(cleaner.id, cleanable.log.name, startOffset, endOffset, cleanerStats) } catch { @@ -490,11 +490,10 @@ private[log] class Cleaner(val id: Int, * Clean the given log * * @param cleanable The log to be cleaned - * @param reason The reason the log is being cleaned, for logging purposes * * @return The first offset not cleaned and the statistics for this round of cleaning */ - private[log] def clean(cleanable: LogToClean, reason: LogCleaningReason = DirtyRatio): (Long, CleanerStats) = { + 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 = @@ -502,11 +501,11 @@ private[log] class Cleaner(val id: Int, case None => 0L case Some(seg) => seg.lastModified - cleanable.log.config.deleteRetentionMs } - doClean(cleanable, time.milliseconds(), legacyDeleteHorizonMs = deleteHorizonMs, logCleaningReason = reason) + doClean(cleanable, time.milliseconds(), legacyDeleteHorizonMs = deleteHorizonMs) } - private[log] def doClean(cleanable: LogToClean, currentTime: Long, legacyDeleteHorizonMs: Long = -1L, logCleaningReason: LogCleaningReason = DirtyRatio): (Long, CleanerStats) = { - info("Beginning cleaning of log %s due to %s.".format(cleanable.log.name, logCleaningReason.getClass.getName)) + private[log] def doClean(cleanable: LogToClean, currentTime: Long, legacyDeleteHorizonMs: Long = -1L): (Long, CleanerStats) = { + info("Beginning cleaning of log %s due to %s.".format(cleanable.log.name, cleanable.logCleaningReason.getClass.getName)) val log = cleanable.log log.latestDeleteHorizon = RecordBatch.NO_TIMESTAMP @@ -581,7 +580,8 @@ private[log] class Cleaner(val id: Int, val retainLegacyDeletesAndTxnMarkers = currentSegment.lastModified > legacyDeleteHorizonMs info(s"Cleaning $currentSegment in log ${log.name} into ${cleaned.baseOffset} " + - s"with legacy deletion horizon $legacyDeleteHorizonMs, " + + s"with an upper bound deletion horizon $legacyDeleteHorizonMs computed from " + + s"the segment last modified time of ${currentSegment.lastModified}," + s"${if(retainLegacyDeletesAndTxnMarkers) "retaining" else "discarding"} deletes.") try { @@ -656,11 +656,10 @@ private[log] class Cleaner(val id: Int, // 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) { - discardBatchRecords = canDiscardBatch && batch.deleteHorizonMs().isPresent && batch.deleteHorizonMs().getAsLong <= currentTime - } else { + if (batch.isControlBatch) + discardBatchRecords = canDiscardBatch && batch.deleteHorizonMs().isPresent && batch.deleteHorizonMs().getAsLong <= currentTime + else discardBatchRecords = canDiscardBatch - } def isBatchLastRecordOfProducer: Boolean = { // We retain the batch in order to preserve the state of active producers. There are three cases: @@ -803,11 +802,12 @@ private[log] class Cleaner(val id: Int, */ val latestOffsetForKey = record.offset() >= foundOffset val legacyRecord = batch.magic() < RecordBatch.MAGIC_VALUE_V2 - val shouldRetainDeletes = + def shouldRetainDeletes = { if (!legacyRecord) !batch.deleteHorizonMs().isPresent || currentTime < batch.deleteHorizonMs().getAsLong else retainDeletesForLegacyRecords + } val isRetainedValue = record.hasValue || shouldRetainDeletes latestOffsetForKey && isRetainedValue } else { @@ -1092,13 +1092,14 @@ private class CleanerStats(time: Time = Time.SYSTEM) { /** * Helper class for a log, its topic/partition, the first cleanable position, the first uncleanable dirty position, - * and whether it needs compaction immediately. + * the reason why it is being cleaned, and whether it needs compaction immediately. */ private case class LogToClean(topicPartition: TopicPartition, log: UnifiedLog, firstDirtyOffset: Long, uncleanableOffset: Long, - needCompactionNow: Boolean = false) extends Ordered[LogToClean] { + needCompactionNow: Boolean = false, + var logCleaningReason: LogCleaningReason = DirtyRatio) 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 diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala index 3749c43e62c6f..953d87d938219 100755 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -170,7 +170,7 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], * the log manager maintains. * Returns a tuple of an Option of the log selected to be cleaned and the reason it was selected. */ - def grabFilthiestCompactedLog(time: Time, preCleanStats: PreCleanStats = new PreCleanStats()): (Option[LogToClean], LogCleaningReason) = { + def grabFilthiestCompactedLog(time: Time, preCleanStats: PreCleanStats = new PreCleanStats()): Option[LogToClean] = { inLock(lock) { val now = time.milliseconds this.timeOfLastRun = now @@ -192,7 +192,7 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], val compactionDelayMs = maxCompactionDelay(log, offsetsToClean.firstDirtyOffset, now) preCleanStats.updateMaxCompactionDelay(compactionDelayMs) - LogToClean(topicPartition, log, offsetsToClean.firstDirtyOffset, offsetsToClean.firstUncleanableDirtyOffset, compactionDelayMs > 0) + LogToClean(topicPartition, log, offsetsToClean.firstDirtyOffset, offsetsToClean.firstUncleanableDirtyOffset, compactionDelayMs > 0, DirtyRatio) } catch { case e: Throwable => throw new LogCleaningException(log, s"Failed to calculate log cleaning stats for partition $topicPartition", e) @@ -205,27 +205,27 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], (ltc.needCompactionNow && ltc.cleanableBytes > 0) || ltc.cleanableRatio > ltc.log.config.minCleanableRatio } - if(cleanableLogs.isEmpty) { - val logsWithTombstonesExpired = dirtyLogs.filter { - case ltc => + if (cleanableLogs.isEmpty) { + val logsWithTombstonesExpired = dirtyLogs.filter { 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 != RecordBatch.NO_TIMESTAMP && ltc.log.latestDeleteHorizon <= time.milliseconds() } - if (!logsWithTombstonesExpired.isEmpty) { + if (logsWithTombstonesExpired.nonEmpty) { val filthiest = logsWithTombstonesExpired.max + filthiest.logCleaningReason = DeleteHorizon inProgress.put(filthiest.topicPartition, LogCleaningInProgress) - (Some(filthiest), DeleteHorizon) + Some(filthiest) } else { - (None, DirtyRatio) + None } } else { preCleanStats.recordCleanablePartitions(cleanableLogs.size) val filthiest = cleanableLogs.max inProgress.put(filthiest.topicPartition, LogCleaningInProgress) - (Some(filthiest), DirtyRatio) + Some(filthiest) } } } diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index adb2c32cc4542..0ec9480ee139a 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -139,7 +139,7 @@ class LogCleanerManagerTest extends Logging { val cleanerManager = createCleanerManagerMock(logsPool) cleanerCheckpoints.put(tp, 1) - val thrownException = assertThrows(classOf[LogCleaningException], () => cleanerManager.grabFilthiestCompactedLog(time)._1.get) + val thrownException = assertThrows(classOf[LogCleaningException], () => cleanerManager.grabFilthiestCompactedLog(time).get) assertEquals(log, thrownException.log) assertTrue(thrownException.getCause.isInstanceOf[IllegalStateException]) } @@ -156,7 +156,8 @@ class LogCleanerManagerTest extends Logging { val cleanerManager = createCleanerManagerMock(logs) partitions.foreach(partition => cleanerCheckpoints.put(partition, 20)) - val filthiestLog: LogToClean = cleanerManager.grabFilthiestCompactedLog(time)._1.get + val filthiestLog: LogToClean = cleanerManager.grabFilthiestCompactedLog(time).get + assertEquals(DirtyRatio, filthiestLog.logCleaningReason) assertEquals(tp2, filthiestLog.topicPartition) assertEquals(tp2, filthiestLog.log.topicPartition) } @@ -175,7 +176,8 @@ class LogCleanerManagerTest extends Logging { cleanerManager.markPartitionUncleanable(logs.get(tp2).dir.getParent, tp2) - val filthiestLog: LogToClean = cleanerManager.grabFilthiestCompactedLog(time)._1.get + val filthiestLog: LogToClean = cleanerManager.grabFilthiestCompactedLog(time).get + assertEquals(DirtyRatio, filthiestLog.logCleaningReason) assertEquals(tp1, filthiestLog.topicPartition) assertEquals(tp1, filthiestLog.log.topicPartition) } @@ -194,8 +196,8 @@ class LogCleanerManagerTest extends Logging { cleanerManager.setCleaningState(tp2, LogCleaningInProgress) - val filthiestLog: LogToClean = cleanerManager.grabFilthiestCompactedLog(time)._1.get - + val filthiestLog: LogToClean = cleanerManager.grabFilthiestCompactedLog(time).get + assertEquals(DirtyRatio, filthiestLog.logCleaningReason) assertEquals(tp1, filthiestLog.topicPartition) assertEquals(tp1, filthiestLog.log.topicPartition) } @@ -215,10 +217,30 @@ class LogCleanerManagerTest extends Logging { cleanerManager.setCleaningState(tp2, LogCleaningInProgress) cleanerManager.markPartitionUncleanable(logs.get(tp1).dir.getParent, tp1) - val filthiestLog: Option[LogToClean] = cleanerManager.grabFilthiestCompactedLog(time)._1 + val filthiestLog: Option[LogToClean] = cleanerManager.grabFilthiestCompactedLog(time) assertEquals(None, filthiestLog) } + @Test + def testGrabFilthiestCompactedLogReturnsLogWithDeleteHorizon(): Unit = { + val tp0 = new TopicPartition("wishing-well", 0) + val tp1 = new TopicPartition("wishing-well", 1) + val tp2 = new TopicPartition("wishing-well", 2) + val partitions = Seq(tp0, tp1, tp2) + + // setup logs without any cleanable ranges: [20, 20], [20, 20], [20, 20] + val logs = setupIncreasinglyFilthyLogs(partitions, startNumBatches = 20, batchIncrement = 0) + // modify latestDeleteHorizon for tp2 to signal that this log has tombstones ready for deletion + logs.get(tp2).latestDeleteHorizon = 0 + val cleanerManager = createCleanerManagerMock(logs) + partitions.foreach(partition => cleanerCheckpoints.put(partition, 20)) + + val filthiestLog: LogToClean = cleanerManager.grabFilthiestCompactedLog(time).get + assertEquals(DeleteHorizon, filthiestLog.logCleaningReason) + assertEquals(tp2, filthiestLog.topicPartition) + assertEquals(tp2, filthiestLog.log.topicPartition) + } + @Test def testDirtyOffsetResetIfLargerThanEndOffset(): Unit = { val tp = new TopicPartition("foo", 0) @@ -226,7 +248,7 @@ class LogCleanerManagerTest extends Logging { val cleanerManager = createCleanerManagerMock(logs) cleanerCheckpoints.put(tp, 200) - val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time)._1.get + val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time).get assertEquals(0L, filthiestLog.firstDirtyOffset) } @@ -240,7 +262,7 @@ class LogCleanerManagerTest extends Logging { val cleanerManager = createCleanerManagerMock(logs) cleanerCheckpoints.put(tp, 0L) - val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time)._1.get + val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time).get assertEquals(10L, filthiestLog.firstDirtyOffset) } @@ -264,7 +286,7 @@ class LogCleanerManagerTest extends Logging { cleanerCheckpoints.put(tp, 0L) // The active segment is uncleanable and hence not filthy from the POV of the CleanerManager. - val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time)._1 + val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time) assertEquals(None, filthiestLog) } @@ -290,7 +312,7 @@ class LogCleanerManagerTest extends Logging { cleanerCheckpoints.put(tp, 3L) // These segments are uncleanable and hence not filthy - val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time)._1 + val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time) assertEquals(None, filthiestLog) } @@ -364,12 +386,12 @@ class LogCleanerManagerTest extends Logging { log.updateConfig(config) // log cleanup inprogress, the log is not available for compaction - val cleanable = cleanerManager.grabFilthiestCompactedLog(time)._1 + val cleanable = cleanerManager.grabFilthiestCompactedLog(time) assertEquals(0, cleanable.size, "should have 0 logs ready to be compacted") // log cleanup finished, and log can be picked up for compaction cleanerManager.resumeCleaning(deletableLog.map(_._1)) - val cleanable2 = cleanerManager.grabFilthiestCompactedLog(time)._1 + val cleanable2 = cleanerManager.grabFilthiestCompactedLog(time) assertEquals(1, cleanable2.size, "should have 1 logs ready to be compacted") // update cleanup policy to delete @@ -753,7 +775,7 @@ class LogCleanerManagerTest extends Logging { val cleanerManager = createCleanerManagerMock(logs) cleanerCheckpoints.put(tp, 15L) - val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time)._1 + val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time) assertEquals(None, filthiestLog, "Log should not be selected for cleaning") assertEquals(20L, cleanerCheckpoints.get(tp).get, "Unselected log should have checkpoint offset updated") } @@ -775,7 +797,7 @@ class LogCleanerManagerTest extends Logging { cleanerCheckpoints.put(tp0, 10L) cleanerCheckpoints.put(tp1, 5L) - val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time)._1.get + val filthiestLog = cleanerManager.grabFilthiestCompactedLog(time).get assertEquals(tp1, filthiestLog.topicPartition, "Dirtier log should be selected") assertEquals(15L, cleanerCheckpoints.get(tp0).get, "Unselected log should have checkpoint offset updated") } From 217624a6cc2978a7e1e3f88d95fd35f97f3e36bf Mon Sep 17 00:00:00 2001 From: Matthew Wong Date: Thu, 9 Sep 2021 14:50:34 -0700 Subject: [PATCH 14/20] address Jason's comment on extra var in shouldRetainRecord --- core/src/main/scala/kafka/log/LogCleaner.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 4231c611d86b8..3fafa3dba53fd 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -687,13 +687,11 @@ private[log] class Cleaner(val id: Int, } 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 + false else - isRecordRetained = Cleaner.this.shouldRetainRecord(map, retainLegacyDeletesAndTxnMarkers, batch, record, stats, currentTime = currentTime) - isRecordRetained + Cleaner.this.shouldRetainRecord(map, retainLegacyDeletesAndTxnMarkers, batch, record, stats, currentTime = currentTime) } } From 890c355572cf696c3bb07bdbd61855aa31119847 Mon Sep 17 00:00:00 2001 From: Matthew Wong Date: Thu, 9 Sep 2021 15:02:53 -0700 Subject: [PATCH 15/20] move legacyDeleteHorizonMs computation to doClean --- core/src/main/scala/kafka/log/LogCleaner.scala | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 3fafa3dba53fd..26c7f0e733460 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -494,18 +494,19 @@ private[log] class Cleaner(val id: Int, * @return The first offset not cleaned and the statistics for this round of cleaning */ private[log] def clean(cleanable: LogToClean): (Long, CleanerStats) = { + doClean(cleanable, time.milliseconds()) + } + + private[log] def doClean(cleanable: LogToClean, currentTime: Long): (Long, CleanerStats) = { + info("Beginning cleaning of log %s due to %s.".format(cleanable.log.name, cleanable.logCleaningReason.getClass.getName)) + // 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 = + val legacyDeleteHorizonMs = cleanable.log.logSegments(0, cleanable.firstDirtyOffset).lastOption match { case None => 0L case Some(seg) => seg.lastModified - cleanable.log.config.deleteRetentionMs - } - doClean(cleanable, time.milliseconds(), legacyDeleteHorizonMs = deleteHorizonMs) - } - - private[log] def doClean(cleanable: LogToClean, currentTime: Long, legacyDeleteHorizonMs: Long = -1L): (Long, CleanerStats) = { - info("Beginning cleaning of log %s due to %s.".format(cleanable.log.name, cleanable.logCleaningReason.getClass.getName)) + } val log = cleanable.log log.latestDeleteHorizon = RecordBatch.NO_TIMESTAMP From 9f6967388e57a55f3326e058ee2d427b83a7389a Mon Sep 17 00:00:00 2001 From: Matthew Wong Date: Fri, 10 Sep 2021 13:52:40 -0700 Subject: [PATCH 16/20] remove DeleteHorizon cleaning pass and LogCleaningReason --- .../src/main/scala/kafka/log/LogCleaner.scala | 5 ++-- .../scala/kafka/log/LogCleanerManager.scala | 27 +++---------------- .../kafka/log/LogCleanerManagerTest.scala | 23 ---------------- .../scala/unit/kafka/log/LogCleanerTest.scala | 6 ++--- 4 files changed, 9 insertions(+), 52 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 26c7f0e733460..9d08abc1911fa 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -498,7 +498,7 @@ private[log] class Cleaner(val id: Int, } private[log] def doClean(cleanable: LogToClean, currentTime: Long): (Long, CleanerStats) = { - info("Beginning cleaning of log %s due to %s.".format(cleanable.log.name, cleanable.logCleaningReason.getClass.getName)) + info("Beginning cleaning of log %s".format(cleanable.log.name)) // 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 @@ -1097,8 +1097,7 @@ private case class LogToClean(topicPartition: TopicPartition, log: UnifiedLog, firstDirtyOffset: Long, uncleanableOffset: Long, - needCompactionNow: Boolean = false, - var logCleaningReason: LogCleaningReason = DirtyRatio) extends Ordered[LogToClean] { + 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 diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala index 953d87d938219..f41594f5d159a 100755 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -30,7 +30,6 @@ 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} @@ -39,10 +38,6 @@ 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] sealed trait LogCleaningReason -private[log] case object DirtyRatio extends LogCleaningReason -private[log] case object DeleteHorizon extends LogCleaningReason - private[log] class LogCleaningException(val log: UnifiedLog, private val message: String, private val cause: Throwable) extends KafkaException(message, cause) @@ -192,7 +187,7 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], val compactionDelayMs = maxCompactionDelay(log, offsetsToClean.firstDirtyOffset, now) preCleanStats.updateMaxCompactionDelay(compactionDelayMs) - LogToClean(topicPartition, log, offsetsToClean.firstDirtyOffset, offsetsToClean.firstUncleanableDirtyOffset, compactionDelayMs > 0, DirtyRatio) + LogToClean(topicPartition, log, offsetsToClean.firstDirtyOffset, offsetsToClean.firstUncleanableDirtyOffset, compactionDelayMs > 0) } catch { case e: Throwable => throw new LogCleaningException(log, s"Failed to calculate log cleaning stats for partition $topicPartition", e) @@ -205,23 +200,9 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], (ltc.needCompactionNow && ltc.cleanableBytes > 0) || ltc.cleanableRatio > ltc.log.config.minCleanableRatio } - if (cleanableLogs.isEmpty) { - val logsWithTombstonesExpired = dirtyLogs.filter { 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 != RecordBatch.NO_TIMESTAMP && ltc.log.latestDeleteHorizon <= time.milliseconds() - } - if (logsWithTombstonesExpired.nonEmpty) { - val filthiest = logsWithTombstonesExpired.max - filthiest.logCleaningReason = DeleteHorizon - inProgress.put(filthiest.topicPartition, LogCleaningInProgress) - Some(filthiest) - } else { - None - } - } else { + if (cleanableLogs.isEmpty) + None + else { preCleanStats.recordCleanablePartitions(cleanableLogs.size) val filthiest = cleanableLogs.max inProgress.put(filthiest.topicPartition, LogCleaningInProgress) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index 0ec9480ee139a..c4a71cc22cc97 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -157,7 +157,6 @@ class LogCleanerManagerTest extends Logging { partitions.foreach(partition => cleanerCheckpoints.put(partition, 20)) val filthiestLog: LogToClean = cleanerManager.grabFilthiestCompactedLog(time).get - assertEquals(DirtyRatio, filthiestLog.logCleaningReason) assertEquals(tp2, filthiestLog.topicPartition) assertEquals(tp2, filthiestLog.log.topicPartition) } @@ -177,7 +176,6 @@ class LogCleanerManagerTest extends Logging { cleanerManager.markPartitionUncleanable(logs.get(tp2).dir.getParent, tp2) val filthiestLog: LogToClean = cleanerManager.grabFilthiestCompactedLog(time).get - assertEquals(DirtyRatio, filthiestLog.logCleaningReason) assertEquals(tp1, filthiestLog.topicPartition) assertEquals(tp1, filthiestLog.log.topicPartition) } @@ -197,7 +195,6 @@ class LogCleanerManagerTest extends Logging { cleanerManager.setCleaningState(tp2, LogCleaningInProgress) val filthiestLog: LogToClean = cleanerManager.grabFilthiestCompactedLog(time).get - assertEquals(DirtyRatio, filthiestLog.logCleaningReason) assertEquals(tp1, filthiestLog.topicPartition) assertEquals(tp1, filthiestLog.log.topicPartition) } @@ -221,26 +218,6 @@ class LogCleanerManagerTest extends Logging { assertEquals(None, filthiestLog) } - @Test - def testGrabFilthiestCompactedLogReturnsLogWithDeleteHorizon(): Unit = { - val tp0 = new TopicPartition("wishing-well", 0) - val tp1 = new TopicPartition("wishing-well", 1) - val tp2 = new TopicPartition("wishing-well", 2) - val partitions = Seq(tp0, tp1, tp2) - - // setup logs without any cleanable ranges: [20, 20], [20, 20], [20, 20] - val logs = setupIncreasinglyFilthyLogs(partitions, startNumBatches = 20, batchIncrement = 0) - // modify latestDeleteHorizon for tp2 to signal that this log has tombstones ready for deletion - logs.get(tp2).latestDeleteHorizon = 0 - val cleanerManager = createCleanerManagerMock(logs) - partitions.foreach(partition => cleanerCheckpoints.put(partition, 20)) - - val filthiestLog: LogToClean = cleanerManager.grabFilthiestCompactedLog(time).get - assertEquals(DeleteHorizon, filthiestLog.logCleaningReason) - assertEquals(tp2, filthiestLog.topicPartition) - assertEquals(tp2, filthiestLog.log.topicPartition) - } - @Test def testDirtyOffsetResetIfLargerThanEndOffset(): Unit = { val tp = new TopicPartition("foo", 0) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index a2f6b36f0ee3e..a220d44fd539c 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -1872,9 +1872,9 @@ class LogCleanerTest { * Returns the first dirty offset in the log as a result of the second cleaning. */ 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 + tombstoneRetentionMs: Long = 86400000) : Long = { + cleaner.doClean(logToClean, currentTime) + cleaner.doClean(logToClean, currentTime + tombstoneRetentionMs + 1)._1 } } From 6f2f2b7fc877f8d06f07e00cc66c65166a5629b0 Mon Sep 17 00:00:00 2001 From: Matthew Wong Date: Fri, 10 Sep 2021 14:14:20 -0700 Subject: [PATCH 17/20] remove logic for delethorizon-triggered cleaning --- .../kafka/common/record/MemoryRecords.java | 13 ------ .../src/main/scala/kafka/log/LogCleaner.scala | 15 +------ .../src/main/scala/kafka/log/UnifiedLog.scala | 6 --- .../kafka/log/LogCleanerIntegrationTest.scala | 43 ++----------------- 4 files changed, 6 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 768b16410cf61..eacc2113b0063 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 @@ -193,8 +193,6 @@ private static FilterResult filterTo(TopicPartition partition, Iterable= RecordBatch.MAGIC_VALUE_V2 && (containsTombstones || containsMarkerForEmptyTxn) && !batch.deleteHorizonMs().isPresent(); if (writeOriginalBatch && !needToSetDeleteHorizon) { - if (batch.deleteHorizonMs().orElse(RecordBatch.NO_TIMESTAMP) > filterResult.latestDeleteHorizon()) - filterResult.updateLatestDeleteHorizon(batch.deleteHorizonMs().getAsLong()); batch.writeTo(bufferOutputStream); filterResult.updateRetainedBatchMetadata(batch, retainedRecords.size(), false); } else { @@ -205,8 +203,6 @@ private static FilterResult filterTo(TopicPartition partition, Iterable filterResult.latestDeleteHorizon()) - filterResult.updateLatestDeleteHorizon(deleteHorizonMs); MemoryRecords records = builder.build(); int filteredBatchSize = records.sizeInBytes(); @@ -407,20 +403,11 @@ 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 9d08abc1911fa..b2f38798f6bdc 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -509,8 +509,6 @@ private[log] class Cleaner(val id: Int, } val log = cleanable.log - log.latestDeleteHorizon = RecordBatch.NO_TIMESTAMP - val stats = new CleanerStats() // build the offset map @@ -586,10 +584,8 @@ private[log] class Cleaner(val id: Int, s"${if(retainLegacyDeletesAndTxnMarkers) "retaining" else "discarding"} deletes.") try { - val latestDeleteHorizon: Long = cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainLegacyDeletesAndTxnMarkers, log.config.deleteRetentionMs, + 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 } catch { case e: LogSegmentOffsetOverflowException => // Split the current segment. It's also safest to abort the current cleaning process, so that we retry from @@ -635,8 +631,6 @@ private[log] class Cleaner(val id: Int, * @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 - * - * @return the latestDeleteHorizon that is found from the FilterResult of the cleaning */ private[log] def cleanInto(topicPartition: TopicPartition, sourceRecords: FileRecords, @@ -648,7 +642,7 @@ private[log] class Cleaner(val id: Int, transactionMetadata: CleanedTransactionMetadata, lastRecordsOfActiveProducers: Map[Long, LastRecord], stats: CleanerStats, - currentTime: Long): Long = { + currentTime: Long): Unit = { val logCleanerFilter: RecordFilter = new RecordFilter(currentTime, deleteRetentionMs) { var discardBatchRecords: Boolean = _ @@ -696,7 +690,6 @@ private[log] class Cleaner(val id: Int, } } - var latestDeleteHorizon: Long = RecordBatch.NO_TIMESTAMP var position = 0 while (position < sourceRecords.sizeInBytes) { checkDone(topicPartition) @@ -708,9 +701,6 @@ 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) @@ -737,7 +727,6 @@ private[log] class Cleaner(val id: Int, growBuffersOrFail(sourceRecords, position, maxLogMessageSize, records) } restoreBuffers() - latestDeleteHorizon } diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index a816416d39de5..029d1fbdec811 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -288,12 +288,6 @@ class UnifiedLog(@volatile var logStartOffset: Long, @volatile var partitionMetadataFile : PartitionMetadataFile = null - /* Keep track of the largest deleteHorizonMs that is present from records in the log. This value helps streamline the - * deletion of tombstones in compacted logs. The LogCleanerManager can use this value to determine if the log can - * be cleaned to remove eligible tombstones without waiting for the log's dirty section to grow large enough. - */ - @volatile var latestDeleteHorizon: Long = RecordBatch.NO_TIMESTAMP - locally { initializePartitionMetadata() updateLogStartOffset(logStartOffset) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 9c639a8f9aec0..5f6a56d21feb0 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -18,7 +18,6 @@ package kafka.log import java.io.PrintWriter -import java.util.Properties import com.yammer.metrics.core.{Gauge, MetricName} import kafka.metrics.{KafkaMetricsGroup, KafkaYammerMetrics} @@ -180,36 +179,6 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K s"log cleaner should have processed at least to offset $secondBlockCleanableSegmentOffset, but lastCleaned=$lastCleaned2") } - @Test - def testTombstoneCleanWithLowThroughput() : 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, backOffMs = 100L) - - 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) - - // roll the active segment - log.roll() - - cleaner.startup() - - val latestOffset: Long = log.logEndOffset - - assertTrue(cleaner.awaitCleaned(new TopicPartition("log-partition", 0), - latestOffset, maxWaitMs = 5000)) - assertEquals(log.latestDeleteHorizon, T0 + tombstoneRetentionMs) - - time.sleep(tombstoneRetentionMs + 1) - TestUtils.waitUntilTrue(() => log.size == 0, "Log should be empty") - } - private def readFromLog(log: UnifiedLog): Iterable[(Int, Int)] = { for (segment <- log.logSegments; record <- segment.log.records.asScala) yield { val key = TestUtils.readString(record.key).toInt @@ -219,17 +188,13 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K } private def writeKeyDups(numKeys: Int, numDups: Int, log: UnifiedLog, codec: CompressionType, timestamp: Long, - startValue: Int, step: Int, isRecordTombstone: Boolean = false): Seq[(Int, Int)] = { + startValue: Int, step: Int): Seq[(Int, Int)] = { var valCounter = startValue for (_ <- 0 until numDups; key <- 0 until numKeys) yield { val curValue = valCounter - 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) - + log.appendAsLeader(TestUtils.singletonRecords(value = curValue.toString.getBytes, codec = codec, + key = key.toString.getBytes, timestamp = timestamp), leaderEpoch = 0) + // move LSO forward to increase compaction bound log.updateHighWatermark(log.logEndOffset) valCounter += step (key, curValue) From 2477f345a2935c689d1ef6996ba4790e4c75ac33 Mon Sep 17 00:00:00 2001 From: Matthew Wong Date: Fri, 10 Sep 2021 15:15:27 -0700 Subject: [PATCH 18/20] address last comments on code comments, readability, logging --- .../src/main/scala/kafka/log/LogCleaner.scala | 8 +++-- .../scala/unit/kafka/log/LogCleanerTest.scala | 30 +++++++++---------- 2 files changed, 20 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index b2f38798f6bdc..c088c2dfc34bf 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -67,6 +67,8 @@ import scala.util.control.ControlThrowable * 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. + * This time is tracked by setting the base timestamp of a record batch with delete markers when the batch is recopied in the first cleaning that encounters + * it. The relative timestamps of the records in the batch are also modified when recopied in this cleaning according to the new base timestamp of the batch. * * Note that cleaning is more complicated with the idempotent/transactional producer capabilities. The following * are the key points: @@ -523,13 +525,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 legacy tombstones prior to %s)...".format(log.name, new Date(cleanableHorizonMs), new Date(legacyDeleteHorizonMs))) + info("Cleaning log %s (cleaning prior to %s, discarding tombstones prior to upper bound deletion horizon %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, legacyDeleteHorizonMs = legacyDeleteHorizonMs) + cleanSegments(log, group, offsetMap, currentTime, stats, transactionMetadata, legacyDeleteHorizonMs) // record buffer utilization stats.bufferUtilization = offsetMap.utilization @@ -557,7 +559,7 @@ private[log] class Cleaner(val id: Int, currentTime: Long, stats: CleanerStats, transactionMetadata: CleanedTransactionMetadata, - legacyDeleteHorizonMs: Long = -1L): Unit = { + legacyDeleteHorizonMs: Long): Unit = { // create a new segment with a suffix appended to the name of the log and indexes val cleaned = UnifiedLog.createNewCleanedSegment(log.dir, log.config, segments.head.baseOffset) transactionMetadata.cleanedIndex = Some(cleaned.txnIndex) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index a220d44fd539c..8f1d241a5d444 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -87,7 +87,7 @@ class LogCleanerTest { val stats = new CleanerStats() val expectedBytesRead = segments.map(_.size).sum val shouldRemain = LogTestUtils.keysInLog(log).filter(!keys.contains(_)) - cleaner.cleanSegments(log, segments, map, 0L, stats, new CleanedTransactionMetadata) + cleaner.cleanSegments(log, segments, map, 0L, stats, new CleanedTransactionMetadata, -1) assertEquals(shouldRemain, LogTestUtils.keysInLog(log)) assertEquals(expectedBytesRead, stats.bytesRead) } @@ -172,7 +172,7 @@ class LogCleanerTest { val segments = log.logSegments(0, log.activeSegment.baseOffset).toSeq val stats = new CleanerStats() cleaner.buildOffsetMap(log, 0, log.activeSegment.baseOffset, offsetMap, stats) - cleaner.cleanSegments(log, segments, offsetMap, 0L, stats, new CleanedTransactionMetadata) + cleaner.cleanSegments(log, segments, offsetMap, 0L, stats, new CleanedTransactionMetadata, -1) // Validate based on the file name that log segment file is renamed exactly once for async deletion assertEquals(expectedFileName, firstLogFile.file().getPath) @@ -736,7 +736,7 @@ class LogCleanerTest { // clean the log val stats = new CleanerStats() - cleaner.cleanSegments(log, Seq(log.logSegments.head), map, 0L, stats, new CleanedTransactionMetadata) + cleaner.cleanSegments(log, Seq(log.logSegments.head), map, 0L, stats, new CleanedTransactionMetadata, -1) val shouldRemain = LogTestUtils.keysInLog(log).filter(!keys.contains(_)) assertEquals(shouldRemain, LogTestUtils.keysInLog(log)) } @@ -749,7 +749,7 @@ class LogCleanerTest { val (log, offsetMap) = createLogWithMessagesLargerThanMaxSize(largeMessageSize = 1024 * 1024) val cleaner = makeCleaner(Int.MaxValue, maxMessageSize=1024) - cleaner.cleanSegments(log, Seq(log.logSegments.head), offsetMap, 0L, new CleanerStats, new CleanedTransactionMetadata) + cleaner.cleanSegments(log, Seq(log.logSegments.head), offsetMap, 0L, new CleanerStats, new CleanedTransactionMetadata, -1) val shouldRemain = LogTestUtils.keysInLog(log).filter(k => !offsetMap.map.containsKey(k.toString)) assertEquals(shouldRemain, LogTestUtils.keysInLog(log)) } @@ -768,7 +768,7 @@ class LogCleanerTest { val cleaner = makeCleaner(Int.MaxValue, maxMessageSize=1024) assertThrows(classOf[CorruptRecordException], () => - cleaner.cleanSegments(log, Seq(log.logSegments.head), offsetMap, 0L, new CleanerStats, new CleanedTransactionMetadata) + cleaner.cleanSegments(log, Seq(log.logSegments.head), offsetMap, 0L, new CleanerStats, new CleanedTransactionMetadata, -1) ) } @@ -785,7 +785,7 @@ class LogCleanerTest { val cleaner = makeCleaner(Int.MaxValue, maxMessageSize=1024) assertThrows(classOf[CorruptRecordException], () => - cleaner.cleanSegments(log, Seq(log.logSegments.head), offsetMap, 0L, new CleanerStats, new CleanedTransactionMetadata) + cleaner.cleanSegments(log, Seq(log.logSegments.head), offsetMap, 0L, new CleanerStats, new CleanedTransactionMetadata, -1) ) } @@ -1120,7 +1120,7 @@ class LogCleanerTest { keys.foreach(k => map.put(key(k), Long.MaxValue)) assertThrows(classOf[LogCleaningAbortedException], () => cleaner.cleanSegments(log, log.logSegments.take(3).toSeq, map, 0L, new CleanerStats(), - new CleanedTransactionMetadata) + new CleanedTransactionMetadata, -1) ) } @@ -1380,7 +1380,7 @@ class LogCleanerTest { // Try to clean segment with offset overflow. This will trigger log split and the cleaning itself must abort. assertThrows(classOf[LogCleaningAbortedException], () => cleaner.cleanSegments(log, Seq(segmentWithOverflow), offsetMap, 0L, new CleanerStats(), - new CleanedTransactionMetadata) + new CleanedTransactionMetadata, -1) ) assertEquals(numSegmentsInitial + 1, log.logSegments.size) assertEquals(allKeys, LogTestUtils.keysInLog(log)) @@ -1389,7 +1389,7 @@ class LogCleanerTest { // Clean each segment now that split is complete. for (segmentToClean <- log.logSegments) cleaner.cleanSegments(log, List(segmentToClean), offsetMap, 0L, new CleanerStats(), - new CleanedTransactionMetadata) + new CleanedTransactionMetadata, -1) assertEquals(expectedKeysAfterCleaning, LogTestUtils.keysInLog(log)) assertFalse(LogTestUtils.hasOffsetOverflow(log)) log.close() @@ -1430,7 +1430,7 @@ class LogCleanerTest { // clean the log cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats(), - new CleanedTransactionMetadata) + new CleanedTransactionMetadata, -1) // clear scheduler so that async deletes don't run time.scheduler.clear() var cleanedKeys = LogTestUtils.keysInLog(log) @@ -1446,7 +1446,7 @@ class LogCleanerTest { // clean again cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats(), - new CleanedTransactionMetadata) + new CleanedTransactionMetadata, -1) // clear scheduler so that async deletes don't run time.scheduler.clear() cleanedKeys = LogTestUtils.keysInLog(log) @@ -1463,7 +1463,7 @@ class LogCleanerTest { // clean again cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats(), - new CleanedTransactionMetadata) + new CleanedTransactionMetadata, -1) // clear scheduler so that async deletes don't run time.scheduler.clear() cleanedKeys = LogTestUtils.keysInLog(log) @@ -1485,7 +1485,7 @@ class LogCleanerTest { for (k <- 1 until messageCount by 2) offsetMap.put(key(k), Long.MaxValue) cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats(), - new CleanedTransactionMetadata) + new CleanedTransactionMetadata, -1) // clear scheduler so that async deletes don't run time.scheduler.clear() cleanedKeys = LogTestUtils.keysInLog(log) @@ -1503,7 +1503,7 @@ class LogCleanerTest { for (k <- 1 until messageCount by 2) offsetMap.put(key(k), Long.MaxValue) cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats(), - new CleanedTransactionMetadata) + new CleanedTransactionMetadata, -1) // clear scheduler so that async deletes don't run time.scheduler.clear() cleanedKeys = LogTestUtils.keysInLog(log) @@ -1521,7 +1521,7 @@ class LogCleanerTest { for (k <- 1 until messageCount by 2) offsetMap.put(key(k), Long.MaxValue) cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats(), - new CleanedTransactionMetadata) + new CleanedTransactionMetadata, -1) // clear scheduler so that async deletes don't run time.scheduler.clear() cleanedKeys = LogTestUtils.keysInLog(log) From b113c470daf640b81c9d44c3e79cb139acac20eb Mon Sep 17 00:00:00 2001 From: Matthew Wong Date: Wed, 15 Sep 2021 12:27:46 -0700 Subject: [PATCH 19/20] address Jun's comments --- core/src/main/scala/kafka/log/LogCleaner.scala | 7 +++---- core/src/main/scala/kafka/log/LogCleanerManager.scala | 1 - 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index c088c2dfc34bf..3ff9853ea39e9 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -504,6 +504,7 @@ private[log] class Cleaner(val id: Int, // 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 + // this timestamp is only used on the older message formats newer than MAGIC_VALUE_V2 val legacyDeleteHorizonMs = cleanable.log.logSegments(0, cleanable.firstDirtyOffset).lastOption match { case None => 0L @@ -680,7 +681,7 @@ private[log] class Cleaner(val id: Int, BatchRetention.DELETE else BatchRetention.DELETE_EMPTY - new RecordFilter.BatchRetentionResult(batchRetention, canDiscardBatch) + new RecordFilter.BatchRetentionResult(batchRetention, canDiscardBatch && batch.isControlBatch) } override def shouldRetainRecord(batch: RecordBatch, record: Record): Boolean = { @@ -1082,7 +1083,7 @@ private class CleanerStats(time: Time = Time.SYSTEM) { /** * Helper class for a log, its topic/partition, the first cleanable position, the first uncleanable dirty position, - * the reason why it is being cleaned, and whether it needs compaction immediately. + * and whether it needs compaction immediately. */ private case class LogToClean(topicPartition: TopicPartition, log: UnifiedLog, @@ -1133,8 +1134,6 @@ private[log] class CleanedTransactionMetadata { 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 diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala index f41594f5d159a..02d6a30ad2b0a 100755 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -163,7 +163,6 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], * 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. - * Returns a tuple of an Option of the log selected to be cleaned and the reason it was selected. */ def grabFilthiestCompactedLog(time: Time, preCleanStats: PreCleanStats = new PreCleanStats()): Option[LogToClean] = { inLock(lock) { From cf7b0a715ba5abfe4465dc3e5e8927d80d469cce Mon Sep 17 00:00:00 2001 From: Matthew Wong Date: Wed, 15 Sep 2021 13:14:48 -0700 Subject: [PATCH 20/20] fix comment --- core/src/main/scala/kafka/log/LogCleaner.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 3ff9853ea39e9..baf937d6415eb 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -504,7 +504,7 @@ private[log] class Cleaner(val id: Int, // 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 - // this timestamp is only used on the older message formats newer than MAGIC_VALUE_V2 + // this timestamp is only used on the older message formats older than MAGIC_VALUE_V2 val legacyDeleteHorizonMs = cleanable.log.logSegments(0, cleanable.firstDirtyOffset).lastOption match { case None => 0L