From e9c0e381a764d7a52ca730abd2248634831d4f02 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 26 Dec 2024 07:55:03 -0800 Subject: [PATCH 01/18] KAFKA-13093: Log compaction should write new segments with record version v2 (KIP-724) --- .../kafka/common/record/MemoryRecords.java | 67 +++++++++---------- .../internals/FetchRequestManagerTest.java | 2 +- .../consumer/internals/FetcherTest.java | 2 +- .../ShareConsumeRequestManagerTest.java | 2 +- .../common/record/MemoryRecordsTest.java | 27 ++++---- .../src/main/scala/kafka/log/LogCleaner.scala | 7 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 6 +- 7 files changed, 57 insertions(+), 56 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 650071474db65..07bdffc17476a 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 @@ -151,8 +151,9 @@ public Integer firstBatchSize() { * @return A FilterResult with a summary of the output (for metrics) and potentially an overflow buffer */ public FilterResult filterTo(TopicPartition partition, RecordFilter filter, ByteBuffer destinationBuffer, - int maxRecordBatchSize, BufferSupplier decompressionBufferSupplier) { - return filterTo(partition, batches(), filter, destinationBuffer, maxRecordBatchSize, decompressionBufferSupplier); + int maxRecordBatchSize, TimestampType timestampTypeConfig, + BufferSupplier decompressionBufferSupplier) { + return filterTo(partition, batches(), filter, destinationBuffer, maxRecordBatchSize, timestampTypeConfig, decompressionBufferSupplier); } /** @@ -161,7 +162,7 @@ public FilterResult filterTo(TopicPartition partition, RecordFilter filter, Byte */ private static FilterResult filterTo(TopicPartition partition, Iterable batches, RecordFilter filter, ByteBuffer destinationBuffer, int maxRecordBatchSize, - BufferSupplier decompressionBufferSupplier) { + TimestampType timestampTypeConfig, BufferSupplier decompressionBufferSupplier) { FilterResult filterResult = new FilterResult(destinationBuffer); ByteBufferOutputStream bufferOutputStream = new ByteBufferOutputStream(destinationBuffer); for (MutableRecordBatch batch : batches) { @@ -174,15 +175,9 @@ private static FilterResult filterTo(TopicPartition partition, Iterable retainedRecords = new ArrayList<>(); - - final BatchFilterResult iterationResult = filterBatch(batch, decompressionBufferSupplier, filterResult, filter, - batchMagic, true, retainedRecords); + final BatchFilterResult iterationResult = filterBatch(batch, decompressionBufferSupplier, filterResult, + filter); + List retainedRecords = iterationResult.retainedRecords; boolean containsTombstones = iterationResult.containsTombstones; boolean writeOriginalBatch = iterationResult.writeOriginalBatch; long maxOffset = iterationResult.maxOffset; @@ -202,7 +197,8 @@ private static FilterResult filterTo(TopicPartition partition, Iterable batch.sizeInBytes() && filteredBatchSize > maxRecordBatchSize) @@ -217,11 +213,11 @@ private static FilterResult filterTo(TopicPartition partition, Iterable retainedRecords) { - long maxOffset = -1; - boolean containsTombstones = false; + RecordFilter filter) { try (final CloseableIterator iterator = batch.streamingIterator(decompressionBufferSupplier)) { + long maxOffset = -1; + boolean containsTombstones = false; + // Convert records with old record versions + boolean writeOriginalBatch = batch.magic() >= RecordBatch.CURRENT_MAGIC_VALUE; + List retainedRecords = new ArrayList<>(); 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(); @@ -272,17 +263,20 @@ private static BatchFilterResult filterBatch(RecordBatch batch, writeOriginalBatch = false; } } - return new BatchFilterResult(writeOriginalBatch, containsTombstones, maxOffset); + return new BatchFilterResult(retainedRecords, writeOriginalBatch, containsTombstones, maxOffset); } } private static class BatchFilterResult { + private final List retainedRecords; private final boolean writeOriginalBatch; private final boolean containsTombstones; private final long maxOffset; - private BatchFilterResult(final boolean writeOriginalBatch, - final boolean containsTombstones, - final long maxOffset) { + private BatchFilterResult(List retainedRecords, + final boolean writeOriginalBatch, + final boolean containsTombstones, + final long maxOffset) { + this.retainedRecords = retainedRecords; this.writeOriginalBatch = writeOriginalBatch; this.containsTombstones = containsTombstones; this.maxOffset = maxOffset; @@ -291,17 +285,20 @@ private BatchFilterResult(final boolean writeOriginalBatch, private static MemoryRecordsBuilder buildRetainedRecordsInto(RecordBatch originalBatch, List retainedRecords, + TimestampType timestampTypeConfig, ByteBufferOutputStream bufferOutputStream, final long deleteHorizonMs) { - byte magic = originalBatch.magic(); Compression compression = Compression.of(originalBatch.compressionType()).build(); - TimestampType timestampType = originalBatch.timestampType(); + // v0 has no timestamp type, use the topic config in that case (like we do when we up-convert produce requests) + TimestampType timestampType = originalBatch.timestampType() == TimestampType.NO_TIMESTAMP_TYPE ? + timestampTypeConfig : originalBatch.timestampType(); long logAppendTime = timestampType == TimestampType.LOG_APPEND_TIME ? originalBatch.maxTimestamp() : RecordBatch.NO_TIMESTAMP; - long baseOffset = magic >= RecordBatch.MAGIC_VALUE_V2 ? + long baseOffset = originalBatch.magic() >= RecordBatch.MAGIC_VALUE_V2 ? originalBatch.baseOffset() : retainedRecords.get(0).offset(); - MemoryRecordsBuilder builder = new MemoryRecordsBuilder(bufferOutputStream, magic, + // Convert records with older record versions to the current one + MemoryRecordsBuilder builder = new MemoryRecordsBuilder(bufferOutputStream, RecordBatch.CURRENT_MAGIC_VALUE, compression, timestampType, baseOffset, logAppendTime, originalBatch.producerId(), originalBatch.producerEpoch(), originalBatch.baseSequence(), originalBatch.isTransactional(), originalBatch.isControlBatch(), originalBatch.partitionLeaderEpoch(), bufferOutputStream.limit(), deleteHorizonMs); @@ -309,7 +306,7 @@ private static MemoryRecordsBuilder buildRetainedRecordsInto(RecordBatch origina for (Record record : retainedRecords) builder.append(record); - if (magic >= RecordBatch.MAGIC_VALUE_V2) + if (originalBatch.magic() >= RecordBatch.MAGIC_VALUE_V2) // we must preserve the last offset from the initial batch in order to ensure that the // last sequence number from the batch remains even after compaction. Otherwise, the producer // could incorrectly see an out of sequence error. diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java index 61ea2e8e56545..7f6a567fe26e7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java @@ -2542,7 +2542,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { return record.key() != null; } - }, ByteBuffer.allocate(1024), Integer.MAX_VALUE, BufferSupplier.NO_CACHING); + }, ByteBuffer.allocate(1024), Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); result.outputBuffer().flip(); MemoryRecords compactedRecords = MemoryRecords.readableRecords(result.outputBuffer()); 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 ab6a9a0c91de3..3aec39d2dedaa 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 @@ -2528,7 +2528,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { return record.key() != null; } - }, ByteBuffer.allocate(1024), Integer.MAX_VALUE, BufferSupplier.NO_CACHING); + }, ByteBuffer.allocate(1024), Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); result.outputBuffer().flip(); MemoryRecords compactedRecords = MemoryRecords.readableRecords(result.outputBuffer()); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java index 1b1ed587203ef..e6f78f54c05da 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java @@ -1259,7 +1259,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { return record.key() != null; } - }, ByteBuffer.allocate(1024), Integer.MAX_VALUE, BufferSupplier.NO_CACHING); + }, ByteBuffer.allocate(1024), Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); result.outputBuffer().flip(); MemoryRecords compactedRecords = MemoryRecords.readableRecords(result.outputBuffer()); 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 80a77d647b474..7dd8e8ced09be 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 @@ -292,7 +292,7 @@ public void testFilterToPreservesPartitionLeaderEpoch(Args args) { ByteBuffer filtered = ByteBuffer.allocate(2048); builder.build().filterTo(new TopicPartition("foo", 0), new RetainNonNullKeysFilter(), filtered, - Integer.MAX_VALUE, BufferSupplier.NO_CACHING); + Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); @@ -345,7 +345,7 @@ protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { // delete the records return false; } - }, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); + }, filtered, Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); // Verify filter result assertEquals(numRecords, filterResult.messagesRead()); @@ -406,7 +406,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { return false; } - }, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); + }, filtered, Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); // Verify filter result assertEquals(0, filterResult.messagesRead()); @@ -453,7 +453,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { return false; } - }, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); + }, filtered, Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); // Verify filter result assertEquals(0, filterResult.outputBuffer().position()); @@ -529,7 +529,8 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { return new BatchRetentionResult(BatchRetention.RETAIN_EMPTY, false); } }; - builder.build().filterTo(new TopicPartition("random", 0), recordFilter, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); + builder.build().filterTo(new TopicPartition("random", 0), recordFilter, filtered, Integer.MAX_VALUE, + TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); @@ -631,7 +632,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { return true; } - }, filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); + }, filtered, Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); @@ -668,7 +669,7 @@ public void testFilterToAlreadyCompactedLog(Args args) { ByteBuffer filtered = ByteBuffer.allocate(2048); MemoryRecords.readableRecords(buffer).filterTo(new TopicPartition("foo", 0), new RetainNonNullKeysFilter(), - filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); + filtered, Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); @@ -744,7 +745,7 @@ public void testFilterToPreservesProducerInfo(Args args) { ByteBuffer filtered = ByteBuffer.allocate(2048); MemoryRecords.readableRecords(buffer).filterTo(new TopicPartition("foo", 0), new RetainNonNullKeysFilter(), - filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); + filtered, Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); @@ -837,7 +838,7 @@ public void testFilterToWithUndersizedBuffer(Args args) { MemoryRecords.FilterResult result = MemoryRecords.readableRecords(buffer) .filterTo(new TopicPartition("foo", 0), new RetainNonNullKeysFilter(), output, Integer.MAX_VALUE, - BufferSupplier.NO_CACHING); + TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); buffer.position(buffer.position() + result.bytesRead()); result.outputBuffer().flip(); @@ -885,7 +886,7 @@ public void testFilterTo(Args args) { ByteBuffer filtered = ByteBuffer.allocate(2048); MemoryRecords.FilterResult result = MemoryRecords.readableRecords(buffer).filterTo( new TopicPartition("foo", 0), new RetainNonNullKeysFilter(), filtered, Integer.MAX_VALUE, - BufferSupplier.NO_CACHING); + TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); filtered.flip(); @@ -928,14 +929,14 @@ public void testFilterTo(Args args) { RecordBatch batch = batches.get(i); assertEquals(expectedStartOffsets.get(i).longValue(), batch.baseOffset()); assertEquals(expectedEndOffsets.get(i).longValue(), batch.lastOffset()); - assertEquals(magic, batch.magic()); + assertEquals(RecordBatch.CURRENT_MAGIC_VALUE, batch.magic()); assertEquals(compression.type(), batch.compressionType()); if (magic >= RecordBatch.MAGIC_VALUE_V1) { assertEquals(expectedMaxTimestamps.get(i).longValue(), batch.maxTimestamp()); assertEquals(TimestampType.CREATE_TIME, batch.timestampType()); } else { assertEquals(RecordBatch.NO_TIMESTAMP, batch.maxTimestamp()); - assertEquals(TimestampType.NO_TIMESTAMP_TYPE, batch.timestampType()); + assertEquals(TimestampType.CREATE_TIME, batch.timestampType()); } } @@ -1004,7 +1005,7 @@ public void testFilterToPreservesLogAppendTime(Args args) { ByteBuffer filtered = ByteBuffer.allocate(2048); MemoryRecords.readableRecords(buffer).filterTo(new TopicPartition("foo", 0), new RetainNonNullKeysFilter(), - filtered, Integer.MAX_VALUE, BufferSupplier.NO_CACHING); + filtered, Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 4f8d545be608e..70b2ab8f055b6 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -684,7 +684,8 @@ private[log] class Cleaner(val id: Int, try { cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainLegacyDeletesAndTxnMarkers, log.config.deleteRetentionMs, - log.config.maxMessageSize, transactionMetadata, lastOffsetOfActiveProducers, upperBoundOffsetOfCleaningRound, stats, currentTime = currentTime) + log.config.maxMessageSize, log.config.messageTimestampType, transactionMetadata, lastOffsetOfActiveProducers, + upperBoundOffsetOfCleaningRound, stats, currentTime = currentTime) } catch { case e: LogSegmentOffsetOverflowException => // Split the current segment. It's also safest to abort the current cleaning process, so that we retry from @@ -741,6 +742,7 @@ private[log] class Cleaner(val id: Int, retainLegacyDeletesAndTxnMarkers: Boolean, deleteRetentionMs: Long, maxLogMessageSize: Int, + timestampTypeConfig: TimestampType, transactionMetadata: CleanedTransactionMetadata, lastRecordsOfActiveProducers: mutable.Map[Long, LastRecord], upperBoundOffsetOfCleaningRound: Long, @@ -810,7 +812,8 @@ private[log] class Cleaner(val id: Int, sourceRecords.readInto(readBuffer, position) val records = MemoryRecords.readableRecords(readBuffer) throttler.maybeThrottle(records.sizeInBytes) - val result = records.filterTo(topicPartition, logCleanerFilter, writeBuffer, maxLogMessageSize, decompressionBufferSupplier) + val result = records.filterTo(topicPartition, logCleanerFilter, writeBuffer, maxLogMessageSize, timestampTypeConfig, + decompressionBufferSupplier) stats.readMessages(result.messagesRead, result.bytesRead) stats.recopyMessages(result.messagesRetained, result.bytesRetained) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 399828ddedc43..5fecc2f8e65e6 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -820,7 +820,7 @@ class UnifiedLogTest { 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, Int.MaxValue, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING) filtered.flip() val filteredRecords = MemoryRecords.readableRecords(filtered) @@ -862,7 +862,7 @@ class UnifiedLogTest { 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, Int.MaxValue, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING) filtered.flip() val filteredRecords = MemoryRecords.readableRecords(filtered) @@ -906,7 +906,7 @@ class UnifiedLogTest { 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, Int.MaxValue, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING) filtered.flip() val filteredRecords = MemoryRecords.readableRecords(filtered) From 8256a06053ae527f386d3f20b0df879b15a9baab Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 27 Dec 2024 08:45:03 -0800 Subject: [PATCH 02/18] Remove warning related to consumers earlier than 0.10.1 - they are no longer supported --- .../java/org/apache/kafka/common/record/MemoryRecords.java | 6 ------ 1 file changed, 6 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 07bdffc17476a..f2db011081213 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 @@ -201,12 +201,6 @@ private static FilterResult filterTo(TopicPartition partition, Iterable batch.sizeInBytes() && filteredBatchSize > maxRecordBatchSize) - log.warn("Record batch from {} with last offset {} exceeded max record batch size {} after cleaning " + - "(new size is {}). Consumers with version earlier than 0.10.1.0 may need to " + - "increase their fetch sizes.", - partition, batch.lastOffset(), maxRecordBatchSize, filteredBatchSize); - MemoryRecordsBuilder.RecordsInfo info = builder.info(); filterResult.updateRetainedBatchMetadata(info.maxTimestamp, info.shallowOffsetOfMaxTimestamp, maxOffset, retainedRecords.size(), filteredBatchSize); From 90c86dbb44bd7c99285bed984425f1cc46935f33 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 27 Dec 2024 10:21:27 -0800 Subject: [PATCH 03/18] UnifiedLog.maybeCreateLeaderEpochCache -> createLeaderEpochCache since it always creates it now --- .../src/main/scala/kafka/log/UnifiedLog.scala | 32 ++++++-------- .../kafka/cluster/PartitionLockTest.scala | 9 ++-- .../unit/kafka/cluster/PartitionTest.scala | 8 ++-- .../kafka/log/LogCleanerManagerTest.scala | 9 ++-- .../scala/unit/kafka/log/LogCleanerTest.scala | 9 ++-- .../scala/unit/kafka/log/LogLoaderTest.scala | 42 +++++++++---------- .../kafka/server/ReplicaManagerTest.scala | 8 ++-- .../unit/kafka/utils/SchedulerTest.scala | 10 ++--- .../storage/internals/log/LogLoader.java | 12 +++--- .../storage/internals/log/LogSegment.java | 12 +++--- .../storage/internals/log/LogSegmentTest.java | 12 +++--- 11 files changed, 75 insertions(+), 88 deletions(-) diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index 3f129deec7f29..db0d302ef38a7 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -509,8 +509,8 @@ class UnifiedLog(@volatile var logStartOffset: Long, } private def initializeLeaderEpochCache(): Unit = lock synchronized { - leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( - dir, topicPartition, logDirFailureChannel, logIdent, leaderEpochCache, scheduler) + leaderEpochCache = Some(UnifiedLog.createLeaderEpochCache( + dir, topicPartition, logDirFailureChannel, leaderEpochCache, scheduler)) } private def updateHighWatermarkWithLogEndOffset(): Unit = { @@ -2015,11 +2015,10 @@ object UnifiedLog extends Logging { // The created leaderEpochCache will be truncated by LogLoader if necessary // so it is guaranteed that the epoch entries will be correct even when on-disk // checkpoint was stale (due to async nature of LeaderEpochFileCache#truncateFromStart/End). - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( + val leaderEpochCache = UnifiedLog.createLeaderEpochCache( dir, topicPartition, logDirFailureChannel, - s"[UnifiedLog partition=$topicPartition, dir=${dir.getParent}] ", None, scheduler) val producerStateManager = new ProducerStateManager(topicPartition, dir, @@ -2036,7 +2035,7 @@ object UnifiedLog extends Logging { segments, logStartOffset, recoveryPoint, - leaderEpochCache.toJava, + leaderEpochCache, producerStateManager, numRemainingSegments, isRemoteLogEnabled, @@ -2047,7 +2046,7 @@ object UnifiedLog extends Logging { localLog, brokerTopicStats, producerIdExpirationCheckIntervalMs, - leaderEpochCache, + Some(leaderEpochCache), producerStateManager, topicId, keepPartitionMetadataFile, @@ -2072,29 +2071,24 @@ object UnifiedLog extends Logging { def parseTopicPartitionName(dir: File): TopicPartition = LocalLog.parseTopicPartitionName(dir) /** - * If the recordVersion is >= RecordVersion.V2, create a new LeaderEpochFileCache instance. - * Loading the epoch entries from the backing checkpoint file or the provided currentCache if not empty. - * Otherwise, the message format is considered incompatible and the existing LeaderEpoch file - * is deleted. + * Create a new LeaderEpochFileCache instance and load the epoch entries from the backing checkpoint file or + * the provided currentCache (if not empty). * * @param dir The directory in which the log will reside * @param topicPartition The topic partition * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure - * @param logPrefix The logging prefix * @param currentCache The current LeaderEpochFileCache instance (if any) * @param scheduler The scheduler for executing asynchronous tasks * @return The new LeaderEpochFileCache instance (if created), none otherwise */ - def maybeCreateLeaderEpochCache(dir: File, - topicPartition: TopicPartition, - logDirFailureChannel: LogDirFailureChannel, - logPrefix: String, - currentCache: Option[LeaderEpochFileCache], - scheduler: Scheduler): Option[LeaderEpochFileCache] = { + def createLeaderEpochCache(dir: File, + topicPartition: TopicPartition, + logDirFailureChannel: LogDirFailureChannel, + currentCache: Option[LeaderEpochFileCache], + scheduler: Scheduler): LeaderEpochFileCache = { val leaderEpochFile = LeaderEpochCheckpointFile.newFile(dir) val checkpointFile = new LeaderEpochCheckpointFile(leaderEpochFile, logDirFailureChannel) - currentCache.map(_.withCheckpoint(checkpointFile)) - .orElse(Some(new LeaderEpochFileCache(topicPartition, checkpointFile, scheduler))) + currentCache.map(_.withCheckpoint(checkpointFile)).getOrElse(new LeaderEpochFileCache(topicPartition, checkpointFile, scheduler)) } private[log] def replaceSegments(existingSegments: LogSegments, diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index 8bdc80e9a4f56..1010613c7eaf9 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -50,7 +50,6 @@ import org.mockito.Mockito.{mock, when} import scala.concurrent.duration._ import scala.jdk.CollectionConverters._ -import scala.jdk.OptionConverters.RichOption /** * Verifies that slow appends to log don't block request threads processing replica fetch requests. @@ -302,8 +301,8 @@ class PartitionLockTest extends Logging { val log = super.createLog(isNew, isFutureReplica, offsetCheckpoints, None, None) val logDirFailureChannel = new LogDirFailureChannel(1) val segments = new LogSegments(log.topicPartition) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( - log.dir, log.topicPartition, logDirFailureChannel, "", None, mockTime.scheduler) + val leaderEpochCache = UnifiedLog.createLeaderEpochCache( + log.dir, log.topicPartition, logDirFailureChannel, None, mockTime.scheduler) val maxTransactionTimeout = 5 * 60 * 1000 val producerStateManagerConfig = new ProducerStateManagerConfig(TransactionLogConfig.PRODUCER_ID_EXPIRATION_MS_DEFAULT, false) val producerStateManager = new ProducerStateManager( @@ -324,7 +323,7 @@ class PartitionLockTest extends Logging { segments, 0L, 0L, - leaderEpochCache.toJava, + leaderEpochCache, producerStateManager, new ConcurrentHashMap[String, Integer], false @@ -332,7 +331,7 @@ class PartitionLockTest extends Logging { val localLog = new LocalLog(log.dir, log.config, segments, offsets.recoveryPoint, offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, log.topicPartition, logDirFailureChannel) - new SlowLog(log, offsets.logStartOffset, localLog, leaderEpochCache, producerStateManager, appendSemaphore) + new SlowLog(log, offsets.logStartOffset, localLog, Some(leaderEpochCache), producerStateManager, appendSemaphore) } } diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index f21522067ca29..cb4105acc53a3 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -447,8 +447,8 @@ class PartitionTest extends AbstractPartitionTest { val log = super.createLog(isNew, isFutureReplica, offsetCheckpoints, None, None) val logDirFailureChannel = new LogDirFailureChannel(1) val segments = new LogSegments(log.topicPartition) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( - log.dir, log.topicPartition, logDirFailureChannel, "", None, time.scheduler) + val leaderEpochCache = UnifiedLog.createLeaderEpochCache( + log.dir, log.topicPartition, logDirFailureChannel, None, time.scheduler) val maxTransactionTimeoutMs = 5 * 60 * 1000 val producerStateManagerConfig = new ProducerStateManagerConfig(TransactionLogConfig.PRODUCER_ID_EXPIRATION_MS_DEFAULT, true) val producerStateManager = new ProducerStateManager( @@ -469,7 +469,7 @@ class PartitionTest extends AbstractPartitionTest { segments, 0L, 0L, - leaderEpochCache.asJava, + leaderEpochCache, producerStateManager, new ConcurrentHashMap[String, Integer], false @@ -477,7 +477,7 @@ class PartitionTest extends AbstractPartitionTest { val localLog = new LocalLog(log.dir, log.config, segments, offsets.recoveryPoint, offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, log.topicPartition, logDirFailureChannel) - new SlowLog(log, offsets.logStartOffset, localLog, leaderEpochCache, producerStateManager, appendSemaphore) + new SlowLog(log, offsets.logStartOffset, localLog, Some(leaderEpochCache), producerStateManager, appendSemaphore) } } diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index 974da551e77e9..b13820a9294a3 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -37,7 +37,6 @@ import java.lang.{Long => JLong} import java.util import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable -import scala.jdk.OptionConverters.RichOption /** * Unit tests for the log cleaning logic @@ -110,8 +109,8 @@ class LogCleanerManagerTest extends Logging { val maxTransactionTimeoutMs = 5 * 60 * 1000 val producerIdExpirationCheckIntervalMs = TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT val segments = new LogSegments(tp) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( - tpDir, topicPartition, logDirFailureChannel, "", None, time.scheduler) + val leaderEpochCache = UnifiedLog.createLeaderEpochCache( + tpDir, topicPartition, logDirFailureChannel, None, time.scheduler) val producerStateManager = new ProducerStateManager(topicPartition, tpDir, maxTransactionTimeoutMs, producerStateManagerConfig, time) val offsets = new LogLoader( tpDir, @@ -124,7 +123,7 @@ class LogCleanerManagerTest extends Logging { segments, 0L, 0L, - leaderEpochCache.toJava, + leaderEpochCache, producerStateManager, new ConcurrentHashMap[String, Integer], false @@ -133,7 +132,7 @@ class LogCleanerManagerTest extends Logging { offsets.nextOffsetMetadata, time.scheduler, time, tp, logDirFailureChannel) // the exception should be caught and the partition that caused it marked as uncleanable class LogMock extends UnifiedLog(offsets.logStartOffset, localLog, new BrokerTopicStats, - producerIdExpirationCheckIntervalMs, leaderEpochCache, + producerIdExpirationCheckIntervalMs, Some(leaderEpochCache), producerStateManager, _topicId = None, keepPartitionMetadataFile = true) { // Throw an error in getFirstBatchTimestampForSegments since it is called in grabFilthiestLog() override def getFirstBatchTimestampForSegments(segments: util.Collection[LogSegment]): util.Collection[java.lang.Long] = diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index be9e11813018c..3c8ab48b54989 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -46,7 +46,6 @@ import java.util.Properties import java.util.concurrent.{ConcurrentHashMap, CountDownLatch, TimeUnit} import scala.collection._ import scala.jdk.CollectionConverters._ -import scala.jdk.OptionConverters.RichOption /** * Unit tests for the log cleaning logic @@ -189,8 +188,8 @@ class LogCleanerTest extends Logging { val maxTransactionTimeoutMs = 5 * 60 * 1000 val producerIdExpirationCheckIntervalMs = TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT val logSegments = new LogSegments(topicPartition) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( - dir, topicPartition, logDirFailureChannel, "", None, time.scheduler) + val leaderEpochCache = UnifiedLog.createLeaderEpochCache( + dir, topicPartition, logDirFailureChannel, None, time.scheduler) val producerStateManager = new ProducerStateManager(topicPartition, dir, maxTransactionTimeoutMs, producerStateManagerConfig, time) val offsets = new LogLoader( @@ -204,7 +203,7 @@ class LogCleanerTest extends Logging { logSegments, 0L, 0L, - leaderEpochCache.toJava, + leaderEpochCache, producerStateManager, new ConcurrentHashMap[String, Integer], false @@ -215,7 +214,7 @@ class LogCleanerTest extends Logging { localLog, brokerTopicStats = new BrokerTopicStats, producerIdExpirationCheckIntervalMs = producerIdExpirationCheckIntervalMs, - leaderEpochCache = leaderEpochCache, + leaderEpochCache = Some(leaderEpochCache), producerStateManager = producerStateManager, _topicId = None, keepPartitionMetadataFile = true) { diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index ec82099f730e9..50d3d7677accc 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -52,7 +52,7 @@ import java.util.{Optional, OptionalLong, Properties} import scala.collection.mutable.ListBuffer import scala.collection.{Iterable, Map, mutable} import scala.jdk.CollectionConverters._ -import scala.jdk.OptionConverters.{RichOption, RichOptional} +import scala.jdk.OptionConverters.RichOptional class LogLoaderTest { var config: KafkaConfig = _ @@ -155,19 +155,19 @@ class LogLoaderTest { val logStartOffset = logStartOffsets.getOrDefault(topicPartition, 0L) val logDirFailureChannel: LogDirFailureChannel = new LogDirFailureChannel(1) val segments = new LogSegments(topicPartition) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( - logDir, topicPartition, logDirFailureChannel, "", None, time.scheduler) + val leaderEpochCache = UnifiedLog.createLeaderEpochCache( + logDir, topicPartition, logDirFailureChannel, None, time.scheduler) val producerStateManager = new ProducerStateManager(topicPartition, logDir, this.maxTransactionTimeoutMs, this.producerStateManagerConfig, time) val logLoader = new LogLoader(logDir, topicPartition, config, time.scheduler, time, logDirFailureChannel, hadCleanShutdown, segments, logStartOffset, logRecoveryPoint, - leaderEpochCache.toJava, producerStateManager, new ConcurrentHashMap[String, Integer], false) + leaderEpochCache, producerStateManager, new ConcurrentHashMap[String, Integer], false) val offsets = logLoader.load() val localLog = new LocalLog(logDir, logConfig, segments, offsets.recoveryPoint, offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, topicPartition, logDirFailureChannel) new UnifiedLog(offsets.logStartOffset, localLog, brokerTopicStats, - this.producerIdExpirationCheckIntervalMs, leaderEpochCache, + this.producerIdExpirationCheckIntervalMs, Some(leaderEpochCache), producerStateManager, None, true) } } @@ -357,13 +357,13 @@ class LogLoaderTest { }.when(wrapper).read(ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any()) Mockito.doAnswer { in => recoveredSegments += wrapper - segment.recover(in.getArgument(0, classOf[ProducerStateManager]), in.getArgument(1, classOf[Optional[LeaderEpochFileCache]])) + segment.recover(in.getArgument(0, classOf[ProducerStateManager]), in.getArgument(1, classOf[LeaderEpochFileCache])) }.when(wrapper).recover(ArgumentMatchers.any(), ArgumentMatchers.any()) super.add(wrapper) } } - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( - logDir, topicPartition, logDirFailureChannel, "", None, mockTime.scheduler) + val leaderEpochCache = UnifiedLog.createLeaderEpochCache( + logDir, topicPartition, logDirFailureChannel, None, mockTime.scheduler) val producerStateManager = new ProducerStateManager(topicPartition, logDir, maxTransactionTimeoutMs, producerStateManagerConfig, mockTime) val logLoader = new LogLoader( @@ -377,7 +377,7 @@ class LogLoaderTest { interceptedLogSegments, 0L, recoveryPoint, - leaderEpochCache.toJava, + leaderEpochCache, producerStateManager, new ConcurrentHashMap[String, Integer], false @@ -387,7 +387,7 @@ class LogLoaderTest { offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, topicPartition, logDirFailureChannel) new UnifiedLog(offsets.logStartOffset, localLog, brokerTopicStats, - producerIdExpirationCheckIntervalMs, leaderEpochCache, producerStateManager, + producerIdExpirationCheckIntervalMs, Some(leaderEpochCache), producerStateManager, None, keepPartitionMetadataFile = true) } @@ -430,8 +430,8 @@ class LogLoaderTest { val logDirFailureChannel: LogDirFailureChannel = new LogDirFailureChannel(1) val config = new LogConfig(new Properties()) val segments = new LogSegments(topicPartition) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( - logDir, topicPartition, logDirFailureChannel, "", None, mockTime.scheduler) + val leaderEpochCache = UnifiedLog.createLeaderEpochCache( + logDir, topicPartition, logDirFailureChannel, None, mockTime.scheduler) val offsets = new LogLoader( logDir, topicPartition, @@ -443,7 +443,7 @@ class LogLoaderTest { segments, 0L, 0L, - leaderEpochCache.toJava, + leaderEpochCache, stateManager, new ConcurrentHashMap[String, Integer], false @@ -455,7 +455,7 @@ class LogLoaderTest { localLog, brokerTopicStats = brokerTopicStats, producerIdExpirationCheckIntervalMs = 30000, - leaderEpochCache = leaderEpochCache, + leaderEpochCache = Some(leaderEpochCache), producerStateManager = stateManager, _topicId = None, keepPartitionMetadataFile = true) @@ -540,8 +540,8 @@ class LogLoaderTest { val config = new LogConfig(new Properties()) val logDirFailureChannel = null val segments = new LogSegments(topicPartition) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( - logDir, topicPartition, logDirFailureChannel, "", None, mockTime.scheduler) + val leaderEpochCache = UnifiedLog.createLeaderEpochCache( + logDir, topicPartition, logDirFailureChannel, None, mockTime.scheduler) val offsets = new LogLoader( logDir, topicPartition, @@ -553,7 +553,7 @@ class LogLoaderTest { segments, 0L, 0L, - leaderEpochCache.toJava, + leaderEpochCache, stateManager, new ConcurrentHashMap[String, Integer], false @@ -565,7 +565,7 @@ class LogLoaderTest { localLog, brokerTopicStats = brokerTopicStats, producerIdExpirationCheckIntervalMs = 30000, - leaderEpochCache = leaderEpochCache, + leaderEpochCache = Some(leaderEpochCache), producerStateManager = stateManager, _topicId = None, keepPartitionMetadataFile = true) @@ -1633,8 +1633,8 @@ class LogLoaderTest { log.logSegments.forEach(segment => segments.add(segment)) assertEquals(5, segments.firstSegment.get.baseOffset) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( - logDir, topicPartition, logDirFailureChannel, "", None, mockTime.scheduler) + val leaderEpochCache = UnifiedLog.createLeaderEpochCache( + logDir, topicPartition, logDirFailureChannel, None, mockTime.scheduler) val offsets = new LogLoader( logDir, topicPartition, @@ -1646,7 +1646,7 @@ class LogLoaderTest { segments, 0L, 0L, - leaderEpochCache.toJava, + leaderEpochCache, stateManager, new ConcurrentHashMap[String, Integer], isRemoteLogEnabled diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 950975f9faf97..d662b64448fbd 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -2901,8 +2901,8 @@ class ReplicaManagerTest { val maxTransactionTimeoutMs = 30000 val maxProducerIdExpirationMs = 30000 val segments = new LogSegments(tp) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( - logDir, tp, mockLogDirFailureChannel, "", None, time.scheduler) + val leaderEpochCache = UnifiedLog.createLeaderEpochCache( + logDir, tp, mockLogDirFailureChannel, None, time.scheduler) val producerStateManager = new ProducerStateManager(tp, logDir, maxTransactionTimeoutMs, new ProducerStateManagerConfig(maxProducerIdExpirationMs, true), time) val offsets = new LogLoader( @@ -2916,7 +2916,7 @@ class ReplicaManagerTest { segments, 0L, 0L, - leaderEpochCache.toJava, + leaderEpochCache, producerStateManager, new ConcurrentHashMap[String, Integer], false @@ -2928,7 +2928,7 @@ class ReplicaManagerTest { localLog = localLog, brokerTopicStats = mockBrokerTopicStats, producerIdExpirationCheckIntervalMs = 30000, - leaderEpochCache = leaderEpochCache, + leaderEpochCache = Some(leaderEpochCache), producerStateManager = producerStateManager, _topicId = topicId, keepPartitionMetadataFile = true) { diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala index cb889fe91a0ad..e36a32186b125 100644 --- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala +++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala @@ -28,8 +28,6 @@ import org.apache.kafka.storage.log.metrics.BrokerTopicStats import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, Timeout} -import scala.jdk.OptionConverters.RichOption - class SchedulerTest { @@ -140,8 +138,8 @@ class SchedulerTest { val topicPartition = UnifiedLog.parseTopicPartitionName(logDir) val logDirFailureChannel = new LogDirFailureChannel(10) val segments = new LogSegments(topicPartition) - val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache( - logDir, topicPartition, logDirFailureChannel, "", None, mockTime.scheduler) + val leaderEpochCache = UnifiedLog.createLeaderEpochCache( + logDir, topicPartition, logDirFailureChannel, None, mockTime.scheduler) val producerStateManager = new ProducerStateManager(topicPartition, logDir, maxTransactionTimeoutMs, new ProducerStateManagerConfig(maxProducerIdExpirationMs, false), mockTime) val offsets = new LogLoader( @@ -155,7 +153,7 @@ class SchedulerTest { segments, 0L, 0L, - leaderEpochCache.toJava, + leaderEpochCache, producerStateManager, new ConcurrentHashMap[String, Integer], false @@ -165,7 +163,7 @@ class SchedulerTest { val log = new UnifiedLog(logStartOffset = offsets.logStartOffset, localLog = localLog, brokerTopicStats, producerIdExpirationCheckIntervalMs, - leaderEpochCache, producerStateManager, + Some(leaderEpochCache), producerStateManager, _topicId = None, keepPartitionMetadataFile = true) assertTrue(scheduler.taskRunning(log.producerExpireCheck)) log.close() diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogLoader.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogLoader.java index 1ba58d1b2a97e..89780686995fb 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogLoader.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogLoader.java @@ -56,7 +56,7 @@ public class LogLoader { private final LogSegments segments; private final long logStartOffsetCheckpoint; private final long recoveryPointCheckpoint; - private final Optional leaderEpochCache; + private final LeaderEpochFileCache leaderEpochCache; private final ProducerStateManager producerStateManager; private final ConcurrentMap numRemainingSegments; private final boolean isRemoteLogEnabled; @@ -74,7 +74,7 @@ public class LogLoader { * @param segments The {@link LogSegments} instance into which segments recovered from disk will be populated * @param logStartOffsetCheckpoint The checkpoint of the log start offset * @param recoveryPointCheckpoint The checkpoint of the offset at which to begin the recovery - * @param leaderEpochCache An optional {@link LeaderEpochFileCache} instance to be updated during recovery + * @param leaderEpochCache A {@link LeaderEpochFileCache} instance to be updated during recovery * @param producerStateManager The {@link ProducerStateManager} instance to be updated during recovery * @param numRemainingSegments The remaining segments to be recovered in this log keyed by recovery thread name * @param isRemoteLogEnabled Boolean flag to indicate whether the remote storage is enabled or not @@ -90,7 +90,7 @@ public LogLoader( LogSegments segments, long logStartOffsetCheckpoint, long recoveryPointCheckpoint, - Optional leaderEpochCache, + LeaderEpochFileCache leaderEpochCache, ProducerStateManager producerStateManager, ConcurrentMap numRemainingSegments, boolean isRemoteLogEnabled) { @@ -215,13 +215,13 @@ public LoadedLogOffsets load() throws IOException { recoveryOffsets = new RecoveryOffsets(0L, 0L); } - leaderEpochCache.ifPresent(lec -> lec.truncateFromEndAsyncFlush(recoveryOffsets.nextOffset)); + leaderEpochCache.truncateFromEndAsyncFlush(recoveryOffsets.nextOffset); long newLogStartOffset = isRemoteLogEnabled ? logStartOffsetCheckpoint : Math.max(logStartOffsetCheckpoint, segments.firstSegment().get().baseOffset()); // The earliest leader epoch may not be flushed during a hard failure. Recover it here. - leaderEpochCache.ifPresent(lec -> lec.truncateFromStartAsyncFlush(logStartOffsetCheckpoint)); + leaderEpochCache.truncateFromStartAsyncFlush(logStartOffsetCheckpoint); // Any segment loading or recovery code must not use producerStateManager, so that we can build the full state here // from scratch. @@ -428,7 +428,7 @@ private Optional deleteSegmentsIfLogStartGreaterThanLogEnd() throws IOExce "is smaller than logStartOffset {}. " + "This could happen if segment files were deleted from the file system.", logEndOffset, logStartOffsetCheckpoint); removeAndDeleteSegmentsAsync(segments.values()); - leaderEpochCache.ifPresent(LeaderEpochFileCache::clearAndFlush); + leaderEpochCache.clearAndFlush(); producerStateManager.truncateFullyAndStartAt(logStartOffsetCheckpoint); return Optional.empty(); } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java index 4ff976bdf78d5..47e506605d4f3 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java @@ -465,11 +465,11 @@ public OptionalLong fetchUpperBoundOffset(OffsetPosition startOffsetPosition, in * * @param producerStateManager Producer state corresponding to the segment's base offset. This is needed to recover * the transaction index. - * @param leaderEpochCache Optionally a cache for updating the leader epoch during recovery. + * @param leaderEpochCache a cache for updating the leader epoch during recovery. * @return The number of bytes truncated from the log * @throws LogSegmentOffsetOverflowException if the log segment contains an offset that causes the index offset to overflow */ - public int recover(ProducerStateManager producerStateManager, Optional leaderEpochCache) throws IOException { + public int recover(ProducerStateManager producerStateManager, LeaderEpochFileCache leaderEpochCache) throws IOException { offsetIndex().reset(); timeIndex().reset(); txnIndex.reset(); @@ -495,11 +495,9 @@ public int recover(ProducerStateManager producerStateManager, Optional= RecordBatch.MAGIC_VALUE_V2) { - leaderEpochCache.ifPresent(cache -> { - if (batch.partitionLeaderEpoch() >= 0 && - (!cache.latestEpoch().isPresent() || batch.partitionLeaderEpoch() > cache.latestEpoch().getAsInt())) - cache.assign(batch.partitionLeaderEpoch(), batch.baseOffset()); - }); + if (batch.partitionLeaderEpoch() >= 0 && + (!leaderEpochCache.latestEpoch().isPresent() || batch.partitionLeaderEpoch() > leaderEpochCache.latestEpoch().getAsInt())) + leaderEpochCache.assign(batch.partitionLeaderEpoch(), batch.baseOffset()); updateProducerState(producerStateManager, batch); } } diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentTest.java index 5e06c073dc5e8..dc6a0cfb3aa78 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentTest.java @@ -440,7 +440,7 @@ public void testRecoveryFixesCorruptIndex() throws Exception { } File indexFile = seg.offsetIndexFile(); writeNonsenseToFile(indexFile, 5, (int) indexFile.length()); - seg.recover(newProducerStateManager(), Optional.empty()); + seg.recover(newProducerStateManager(), mock(LeaderEpochFileCache.class)); for (int i = 0; i < 100; i++) { Iterable records = seg.read(i, 1, Optional.of((long) seg.size()), true).records.records(); assertEquals(i, records.iterator().next().offset()); @@ -482,7 +482,7 @@ public void testRecoverTransactionIndex() throws Exception { 107L, endTxnRecords(ControlRecordType.COMMIT, pid1, producerEpoch, 107L)); ProducerStateManager stateManager = newProducerStateManager(); - segment.recover(stateManager, Optional.empty()); + segment.recover(stateManager, mock(LeaderEpochFileCache.class)); assertEquals(108L, stateManager.mapEndOffset()); List abortedTxns = segment.txnIndex().allAbortedTxns(); @@ -498,7 +498,7 @@ public void testRecoverTransactionIndex() throws Exception { stateManager.loadProducerEntry(new ProducerStateEntry(pid2, producerEpoch, 0, RecordBatch.NO_TIMESTAMP, OptionalLong.of(75L), Optional.of(new BatchMetadata(10, 10L, 5, RecordBatch.NO_TIMESTAMP)))); - segment.recover(stateManager, Optional.empty()); + segment.recover(stateManager, mock(LeaderEpochFileCache.class)); assertEquals(108L, stateManager.mapEndOffset()); abortedTxns = segment.txnIndex().allAbortedTxns(); @@ -533,7 +533,7 @@ public void testRecoveryRebuildsEpochCache() throws Exception { seg.append(111L, RecordBatch.NO_TIMESTAMP, 110L, MemoryRecords.withRecords(110L, Compression.NONE, 2, new SimpleRecord("a".getBytes()), new SimpleRecord("b".getBytes()))); - seg.recover(newProducerStateManager(), Optional.of(cache)); + seg.recover(newProducerStateManager(), cache); assertEquals(Arrays.asList( new EpochEntry(0, 104L), new EpochEntry(1, 106L), @@ -570,7 +570,7 @@ public void testRecoveryFixesCorruptTimeIndex() throws IOException { } File timeIndexFile = seg.timeIndexFile(); writeNonsenseToFile(timeIndexFile, 5, (int) timeIndexFile.length()); - seg.recover(newProducerStateManager(), Optional.empty()); + seg.recover(newProducerStateManager(), mock(LeaderEpochFileCache.class)); for (int i = 0; i < 100; i++) { assertEquals(i, seg.findOffsetByTimestamp(i * 10, 0L).get().offset); if (i < 99) { @@ -597,7 +597,7 @@ public void testRecoveryWithCorruptMessage() throws IOException { FileRecords.LogOffsetPosition recordPosition = seg.log().searchForOffsetWithSize(offsetToBeginCorruption, 0); int position = recordPosition.position + TestUtils.RANDOM.nextInt(15); writeNonsenseToFile(seg.log().file(), position, (int) (seg.log().file().length() - position)); - seg.recover(newProducerStateManager(), Optional.empty()); + seg.recover(newProducerStateManager(), mock(LeaderEpochFileCache.class)); List expectList = new ArrayList<>(); for (long j = 0; j < offsetToBeginCorruption; j++) { From bb286072e7eef0dc36268ea0e2a1ab07ccf2e178 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 27 Dec 2024 10:48:57 -0800 Subject: [PATCH 04/18] Add back `testLeaderEpochCacheCreatedAfterMessageFormatUpgrade` as it's still a useful test --- core/src/main/scala/kafka/log/UnifiedLog.scala | 17 +++++++++++++++-- .../scala/unit/kafka/log/LogTestUtils.scala | 7 ------- .../scala/unit/kafka/log/UnifiedLogTest.scala | 17 +++++++++++++++++ 3 files changed, 32 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index db0d302ef38a7..4afe21761b5f0 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -713,6 +713,18 @@ class UnifiedLog(@volatile var logStartOffset: Long, append(records, origin, interBrokerProtocolVersion, validateAndAssignOffsets, leaderEpoch, Some(requestLocal), verificationGuard, ignoreRecordSize = false) } + /** + * Even though we always write to disk with record version v2 since Apache Kafka 4.0, older record versions may have + * been persisted to disk before that. In order to test such scenarios, we need the ability to append with older + * record versions. This method exists for that purpose and hence it should only be used from test code. + * + * Also see #appendAsLeader. + */ + private[log] def appendAsLeaderWithRecordVersion(records: MemoryRecords, leaderEpoch: Int, recordVersion: RecordVersion): Unit = { + append(records, AppendOrigin.CLIENT, MetadataVersion.latestProduction, true, leaderEpoch, Some(RequestLocal.noCaching), + VerificationGuard.SENTINEL, ignoreRecordSize = false, recordVersion.value) + } + /** * Append this message set to the active segment of the local log without assigning offsets or Partition Leader Epochs * @@ -757,7 +769,8 @@ class UnifiedLog(@volatile var logStartOffset: Long, leaderEpoch: Int, requestLocal: Option[RequestLocal], verificationGuard: VerificationGuard, - ignoreRecordSize: Boolean): LogAppendInfo = { + ignoreRecordSize: Boolean, + toMagic: Byte = RecordBatch.CURRENT_MAGIC_VALUE): LogAppendInfo = { // We want to ensure the partition metadata file is written to the log dir before any log data is written to disk. // This will ensure that any log data can be recovered with the correct topic ID in the case of failure. maybeFlushMetadataFile() @@ -787,7 +800,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, appendInfo.sourceCompression, targetCompression, config.compact, - RecordBatch.CURRENT_MAGIC_VALUE, + toMagic, config.messageTimestampType, config.messageTimestampBeforeMaxMs, config.messageTimestampAfterMaxMs, diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index 6e27ea75944fe..e98028ab86fe4 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -35,7 +35,6 @@ import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.storage.log.FetchIsolation import org.apache.kafka.server.util.Scheduler -import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile import org.apache.kafka.storage.internals.log.LogConfig.{DEFAULT_REMOTE_LOG_COPY_DISABLE_CONFIG, DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG} import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, FetchDataInfo, LazyIndex, LogAppendInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetsListener, LogSegment, ProducerStateManager, ProducerStateManagerConfig, TransactionIndex} import org.apache.kafka.storage.log.metrics.BrokerTopicStats @@ -262,12 +261,6 @@ object LogTestUtils { def listProducerSnapshotOffsets(logDir: File): Seq[Long] = ProducerStateManager.listSnapshotFiles(logDir).asScala.map(_.offset).sorted.toSeq - def assertLeaderEpochCacheEmpty(log: UnifiedLog): Unit = { - assertEquals(None, log.leaderEpochCache) - assertEquals(None, log.latestEpoch) - assertFalse(LeaderEpochCheckpointFile.newFile(log.dir).exists()) - } - def appendNonTransactionalAsLeader(log: UnifiedLog, numRecords: Int): Unit = { val simpleRecords = (0 until numRecords).map { seq => new SimpleRecord(s"$seq".getBytes) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 5fecc2f8e65e6..a26b9d9fd9bb1 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -2580,6 +2580,23 @@ class UnifiedLogTest { assertEquals(None, log.leaderEpochCache.flatMap(_.latestEpoch.toScala)) } + @Test + def testLeaderEpochCacheCreatedAfterMessageFormatUpgrade(): Unit = { + val logProps = new Properties() + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1000") + logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, "1") + logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, "65536") + val logConfig = new LogConfig(logProps) + val log = createLog(logDir, logConfig) + log.appendAsLeaderWithRecordVersion(TestUtils.records(List(new SimpleRecord("bar".getBytes())), + magicValue = RecordVersion.V1.value), leaderEpoch = 5, RecordVersion.V1) + assertEquals(None, log.latestEpoch) + + log.appendAsLeader(TestUtils.records(List(new SimpleRecord("foo".getBytes())), + magicValue = RecordVersion.V2.value), leaderEpoch = 5) + assertEquals(Some(5), log.latestEpoch) + } + @Test def testSplitOnOffsetOverflow(): Unit = { // create a log such that one log segment has offsets that overflow, and call the split API on that segment From 63b10f743c0a071fa3e534c282c9059320be2cea Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Sat, 28 Dec 2024 19:09:05 -0800 Subject: [PATCH 05/18] Remove unused parameters for LogCleaner.filterInfo --- .../kafka/common/record/MemoryRecords.java | 15 +++---- .../internals/FetchRequestManagerTest.java | 4 +- .../consumer/internals/FetcherTest.java | 4 +- .../ShareConsumeRequestManagerTest.java | 4 +- .../common/record/MemoryRecordsTest.java | 42 +++++++++---------- .../src/main/scala/kafka/log/LogCleaner.scala | 3 +- .../src/main/scala/kafka/log/UnifiedLog.scala | 2 +- .../AbstractLogCleanerIntegrationTest.scala | 6 +-- .../scala/unit/kafka/log/UnifiedLogTest.scala | 12 +++--- 9 files changed, 41 insertions(+), 51 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index f2db011081213..db63f3846fff6 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 @@ -137,12 +137,8 @@ public Integer firstBatchSize() { /** * Filter the records into the provided ByteBuffer. * - * @param partition The partition that is filtered (used only for logging) * @param filter The filter function * @param destinationBuffer The byte buffer to write the filtered records to - * @param maxRecordBatchSize The maximum record batch size. Note this is not a hard limit: if a batch - * exceeds this after filtering, we log a warning, but the batch will still be - * created. * @param decompressionBufferSupplier The supplier of ByteBuffer(s) used for decompression if supported. For small * record batches, allocating a potentially large buffer (64 KB for LZ4) will * dominate the cost of decompressing and iterating over the records in the @@ -150,19 +146,18 @@ public Integer firstBatchSize() { * performance impact. * @return A FilterResult with a summary of the output (for metrics) and potentially an overflow buffer */ - public FilterResult filterTo(TopicPartition partition, RecordFilter filter, ByteBuffer destinationBuffer, - int maxRecordBatchSize, TimestampType timestampTypeConfig, + public FilterResult filterTo(RecordFilter filter, ByteBuffer destinationBuffer, TimestampType timestampTypeConfig, BufferSupplier decompressionBufferSupplier) { - return filterTo(partition, batches(), filter, destinationBuffer, maxRecordBatchSize, timestampTypeConfig, decompressionBufferSupplier); + return filterTo(batches(), filter, destinationBuffer, timestampTypeConfig, 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, - TimestampType timestampTypeConfig, BufferSupplier decompressionBufferSupplier) { + private static FilterResult filterTo(Iterable batches, RecordFilter filter, + ByteBuffer destinationBuffer, TimestampType timestampTypeConfig, + BufferSupplier decompressionBufferSupplier) { FilterResult filterResult = new FilterResult(destinationBuffer); ByteBufferOutputStream bufferOutputStream = new ByteBufferOutputStream(destinationBuffer); for (MutableRecordBatch batch : batches) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java index 7f6a567fe26e7..3a47b6f9c0739 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java @@ -2532,7 +2532,7 @@ public void testUpdatePositionWithLastRecordMissingFromBatch() { new SimpleRecord(null, "value".getBytes())); // Remove the last record to simulate compaction - MemoryRecords.FilterResult result = records.filterTo(tp0, new MemoryRecords.RecordFilter(0, 0) { + MemoryRecords.FilterResult result = records.filterTo(new MemoryRecords.RecordFilter(0, 0) { @Override protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { return new BatchRetentionResult(BatchRetention.DELETE_EMPTY, false); @@ -2542,7 +2542,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { return record.key() != null; } - }, ByteBuffer.allocate(1024), Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + }, ByteBuffer.allocate(1024), TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); result.outputBuffer().flip(); MemoryRecords compactedRecords = MemoryRecords.readableRecords(result.outputBuffer()); 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 3aec39d2dedaa..cb4b1fb23347e 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 @@ -2518,7 +2518,7 @@ public void testUpdatePositionWithLastRecordMissingFromBatch() { new SimpleRecord(null, "value".getBytes())); // Remove the last record to simulate compaction - MemoryRecords.FilterResult result = records.filterTo(tp0, new MemoryRecords.RecordFilter(0, 0) { + MemoryRecords.FilterResult result = records.filterTo(new MemoryRecords.RecordFilter(0, 0) { @Override protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { return new BatchRetentionResult(BatchRetention.DELETE_EMPTY, false); @@ -2528,7 +2528,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { return record.key() != null; } - }, ByteBuffer.allocate(1024), Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + }, ByteBuffer.allocate(1024), TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); result.outputBuffer().flip(); MemoryRecords compactedRecords = MemoryRecords.readableRecords(result.outputBuffer()); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java index e6f78f54c05da..eedf4f432f4e0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java @@ -1249,7 +1249,7 @@ public void testFetchWithLastRecordMissingFromBatch() { new SimpleRecord(null, "value".getBytes())); // Remove the last record to simulate compaction - MemoryRecords.FilterResult result = records.filterTo(tp0, new MemoryRecords.RecordFilter(0, 0) { + MemoryRecords.FilterResult result = records.filterTo(new MemoryRecords.RecordFilter(0, 0) { @Override protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { return new BatchRetentionResult(BatchRetention.DELETE_EMPTY, false); @@ -1259,7 +1259,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { return record.key() != null; } - }, ByteBuffer.allocate(1024), Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + }, ByteBuffer.allocate(1024), TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); result.outputBuffer().flip(); MemoryRecords compactedRecords = MemoryRecords.readableRecords(result.outputBuffer()); 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 7dd8e8ced09be..15b3c2767019a 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 @@ -291,8 +291,7 @@ public void testFilterToPreservesPartitionLeaderEpoch(Args args) { builder.append(12L, null, "c".getBytes()); ByteBuffer filtered = ByteBuffer.allocate(2048); - builder.build().filterTo(new TopicPartition("foo", 0), new RetainNonNullKeysFilter(), filtered, - Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + builder.build().filterTo(new RetainNonNullKeysFilter(), filtered, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); @@ -332,7 +331,7 @@ public void testFilterToEmptyBatchRetention(Args args) { builder.close(); MemoryRecords records = builder.build(); ByteBuffer filtered = ByteBuffer.allocate(2048); - MemoryRecords.FilterResult filterResult = records.filterTo(new TopicPartition("foo", 0), + MemoryRecords.FilterResult filterResult = records.filterTo( new MemoryRecords.RecordFilter(0, 0) { @Override protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { @@ -345,7 +344,7 @@ protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { // delete the records return false; } - }, filtered, Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + }, filtered, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); // Verify filter result assertEquals(numRecords, filterResult.messagesRead()); @@ -394,7 +393,7 @@ public void testEmptyBatchRetention() { ByteBuffer filtered = ByteBuffer.allocate(2048); MemoryRecords records = MemoryRecords.readableRecords(buffer); - MemoryRecords.FilterResult filterResult = records.filterTo(new TopicPartition("foo", 0), + MemoryRecords.FilterResult filterResult = records.filterTo( new MemoryRecords.RecordFilter(0, 0) { @Override protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { @@ -406,7 +405,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { return false; } - }, filtered, Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + }, filtered, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); // Verify filter result assertEquals(0, filterResult.messagesRead()); @@ -442,7 +441,7 @@ public void testEmptyBatchDeletion() { ByteBuffer filtered = ByteBuffer.allocate(2048); MemoryRecords records = MemoryRecords.readableRecords(buffer); - MemoryRecords.FilterResult filterResult = records.filterTo(new TopicPartition("foo", 0), + MemoryRecords.FilterResult filterResult = records.filterTo( new MemoryRecords.RecordFilter(0, 0) { @Override protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { @@ -453,7 +452,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { return false; } - }, filtered, Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + }, filtered, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); // Verify filter result assertEquals(0, filterResult.outputBuffer().position()); @@ -529,8 +528,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { return new BatchRetentionResult(BatchRetention.RETAIN_EMPTY, false); } }; - builder.build().filterTo(new TopicPartition("random", 0), recordFilter, filtered, Integer.MAX_VALUE, - TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + builder.build().filterTo(recordFilter, filtered, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); @@ -619,7 +617,7 @@ public void testFilterToBatchDiscard(Args args) { buffer.flip(); ByteBuffer filtered = ByteBuffer.allocate(2048); - MemoryRecords.readableRecords(buffer).filterTo(new TopicPartition("foo", 0), new MemoryRecords.RecordFilter(0, 0) { + MemoryRecords.readableRecords(buffer).filterTo(new MemoryRecords.RecordFilter(0, 0) { @Override protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { // discard the second and fourth batches @@ -632,7 +630,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { return true; } - }, filtered, Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + }, filtered, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); @@ -668,8 +666,8 @@ public void testFilterToAlreadyCompactedLog(Args args) { buffer.flip(); ByteBuffer filtered = ByteBuffer.allocate(2048); - MemoryRecords.readableRecords(buffer).filterTo(new TopicPartition("foo", 0), new RetainNonNullKeysFilter(), - filtered, Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + MemoryRecords.readableRecords(buffer).filterTo(new RetainNonNullKeysFilter(), filtered, + TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); @@ -744,8 +742,8 @@ public void testFilterToPreservesProducerInfo(Args args) { buffer.flip(); ByteBuffer filtered = ByteBuffer.allocate(2048); - MemoryRecords.readableRecords(buffer).filterTo(new TopicPartition("foo", 0), new RetainNonNullKeysFilter(), - filtered, Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + MemoryRecords.readableRecords(buffer).filterTo(new RetainNonNullKeysFilter(), filtered, + TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); @@ -836,9 +834,8 @@ public void testFilterToWithUndersizedBuffer(Args args) { while (buffer.hasRemaining()) { output.rewind(); - MemoryRecords.FilterResult result = MemoryRecords.readableRecords(buffer) - .filterTo(new TopicPartition("foo", 0), new RetainNonNullKeysFilter(), output, Integer.MAX_VALUE, - TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + MemoryRecords.FilterResult result = MemoryRecords.readableRecords(buffer).filterTo( + new RetainNonNullKeysFilter(), output, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); buffer.position(buffer.position() + result.bytesRead()); result.outputBuffer().flip(); @@ -885,8 +882,7 @@ public void testFilterTo(Args args) { ByteBuffer filtered = ByteBuffer.allocate(2048); MemoryRecords.FilterResult result = MemoryRecords.readableRecords(buffer).filterTo( - new TopicPartition("foo", 0), new RetainNonNullKeysFilter(), filtered, Integer.MAX_VALUE, - TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + new RetainNonNullKeysFilter(), filtered, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); filtered.flip(); @@ -1004,8 +1000,8 @@ public void testFilterToPreservesLogAppendTime(Args args) { buffer.flip(); ByteBuffer filtered = ByteBuffer.allocate(2048); - MemoryRecords.readableRecords(buffer).filterTo(new TopicPartition("foo", 0), new RetainNonNullKeysFilter(), - filtered, Integer.MAX_VALUE, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + MemoryRecords.readableRecords(buffer).filterTo(new RetainNonNullKeysFilter(), filtered, TimestampType.CREATE_TIME, + BufferSupplier.NO_CACHING); filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 70b2ab8f055b6..ab864ebdeeb47 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -812,8 +812,7 @@ private[log] class Cleaner(val id: Int, sourceRecords.readInto(readBuffer, position) val records = MemoryRecords.readableRecords(readBuffer) throttler.maybeThrottle(records.sizeInBytes) - val result = records.filterTo(topicPartition, logCleanerFilter, writeBuffer, maxLogMessageSize, timestampTypeConfig, - decompressionBufferSupplier) + val result = records.filterTo(logCleanerFilter, writeBuffer, timestampTypeConfig, decompressionBufferSupplier) stats.readMessages(result.messagesRead, result.bytesRead) stats.recopyMessages(result.messagesRetained, result.bytesRetained) diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index 4afe21761b5f0..9349ca0baefef 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -720,7 +720,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, * * Also see #appendAsLeader. */ - private[log] def appendAsLeaderWithRecordVersion(records: MemoryRecords, leaderEpoch: Int, recordVersion: RecordVersion): Unit = { + private[log] def appendAsLeaderWithRecordVersion(records: MemoryRecords, leaderEpoch: Int, recordVersion: RecordVersion): LogAppendInfo = { append(records, AppendOrigin.CLIENT, MetadataVersion.latestProduction, true, leaderEpoch, Some(RequestLocal.noCaching), VerificationGuard.SENTINEL, ignoreRecordSize = false, recordVersion.value) } diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala index 874701685274c..e0a6724d081e9 100644 --- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala @@ -24,7 +24,7 @@ import kafka.utils.Implicits._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.config.TopicConfig -import org.apache.kafka.common.record.{MemoryRecords, RecordBatch} +import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion} import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.server.util.MockTime @@ -147,8 +147,8 @@ abstract class AbstractLogCleanerIntegrationTest { startKey: Int = 0, magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE): Seq[(Int, String, Long)] = { for (_ <- 0 until numDups; key <- startKey until (startKey + numKeys)) yield { val value = counter.toString - val appendInfo = log.appendAsLeader(TestUtils.singletonRecords(value = value.getBytes, codec = codec, - key = key.toString.getBytes, magicValue = magicValue), leaderEpoch = 0) + val appendInfo = log.appendAsLeaderWithRecordVersion(TestUtils.singletonRecords(value = value.getBytes, codec = codec, + key = key.toString.getBytes, magicValue = magicValue), leaderEpoch = 0, recordVersion = RecordVersion.lookup(magicValue)) // move LSO forward to increase compaction bound log.updateHighWatermark(log.logEndOffset) incCounter() diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index a26b9d9fd9bb1..8f931156eafe8 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -816,11 +816,11 @@ class UnifiedLogTest { records.batches.forEach(_.setPartitionLeaderEpoch(0)) val filtered = ByteBuffer.allocate(2048) - records.filterTo(new TopicPartition("foo", 0), new RecordFilter(0, 0) { + records.filterTo(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, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING) + }, filtered, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING) filtered.flip() val filteredRecords = MemoryRecords.readableRecords(filtered) @@ -858,11 +858,11 @@ class UnifiedLogTest { records.batches.forEach(_.setPartitionLeaderEpoch(0)) val filtered = ByteBuffer.allocate(2048) - records.filterTo(new TopicPartition("foo", 0), new RecordFilter(0, 0) { + records.filterTo(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, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING) + }, filtered, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING) filtered.flip() val filteredRecords = MemoryRecords.readableRecords(filtered) @@ -902,11 +902,11 @@ class UnifiedLogTest { records.batches.forEach(_.setPartitionLeaderEpoch(0)) val filtered = ByteBuffer.allocate(2048) - records.filterTo(new TopicPartition("foo", 0), new RecordFilter(0, 0) { + records.filterTo(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, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING) + }, filtered, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING) filtered.flip() val filteredRecords = MemoryRecords.readableRecords(filtered) From 021d86d1a23aecc5b4ef702581f62751e4126f1f Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Sat, 28 Dec 2024 19:09:41 -0800 Subject: [PATCH 06/18] Remove unnecessary version check in LogCleaner.filterInfo --- .../java/org/apache/kafka/common/record/MemoryRecords.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index db63f3846fff6..85c961d74ea15 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,8 +181,8 @@ private static FilterResult filterTo(Iterable batches, Recor // 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() >= RecordBatch.MAGIC_VALUE_V2 && (containsTombstones || containsMarkerForEmptyTxn) - && batch.deleteHorizonMs().isEmpty(); + boolean needToSetDeleteHorizon = (containsTombstones || containsMarkerForEmptyTxn) && + batch.deleteHorizonMs().isEmpty(); if (writeOriginalBatch && !needToSetDeleteHorizon) { batch.writeTo(bufferOutputStream); filterResult.updateRetainedBatchMetadata(batch, retainedRecords.size(), false); From b92c3d6666bed789940ac43d2ceb2cc4deafe3ca Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Sun, 29 Dec 2024 00:05:46 -0800 Subject: [PATCH 07/18] Adjust the conversion of V0 to V2 during compaction --- .../kafka/common/record/MemoryRecords.java | 18 ++++++------- .../internals/FetchRequestManagerTest.java | 2 +- .../consumer/internals/FetcherTest.java | 2 +- .../ShareConsumeRequestManagerTest.java | 2 +- .../common/record/MemoryRecordsTest.java | 25 ++++++++----------- .../src/main/scala/kafka/log/LogCleaner.scala | 2 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 6 ++--- 7 files changed, 27 insertions(+), 30 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 85c961d74ea15..434cf6bf47147 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 @@ -146,9 +146,8 @@ public Integer firstBatchSize() { * performance impact. * @return A FilterResult with a summary of the output (for metrics) and potentially an overflow buffer */ - public FilterResult filterTo(RecordFilter filter, ByteBuffer destinationBuffer, TimestampType timestampTypeConfig, - BufferSupplier decompressionBufferSupplier) { - return filterTo(batches(), filter, destinationBuffer, timestampTypeConfig, decompressionBufferSupplier); + public FilterResult filterTo(RecordFilter filter, ByteBuffer destinationBuffer, BufferSupplier decompressionBufferSupplier) { + return filterTo(batches(), filter, destinationBuffer, decompressionBufferSupplier); } /** @@ -156,8 +155,7 @@ public FilterResult filterTo(RecordFilter filter, ByteBuffer destinationBuffer, * to the delete horizon of the tombstones or txn markers which are present in the batch. */ private static FilterResult filterTo(Iterable batches, RecordFilter filter, - ByteBuffer destinationBuffer, TimestampType timestampTypeConfig, - BufferSupplier decompressionBufferSupplier) { + ByteBuffer destinationBuffer, BufferSupplier decompressionBufferSupplier) { FilterResult filterResult = new FilterResult(destinationBuffer); ByteBufferOutputStream bufferOutputStream = new ByteBufferOutputStream(destinationBuffer); for (MutableRecordBatch batch : batches) { @@ -193,7 +191,7 @@ private static FilterResult filterTo(Iterable batches, Recor else deleteHorizonMs = batch.deleteHorizonMs().orElse(RecordBatch.NO_TIMESTAMP); try (final MemoryRecordsBuilder builder = buildRetainedRecordsInto(batch, retainedRecords, - timestampTypeConfig, bufferOutputStream, deleteHorizonMs)) { + bufferOutputStream, deleteHorizonMs)) { MemoryRecords records = builder.build(); int filteredBatchSize = records.sizeInBytes(); MemoryRecordsBuilder.RecordsInfo info = builder.info(); @@ -274,13 +272,15 @@ private BatchFilterResult(List retainedRecords, private static MemoryRecordsBuilder buildRetainedRecordsInto(RecordBatch originalBatch, List retainedRecords, - TimestampType timestampTypeConfig, ByteBufferOutputStream bufferOutputStream, final long deleteHorizonMs) { Compression compression = Compression.of(originalBatch.compressionType()).build(); - // v0 has no timestamp type, use the topic config in that case (like we do when we up-convert produce requests) + // V0 has no timestamp type or timestamp, so we set the timestamp to CREATE_TIME and timestamp to NO_TIMESTAMP. + // Note that this differs from produce up-conversion where the timestamp type topic config is used and the log append + // time is generated if the config is LOG_APPEND_TIME. The reason for the different behavior is that there is + // no appropriate log append time we can generate at compaction time. TimestampType timestampType = originalBatch.timestampType() == TimestampType.NO_TIMESTAMP_TYPE ? - timestampTypeConfig : originalBatch.timestampType(); + TimestampType.CREATE_TIME : originalBatch.timestampType(); long logAppendTime = timestampType == TimestampType.LOG_APPEND_TIME ? originalBatch.maxTimestamp() : RecordBatch.NO_TIMESTAMP; long baseOffset = originalBatch.magic() >= RecordBatch.MAGIC_VALUE_V2 ? diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java index 3a47b6f9c0739..8657dcfc1e99d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java @@ -2542,7 +2542,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { return record.key() != null; } - }, ByteBuffer.allocate(1024), TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + }, ByteBuffer.allocate(1024), BufferSupplier.NO_CACHING); result.outputBuffer().flip(); MemoryRecords compactedRecords = MemoryRecords.readableRecords(result.outputBuffer()); 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 cb4b1fb23347e..ede973c5f9b4f 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 @@ -2528,7 +2528,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { return record.key() != null; } - }, ByteBuffer.allocate(1024), TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + }, ByteBuffer.allocate(1024), BufferSupplier.NO_CACHING); result.outputBuffer().flip(); MemoryRecords compactedRecords = MemoryRecords.readableRecords(result.outputBuffer()); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java index eedf4f432f4e0..37df4cbf43a2f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java @@ -1259,7 +1259,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { return record.key() != null; } - }, ByteBuffer.allocate(1024), TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + }, ByteBuffer.allocate(1024), BufferSupplier.NO_CACHING); result.outputBuffer().flip(); MemoryRecords compactedRecords = MemoryRecords.readableRecords(result.outputBuffer()); 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 15b3c2767019a..25c8242455ef1 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 @@ -291,7 +291,7 @@ public void testFilterToPreservesPartitionLeaderEpoch(Args args) { builder.append(12L, null, "c".getBytes()); ByteBuffer filtered = ByteBuffer.allocate(2048); - builder.build().filterTo(new RetainNonNullKeysFilter(), filtered, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + builder.build().filterTo(new RetainNonNullKeysFilter(), filtered, BufferSupplier.NO_CACHING); filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); @@ -344,7 +344,7 @@ protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { // delete the records return false; } - }, filtered, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + }, filtered, BufferSupplier.NO_CACHING); // Verify filter result assertEquals(numRecords, filterResult.messagesRead()); @@ -405,7 +405,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { return false; } - }, filtered, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + }, filtered, BufferSupplier.NO_CACHING); // Verify filter result assertEquals(0, filterResult.messagesRead()); @@ -452,7 +452,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { return false; } - }, filtered, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + }, filtered, BufferSupplier.NO_CACHING); // Verify filter result assertEquals(0, filterResult.outputBuffer().position()); @@ -528,7 +528,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { return new BatchRetentionResult(BatchRetention.RETAIN_EMPTY, false); } }; - builder.build().filterTo(recordFilter, filtered, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + builder.build().filterTo(recordFilter, filtered, BufferSupplier.NO_CACHING); filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); @@ -630,7 +630,7 @@ protected BatchRetentionResult checkBatchRetention(RecordBatch batch) { protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { return true; } - }, filtered, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + }, filtered, BufferSupplier.NO_CACHING); filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); @@ -666,8 +666,7 @@ public void testFilterToAlreadyCompactedLog(Args args) { buffer.flip(); ByteBuffer filtered = ByteBuffer.allocate(2048); - MemoryRecords.readableRecords(buffer).filterTo(new RetainNonNullKeysFilter(), filtered, - TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + MemoryRecords.readableRecords(buffer).filterTo(new RetainNonNullKeysFilter(), filtered, BufferSupplier.NO_CACHING); filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); @@ -742,8 +741,7 @@ public void testFilterToPreservesProducerInfo(Args args) { buffer.flip(); ByteBuffer filtered = ByteBuffer.allocate(2048); - MemoryRecords.readableRecords(buffer).filterTo(new RetainNonNullKeysFilter(), filtered, - TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + MemoryRecords.readableRecords(buffer).filterTo(new RetainNonNullKeysFilter(), filtered, BufferSupplier.NO_CACHING); filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); @@ -835,7 +833,7 @@ public void testFilterToWithUndersizedBuffer(Args args) { output.rewind(); MemoryRecords.FilterResult result = MemoryRecords.readableRecords(buffer).filterTo( - new RetainNonNullKeysFilter(), output, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + new RetainNonNullKeysFilter(), output, BufferSupplier.NO_CACHING); buffer.position(buffer.position() + result.bytesRead()); result.outputBuffer().flip(); @@ -882,7 +880,7 @@ public void testFilterTo(Args args) { ByteBuffer filtered = ByteBuffer.allocate(2048); MemoryRecords.FilterResult result = MemoryRecords.readableRecords(buffer).filterTo( - new RetainNonNullKeysFilter(), filtered, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING); + new RetainNonNullKeysFilter(), filtered, BufferSupplier.NO_CACHING); filtered.flip(); @@ -1000,8 +998,7 @@ public void testFilterToPreservesLogAppendTime(Args args) { buffer.flip(); ByteBuffer filtered = ByteBuffer.allocate(2048); - MemoryRecords.readableRecords(buffer).filterTo(new RetainNonNullKeysFilter(), filtered, TimestampType.CREATE_TIME, - BufferSupplier.NO_CACHING); + MemoryRecords.readableRecords(buffer).filterTo(new RetainNonNullKeysFilter(), filtered, BufferSupplier.NO_CACHING); filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index ab864ebdeeb47..8deb12f742dab 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -812,7 +812,7 @@ private[log] class Cleaner(val id: Int, sourceRecords.readInto(readBuffer, position) val records = MemoryRecords.readableRecords(readBuffer) throttler.maybeThrottle(records.sizeInBytes) - val result = records.filterTo(logCleanerFilter, writeBuffer, timestampTypeConfig, decompressionBufferSupplier) + val result = records.filterTo(logCleanerFilter, writeBuffer, decompressionBufferSupplier) stats.readMessages(result.messagesRead, result.bytesRead) stats.recopyMessages(result.messagesRetained, result.bytesRetained) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 8f931156eafe8..263b5a5faf5de 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -820,7 +820,7 @@ class UnifiedLogTest { 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, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING) + }, filtered, BufferSupplier.NO_CACHING) filtered.flip() val filteredRecords = MemoryRecords.readableRecords(filtered) @@ -862,7 +862,7 @@ class UnifiedLogTest { 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, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING) + }, filtered, BufferSupplier.NO_CACHING) filtered.flip() val filteredRecords = MemoryRecords.readableRecords(filtered) @@ -906,7 +906,7 @@ class UnifiedLogTest { 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, TimestampType.CREATE_TIME, BufferSupplier.NO_CACHING) + }, filtered, BufferSupplier.NO_CACHING) filtered.flip() val filteredRecords = MemoryRecords.readableRecords(filtered) From 58e1547996a67c332f9cfcabe56ad096e141fcaf Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Sun, 29 Dec 2024 00:52:51 -0800 Subject: [PATCH 08/18] Add tests --- ...gCleanerParameterizedIntegrationTest.scala | 150 +++++++++++++++++- 1 file changed, 149 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala index 0b53960dd04c4..986a439b8497f 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala @@ -25,10 +25,11 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record._ +import org.apache.kafka.common.utils.Time import org.apache.kafka.server.config.ServerConfigs import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.checkpoint.OffsetCheckpointFile -import org.apache.kafka.storage.internals.log.CleanerConfig +import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.extension.ExtensionContext import org.junit.jupiter.params.ParameterizedTest @@ -134,6 +135,131 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati assertEquals(toMap(messages), toMap(read), "Contents of the map shouldn't change") } + @ParameterizedTest + @ArgumentsSource(classOf[LogCleanerParameterizedIntegrationTest.ExcludeZstd]) + def testCleanerWithMessageFormatV0V1V2(compressionType: CompressionType): Unit = { + val compression = Compression.of(compressionType).build() + val largeMessageKey = 20 + val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, RecordBatch.MAGIC_VALUE_V0, compression) + val maxMessageSize = compression match { + case Compression.NONE => largeMessageSet.sizeInBytes + case _ => + // the broker assigns absolute offsets for message format 0 which potentially causes the compressed size to + // increase because the broker offsets are larger than the ones assigned by the client + // adding `6` to the message set size is good enough for this test: it covers the increased message size while + // still being less than the overhead introduced by the conversion from message format version 0 to 1 + largeMessageSet.sizeInBytes + 6 + } + + cleaner = makeCleaner(partitions = topicPartitions, maxMessageSize = maxMessageSize) + + val log = cleaner.logs.get(topicPartitions(0)) + val props = logConfigProperties(maxMessageSize = maxMessageSize) + props.put(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, TimestampType.LOG_APPEND_TIME.name) + val logConfig = new LogConfig(props) + log.updateConfig(logConfig) + + val appends1 = writeDups(numKeys = 100, numDups = 3, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V0) + val startSize = log.size + cleaner.startup() + + val firstDirty = log.activeSegment.baseOffset + checkLastCleaned("log", 0, firstDirty) + val compactedSize = log.logSegments.asScala.map(_.size).sum + assertTrue(startSize > compactedSize, s"log should have been compacted: startSize=$startSize compactedSize=$compactedSize") + + checkLogAfterAppendingDups(log, startSize, appends1) + + val dupsV0 = writeDups(numKeys = 40, numDups = 3, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V0) + val appendInfo = log.appendAsLeaderWithRecordVersion(largeMessageSet, leaderEpoch = 0, recordVersion = RecordVersion.V0) + // move LSO forward to increase compaction bound + log.updateHighWatermark(log.logEndOffset) + val largeMessageOffset = appendInfo.firstOffset + + // also add some messages with version 1 and version 2 to check that we handle mixed format versions correctly + val dupsV1 = writeDups(startKey = 30, numKeys = 40, numDups = 3, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V1) + val dupsV2 = writeDups(startKey = 15, numKeys = 5, numDups = 3, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V2) + + val v0RecordKeysWithNoV1V2Updates = (appends1.map(_._1).toSet -- dupsV1.map(_._1) -- dupsV2.map(_._1)).map(_.toString) + val appends2: Seq[(Int, String, Long)] = + appends1 ++ dupsV0 ++ Seq((largeMessageKey, largeMessageValue, largeMessageOffset)) ++ dupsV1 ++ dupsV2 + + // roll the log so that all appended messages can be compacted + log.roll() + val firstDirty2 = log.activeSegment.baseOffset + checkLastCleaned("log", 0, firstDirty2) + + checkLogAfterAppendingDups(log, startSize, appends2) + checkLogAfterConvertingToV2(compressionType, log, logConfig.messageTimestampType, v0RecordKeysWithNoV1V2Updates) + } + + @ParameterizedTest + @ArgumentsSource(classOf[LogCleanerParameterizedIntegrationTest.ExcludeZstd]) + def testCleaningNestedMessagesWithV0V1(compressionType: CompressionType): Unit = { + val compression = Compression.of(compressionType).build() + val maxMessageSize = 192 + cleaner = makeCleaner(partitions = topicPartitions, maxMessageSize = maxMessageSize, segmentSize = 256) + + val log = cleaner.logs.get(topicPartitions(0)) + val logConfig = new LogConfig(logConfigProperties(maxMessageSize = maxMessageSize, segmentSize = 256)) + log.updateConfig(logConfig) + + // with compression enabled, these messages will be written as a single message containing all the individual messages + var appendsV0 = writeDupsSingleMessageSet(numKeys = 2, numDups = 3, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V0) + appendsV0 ++= writeDupsSingleMessageSet(numKeys = 2, startKey = 3, numDups = 2, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V0) + + var appendsV1 = writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V1) + appendsV1 ++= writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V1) + appendsV1 ++= writeDupsSingleMessageSet(startKey = 6, numKeys = 2, numDups = 2, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V1) + + val appends = appendsV0 ++ appendsV1 + + val v0RecordKeysWithNoV1V2Updates = (appendsV0.map(_._1).toSet -- appendsV1.map(_._1)).map(_.toString) + + // roll the log so that all appended messages can be compacted + log.roll() + val startSize = log.size + cleaner.startup() + + val firstDirty = log.activeSegment.baseOffset + assertTrue(firstDirty > appendsV0.size) // ensure we clean data from V0 and V1 + + checkLastCleaned("log", 0, firstDirty) + val compactedSize = log.logSegments.asScala.map(_.size).sum + assertTrue(startSize > compactedSize, s"log should have been compacted: startSize=$startSize compactedSize=$compactedSize") + + checkLogAfterAppendingDups(log, startSize, appends) + checkLogAfterConvertingToV2(compressionType, log, logConfig.messageTimestampType, v0RecordKeysWithNoV1V2Updates) + } + + private def checkLogAfterConvertingToV2(compressionType: CompressionType, log: UnifiedLog, timestampType: TimestampType, + keysForV0RecordsWithNoV1V2Updates: Set[String]): Unit = { + for (segment <- log.logSegments.asScala; recordBatch <- segment.log.batches.asScala) { + // Uncompressed v0/v1 records are always converted into single record v2 batches via compaction if they are retained + // Compressed v0/v1 record batches are converted into record batches v2 with one or more records (depending on the + // number of retained records after compaction) + assertEquals(RecordVersion.V2.value, recordBatch.magic) + if (compressionType == CompressionType.NONE) + assertEquals(1, recordBatch.iterator().asScala.size) + else + assertTrue(recordBatch.iterator().asScala.size >= 1) + + val firstRecordKey = TestUtils.readString(recordBatch.iterator().next().key()) + if (keysForV0RecordsWithNoV1V2Updates.contains(firstRecordKey)) + assertEquals(TimestampType.CREATE_TIME, recordBatch.timestampType) + else + assertEquals(timestampType, recordBatch.timestampType) + + recordBatch.iterator.asScala.foreach { record => + val recordKey = TestUtils.readString(record.key) + if (keysForV0RecordsWithNoV1V2Updates.contains(recordKey)) + assertEquals(RecordBatch.NO_TIMESTAMP, record.timestamp, "Record " + recordKey + " with unexpected timestamp ") + else + assertNotEquals(RecordBatch.NO_TIMESTAMP, record.timestamp, "Record " + recordKey + " with unexpected timestamp " + RecordBatch.NO_TIMESTAMP) + } + } + } + @ParameterizedTest @ArgumentsSource(classOf[LogCleanerParameterizedIntegrationTest.AllCompressions]) def cleanerConfigUpdateTest(compressionType: CompressionType): Unit = { @@ -213,6 +339,28 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati (key, value, deepLogEntry.offset) } } + + private def writeDupsSingleMessageSet(numKeys: Int, numDups: Int, log: UnifiedLog, codec: Compression, + startKey: Int = 0, magicValue: Byte): Seq[(Int, String, Long)] = { + val kvs = for (_ <- 0 until numDups; key <- startKey until (startKey + numKeys)) yield { + val payload = counter.toString + incCounter() + (key, payload) + } + + val records = kvs.map { case (key, payload) => + new SimpleRecord(Time.SYSTEM.milliseconds(), key.toString.getBytes, payload.getBytes) + } + + val appendInfo = log.appendAsLeaderWithRecordVersion(MemoryRecords.withRecords(magicValue, codec, records: _*), + leaderEpoch = 0, recordVersion = RecordVersion.lookup(magicValue)) + // move LSO forward to increase compaction bound + log.updateHighWatermark(log.logEndOffset) + val offsets = appendInfo.firstOffset to appendInfo.lastOffset + + kvs.zip(offsets).map { case (kv, offset) => (kv._1, kv._2, offset) } + } + } object LogCleanerParameterizedIntegrationTest { From 593dc8394cdba9d37aab1066edf99dc91fd99fba Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Sun, 29 Dec 2024 00:55:41 -0800 Subject: [PATCH 09/18] Remove unused imports --- .../main/java/org/apache/kafka/common/record/MemoryRecords.java | 1 - .../java/org/apache/kafka/common/record/MemoryRecordsTest.java | 1 - 2 files changed, 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index 434cf6bf47147..3aee889aded6e 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 @@ -16,7 +16,6 @@ */ package org.apache.kafka.common.record; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.message.KRaftVersionRecord; 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 25c8242455ef1..3818976e423fd 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 @@ -16,7 +16,6 @@ */ package org.apache.kafka.common.record; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.header.internals.RecordHeaders; From a50bd5d5d34f921895bd84a565b557e863f91914 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Sun, 29 Dec 2024 09:32:39 -0800 Subject: [PATCH 10/18] Remove unused method parameter --- core/src/main/scala/kafka/log/LogCleaner.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 8deb12f742dab..43193016fd06b 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -684,7 +684,7 @@ private[log] class Cleaner(val id: Int, try { cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainLegacyDeletesAndTxnMarkers, log.config.deleteRetentionMs, - log.config.maxMessageSize, log.config.messageTimestampType, transactionMetadata, lastOffsetOfActiveProducers, + log.config.maxMessageSize, transactionMetadata, lastOffsetOfActiveProducers, upperBoundOffsetOfCleaningRound, stats, currentTime = currentTime) } catch { case e: LogSegmentOffsetOverflowException => @@ -742,7 +742,6 @@ private[log] class Cleaner(val id: Int, retainLegacyDeletesAndTxnMarkers: Boolean, deleteRetentionMs: Long, maxLogMessageSize: Int, - timestampTypeConfig: TimestampType, transactionMetadata: CleanedTransactionMetadata, lastRecordsOfActiveProducers: mutable.Map[Long, LastRecord], upperBoundOffsetOfCleaningRound: Long, From ed6e24020509dc60f88f7cb135523244db5a3100 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 7 Jan 2025 20:51:10 -0800 Subject: [PATCH 11/18] Change UnifiedLog.leaderEpochCache from Option[LeaderEpochFileCache] to LeaderEpochFileCache --- .../kafka/log/remote/RemoteLogManager.java | 93 ++++++++----------- .../java/kafka/server/TierStateMachine.java | 4 +- .../main/scala/kafka/cluster/Partition.scala | 2 +- .../src/main/scala/kafka/log/UnifiedLog.scala | 90 +++++++----------- .../scala/kafka/raft/KafkaMetadataLog.scala | 2 +- .../kafka/server/LocalLeaderEndPoint.scala | 6 +- .../log/remote/RemoteLogManagerTest.java | 72 +++++++------- .../kafka/cluster/PartitionLockTest.scala | 4 +- .../unit/kafka/cluster/PartitionTest.scala | 8 +- .../kafka/log/LogCleanerManagerTest.scala | 2 +- .../scala/unit/kafka/log/LogCleanerTest.scala | 2 +- .../scala/unit/kafka/log/LogLoaderTest.scala | 12 +-- .../scala/unit/kafka/log/UnifiedLogTest.scala | 40 ++++---- .../kafka/server/ReplicaManagerTest.scala | 67 +------------ .../unit/kafka/utils/SchedulerTest.scala | 2 +- .../storage/TieredStorageTestContext.java | 6 +- .../ExpectLeaderEpochCheckpointAction.java | 8 +- 17 files changed, 151 insertions(+), 269 deletions(-) diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index bcdd718baa15b..60fd5ccb61e6b 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -777,11 +777,7 @@ public boolean isCancelled() { * @return the leader epoch entries */ List getLeaderEpochEntries(UnifiedLog log, long startOffset, long endOffset) { - if (log.leaderEpochCache().isDefined()) { - return log.leaderEpochCache().get().epochEntriesInRange(startOffset, endOffset); - } else { - return Collections.emptyList(); - } + return log.leaderEpochCache().epochEntriesInRange(startOffset, endOffset); } // VisibleForTesting @@ -1249,11 +1245,6 @@ void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionE } final UnifiedLog log = logOptional.get(); - final Option leaderEpochCacheOption = log.leaderEpochCache(); - if (leaderEpochCacheOption.isEmpty()) { - logger.debug("No leader epoch cache available for partition: {}", topicIdPartition); - return; - } // Cleanup remote log segments and update the log start offset if applicable. final Iterator segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition); @@ -1281,7 +1272,7 @@ void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionE final List remoteLeaderEpochs = new ArrayList<>(epochsSet); Collections.sort(remoteLeaderEpochs); - LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get(); + LeaderEpochFileCache leaderEpochCache = log.leaderEpochCache(); // Build the leader epoch map by filtering the epochs that do not have any records. NavigableMap epochWithOffsets = buildFilteredLeaderEpochMap(leaderEpochCache.epochWithOffsets()); @@ -1680,9 +1671,9 @@ public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws OptionalInt epoch = OptionalInt.empty(); if (logOptional.isPresent()) { - Option leaderEpochCache = logOptional.get().leaderEpochCache(); - if (leaderEpochCache != null && leaderEpochCache.isDefined()) { - epoch = leaderEpochCache.get().epochForOffset(offset); + LeaderEpochFileCache leaderEpochCache = logOptional.get().leaderEpochCache(); + if (leaderEpochCache != null) { + epoch = leaderEpochCache.epochForOffset(offset); } } @@ -1819,7 +1810,7 @@ private void collectAbortedTransactions(long startOffset, UnifiedLog log) throws RemoteStorageException { TopicPartition tp = segmentMetadata.topicIdPartition().topicPartition(); boolean isSearchComplete = false; - LeaderEpochFileCache leaderEpochCache = log.leaderEpochCache().getOrElse(null); + LeaderEpochFileCache leaderEpochCache = log.leaderEpochCache(); Optional currentMetadataOpt = Optional.of(segmentMetadata); while (!isSearchComplete && currentMetadataOpt.isPresent()) { RemoteLogSegmentMetadata currentMetadata = currentMetadataOpt.get(); @@ -1866,13 +1857,9 @@ private void collectAbortedTransactionInLocalSegments(long startOffset, // visible for testing. Optional findNextSegmentMetadata(RemoteLogSegmentMetadata segmentMetadata, - Option leaderEpochFileCacheOption) throws RemoteStorageException { - if (leaderEpochFileCacheOption.isEmpty()) { - return Optional.empty(); - } - + LeaderEpochFileCache leaderEpochFileCacheOption) throws RemoteStorageException { long nextSegmentBaseOffset = segmentMetadata.endOffset() + 1; - OptionalInt epoch = leaderEpochFileCacheOption.get().epochForOffset(nextSegmentBaseOffset); + OptionalInt epoch = leaderEpochFileCacheOption.epochForOffset(nextSegmentBaseOffset); return epoch.isPresent() ? fetchRemoteLogSegmentMetadata(segmentMetadata.topicIdPartition().topicPartition(), epoch.getAsInt(), nextSegmentBaseOffset) : Optional.empty(); @@ -1933,30 +1920,27 @@ EnrichedRecordBatch findFirstBatch(RemoteLogInputStream remoteLogInputStream, lo OffsetAndEpoch findHighestRemoteOffset(TopicIdPartition topicIdPartition, UnifiedLog log) throws RemoteStorageException { OffsetAndEpoch offsetAndEpoch = null; - Option leaderEpochCacheOpt = log.leaderEpochCache(); - if (leaderEpochCacheOpt.isDefined()) { - LeaderEpochFileCache cache = leaderEpochCacheOpt.get(); - Optional maybeEpochEntry = cache.latestEntry(); - while (offsetAndEpoch == null && maybeEpochEntry.isPresent()) { - int epoch = maybeEpochEntry.get().epoch; - Optional highestRemoteOffsetOpt = - remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch); - if (highestRemoteOffsetOpt.isPresent()) { - Map.Entry entry = cache.endOffsetFor(epoch, log.logEndOffset()); - int requestedEpoch = entry.getKey(); - long endOffset = entry.getValue(); - long highestRemoteOffset = highestRemoteOffsetOpt.get(); - if (endOffset <= highestRemoteOffset) { - LOGGER.info("The end-offset for epoch {}: ({}, {}) is less than or equal to the " + - "highest-remote-offset: {} for partition: {}", epoch, requestedEpoch, endOffset, - highestRemoteOffset, topicIdPartition); - offsetAndEpoch = new OffsetAndEpoch(endOffset - 1, requestedEpoch); - } else { - offsetAndEpoch = new OffsetAndEpoch(highestRemoteOffset, epoch); - } + LeaderEpochFileCache leaderEpochCache = log.leaderEpochCache(); + Optional maybeEpochEntry = leaderEpochCache.latestEntry(); + while (offsetAndEpoch == null && maybeEpochEntry.isPresent()) { + int epoch = maybeEpochEntry.get().epoch; + Optional highestRemoteOffsetOpt = + remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch); + if (highestRemoteOffsetOpt.isPresent()) { + Map.Entry entry = leaderEpochCache.endOffsetFor(epoch, log.logEndOffset()); + int requestedEpoch = entry.getKey(); + long endOffset = entry.getValue(); + long highestRemoteOffset = highestRemoteOffsetOpt.get(); + if (endOffset <= highestRemoteOffset) { + LOGGER.info("The end-offset for epoch {}: ({}, {}) is less than or equal to the " + + "highest-remote-offset: {} for partition: {}", epoch, requestedEpoch, endOffset, + highestRemoteOffset, topicIdPartition); + offsetAndEpoch = new OffsetAndEpoch(endOffset - 1, requestedEpoch); + } else { + offsetAndEpoch = new OffsetAndEpoch(highestRemoteOffset, epoch); } - maybeEpochEntry = cache.previousEntry(epoch); } + maybeEpochEntry = leaderEpochCache.previousEntry(epoch); } if (offsetAndEpoch == null) { offsetAndEpoch = new OffsetAndEpoch(-1L, RecordBatch.NO_PARTITION_LEADER_EPOCH); @@ -1966,20 +1950,17 @@ OffsetAndEpoch findHighestRemoteOffset(TopicIdPartition topicIdPartition, Unifie long findLogStartOffset(TopicIdPartition topicIdPartition, UnifiedLog log) throws RemoteStorageException { Optional logStartOffset = Optional.empty(); - Option maybeLeaderEpochFileCache = log.leaderEpochCache(); - if (maybeLeaderEpochFileCache.isDefined()) { - LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get(); - OptionalInt earliestEpochOpt = cache.earliestEntry() - .map(epochEntry -> OptionalInt.of(epochEntry.epoch)) - .orElseGet(OptionalInt::empty); - while (logStartOffset.isEmpty() && earliestEpochOpt.isPresent()) { - Iterator iterator = - remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, earliestEpochOpt.getAsInt()); - if (iterator.hasNext()) { - logStartOffset = Optional.of(iterator.next().startOffset()); - } - earliestEpochOpt = cache.nextEpoch(earliestEpochOpt.getAsInt()); + LeaderEpochFileCache leaderEpochCache = log.leaderEpochCache(); + OptionalInt earliestEpochOpt = leaderEpochCache.earliestEntry() + .map(epochEntry -> OptionalInt.of(epochEntry.epoch)) + .orElseGet(OptionalInt::empty); + while (logStartOffset.isEmpty() && earliestEpochOpt.isPresent()) { + Iterator iterator = + remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, earliestEpochOpt.getAsInt()); + if (iterator.hasNext()) { + logStartOffset = Optional.of(iterator.next().startOffset()); } + earliestEpochOpt = leaderEpochCache.nextEpoch(earliestEpochOpt.getAsInt()); } return logStartOffset.orElseGet(log::localLogStartOffset); } diff --git a/core/src/main/java/kafka/server/TierStateMachine.java b/core/src/main/java/kafka/server/TierStateMachine.java index ddb19e86aeca1..d316e70da2e3e 100644 --- a/core/src/main/java/kafka/server/TierStateMachine.java +++ b/core/src/main/java/kafka/server/TierStateMachine.java @@ -247,9 +247,7 @@ private Long buildRemoteLogAuxState(TopicPartition topicPartition, // Build leader epoch cache. List epochs = readLeaderEpochCheckpoint(rlm, remoteLogSegmentMetadata); - if (unifiedLog.leaderEpochCache().isDefined()) { - unifiedLog.leaderEpochCache().get().assign(epochs); - } + unifiedLog.leaderEpochCache().assign(epochs); log.info("Updated the epoch cache from remote tier till offset: {} with size: {} for {}", leaderLocalLogStartOffset, epochs.size(), partition); diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index e58f5824e767b..e4d2f01cdd607 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -806,7 +806,7 @@ class Partition(val topicPartition: TopicPartition, // to ensure that these followers can truncate to the right offset, we must cache the new // leader epoch and the start offset since it should be larger than any epoch that a follower // would try to query. - leaderLog.maybeAssignEpochStartOffset(partitionState.leaderEpoch, leaderEpochStartOffset) + leaderLog.assignEpochStartOffset(partitionState.leaderEpoch, leaderEpochStartOffset) // Initialize lastCaughtUpTime of replicas as well as their lastFetchTimeMs and // lastFetchLeaderLogEndOffset. diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index 9349ca0baefef..2c7441ca39d1f 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -99,7 +99,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, private val localLog: LocalLog, val brokerTopicStats: BrokerTopicStats, val producerIdExpirationCheckIntervalMs: Int, - @volatile var leaderEpochCache: Option[LeaderEpochFileCache], + @volatile var leaderEpochCache: LeaderEpochFileCache, val producerStateManager: ProducerStateManager, @volatile private var _topicId: Option[Uuid], val keepPartitionMetadataFile: Boolean, @@ -509,8 +509,8 @@ class UnifiedLog(@volatile var logStartOffset: Long, } private def initializeLeaderEpochCache(): Unit = lock synchronized { - leaderEpochCache = Some(UnifiedLog.createLeaderEpochCache( - dir, topicPartition, logDirFailureChannel, leaderEpochCache, scheduler)) + leaderEpochCache = UnifiedLog.createLeaderEpochCache( + dir, topicPartition, logDirFailureChannel, Option.apply(leaderEpochCache), scheduler) } private def updateHighWatermarkWithLogEndOffset(): Unit = { @@ -675,7 +675,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, initializeLeaderEpochCache() initializePartitionMetadata() } else { - leaderEpochCache = None + leaderEpochCache = null partitionMetadataFile = None } } @@ -863,14 +863,14 @@ class UnifiedLog(@volatile var logStartOffset: Long, // update the epoch cache with the epoch stamped onto the message by the leader validRecords.batches.forEach { batch => if (batch.magic >= RecordBatch.MAGIC_VALUE_V2) { - maybeAssignEpochStartOffset(batch.partitionLeaderEpoch, batch.baseOffset) + assignEpochStartOffset(batch.partitionLeaderEpoch, batch.baseOffset) } else { // In partial upgrade scenarios, we may get a temporary regression to the message format. In // order to ensure the safety of leader election, we clear the epoch cache so that we revert // to truncation by high watermark after the next leader election. - leaderEpochCache.filter(_.nonEmpty).foreach { cache => + if (leaderEpochCache.nonEmpty) { warn(s"Clearing leader epoch cache after unexpected append with message format v${batch.magic}") - cache.clearAndFlush() + leaderEpochCache.clearAndFlush() } } } @@ -941,23 +941,18 @@ class UnifiedLog(@volatile var logStartOffset: Long, } } - def maybeAssignEpochStartOffset(leaderEpoch: Int, startOffset: Long): Unit = { - leaderEpochCache.foreach { cache => - cache.assign(leaderEpoch, startOffset) - } - } + def assignEpochStartOffset(leaderEpoch: Int, startOffset: Long): Unit = + leaderEpochCache.assign(leaderEpoch, startOffset) - def latestEpoch: Option[Int] = leaderEpochCache.flatMap(_.latestEpoch.toScala) + def latestEpoch: Option[Int] = leaderEpochCache.latestEpoch.toScala def endOffsetForEpoch(leaderEpoch: Int): Option[OffsetAndEpoch] = { - leaderEpochCache.flatMap { cache => - val entry = cache.endOffsetFor(leaderEpoch, logEndOffset) - val (foundEpoch, foundOffset) = (entry.getKey, entry.getValue) - if (foundOffset == UNDEFINED_EPOCH_OFFSET) - None - else - Some(new OffsetAndEpoch(foundOffset, foundEpoch)) - } + val entry = leaderEpochCache.endOffsetFor(leaderEpoch, logEndOffset) + val (foundEpoch, foundOffset) = (entry.getKey, entry.getValue) + if (foundOffset == UNDEFINED_EPOCH_OFFSET) + None + else + Some(new OffsetAndEpoch(foundOffset, foundEpoch)) } private def maybeIncrementFirstUnstableOffset(): Unit = lock synchronized { @@ -1017,7 +1012,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, updatedLogStartOffset = true updateLogStartOffset(newLogStartOffset) info(s"Incremented log start offset to $newLogStartOffset due to $reason") - leaderEpochCache.foreach(_.truncateFromStartAsyncFlush(logStartOffset)) + leaderEpochCache.truncateFromStartAsyncFlush(logStartOffset) producerStateManager.onLogStartOffsetIncremented(newLogStartOffset) maybeIncrementFirstUnstableOffset() } @@ -1284,7 +1279,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, // The first cached epoch usually corresponds to the log start offset, but we have to verify this since // it may not be true following a message format version bump as the epoch will not be available for // log entries written in the older format. - val earliestEpochEntry = leaderEpochCache.toJava.flatMap(_.earliestEntry()) + val earliestEpochEntry = leaderEpochCache.earliestEntry() val epochOpt = if (earliestEpochEntry.isPresent && earliestEpochEntry.get().startOffset <= logStartOffset) { Optional.of[Integer](earliestEpochEntry.get().epoch) } else Optional.empty[Integer]() @@ -1293,41 +1288,24 @@ class UnifiedLog(@volatile var logStartOffset: Long, } else if (targetTimestamp == ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP) { val curLocalLogStartOffset = localLogStartOffset() - val epochResult: Optional[Integer] = - if (leaderEpochCache.isDefined) { - val epochOpt = leaderEpochCache.get.epochForOffset(curLocalLogStartOffset) - if (epochOpt.isPresent) Optional.of(epochOpt.getAsInt) else Optional.empty() - } else { - Optional.empty() - } + val epochResult: Optional[Integer] = { + val epochOpt = leaderEpochCache.epochForOffset(curLocalLogStartOffset) + if (epochOpt.isPresent) Optional.of(epochOpt.getAsInt) else Optional.empty() + } new OffsetResultHolder(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, curLocalLogStartOffset, epochResult)) } else if (targetTimestamp == ListOffsetsRequest.LATEST_TIMESTAMP) { - val epoch = leaderEpochCache match { - case Some(cache) => - val latestEpoch = cache.latestEpoch() - if (latestEpoch.isPresent) Optional.of[Integer](latestEpoch.getAsInt) else Optional.empty[Integer]() - case None => Optional.empty[Integer]() - } + val latestEpoch = leaderEpochCache.latestEpoch() + val epoch = if (latestEpoch.isPresent) Optional.of[Integer](latestEpoch.getAsInt) else Optional.empty[Integer]() new OffsetResultHolder(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, logEndOffset, epoch)) } else if (targetTimestamp == ListOffsetsRequest.LATEST_TIERED_TIMESTAMP) { if (remoteLogEnabled()) { val curHighestRemoteOffset = highestOffsetInRemoteStorage() - + val epochOpt = leaderEpochCache.epochForOffset(curHighestRemoteOffset) val epochResult: Optional[Integer] = - if (leaderEpochCache.isDefined) { - val epochOpt = leaderEpochCache.get.epochForOffset(curHighestRemoteOffset) - if (epochOpt.isPresent) { - Optional.of(epochOpt.getAsInt) - } else if (curHighestRemoteOffset == -1) { - Optional.of(RecordBatch.NO_PARTITION_LEADER_EPOCH) - } else { - Optional.empty() - } - } else { - Optional.empty() - } - + if (epochOpt.isPresent) Optional.of(epochOpt.getAsInt) + else if (curHighestRemoteOffset == -1) Optional.of(RecordBatch.NO_PARTITION_LEADER_EPOCH) + else Optional.empty() new OffsetResultHolder(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, curHighestRemoteOffset, epochResult)) } else { new OffsetResultHolder(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, -1L, Optional.of(-1))) @@ -1353,7 +1331,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, } val asyncOffsetReadFutureHolder = remoteLogManager.get.asyncOffsetRead(topicPartition, targetTimestamp, - logStartOffset, leaderEpochCache.get, () => searchOffsetInLocalLog(targetTimestamp, localLogStartOffset())) + logStartOffset, leaderEpochCache, () => searchOffsetInLocalLog(targetTimestamp, localLogStartOffset())) new OffsetResultHolder(Optional.empty(), Optional.of(asyncOffsetReadFutureHolder)) } else { @@ -1781,7 +1759,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, lock synchronized { localLog.checkIfMemoryMappedBufferClosed() producerExpireCheck.cancel(true) - leaderEpochCache.foreach(_.clear()) + leaderEpochCache.clear() val deletedSegments = localLog.deleteAllSegments() deleteProducerSnapshots(deletedSegments, asyncDelete = false) localLog.deleteEmptyDir() @@ -1834,7 +1812,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, // and inserted the first start offset entry, but then failed to append any entries // before another leader was elected. lock synchronized { - leaderEpochCache.foreach(_.truncateFromEndAsyncFlush(logEndOffset)) + leaderEpochCache.truncateFromEndAsyncFlush(logEndOffset) } false @@ -1847,7 +1825,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, } else { val deletedSegments = localLog.truncateTo(targetOffset) deleteProducerSnapshots(deletedSegments, asyncDelete = true) - leaderEpochCache.foreach(_.truncateFromEndAsyncFlush(targetOffset)) + leaderEpochCache.truncateFromEndAsyncFlush(targetOffset) logStartOffset = math.min(targetOffset, logStartOffset) rebuildProducerState(targetOffset, producerStateManager) if (highWatermark >= localLog.logEndOffset) @@ -1871,7 +1849,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, debug(s"Truncate and start at offset $newOffset, logStartOffset: ${logStartOffsetOpt.getOrElse(newOffset)}") lock synchronized { localLog.truncateFullyAndStartAt(newOffset) - leaderEpochCache.foreach(_.clearAndFlush()) + leaderEpochCache.clearAndFlush() producerStateManager.truncateFullyAndStartAt(newOffset) logStartOffset = logStartOffsetOpt.getOrElse(newOffset) if (remoteLogEnabled()) _localLogStartOffset = newOffset @@ -2059,7 +2037,7 @@ object UnifiedLog extends Logging { localLog, brokerTopicStats, producerIdExpirationCheckIntervalMs, - Some(leaderEpochCache), + leaderEpochCache, producerStateManager, topicId, keepPartitionMetadataFile, diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index 451bb5a851132..bd80c0aca4d10 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -197,7 +197,7 @@ final class KafkaMetadataLog private ( } override def initializeLeaderEpoch(epoch: Int): Unit = { - log.maybeAssignEpochStartOffset(epoch, log.logEndOffset) + log.assignEpochStartOffset(epoch, log.logEndOffset) } override def updateHighWatermark(offsetMetadata: LogOffsetMetadata): Unit = { diff --git a/core/src/main/scala/kafka/server/LocalLeaderEndPoint.scala b/core/src/main/scala/kafka/server/LocalLeaderEndPoint.scala index 03258295a41a3..1e2a6cd033e48 100644 --- a/core/src/main/scala/kafka/server/LocalLeaderEndPoint.scala +++ b/core/src/main/scala/kafka/server/LocalLeaderEndPoint.scala @@ -118,21 +118,21 @@ class LocalLeaderEndPoint(sourceBroker: BrokerEndPoint, override def fetchEarliestOffset(topicPartition: TopicPartition, currentLeaderEpoch: Int): OffsetAndEpoch = { val partition = replicaManager.getPartitionOrException(topicPartition) val logStartOffset = partition.localLogOrException.logStartOffset - val epoch = partition.localLogOrException.leaderEpochCache.get.epochForOffset(logStartOffset) + val epoch = partition.localLogOrException.leaderEpochCache.epochForOffset(logStartOffset) new OffsetAndEpoch(logStartOffset, epoch.orElse(0)) } override def fetchLatestOffset(topicPartition: TopicPartition, currentLeaderEpoch: Int): OffsetAndEpoch = { val partition = replicaManager.getPartitionOrException(topicPartition) val logEndOffset = partition.localLogOrException.logEndOffset - val epoch = partition.localLogOrException.leaderEpochCache.get.epochForOffset(logEndOffset) + val epoch = partition.localLogOrException.leaderEpochCache.epochForOffset(logEndOffset) new OffsetAndEpoch(logEndOffset, epoch.orElse(0)) } override def fetchEarliestLocalOffset(topicPartition: TopicPartition, currentLeaderEpoch: Int): OffsetAndEpoch = { val partition = replicaManager.getPartitionOrException(topicPartition) val localLogStartOffset = partition.localLogOrException.localLogStartOffset() - val epoch = partition.localLogOrException.leaderEpochCache.get.epochForOffset(localLogStartOffset) + val epoch = partition.localLogOrException.leaderEpochCache.epochForOffset(localLogStartOffset) new OffsetAndEpoch(localLogStartOffset, epoch.orElse(0)) } diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index 2ae2a18467065..5f21a758c67f5 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -279,7 +279,7 @@ void tearDown() { void testGetLeaderEpochCheckpoint() { checkpoint.write(totalEpochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); assertEquals(totalEpochEntries, remoteLogManager.getLeaderEpochEntries(mockLog, 0, 300)); List epochEntries = remoteLogManager.getLeaderEpochEntries(mockLog, 100, 200); @@ -295,7 +295,7 @@ void testFindHighestRemoteOffsetOnEmptyRemoteStorage() throws RemoteStorageExce ); checkpoint.write(totalEpochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); TopicIdPartition tpId = new TopicIdPartition(Uuid.randomUuid(), tp); OffsetAndEpoch offsetAndEpoch = remoteLogManager.findHighestRemoteOffset(tpId, mockLog); assertEquals(new OffsetAndEpoch(-1L, -1), offsetAndEpoch); @@ -309,7 +309,7 @@ void testFindHighestRemoteOffset() throws RemoteStorageException { ); checkpoint.write(totalEpochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); TopicIdPartition tpId = new TopicIdPartition(Uuid.randomUuid(), tp); when(remoteLogMetadataManager.highestOffsetForEpoch(eq(tpId), anyInt())).thenAnswer(ans -> { Integer epoch = ans.getArgument(1, Integer.class); @@ -332,7 +332,7 @@ void testFindHighestRemoteOffsetWithUncleanLeaderElection() throws RemoteStorage ); checkpoint.write(totalEpochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); TopicIdPartition tpId = new TopicIdPartition(Uuid.randomUuid(), tp); when(remoteLogMetadataManager.highestOffsetForEpoch(eq(tpId), anyInt())).thenAnswer(ans -> { Integer epoch = ans.getArgument(1, Integer.class); @@ -501,7 +501,7 @@ private void assertCopyExpectedLogSegmentsToRemote(long oldSegmentStartOffset, // leader epoch preparation checkpoint.write(totalEpochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())).thenReturn(Optional.of(-1L)); File tempFile = TestUtils.tempFile(); @@ -615,7 +615,7 @@ void testCustomMetadataSizeExceedsLimit() throws Exception { // leader epoch preparation checkpoint.write(totalEpochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())).thenReturn(Optional.of(-1L)); File tempFile = TestUtils.tempFile(); @@ -707,7 +707,7 @@ void testFailedCopyShouldDeleteTheDanglingSegment() throws Exception { // leader epoch preparation checkpoint.write(totalEpochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())).thenReturn(Optional.of(-1L)); File tempFile = TestUtils.tempFile(); @@ -797,7 +797,7 @@ void testRemoteLogManagerTasksAvgIdlePercentAndMetadataCountMetrics() throws Exc // leader epoch preparation checkpoint.write(totalEpochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())).thenReturn(Optional.of(0L)); File tempFile = TestUtils.tempFile(); @@ -916,7 +916,7 @@ void testRemoteLogTaskUpdateRemoteLogSegmentMetadataAfterLogDirChanged() throws // leader epoch preparation checkpoint.write(totalEpochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())) .thenReturn(Optional.of(0L)) .thenReturn(Optional.of(nextSegmentStartOffset - 1)); @@ -995,7 +995,7 @@ void testRemoteLogTaskUpdateRemoteLogSegmentMetadataAfterLogDirChanged() throws // simulate altering log dir completes, and the new partition leader changes to the same broker in different log dir (dir2) mockLog = mock(UnifiedLog.class); when(mockLog.parentDir()).thenReturn("dir2"); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(mockLog.config()).thenReturn(logConfig); when(mockLog.logEndOffset()).thenReturn(500L); @@ -1031,7 +1031,7 @@ void testRemoteLogManagerRemoteMetrics() throws Exception { // leader epoch preparation checkpoint.write(totalEpochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())).thenReturn(Optional.of(0L)); File tempFile = TestUtils.tempFile(); @@ -1195,7 +1195,7 @@ void testMetricsUpdateOnCopyLogSegmentsFailure() throws Exception { // leader epoch preparation checkpoint.write(totalEpochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())).thenReturn(Optional.of(0L)); File tempFile = TestUtils.tempFile(); @@ -1270,7 +1270,7 @@ void testRLMTaskDoesNotUploadSegmentsWhenRemoteLogMetadataManagerIsNotInitialize // leader epoch preparation checkpoint.write(totalEpochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); // Throw a retryable exception so indicate that the remote log metadata manager is not initialized yet when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())) @@ -1440,7 +1440,7 @@ public void testFetchNextSegmentWithTxnIndex() throws RemoteStorageException { public void testFindNextSegmentWithTxnIndex() throws RemoteStorageException { checkpoint.write(totalEpochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())) .thenReturn(Optional.of(0L)); @@ -1471,7 +1471,7 @@ public void testFindNextSegmentWithTxnIndex() throws RemoteStorageException { public void testFindNextSegmentWithTxnIndexTraversesNextEpoch() throws RemoteStorageException { checkpoint.write(totalEpochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())) .thenReturn(Optional.of(0L)); @@ -1696,7 +1696,7 @@ void testFetchOffsetByTimestampWithTieredStorageDoesNotFetchIndexWhenExistsLocal epochEntries.add(new EpochEntry(5, 200L)); checkpoint.write(epochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); long timestamp = time.milliseconds(); RemoteLogSegmentMetadata metadata0 = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(tpId, Uuid.randomUuid()), @@ -2187,7 +2187,7 @@ public void testFindLogStartOffset() throws RemoteStorageException, IOException checkpoint.write(epochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); long timestamp = time.milliseconds(); int segmentSize = 1024; @@ -2225,7 +2225,7 @@ public void testFindLogStartOffsetFallbackToLocalLogStartOffsetWhenRemoteIsEmpty checkpoint.write(epochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(mockLog.localLogStartOffset()).thenReturn(250L); when(remoteLogMetadataManager.listRemoteLogSegments(eq(leaderTopicIdPartition), anyInt())) .thenReturn(Collections.emptyIterator()); @@ -2250,7 +2250,7 @@ public void testLogStartOffsetUpdatedOnStartup() throws RemoteStorageException, checkpoint.write(epochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); RemoteLogSegmentMetadata metadata = mock(RemoteLogSegmentMetadata.class); when(metadata.startOffset()).thenReturn(600L); @@ -2350,7 +2350,7 @@ long findLogStartOffset(TopicIdPartition topicIdPartition, UnifiedLog log) { // leader epoch preparation checkpoint.write(Collections.singletonList(epochEntry0)); LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); // create 2 log segments, with 0 and 150 as log start offset LogSegment oldSegment = mock(LogSegment.class); @@ -2455,7 +2455,7 @@ public void testDeletionOnRetentionBreachedSegments(long retentionSize, List epochEntries = Collections.singletonList(epochEntry0); checkpoint.write(epochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); when(mockLog.logEndOffset()).thenReturn(200L); @@ -2507,7 +2507,7 @@ public void testDeletionOnOverlappingRetentionBreachedSegments(long retentionSiz List epochEntries = Collections.singletonList(epochEntry0); checkpoint.write(epochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); when(mockLog.logEndOffset()).thenReturn(200L); @@ -2575,7 +2575,7 @@ public void testRemoteDeleteLagsOnRetentionBreachedSegments(long retentionSize, List epochEntries = Collections.singletonList(epochEntry0); checkpoint.write(epochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); when(mockLog.logEndOffset()).thenReturn(200L); @@ -2622,7 +2622,7 @@ public void testRemoteLogSizeRetentionShouldFilterOutCopySegmentStartState() List epochEntries = Collections.singletonList(epochEntry0); checkpoint.write(epochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); when(mockLog.logEndOffset()).thenReturn(2000L); @@ -2716,7 +2716,7 @@ public void testDeleteRetentionMsBeingCancelledBeforeSecondDelete() throws Remot checkpoint.write(epochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); Map logProps = new HashMap<>(); logProps.put("retention.bytes", -1L); @@ -2786,7 +2786,7 @@ public void testDeleteRetentionMsBiggerThanTimeMs() throws RemoteStorageExceptio checkpoint.write(epochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); assertDoesNotThrow(leaderTask::cleanupExpiredRemoteLogSegments); @@ -2806,7 +2806,7 @@ public void testFailedDeleteExpiredSegments(long retentionSize, List epochEntries = Collections.singletonList(epochEntry0); checkpoint.write(epochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); when(mockLog.logEndOffset()).thenReturn(200L); @@ -2876,7 +2876,7 @@ public void testDeleteLogSegmentDueToRetentionSizeBreach(int segmentCount, long localLogStartOffset = (long) segmentCount * recordsPerSegment; long logEndOffset = ((long) segmentCount * recordsPerSegment) + 1; - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(mockLog.localLogStartOffset()).thenReturn(localLogStartOffset); when(mockLog.logEndOffset()).thenReturn(logEndOffset); when(mockLog.onlyLocalLogSegmentsSize()).thenReturn(localLogSegmentsSize); @@ -2914,7 +2914,7 @@ public void testDeleteLogSegmentDueToRetentionTimeBreach(int segmentCount, long localLogStartOffset = (long) segmentCount * recordsPerSegment; long logEndOffset = ((long) segmentCount * recordsPerSegment) + 1; - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(mockLog.localLogStartOffset()).thenReturn(localLogStartOffset); when(mockLog.logEndOffset()).thenReturn(logEndOffset); when(mockLog.onlyLocalLogSegmentsSize()).thenReturn(localLogSegmentsSize); @@ -3001,7 +3001,7 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() { checkpoint.write(epochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); Map logProps = new HashMap<>(); logProps.put("retention.bytes", -1L); @@ -3119,7 +3119,7 @@ public void testReadForMissingFirstBatchInRemote() throws RemoteStorageException when(remoteStorageManager.fetchLogSegment(any(RemoteLogSegmentMetadata.class), anyInt())) .thenAnswer(a -> fileInputStream); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); int fetchOffset = 0; int fetchMaxBytes = 10; @@ -3189,7 +3189,7 @@ public void testReadForFirstBatchMoreThanMaxFetchBytes(boolean minOneMessage) th when(remoteStorageManager.fetchLogSegment(any(RemoteLogSegmentMetadata.class), anyInt())) .thenAnswer(a -> fileInputStream); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); int fetchOffset = 0; int fetchMaxBytes = 10; @@ -3264,7 +3264,7 @@ public void testReadForFirstBatchInLogCompaction() throws RemoteStorageException RemoteLogSegmentMetadata segmentMetadata = mock(RemoteLogSegmentMetadata.class); LeaderEpochFileCache cache = mock(LeaderEpochFileCache.class); when(cache.epochForOffset(anyLong())).thenReturn(OptionalInt.of(1)); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); int fetchOffset = 0; int fetchMaxBytes = 10; @@ -3469,7 +3469,7 @@ private RemoteLogManager.RLMCopyTask setupRLMTask(boolean quotaExceeded) throws // leader epoch preparation checkpoint.write(totalEpochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(mockLog.parentDir()).thenReturn("dir1"); when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())).thenReturn(Optional.of(0L)); @@ -3532,7 +3532,7 @@ public void testCopyThrottling() throws Exception { // leader epoch preparation checkpoint.write(totalEpochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())).thenReturn(Optional.of(0L)); // create 3 log segments @@ -3631,7 +3631,7 @@ public void testTierLagResetsToZeroOnBecomingFollower() { public void testRemoteReadFetchDataInfo() throws RemoteStorageException, IOException { checkpoint.write(totalEpochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); - when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.leaderEpochCache()).thenReturn(cache); when(remoteLogMetadataManager.remoteLogSegmentMetadata(eq(leaderTopicIdPartition), anyInt(), anyLong())) .thenAnswer(ans -> { long offset = ans.getArgument(2); diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index 1010613c7eaf9..c96eef55420e2 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -331,7 +331,7 @@ class PartitionLockTest extends Logging { val localLog = new LocalLog(log.dir, log.config, segments, offsets.recoveryPoint, offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, log.topicPartition, logDirFailureChannel) - new SlowLog(log, offsets.logStartOffset, localLog, Some(leaderEpochCache), producerStateManager, appendSemaphore) + new SlowLog(log, offsets.logStartOffset, localLog, leaderEpochCache, producerStateManager, appendSemaphore) } } @@ -443,7 +443,7 @@ class PartitionLockTest extends Logging { log: UnifiedLog, logStartOffset: Long, localLog: LocalLog, - leaderEpochCache: Option[LeaderEpochFileCache], + leaderEpochCache: LeaderEpochFileCache, producerStateManager: ProducerStateManager, appendSemaphore: Semaphore ) extends UnifiedLog( diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index ed8e22c2ea309..3dbcb952fa0c5 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -475,7 +475,7 @@ class PartitionTest extends AbstractPartitionTest { val localLog = new LocalLog(log.dir, log.config, segments, offsets.recoveryPoint, offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, log.topicPartition, logDirFailureChannel) - new SlowLog(log, offsets.logStartOffset, localLog, Some(leaderEpochCache), producerStateManager, appendSemaphore) + new SlowLog(log, offsets.logStartOffset, localLog, leaderEpochCache, producerStateManager, appendSemaphore) } } @@ -3124,7 +3124,7 @@ class PartitionTest extends AbstractPartitionTest { assertEquals(Some(0L), partition.leaderEpochStartOffsetOpt) val leaderLog = partition.localLogOrException - assertEquals(Optional.of(new EpochEntry(leaderEpoch, 0L)), leaderLog.leaderEpochCache.toJava.flatMap(_.latestEntry)) + assertEquals(Optional.of(new EpochEntry(leaderEpoch, 0L)), leaderLog.leaderEpochCache.latestEntry) // Write to the log to increment the log end offset. leaderLog.appendAsLeader(MemoryRecords.withRecords(0L, Compression.NONE, 0, @@ -3148,7 +3148,7 @@ class PartitionTest extends AbstractPartitionTest { assertEquals(leaderEpoch, partition.getLeaderEpoch) assertEquals(Set(leaderId), partition.partitionState.isr) assertEquals(Some(0L), partition.leaderEpochStartOffsetOpt) - assertEquals(Optional.of(new EpochEntry(leaderEpoch, 0L)), leaderLog.leaderEpochCache.toJava.flatMap(_.latestEntry)) + assertEquals(Optional.of(new EpochEntry(leaderEpoch, 0L)), leaderLog.leaderEpochCache.latestEntry) } @Test @@ -3628,7 +3628,7 @@ class PartitionTest extends AbstractPartitionTest { log: UnifiedLog, logStartOffset: Long, localLog: LocalLog, - leaderEpochCache: Option[LeaderEpochFileCache], + leaderEpochCache: LeaderEpochFileCache, producerStateManager: ProducerStateManager, appendSemaphore: Semaphore ) extends UnifiedLog( diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index b13820a9294a3..796536780b150 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -132,7 +132,7 @@ class LogCleanerManagerTest extends Logging { offsets.nextOffsetMetadata, time.scheduler, time, tp, logDirFailureChannel) // the exception should be caught and the partition that caused it marked as uncleanable class LogMock extends UnifiedLog(offsets.logStartOffset, localLog, new BrokerTopicStats, - producerIdExpirationCheckIntervalMs, Some(leaderEpochCache), + producerIdExpirationCheckIntervalMs, leaderEpochCache, producerStateManager, _topicId = None, keepPartitionMetadataFile = true) { // Throw an error in getFirstBatchTimestampForSegments since it is called in grabFilthiestLog() override def getFirstBatchTimestampForSegments(segments: util.Collection[LogSegment]): util.Collection[java.lang.Long] = diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 5e30e13484939..9100cc7af211a 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -214,7 +214,7 @@ class LogCleanerTest extends Logging { localLog, brokerTopicStats = new BrokerTopicStats, producerIdExpirationCheckIntervalMs = producerIdExpirationCheckIntervalMs, - leaderEpochCache = Some(leaderEpochCache), + leaderEpochCache = leaderEpochCache, producerStateManager = producerStateManager, _topicId = None, keepPartitionMetadataFile = true) { diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index bfa15e2248080..8043c53e30c7b 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -167,7 +167,7 @@ class LogLoaderTest { offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, topicPartition, logDirFailureChannel) new UnifiedLog(offsets.logStartOffset, localLog, brokerTopicStats, - this.producerIdExpirationCheckIntervalMs, Some(leaderEpochCache), + this.producerIdExpirationCheckIntervalMs, leaderEpochCache, producerStateManager, None, true) } } @@ -387,7 +387,7 @@ class LogLoaderTest { offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, topicPartition, logDirFailureChannel) new UnifiedLog(offsets.logStartOffset, localLog, brokerTopicStats, - producerIdExpirationCheckIntervalMs, Some(leaderEpochCache), producerStateManager, + producerIdExpirationCheckIntervalMs, leaderEpochCache, producerStateManager, None, keepPartitionMetadataFile = true) } @@ -455,7 +455,7 @@ class LogLoaderTest { localLog, brokerTopicStats = brokerTopicStats, producerIdExpirationCheckIntervalMs = 30000, - leaderEpochCache = Some(leaderEpochCache), + leaderEpochCache = leaderEpochCache, producerStateManager = stateManager, _topicId = None, keepPartitionMetadataFile = true) @@ -565,7 +565,7 @@ class LogLoaderTest { localLog, brokerTopicStats = brokerTopicStats, producerIdExpirationCheckIntervalMs = 30000, - leaderEpochCache = Some(leaderEpochCache), + leaderEpochCache = leaderEpochCache, producerStateManager = stateManager, _topicId = None, keepPartitionMetadataFile = true) @@ -1215,7 +1215,7 @@ class LogLoaderTest { @Test def testLogRecoversForLeaderEpoch(): Unit = { val log = createLog(logDir, new LogConfig(new Properties)) - val leaderEpochCache = log.leaderEpochCache.get + val leaderEpochCache = log.leaderEpochCache val firstBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 1, offset = 0) log.appendAsFollower(records = firstBatch) @@ -1237,7 +1237,7 @@ class LogLoaderTest { // reopen the log and recover from the beginning val recoveredLog = createLog(logDir, new LogConfig(new Properties), lastShutdownClean = false) - val recoveredLeaderEpochCache = recoveredLog.leaderEpochCache.get + val recoveredLeaderEpochCache = recoveredLog.leaderEpochCache // epoch entries should be recovered assertEquals(java.util.Arrays.asList(new EpochEntry(1, 0), new EpochEntry(2, 1), new EpochEntry(3, 3)), recoveredLeaderEpochCache.epochEntries) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 3bd9eb79c2e2f..523313c4ae99d 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -57,11 +57,10 @@ import java.io._ import java.nio.ByteBuffer import java.nio.file.Files import java.util.concurrent.{Callable, ConcurrentHashMap, Executors, TimeUnit} -import java.util.{Optional, OptionalLong, Properties} +import java.util.{Optional, OptionalInt, OptionalLong, Properties} import scala.collection.immutable.SortedSet import scala.collection.mutable.ListBuffer import scala.jdk.CollectionConverters._ -import scala.jdk.OptionConverters.{RichOptional, RichOptionalInt} class UnifiedLogTest { var config: KafkaConfig = _ @@ -655,23 +654,20 @@ class UnifiedLogTest { val records = TestUtils.records(List(new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)), baseOffset = 27) appendAsFollower(log, records, leaderEpoch = 19) - assertEquals(Some(new EpochEntry(19, 27)), - log.leaderEpochCache.flatMap(_.latestEntry.toScala)) + assertEquals(Optional.of(new EpochEntry(19, 27)), log.leaderEpochCache.latestEntry) assertEquals(29, log.logEndOffset) def verifyTruncationClearsEpochCache(epoch: Int, truncationOffset: Long): Unit = { // Simulate becoming a leader - log.maybeAssignEpochStartOffset(leaderEpoch = epoch, startOffset = log.logEndOffset) - assertEquals(Some(new EpochEntry(epoch, 29)), - log.leaderEpochCache.flatMap(_.latestEntry.toScala)) + log.assignEpochStartOffset(leaderEpoch = epoch, startOffset = log.logEndOffset) + assertEquals(Optional.of(new EpochEntry(epoch, 29)), log.leaderEpochCache.latestEntry) assertEquals(29, log.logEndOffset) // Now we become the follower and truncate to an offset greater // than or equal to the log end offset. The trivial epoch entry // at the end of the log should be gone log.truncateTo(truncationOffset) - assertEquals(Some(new EpochEntry(19, 27)), - log.leaderEpochCache.flatMap(_.latestEntry.toScala)) + assertEquals(Optional.of(new EpochEntry(19, 27)), log.leaderEpochCache.latestEntry) assertEquals(29, log.logEndOffset) } @@ -2060,7 +2056,7 @@ class UnifiedLogTest { // The cache can be updated directly after a leader change. // The new latest offset should reflect the updated epoch. - log.maybeAssignEpochStartOffset(2, 2L) + log.assignEpochStartOffset(2, 2L) assertEquals(new OffsetResultHolder(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 2L, Optional.of(2))), log.fetchOffsetByTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP)) @@ -2136,7 +2132,7 @@ class UnifiedLogTest { .filter(_ == firstTimestamp) .map[TimestampAndOffset](x => new TimestampAndOffset(x, 0L, Optional.of(firstLeaderEpoch))) }).when(remoteLogManager).findOffsetByTimestamp(ArgumentMatchers.eq(log.topicPartition), - anyLong(), anyLong(), ArgumentMatchers.eq(log.leaderEpochCache.get)) + anyLong(), anyLong(), ArgumentMatchers.eq(log.leaderEpochCache)) log._localLogStartOffset = 1 def assertFetchOffsetByTimestamp(expected: Option[TimestampAndOffset], timestamp: Long): Unit = { @@ -2161,7 +2157,7 @@ class UnifiedLogTest { // The cache can be updated directly after a leader change. // The new latest offset should reflect the updated epoch. - log.maybeAssignEpochStartOffset(2, 2L) + log.assignEpochStartOffset(2, 2L) assertEquals(new OffsetResultHolder(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 2L, Optional.of(2))), log.fetchOffsetByTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, Some(remoteLogManager))) @@ -2235,7 +2231,7 @@ class UnifiedLogTest { .filter(_ == firstTimestamp) .map[TimestampAndOffset](x => new TimestampAndOffset(x, 0L, Optional.of(firstLeaderEpoch))) }).when(remoteLogManager).findOffsetByTimestamp(ArgumentMatchers.eq(log.topicPartition), - anyLong(), anyLong(), ArgumentMatchers.eq(log.leaderEpochCache.get)) + anyLong(), anyLong(), ArgumentMatchers.eq(log.leaderEpochCache)) log._localLogStartOffset = 1 log._highestOffsetInRemoteStorage = 0 @@ -2263,7 +2259,7 @@ class UnifiedLogTest { // The cache can be updated directly after a leader change. // The new latest offset should reflect the updated epoch. - log.maybeAssignEpochStartOffset(2, 2L) + log.assignEpochStartOffset(2, 2L) assertEquals(new OffsetResultHolder(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 2L, Optional.of(2))), log.fetchOffsetByTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, Some(remoteLogManager))) @@ -2578,12 +2574,12 @@ class UnifiedLogTest { val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.records(List(new SimpleRecord("foo".getBytes()))), leaderEpoch = 5) - assertEquals(Some(5), log.leaderEpochCache.flatMap(_.latestEpoch.toScala)) + assertEquals(OptionalInt.of(5), log.leaderEpochCache.latestEpoch) log.appendAsFollower(TestUtils.records(List(new SimpleRecord("foo".getBytes())), baseOffset = 1L, magicValue = RecordVersion.V1.value)) - assertEquals(None, log.leaderEpochCache.flatMap(_.latestEpoch.toScala)) + assertEquals(OptionalInt.empty, log.leaderEpochCache.latestEpoch) } @Test @@ -2688,8 +2684,8 @@ class UnifiedLogTest { for (_ <- 0 until 100) log.appendAsLeader(createRecords, leaderEpoch = 0) - log.maybeAssignEpochStartOffset(0, 40) - log.maybeAssignEpochStartOffset(1, 90) + log.assignEpochStartOffset(0, 40) + log.assignEpochStartOffset(1, 90) // segments are not eligible for deletion if no high watermark has been set val numSegments = log.numberOfSegments @@ -2774,9 +2770,7 @@ class UnifiedLogTest { assertEquals(log.logStartOffset, 15) } - def epochCache(log: UnifiedLog): LeaderEpochFileCache = { - log.leaderEpochCache.get - } + def epochCache(log: UnifiedLog): LeaderEpochFileCache = log.leaderEpochCache @Test def shouldDeleteSizeBasedSegments(): Unit = { @@ -2905,7 +2899,7 @@ class UnifiedLogTest { //Given this partition is on leader epoch 72 val epoch = 72 val log = createLog(logDir, new LogConfig(new Properties)) - log.maybeAssignEpochStartOffset(epoch, records.length) + log.assignEpochStartOffset(epoch, records.length) //When appending messages as a leader (i.e. assignOffsets = true) for (record <- records) @@ -3679,7 +3673,7 @@ class UnifiedLogTest { assertTrue(newDir.exists()) log.renameDir(newDir.getName, false) - assertTrue(log.leaderEpochCache.isEmpty) + assertNull(log.leaderEpochCache) assertTrue(log.partitionMetadataFile.isEmpty) assertEquals(0, log.logEndOffset) // verify that records appending can still succeed diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index d036724909e07..425ddd3affd72 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -67,6 +67,7 @@ import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams, FetchPa import org.apache.kafka.server.util.timer.MockTimer import org.apache.kafka.server.util.{MockScheduler, MockTime} import org.apache.kafka.storage.internals.checkpoint.{LazyOffsetCheckpoints, OffsetCheckpointFile, PartitionMetadataFile} +import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchDataInfo, LocalLog, LogConfig, LogDirFailureChannel, LogLoader, LogOffsetMetadata, LogOffsetSnapshot, LogSegments, LogStartOffsetIncrementReason, ProducerStateManager, ProducerStateManagerConfig, RemoteStorageFetchInfo, VerificationGuard} import org.apache.kafka.storage.log.metrics.BrokerTopicStats import org.junit.jupiter.api.Assertions._ @@ -264,68 +265,6 @@ class ReplicaManagerTest { when(cache.getAliveBrokerNodes(any[ListenerName])).thenReturn(aliveBrokers) } - @Test - def testMaybeAddLogDirFetchersWithoutEpochCache(): Unit = { - val dir1 = TestUtils.tempDir() - val dir2 = TestUtils.tempDir() - val props = TestUtils.createBrokerConfig(0) - props.put("log.dirs", dir1.getAbsolutePath + "," + dir2.getAbsolutePath) - val config = KafkaConfig.fromProps(props) - val logManager = TestUtils.createLogManager(config.logDirs.map(new File(_)), new LogConfig(new Properties())) - val metadataCache: MetadataCache = mock(classOf[MetadataCache]) - mockGetAliveBrokerFunctions(metadataCache, Seq(new Node(0, "host0", 0))) - when(metadataCache.metadataVersion()).thenReturn(config.interBrokerProtocolVersion) - val rm = new ReplicaManager( - metrics = metrics, - config = config, - time = time, - scheduler = new MockScheduler(time), - logManager = logManager, - quotaManagers = quotaManager, - metadataCache = metadataCache, - logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), - alterPartitionManager = alterPartitionManager) - - try { - val partition = rm.createPartition(new TopicPartition(topic, 0)) - partition.createLogIfNotExists(isNew = false, isFutureReplica = false, - new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints.asJava), None) - - rm.becomeLeaderOrFollower(0, new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, - Seq(new LeaderAndIsrPartitionState() - .setTopicName(topic) - .setPartitionIndex(0) - .setControllerEpoch(0) - .setLeader(0) - .setLeaderEpoch(0) - .setIsr(Seq[Integer](0).asJava) - .setPartitionEpoch(0) - .setReplicas(Seq[Integer](0).asJava) - .setIsNew(false)).asJava, - Collections.singletonMap(topic, Uuid.randomUuid()), - Set(new Node(0, "host1", 0)).asJava).build(), (_, _) => ()) - appendRecords(rm, new TopicPartition(topic, 0), - MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("first message".getBytes()), new SimpleRecord("second message".getBytes()))) - logManager.maybeUpdatePreferredLogDir(new TopicPartition(topic, 0), dir2.getAbsolutePath) - - partition.createLogIfNotExists(isNew = true, isFutureReplica = true, - new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints.asJava), None) - // remove cache to disable OffsetsForLeaderEpoch API - partition.futureLog.get.leaderEpochCache = None - - // this method should use hw of future log to create log dir fetcher. Otherwise, it causes offset mismatch error - rm.maybeAddLogDirFetchers(Set(partition), new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints.asJava), _ => None) - rm.replicaAlterLogDirsManager.fetcherThreadMap.values.foreach(t => t.fetchState(new TopicPartition(topic, 0)).foreach(s => assertEquals(0L, s.fetchOffset))) - // make sure alter log dir thread has processed the data - rm.replicaAlterLogDirsManager.fetcherThreadMap.values.foreach(t => t.doWork()) - assertEquals(Set.empty, rm.replicaAlterLogDirsManager.failedPartitions.partitions()) - // the future log becomes the current log, so the partition state should get removed - rm.replicaAlterLogDirsManager.fetcherThreadMap.values.foreach(t => assertEquals(None, t.fetchState(new TopicPartition(topic, 0)))) - } finally { - rm.shutdown(checkpointHW = false) - } - } - @ParameterizedTest(name = "testMaybeAddLogDirFetchersPausingCleaning with futureLogCreated: {0}") @ValueSource(booleans = Array(true, false)) def testMaybeAddLogDirFetchersPausingCleaning(futureLogCreated: Boolean): Unit = { @@ -2928,7 +2867,7 @@ class ReplicaManagerTest { localLog = localLog, brokerTopicStats = mockBrokerTopicStats, producerIdExpirationCheckIntervalMs = 30000, - leaderEpochCache = Some(leaderEpochCache), + leaderEpochCache = leaderEpochCache, producerStateManager = producerStateManager, _topicId = topicId, keepPartitionMetadataFile = true) { @@ -4517,7 +4456,7 @@ class ReplicaManagerTest { when(mockLog.logStartOffset).thenReturn(endOffset).thenReturn(startOffset) when(mockLog.logEndOffset).thenReturn(endOffset) when(mockLog.localLogStartOffset()).thenReturn(endOffset - 10) - when(mockLog.leaderEpochCache).thenReturn(None) + when(mockLog.leaderEpochCache).thenReturn(mock(classOf[LeaderEpochFileCache])) when(mockLog.latestEpoch).thenReturn(Some(0)) val producerStateManager = mock(classOf[ProducerStateManager]) when(mockLog.producerStateManager).thenReturn(producerStateManager) diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala index e36a32186b125..7afa2178f7300 100644 --- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala +++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala @@ -163,7 +163,7 @@ class SchedulerTest { val log = new UnifiedLog(logStartOffset = offsets.logStartOffset, localLog = localLog, brokerTopicStats, producerIdExpirationCheckIntervalMs, - Some(leaderEpochCache), producerStateManager, + leaderEpochCache, producerStateManager, _topicId = None, keepPartitionMetadataFile = true) assertTrue(scheduler.taskRunning(log.producerExpireCheck)) log.close() diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestContext.java b/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestContext.java index ff61e29d93c28..2fdb9483fe628 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestContext.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestContext.java @@ -302,11 +302,7 @@ public LocalTieredStorage remoteStorageManager(int brokerId) { // unused now, but it can be reused later as this is an utility method. public Optional leaderEpochFileCache(int brokerId, TopicPartition partition) { - Optional unifiedLogOpt = log(brokerId, partition); - if (unifiedLogOpt.isPresent() && unifiedLogOpt.get().leaderEpochCache().isDefined()) { - return Optional.of(unifiedLogOpt.get().leaderEpochCache().get()); - } - return Optional.empty(); + return log(brokerId, partition).map(log -> log.leaderEpochCache()); } public List remoteStorageManagers() { diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectLeaderEpochCheckpointAction.java b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectLeaderEpochCheckpointAction.java index 10231ad06ff04..da6839799834a 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectLeaderEpochCheckpointAction.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectLeaderEpochCheckpointAction.java @@ -30,8 +30,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicReference; -import scala.Option; - public final class ExpectLeaderEpochCheckpointAction implements TieredStorageTestAction { private final Integer brokerId; @@ -56,10 +54,8 @@ public void doExecute(TieredStorageTestContext context) throws InterruptedExcept EpochEntry earliestEntry = null; Optional log = context.log(brokerId, partition); if (log.isPresent()) { - Option leaderEpochCache = log.get().leaderEpochCache(); - if (leaderEpochCache.isDefined()) { - earliestEntry = leaderEpochCache.get().earliestEntry().orElse(null); - } + LeaderEpochFileCache leaderEpochCache = log.get().leaderEpochCache(); + earliestEntry = leaderEpochCache.earliestEntry().orElse(null); } earliestEntryOpt.set(earliestEntry); return earliestEntry != null && beginEpoch == earliestEntry.epoch From b6491793f00b500f4663a60aca28db7d3b49453a Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 8 Jan 2025 06:51:56 -0800 Subject: [PATCH 12/18] Fix issue where method was not overriden due to change in signature Also added @Override annotations to catch it at compile time in the future. --- .../test/java/kafka/log/remote/RemoteLogManagerTest.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index 5f21a758c67f5..4e8a320635221 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -3149,21 +3149,25 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() { return remoteLogMetadataManager; } + @Override public Optional fetchRemoteLogSegmentMetadata(TopicPartition topicPartition, int epochForOffset, long offset) { return Optional.of(segmentMetadata); } + @Override public Optional findNextSegmentMetadata(RemoteLogSegmentMetadata segmentMetadata, - Option leaderEpochFileCacheOption) { + LeaderEpochFileCache leaderEpochFileCacheOption) { return Optional.empty(); } + @Override int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) { return 1; } // This is the key scenario that we are testing here + @Override EnrichedRecordBatch findFirstBatch(RemoteLogInputStream remoteLogInputStream, long offset) { return new EnrichedRecordBatch(null, 0); } From 7628099dfe78dc350005fcf26a959b29174d8329 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 8 Jan 2025 06:59:54 -0800 Subject: [PATCH 13/18] Fix NPE in `UnifiedLog.delete` --- core/src/main/scala/kafka/log/UnifiedLog.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index 2c7441ca39d1f..d28ac53c0a5ba 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -1759,7 +1759,9 @@ class UnifiedLog(@volatile var logStartOffset: Long, lock synchronized { localLog.checkIfMemoryMappedBufferClosed() producerExpireCheck.cancel(true) - leaderEpochCache.clear() + // `renameDir` with `shouldReinitialize=false` sets this to `null` and it's usually (but not always) called before this method + if (leaderEpochCache != null) + leaderEpochCache.clear() val deletedSegments = localLog.deleteAllSegments() deleteProducerSnapshots(deletedSegments, asyncDelete = false) localLog.deleteEmptyDir() From 56a93df80dd6f45c3c2ef345c2b68dc29a8aea19 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 8 Jan 2025 07:17:54 -0800 Subject: [PATCH 14/18] Adjust assert to address review comment --- .../unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala index 9d5bf3debe69b..df461855a9fa9 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala @@ -222,7 +222,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati cleaner.startup() val firstDirty = log.activeSegment.baseOffset - assertTrue(firstDirty > appendsV0.size) // ensure we clean data from V0 and V1 + assertTrue(firstDirty >= appends.size) // ensure we clean data from V0 and V1 checkLastCleaned("log", 0, firstDirty) val compactedSize = log.logSegments.asScala.map(_.size).sum From fb0f06f50c2cdaeadad41ada227c9960152426a1 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 8 Jan 2025 11:42:52 -0800 Subject: [PATCH 15/18] leaderEpochCache is never null --- .../src/main/java/kafka/log/remote/RemoteLogManager.java | 9 ++------- core/src/main/scala/kafka/log/UnifiedLog.scala | 6 ++---- 2 files changed, 4 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index 60fd5ccb61e6b..b5f9e408c9442 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -1672,9 +1672,7 @@ public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws if (logOptional.isPresent()) { LeaderEpochFileCache leaderEpochCache = logOptional.get().leaderEpochCache(); - if (leaderEpochCache != null) { - epoch = leaderEpochCache.epochForOffset(offset); - } + epoch = leaderEpochCache.epochForOffset(offset); } Optional rlsMetadataOptional = epoch.isPresent() @@ -1874,7 +1872,7 @@ Optional findNextSegmentMetadata(RemoteLogSegmentMetad * Visible for testing * @param tp The topic partition. * @param offset The offset to start the search. - * @param leaderEpochCache The leader epoch file cache, this could be null. + * @param leaderEpochCache The leader epoch file cache. * @return The next segment metadata that contains the transaction index. The transaction index may or may not exist * in that segment metadata which depends on the RLMM plugin implementation. The caller of this method should handle * for both the cases. @@ -1883,9 +1881,6 @@ Optional findNextSegmentMetadata(RemoteLogSegmentMetad Optional findNextSegmentWithTxnIndex(TopicPartition tp, long offset, LeaderEpochFileCache leaderEpochCache) throws RemoteStorageException { - if (leaderEpochCache == null) { - return Optional.empty(); - } OptionalInt initialEpochOpt = leaderEpochCache.epochForOffset(offset); if (initialEpochOpt.isEmpty()) { return Optional.empty(); diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index d28ac53c0a5ba..9acedabd7a9b2 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -675,7 +675,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, initializeLeaderEpochCache() initializePartitionMetadata() } else { - leaderEpochCache = null + leaderEpochCache.clear() partitionMetadataFile = None } } @@ -1759,9 +1759,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, lock synchronized { localLog.checkIfMemoryMappedBufferClosed() producerExpireCheck.cancel(true) - // `renameDir` with `shouldReinitialize=false` sets this to `null` and it's usually (but not always) called before this method - if (leaderEpochCache != null) - leaderEpochCache.clear() + leaderEpochCache.clear() val deletedSegments = localLog.deleteAllSegments() deleteProducerSnapshots(deletedSegments, asyncDelete = false) localLog.deleteEmptyDir() From ef3ec426d9cf2f6475f73146a3f43a795ea91f8a Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 8 Jan 2025 11:52:16 -0800 Subject: [PATCH 16/18] Add back test with adjustments (including name) --- .../kafka/server/ReplicaManagerTest.scala | 60 +++++++++++++++++++ 1 file changed, 60 insertions(+) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 425ddd3affd72..a3081f17ed343 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -265,6 +265,66 @@ class ReplicaManagerTest { when(cache.getAliveBrokerNodes(any[ListenerName])).thenReturn(aliveBrokers) } + @Test + def testMaybeAddLogDirFetchers(): Unit = { + val dir1 = TestUtils.tempDir() + val dir2 = TestUtils.tempDir() + val props = TestUtils.createBrokerConfig(0) + props.put("log.dirs", dir1.getAbsolutePath + "," + dir2.getAbsolutePath) + val config = KafkaConfig.fromProps(props) + val logManager = TestUtils.createLogManager(config.logDirs.map(new File(_)), new LogConfig(new Properties())) + val metadataCache: MetadataCache = mock(classOf[MetadataCache]) + mockGetAliveBrokerFunctions(metadataCache, Seq(new Node(0, "host0", 0))) + when(metadataCache.metadataVersion()).thenReturn(config.interBrokerProtocolVersion) + val rm = new ReplicaManager( + metrics = metrics, + config = config, + time = time, + scheduler = new MockScheduler(time), + logManager = logManager, + quotaManagers = quotaManager, + metadataCache = metadataCache, + logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), + alterPartitionManager = alterPartitionManager) + + try { + val partition = rm.createPartition(new TopicPartition(topic, 0)) + partition.createLogIfNotExists(isNew = false, isFutureReplica = false, + new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints.asJava), None) + + rm.becomeLeaderOrFollower(0, new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, + Seq(new LeaderAndIsrPartitionState() + .setTopicName(topic) + .setPartitionIndex(0) + .setControllerEpoch(0) + .setLeader(0) + .setLeaderEpoch(0) + .setIsr(Seq[Integer](0).asJava) + .setPartitionEpoch(0) + .setReplicas(Seq[Integer](0).asJava) + .setIsNew(false)).asJava, + Collections.singletonMap(topic, Uuid.randomUuid()), + Set(new Node(0, "host1", 0)).asJava).build(), (_, _) => ()) + appendRecords(rm, new TopicPartition(topic, 0), + MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("first message".getBytes()), new SimpleRecord("second message".getBytes()))) + logManager.maybeUpdatePreferredLogDir(new TopicPartition(topic, 0), dir2.getAbsolutePath) + + partition.createLogIfNotExists(isNew = true, isFutureReplica = true, + new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints.asJava), None) + + // this method should use hw of future log to create log dir fetcher. Otherwise, it causes offset mismatch error + rm.maybeAddLogDirFetchers(Set(partition), new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints.asJava), _ => None) + rm.replicaAlterLogDirsManager.fetcherThreadMap.values.foreach(t => t.fetchState(new TopicPartition(topic, 0)).foreach(s => assertEquals(0L, s.fetchOffset))) + // make sure alter log dir thread has processed the data + rm.replicaAlterLogDirsManager.fetcherThreadMap.values.foreach(t => t.doWork()) + assertEquals(Set.empty, rm.replicaAlterLogDirsManager.failedPartitions.partitions()) + // the future log becomes the current log, so the partition state should get removed + rm.replicaAlterLogDirsManager.fetcherThreadMap.values.foreach(t => assertEquals(None, t.fetchState(new TopicPartition(topic, 0)))) + } finally { + rm.shutdown(checkpointHW = false) + } + } + @ParameterizedTest(name = "testMaybeAddLogDirFetchersPausingCleaning with futureLogCreated: {0}") @ValueSource(booleans = Array(true, false)) def testMaybeAddLogDirFetchersPausingCleaning(futureLogCreated: Boolean): Unit = { From 525d8dfe843d5453cf39a1d076cf2c8931ce9e21 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 8 Jan 2025 15:58:01 -0800 Subject: [PATCH 17/18] Fix test --- core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 523313c4ae99d..feb2a9770ec6c 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -3673,14 +3673,9 @@ class UnifiedLogTest { assertTrue(newDir.exists()) log.renameDir(newDir.getName, false) - assertNull(log.leaderEpochCache) + assertFalse(log.leaderEpochCache.nonEmpty) assertTrue(log.partitionMetadataFile.isEmpty) assertEquals(0, log.logEndOffset) - // verify that records appending can still succeed - // even with the uninitialized leaderEpochCache and partitionMetadataFile - val records = TestUtils.records(List(new SimpleRecord(mockTime.milliseconds, "key".getBytes, "value".getBytes))) - log.appendAsLeader(records, leaderEpoch = 0) - assertEquals(1, log.logEndOffset) // verify that the background deletion can succeed log.delete() From 97448ce6d647d01ed1b52a6481021d13476d5820 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 8 Jan 2025 15:59:05 -0800 Subject: [PATCH 18/18] Rename `initializeLeaderEpochCache` to `reinitializeLeaderEpochCache` --- core/src/main/scala/kafka/log/UnifiedLog.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index 9acedabd7a9b2..b3d6588de066f 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -508,7 +508,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, } } - private def initializeLeaderEpochCache(): Unit = lock synchronized { + private def reinitializeLeaderEpochCache(): Unit = lock synchronized { leaderEpochCache = UnifiedLog.createLeaderEpochCache( dir, topicPartition, logDirFailureChannel, Option.apply(leaderEpochCache), scheduler) } @@ -672,7 +672,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, if (shouldReinitialize) { // re-initialize leader epoch cache so that LeaderEpochCheckpointFile.checkpoint can correctly reference // the checkpoint file in renamed log directory - initializeLeaderEpochCache() + reinitializeLeaderEpochCache() initializePartitionMetadata() } else { leaderEpochCache.clear()