From 4ae681446f18f9a6d7a59f317013ef82379a4ba4 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Sun, 16 Mar 2025 02:51:49 +0800 Subject: [PATCH 01/72] KAFKA-18993 Remove confusing notable change section from upgrade.html (#19212) Currently, the "Notable changes in 4.0.0" for the client is very confusing. We should remove it. Reviewers: mingdaoy , Luke Chen , Ken Huang , David Jacot , Chia-Ping Tsai --- docs/upgrade.html | 6 ------ 1 file changed, 6 deletions(-) diff --git a/docs/upgrade.html b/docs/upgrade.html index eba558f39c..4e24c14932 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -32,12 +32,6 @@
Upgrading Client or KIP-1124. -
Notable changes in 4.0.0
- -
    -
  • Please see notable changes in the server section.
  • -
-
Upgrading Servers to 4.0.0 from any version 3.3.x through 3.9.x

Note: Apache Kafka 4.0 only supports KRaft mode - ZooKeeper mode has been removed. As such, broker upgrades to 4.0.0 (and higher) require KRaft mode and From 16da5a885e6d2a1fa12a0013b6c60bbe6cb2706a Mon Sep 17 00:00:00 2001 From: David Jacot Date: Tue, 18 Mar 2025 11:56:02 +0100 Subject: [PATCH 02/72] MINOR: Bump to 4.0.1-SNAPSHOT (#19224) Reviewers: Chia-Ping Tsai --- committer-tools/kafka-merge-pr.py | 2 +- docs/js/templateData.js | 2 +- gradle.properties | 2 +- streams/quickstart/java/pom.xml | 2 +- .../java/src/main/resources/archetype-resources/pom.xml | 2 +- streams/quickstart/pom.xml | 2 +- tests/kafkatest/__init__.py | 2 +- tests/kafkatest/version.py | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/committer-tools/kafka-merge-pr.py b/committer-tools/kafka-merge-pr.py index 63439f3e0e..175dbd7601 100755 --- a/committer-tools/kafka-merge-pr.py +++ b/committer-tools/kafka-merge-pr.py @@ -70,7 +70,7 @@ DEV_BRANCH_NAME = "trunk" -DEFAULT_FIX_VERSION = os.environ.get("DEFAULT_FIX_VERSION", "4.0.0") +DEFAULT_FIX_VERSION = os.environ.get("DEFAULT_FIX_VERSION", "4.0.1") ORIGINAL_HEAD = "" diff --git a/docs/js/templateData.js b/docs/js/templateData.js index 34c8a96b7c..fedf50f2ac 100644 --- a/docs/js/templateData.js +++ b/docs/js/templateData.js @@ -19,6 +19,6 @@ limitations under the License. var context={ "version": "40", "dotVersion": "4.0", - "fullDotVersion": "4.0.0", + "fullDotVersion": "4.0.1", "scalaVersion": "2.13" }; diff --git a/gradle.properties b/gradle.properties index e47e6eac01..46bfc41b7e 100644 --- a/gradle.properties +++ b/gradle.properties @@ -23,7 +23,7 @@ group=org.apache.kafka # - streams/quickstart/pom.xml # - streams/quickstart/java/src/main/resources/archetype-resources/pom.xml # - streams/quickstart/java/pom.xml -version=4.0.0-SNAPSHOT +version=4.1.0-SNAPSHOT scalaVersion=2.13.15 # Adding swaggerVersion in gradle.properties to have a single version in place for swagger swaggerVersion=2.2.25 diff --git a/streams/quickstart/java/pom.xml b/streams/quickstart/java/pom.xml index e58187bb6e..ffbf57e2fd 100644 --- a/streams/quickstart/java/pom.xml +++ b/streams/quickstart/java/pom.xml @@ -26,7 +26,7 @@ org.apache.kafka streams-quickstart - 4.0.0-SNAPSHOT + 4.0.1-SNAPSHOT .. diff --git a/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml b/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml index 8b84c17024..14bf05d7f9 100644 --- a/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml +++ b/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml @@ -29,7 +29,7 @@ UTF-8 - 4.0.0-SNAPSHOT + 4.0.1-SNAPSHOT 2.0.16 diff --git a/streams/quickstart/pom.xml b/streams/quickstart/pom.xml index aa7f1884d1..7f267a1828 100644 --- a/streams/quickstart/pom.xml +++ b/streams/quickstart/pom.xml @@ -22,7 +22,7 @@ org.apache.kafka streams-quickstart pom - 4.0.0-SNAPSHOT + 4.0.1-SNAPSHOT Kafka Streams :: Quickstart diff --git a/tests/kafkatest/__init__.py b/tests/kafkatest/__init__.py index cf5a5d17f5..2f93446fdc 100644 --- a/tests/kafkatest/__init__.py +++ b/tests/kafkatest/__init__.py @@ -22,4 +22,4 @@ # Instead, in development branches, the version should have a suffix of the form ".devN" # # For example, when Kafka is at version 1.0.0-SNAPSHOT, this should be something like "1.0.0.dev0" -__version__ = '4.0.0.dev0' +__version__ = '4.0.1.dev0' diff --git a/tests/kafkatest/version.py b/tests/kafkatest/version.py index 38d6cb443c..d690da556e 100644 --- a/tests/kafkatest/version.py +++ b/tests/kafkatest/version.py @@ -110,7 +110,7 @@ def get_version(node=None): return DEV_BRANCH DEV_BRANCH = KafkaVersion("dev") -DEV_VERSION = KafkaVersion("4.0.0-SNAPSHOT") +DEV_VERSION = KafkaVersion("4.0.1-SNAPSHOT") LATEST_STABLE_TRANSACTION_VERSION = 2 # This should match the LATEST_PRODUCTION version defined in MetadataVersion.java From 4dd893ba212d05fb2201d884d962f0c0f81ebb10 Mon Sep 17 00:00:00 2001 From: PoAn Yang Date: Thu, 20 Mar 2025 16:38:54 +0800 Subject: [PATCH 03/72] KAFKA-806 Index may not always observe log.index.interval.bytes (#18842) Currently, each log.append() will add at most 1 index entry, even when the appended data is larger than log.index.interval.bytes. One potential issue is that if a follower restarts after being down for a long time, it may fetch data much bigger than log.index.interval.bytes at a time. This means that fewer index entries are created, which can increase the fetch time from the consumers. (cherry picked from commit e124d3975bdb3a9ec85eee2fba7a1b0a6967d3a6) Reviewers: Chia-Ping Tsai --- .../src/main/scala/kafka/log/LogCleaner.scala | 2 +- .../src/main/scala/kafka/log/UnifiedLog.scala | 5 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 4 +- .../unit/kafka/server/MockFetcherThread.scala | 1 - .../server/ReplicaFetcherThreadTest.scala | 1 - .../kafka/tools/DumpLogSegmentsTest.scala | 4 +- .../kafka/storage/internals/log/LocalLog.java | 4 +- .../storage/internals/log/LogAppendInfo.java | 75 ++++----- .../storage/internals/log/LogSegment.java | 46 +++--- .../storage/internals/log/LogValidator.java | 35 +--- .../storage/internals/log/LocalLogTest.java | 2 - .../storage/internals/log/LogSegmentTest.java | 155 +++++++++++++----- .../internals/log/LogValidatorTest.java | 24 --- 13 files changed, 181 insertions(+), 177 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index a4f96ff7e6..5d7ee51896 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -824,7 +824,7 @@ private[log] class Cleaner(val id: Int, val retained = MemoryRecords.readableRecords(outputBuffer) // it's OK not to hold the Log's lock in this case, because this segment is only accessed by other threads // after `Log.replaceSegments` (which acquires the lock) is called - dest.append(result.maxOffset, result.maxTimestamp, result.shallowOffsetOfMaxTimestamp(), retained) + dest.append(result.maxOffset, retained) throttler.maybeThrottle(outputBuffer.limit()) } diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index 9a977a262b..3c1a157087 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -816,7 +816,6 @@ class UnifiedLog(@volatile var logStartOffset: Long, validRecords = validateAndOffsetAssignResult.validatedRecords appendInfo.setMaxTimestamp(validateAndOffsetAssignResult.maxTimestampMs) - appendInfo.setShallowOffsetOfMaxTimestamp(validateAndOffsetAssignResult.shallowOffsetOfMaxTimestamp) appendInfo.setLastOffset(offset.value - 1) appendInfo.setRecordValidationStats(validateAndOffsetAssignResult.recordValidationStats) if (config.messageTimestampType == TimestampType.LOG_APPEND_TIME) @@ -902,7 +901,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, // will be cleaned up after the log directory is recovered. Note that the end offset of the // ProducerStateManager will not be updated and the last stable offset will not advance // if the append to the transaction index fails. - localLog.append(appendInfo.lastOffset, appendInfo.maxTimestamp, appendInfo.shallowOffsetOfMaxTimestamp, validRecords) + localLog.append(appendInfo.lastOffset, validRecords) updateHighWatermarkWithLogEndOffset() // update the producer state @@ -1183,7 +1182,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, else OptionalInt.empty() - new LogAppendInfo(firstOffset, lastOffset, lastLeaderEpochOpt, maxTimestamp, shallowOffsetOfMaxTimestamp, + new LogAppendInfo(firstOffset, lastOffset, lastLeaderEpochOpt, maxTimestamp, RecordBatch.NO_TIMESTAMP, logStartOffset, RecordValidationStats.EMPTY, sourceCompression, validBytesCount, lastOffsetOfFirstBatch, Collections.emptyList[RecordError], LeaderHwChange.NONE) } diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index bbcda01451..9d208055a5 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -4455,8 +4455,8 @@ class UnifiedLogTest { segments.add(seg2) assertEquals(Seq(Long.MaxValue, Long.MaxValue), log.getFirstBatchTimestampForSegments(segments).asScala.toSeq) - seg1.append(1, 1000L, 1, MemoryRecords.withRecords(1, Compression.NONE, new SimpleRecord("one".getBytes))) - seg2.append(2, 2000L, 1, MemoryRecords.withRecords(2, Compression.NONE, new SimpleRecord("two".getBytes))) + seg1.append(1, MemoryRecords.withRecords(1, Compression.NONE, new SimpleRecord(1000L, "one".getBytes))) + seg2.append(2, MemoryRecords.withRecords(2, Compression.NONE, new SimpleRecord(2000L, "two".getBytes))) assertEquals(Seq(1000L, 2000L), log.getFirstBatchTimestampForSegments(segments).asScala.toSeq) seg1.close() diff --git a/core/src/test/scala/unit/kafka/server/MockFetcherThread.scala b/core/src/test/scala/unit/kafka/server/MockFetcherThread.scala index 2754685b8f..5d50de0409 100644 --- a/core/src/test/scala/unit/kafka/server/MockFetcherThread.scala +++ b/core/src/test/scala/unit/kafka/server/MockFetcherThread.scala @@ -107,7 +107,6 @@ class MockFetcherThread(val mockLeader: MockLeaderEndPoint, lastOffset, lastEpoch, maxTimestamp, - shallowOffsetOfMaxTimestamp, Time.SYSTEM.milliseconds(), state.logStartOffset, RecordValidationStats.EMPTY, diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala index ff556f586c..6526d6628c 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala @@ -465,7 +465,6 @@ class ReplicaFetcherThreadTest { 0, OptionalInt.empty, RecordBatch.NO_TIMESTAMP, - -1L, RecordBatch.NO_TIMESTAMP, -1L, RecordValidationStats.EMPTY, diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index b86a5608c3..bf8cafac62 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -36,7 +36,7 @@ import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.metadata.{PartitionChangeRecord, RegisterBrokerRecord, TopicRecord} import org.apache.kafka.common.protocol.{ByteBufferAccessor, ObjectSerializationCache} -import org.apache.kafka.common.record.{ControlRecordType, EndTransactionMarker, MemoryRecords, Record, RecordBatch, RecordVersion, SimpleRecord} +import org.apache.kafka.common.record.{ControlRecordType, EndTransactionMarker, MemoryRecords, Record, RecordVersion, SimpleRecord} import org.apache.kafka.common.utils.{Exit, Utils} import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord import org.apache.kafka.coordinator.group.GroupCoordinatorRecordSerde @@ -402,7 +402,7 @@ class DumpLogSegmentsTest { log = LogTestUtils.createLog(logDir, logConfig, new BrokerTopicStats, time.scheduler, time) log.appendAsLeader(MemoryRecords.withRecords(Compression.NONE, metadataRecords:_*), leaderEpoch = 0) val secondSegment = log.roll() - secondSegment.append(1L, RecordBatch.NO_TIMESTAMP, 1L, MemoryRecords.withRecords(Compression.NONE, metadataRecords:_*)) + secondSegment.append(1L, MemoryRecords.withRecords(Compression.NONE, metadataRecords: _*)) secondSegment.flush() log.flush(true) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LocalLog.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LocalLog.java index 027196a5de..817da5c831 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LocalLog.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LocalLog.java @@ -525,8 +525,8 @@ else if (segment.baseOffset() == maxOffsetMetadata.segmentBaseOffset && !maxOffs ); } - public void append(long lastOffset, long largestTimestamp, long shallowOffsetOfMaxTimestamp, MemoryRecords records) throws IOException { - segments.activeSegment().append(lastOffset, largestTimestamp, shallowOffsetOfMaxTimestamp, records); + public void append(long lastOffset, MemoryRecords records) throws IOException { + segments.activeSegment().append(lastOffset, records); updateLogEndOffset(lastOffset + 1); } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogAppendInfo.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogAppendInfo.java index 05e162a304..63a8a51081 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogAppendInfo.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogAppendInfo.java @@ -31,13 +31,12 @@ public class LogAppendInfo { public static final LogAppendInfo UNKNOWN_LOG_APPEND_INFO = new LogAppendInfo(-1, -1, OptionalInt.empty(), - RecordBatch.NO_TIMESTAMP, -1L, RecordBatch.NO_TIMESTAMP, -1L, + RecordBatch.NO_TIMESTAMP, RecordBatch.NO_TIMESTAMP, -1L, RecordValidationStats.EMPTY, CompressionType.NONE, -1, -1L); private long firstOffset; private long lastOffset; private long maxTimestamp; - private long shallowOffsetOfMaxTimestamp; private long logAppendTime; private long logStartOffset; private RecordValidationStats recordValidationStats; @@ -52,31 +51,29 @@ public class LogAppendInfo { /** * Creates an instance with the given params. * - * @param firstOffset The first offset in the message set unless the message format is less than V2 and we are appending - * to the follower. - * @param lastOffset The last offset in the message set - * @param lastLeaderEpoch The partition leader epoch corresponding to the last offset, if available. - * @param maxTimestamp The maximum timestamp of the message set. - * @param shallowOffsetOfMaxTimestamp The last offset of the batch with the maximum timestamp. - * @param logAppendTime The log append time (if used) of the message set, otherwise Message.NoTimestamp - * @param logStartOffset The start offset of the log at the time of this append. - * @param recordValidationStats Statistics collected during record processing, `null` if `assignOffsets` is `false` - * @param sourceCompression The source codec used in the message set (send by the producer) - * @param validBytes The number of valid bytes - * @param lastOffsetOfFirstBatch The last offset of the first batch + * @param firstOffset The first offset in the message set unless the message format is less than V2 and we are appending + * to the follower. + * @param lastOffset The last offset in the message set + * @param lastLeaderEpoch The partition leader epoch corresponding to the last offset, if available. + * @param maxTimestamp The maximum timestamp of the message set. + * @param logAppendTime The log append time (if used) of the message set, otherwise Message.NoTimestamp + * @param logStartOffset The start offset of the log at the time of this append. + * @param recordValidationStats Statistics collected during record processing, `null` if `assignOffsets` is `false` + * @param sourceCompression The source codec used in the message set (send by the producer) + * @param validBytes The number of valid bytes + * @param lastOffsetOfFirstBatch The last offset of the first batch */ public LogAppendInfo(long firstOffset, long lastOffset, OptionalInt lastLeaderEpoch, long maxTimestamp, - long shallowOffsetOfMaxTimestamp, long logAppendTime, long logStartOffset, RecordValidationStats recordValidationStats, CompressionType sourceCompression, int validBytes, long lastOffsetOfFirstBatch) { - this(firstOffset, lastOffset, lastLeaderEpoch, maxTimestamp, shallowOffsetOfMaxTimestamp, logAppendTime, logStartOffset, + this(firstOffset, lastOffset, lastLeaderEpoch, maxTimestamp, logAppendTime, logStartOffset, recordValidationStats, sourceCompression, validBytes, lastOffsetOfFirstBatch, Collections.emptyList(), LeaderHwChange.NONE); } @@ -84,27 +81,25 @@ public LogAppendInfo(long firstOffset, /** * Creates an instance with the given params. * - * @param firstOffset The first offset in the message set unless the message format is less than V2 and we are appending - * to the follower. - * @param lastOffset The last offset in the message set - * @param lastLeaderEpoch The partition leader epoch corresponding to the last offset, if available. - * @param maxTimestamp The maximum timestamp of the message set. - * @param shallowOffsetOfMaxTimestamp The last offset of the batch with the maximum timestamp. - * @param logAppendTime The log append time (if used) of the message set, otherwise Message.NoTimestamp - * @param logStartOffset The start offset of the log at the time of this append. - * @param recordValidationStats Statistics collected during record processing, `null` if `assignOffsets` is `false` - * @param sourceCompression The source codec used in the message set (send by the producer) - * @param validBytes The number of valid bytes - * @param lastOffsetOfFirstBatch The last offset of the first batch - * @param recordErrors List of record errors that caused the respective batch to be dropped - * @param leaderHwChange Incremental if the high watermark needs to be increased after appending record - * Same if high watermark is not changed. None is the default value and it means append failed + * @param firstOffset The first offset in the message set unless the message format is less than V2 and we are appending + * to the follower. + * @param lastOffset The last offset in the message set + * @param lastLeaderEpoch The partition leader epoch corresponding to the last offset, if available. + * @param maxTimestamp The maximum timestamp of the message set. + * @param logAppendTime The log append time (if used) of the message set, otherwise Message.NoTimestamp + * @param logStartOffset The start offset of the log at the time of this append. + * @param recordValidationStats Statistics collected during record processing, `null` if `assignOffsets` is `false` + * @param sourceCompression The source codec used in the message set (send by the producer) + * @param validBytes The number of valid bytes + * @param lastOffsetOfFirstBatch The last offset of the first batch + * @param recordErrors List of record errors that caused the respective batch to be dropped + * @param leaderHwChange Incremental if the high watermark needs to be increased after appending record + * Same if high watermark is not changed. None is the default value and it means append failed */ public LogAppendInfo(long firstOffset, long lastOffset, OptionalInt lastLeaderEpoch, long maxTimestamp, - long shallowOffsetOfMaxTimestamp, long logAppendTime, long logStartOffset, RecordValidationStats recordValidationStats, @@ -117,7 +112,6 @@ public LogAppendInfo(long firstOffset, this.lastOffset = lastOffset; this.lastLeaderEpoch = lastLeaderEpoch; this.maxTimestamp = maxTimestamp; - this.shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp; this.logAppendTime = logAppendTime; this.logStartOffset = logStartOffset; this.recordValidationStats = recordValidationStats; @@ -156,14 +150,6 @@ public void setMaxTimestamp(long maxTimestamp) { this.maxTimestamp = maxTimestamp; } - public long shallowOffsetOfMaxTimestamp() { - return shallowOffsetOfMaxTimestamp; - } - - public void setShallowOffsetOfMaxTimestamp(long shallowOffsetOfMaxTimestamp) { - this.shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp; - } - public long logAppendTime() { return logAppendTime; } @@ -233,12 +219,12 @@ public long numMessages() { * @return a new instance with the given LeaderHwChange */ public LogAppendInfo copy(LeaderHwChange newLeaderHwChange) { - return new LogAppendInfo(firstOffset, lastOffset, lastLeaderEpoch, maxTimestamp, shallowOffsetOfMaxTimestamp, logAppendTime, logStartOffset, recordValidationStats, + return new LogAppendInfo(firstOffset, lastOffset, lastLeaderEpoch, maxTimestamp, logAppendTime, logStartOffset, recordValidationStats, sourceCompression, validBytes, lastOffsetOfFirstBatch, recordErrors, newLeaderHwChange); } public static LogAppendInfo unknownLogAppendInfoWithLogStartOffset(long logStartOffset) { - return new LogAppendInfo(-1, -1, OptionalInt.empty(), RecordBatch.NO_TIMESTAMP, -1L, RecordBatch.NO_TIMESTAMP, logStartOffset, + return new LogAppendInfo(-1, -1, OptionalInt.empty(), RecordBatch.NO_TIMESTAMP, RecordBatch.NO_TIMESTAMP, logStartOffset, RecordValidationStats.EMPTY, CompressionType.NONE, -1, -1L); } @@ -248,7 +234,7 @@ public static LogAppendInfo unknownLogAppendInfoWithLogStartOffset(long logStart * in unknownLogAppendInfoWithLogStartOffset, but with additional fields recordErrors */ public static LogAppendInfo unknownLogAppendInfoWithAdditionalInfo(long logStartOffset, List recordErrors) { - return new LogAppendInfo(-1, -1, OptionalInt.empty(), RecordBatch.NO_TIMESTAMP, -1L, RecordBatch.NO_TIMESTAMP, logStartOffset, + return new LogAppendInfo(-1, -1, OptionalInt.empty(), RecordBatch.NO_TIMESTAMP, RecordBatch.NO_TIMESTAMP, logStartOffset, RecordValidationStats.EMPTY, CompressionType.NONE, -1, -1L, recordErrors, LeaderHwChange.NONE); } @@ -259,7 +245,6 @@ public String toString() { ", lastOffset=" + lastOffset + ", lastLeaderEpoch=" + lastLeaderEpoch + ", maxTimestamp=" + maxTimestamp + - ", shallowOffsetOfMaxTimestamp=" + shallowOffsetOfMaxTimestamp + ", logAppendTime=" + logAppendTime + ", logStartOffset=" + logStartOffset + ", recordConversionStats=" + recordValidationStats + 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 5148408c0e..b388af1f79 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 @@ -232,38 +232,38 @@ private boolean canConvertToRelativeOffset(long offset) throws IOException { * It is assumed this method is being called from within a lock, it is not thread-safe otherwise. * * @param largestOffset The last offset in the message set - * @param largestTimestampMs The largest timestamp in the message set. - * @param shallowOffsetOfMaxTimestamp The last offset of earliest batch with max timestamp in the messages to append. - * @param records The log entries to append. + * @param records The log entries to append. * @throws LogSegmentOffsetOverflowException if the largest offset causes index offset overflow */ public void append(long largestOffset, - long largestTimestampMs, - long shallowOffsetOfMaxTimestamp, MemoryRecords records) throws IOException { if (records.sizeInBytes() > 0) { - LOGGER.trace("Inserting {} bytes at end offset {} at position {} with largest timestamp {} at offset {}", - records.sizeInBytes(), largestOffset, log.sizeInBytes(), largestTimestampMs, shallowOffsetOfMaxTimestamp); + LOGGER.trace("Inserting {} bytes at end offset {} at position {}", + records.sizeInBytes(), largestOffset, log.sizeInBytes()); int physicalPosition = log.sizeInBytes(); - if (physicalPosition == 0) - rollingBasedTimestamp = OptionalLong.of(largestTimestampMs); ensureOffsetInRange(largestOffset); // append the messages long appendedBytes = log.append(records); LOGGER.trace("Appended {} to {} at end offset {}", appendedBytes, log.file(), largestOffset); - // Update the in memory max timestamp and corresponding offset. - if (largestTimestampMs > maxTimestampSoFar()) { - maxTimestampAndOffsetSoFar = new TimestampOffset(largestTimestampMs, shallowOffsetOfMaxTimestamp); - } - // append an entry to the index (if needed) - if (bytesSinceLastIndexEntry > indexIntervalBytes) { - offsetIndex().append(largestOffset, physicalPosition); - timeIndex().maybeAppend(maxTimestampSoFar(), shallowOffsetOfMaxTimestampSoFar()); - bytesSinceLastIndexEntry = 0; + + for (RecordBatch batch : records.batches()) { + long batchMaxTimestamp = batch.maxTimestamp(); + long batchLastOffset = batch.lastOffset(); + if (batchMaxTimestamp > maxTimestampSoFar()) { + maxTimestampAndOffsetSoFar = new TimestampOffset(batchMaxTimestamp, batchLastOffset); + } + + if (bytesSinceLastIndexEntry > indexIntervalBytes) { + offsetIndex().append(batchLastOffset, physicalPosition); + timeIndex().maybeAppend(maxTimestampSoFar(), shallowOffsetOfMaxTimestampSoFar()); + bytesSinceLastIndexEntry = 0; + } + var sizeInBytes = batch.sizeInBytes(); + physicalPosition += sizeInBytes; + bytesSinceLastIndexEntry += sizeInBytes; } - bytesSinceLastIndexEntry += records.sizeInBytes(); } } @@ -274,8 +274,6 @@ private void ensureOffsetInRange(long offset) throws IOException { private int appendChunkFromFile(FileRecords records, int position, BufferSupplier bufferSupplier) throws IOException { int bytesToAppend = 0; - long maxTimestamp = Long.MIN_VALUE; - long shallowOffsetOfMaxTimestamp = Long.MIN_VALUE; long maxOffset = Long.MIN_VALUE; ByteBuffer readBuffer = bufferSupplier.get(1024 * 1024); @@ -284,10 +282,6 @@ private int appendChunkFromFile(FileRecords records, int position, BufferSupplie Iterator nextBatches = records.batchesFrom(position).iterator(); FileChannelRecordBatch batch; while ((batch = nextAppendableBatch(nextBatches, readBuffer, bytesToAppend)) != null) { - if (batch.maxTimestamp() > maxTimestamp) { - maxTimestamp = batch.maxTimestamp(); - shallowOffsetOfMaxTimestamp = batch.lastOffset(); - } maxOffset = batch.lastOffset(); bytesToAppend += batch.sizeInBytes(); } @@ -300,7 +294,7 @@ private int appendChunkFromFile(FileRecords records, int position, BufferSupplie readBuffer.limit(bytesToAppend); records.readInto(readBuffer, position); - append(maxOffset, maxTimestamp, shallowOffsetOfMaxTimestamp, MemoryRecords.readableRecords(readBuffer)); + append(maxOffset, MemoryRecords.readableRecords(readBuffer)); } bufferSupplier.release(readBuffer); diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java index fc2ba45d10..d74ac0a8b3 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java @@ -65,20 +65,15 @@ public static class ValidationResult { public final long logAppendTimeMs; public final MemoryRecords validatedRecords; public final long maxTimestampMs; - // we only maintain batch level offset for max timestamp since we want to align the behavior of updating time - // indexing entries. The paths of follower append and replica recovery do not iterate all records, so they have no - // idea about record level offset for max timestamp. - public final long shallowOffsetOfMaxTimestamp; public final boolean messageSizeMaybeChanged; public final RecordValidationStats recordValidationStats; public ValidationResult(long logAppendTimeMs, MemoryRecords validatedRecords, long maxTimestampMs, - long shallowOffsetOfMaxTimestamp, boolean messageSizeMaybeChanged, + boolean messageSizeMaybeChanged, RecordValidationStats recordValidationStats) { this.logAppendTimeMs = logAppendTimeMs; this.validatedRecords = validatedRecords; this.maxTimestampMs = maxTimestampMs; - this.shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp; this.messageSizeMaybeChanged = messageSizeMaybeChanged; this.recordValidationStats = recordValidationStats; } @@ -229,7 +224,6 @@ private ValidationResult convertAndAssignOffsetsNonCompressed(LongRef offsetCoun now, convertedRecords, info.maxTimestamp, - info.shallowOffsetOfMaxTimestamp, true, recordValidationStats); } @@ -239,8 +233,6 @@ public ValidationResult assignOffsetsNonCompressed(LongRef offsetCounter, MetricsRecorder metricsRecorder) { long now = time.milliseconds(); long maxTimestamp = RecordBatch.NO_TIMESTAMP; - long shallowOffsetOfMaxTimestamp = -1L; - long initialOffset = offsetCounter.value; RecordBatch firstBatch = getFirstBatchAndMaybeValidateNoMoreBatches(records, CompressionType.NONE); @@ -269,7 +261,6 @@ public ValidationResult assignOffsetsNonCompressed(LongRef offsetCounter, if (batch.magic() > RecordBatch.MAGIC_VALUE_V0 && maxBatchTimestamp > maxTimestamp) { maxTimestamp = maxBatchTimestamp; - shallowOffsetOfMaxTimestamp = offsetCounter.value - 1; } batch.setLastOffset(offsetCounter.value - 1); @@ -286,23 +277,10 @@ public ValidationResult assignOffsetsNonCompressed(LongRef offsetCounter, } if (timestampType == TimestampType.LOG_APPEND_TIME) { - maxTimestamp = now; - // those checks should be equal to MemoryRecordsBuilder#info - switch (toMagic) { - case RecordBatch.MAGIC_VALUE_V0: - maxTimestamp = RecordBatch.NO_TIMESTAMP; - // value will be the default value: -1 - shallowOffsetOfMaxTimestamp = -1; - break; - case RecordBatch.MAGIC_VALUE_V1: - // Those single-record batches have same max timestamp, so the initial offset is equal with - // the last offset of earliest batch - shallowOffsetOfMaxTimestamp = initialOffset; - break; - default: - // there is only one batch so use the last offset - shallowOffsetOfMaxTimestamp = offsetCounter.value - 1; - break; + if (toMagic == RecordBatch.MAGIC_VALUE_V0) { + maxTimestamp = RecordBatch.NO_TIMESTAMP; + } else { + maxTimestamp = now; } } @@ -310,7 +288,6 @@ public ValidationResult assignOffsetsNonCompressed(LongRef offsetCounter, now, records, maxTimestamp, - shallowOffsetOfMaxTimestamp, false, RecordValidationStats.EMPTY); } @@ -434,7 +411,6 @@ public ValidationResult validateMessagesAndAssignOffsetsCompressed(LongRef offse now, records, maxTimestamp, - lastOffset, false, recordValidationStats); } @@ -476,7 +452,6 @@ private ValidationResult buildRecordsAndAssignOffsets(LongRef offsetCounter, logAppendTime, records, info.maxTimestamp, - info.shallowOffsetOfMaxTimestamp, true, recordValidationStats); } diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java index 00b53de34e..a638f03abd 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java @@ -114,8 +114,6 @@ private List recordsToKvs(Iterable records) { private void appendRecords(List records, long initialOffset) throws IOException { log.append(initialOffset + records.size() - 1, - records.get(0).timestamp(), - initialOffset, MemoryRecords.withRecords(initialOffset, Compression.NONE, 0, records.toArray(new SimpleRecord[0]))); } 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 56bdcf7240..b798378f1a 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 @@ -24,6 +24,7 @@ import org.apache.kafka.common.record.FileLogInputStream; import org.apache.kafka.common.record.FileRecords; import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.MemoryRecordsBuilder; import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.Records; @@ -48,6 +49,7 @@ import java.io.File; import java.io.IOException; import java.io.RandomAccessFile; +import java.nio.ByteBuffer; import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; @@ -145,7 +147,7 @@ public void testAppendForLogSegmentOffsetOverflowException(long baseOffset, long try (LogSegment seg = createSegment(baseOffset, 10, Time.SYSTEM)) { long currentTime = Time.SYSTEM.milliseconds(); MemoryRecords memoryRecords = v1Records(0, "hello"); - assertThrows(LogSegmentOffsetOverflowException.class, () -> seg.append(largestOffset, currentTime, largestOffset, memoryRecords)); + assertThrows(LogSegmentOffsetOverflowException.class, () -> seg.append(largestOffset, memoryRecords)); } } @@ -168,7 +170,7 @@ public void testReadOnEmptySegment() throws IOException { public void testReadBeforeFirstOffset() throws IOException { try (LogSegment seg = createSegment(40)) { MemoryRecords ms = v1Records(50, "hello", "there", "little", "bee"); - seg.append(53, RecordBatch.NO_TIMESTAMP, -1L, ms); + seg.append(53, ms); Records read = seg.read(41, 300).records; checkEquals(ms.records().iterator(), read.records().iterator()); } @@ -183,7 +185,7 @@ public void testReadFromMiddleOfBatch() throws IOException { long batchBaseOffset = 50; try (LogSegment seg = createSegment(40)) { MemoryRecords ms = v2Records(batchBaseOffset, "hello", "there", "little", "bee"); - seg.append(53, RecordBatch.NO_TIMESTAMP, -1L, ms); + seg.append(53, ms); FetchDataInfo readInfo = seg.read(52, 300); assertEquals(batchBaseOffset, readInfo.fetchOffsetMetadata.messageOffset); } @@ -196,7 +198,7 @@ public void testReadFromMiddleOfBatch() throws IOException { public void testReadAfterLast() throws IOException { try (LogSegment seg = createSegment(40)) { MemoryRecords ms = v1Records(50, "hello", "there"); - seg.append(51, RecordBatch.NO_TIMESTAMP, -1L, ms); + seg.append(51, ms); FetchDataInfo read = seg.read(52, 200); assertNull(read, "Read beyond the last offset in the segment should give null"); } @@ -210,9 +212,9 @@ public void testReadAfterLast() throws IOException { public void testReadFromGap() throws IOException { try (LogSegment seg = createSegment(40)) { MemoryRecords ms = v1Records(50, "hello", "there"); - seg.append(51, RecordBatch.NO_TIMESTAMP, -1L, ms); + seg.append(51, ms); MemoryRecords ms2 = v1Records(60, "alpha", "beta"); - seg.append(61, RecordBatch.NO_TIMESTAMP, -1L, ms2); + seg.append(61, ms2); FetchDataInfo read = seg.read(55, 200); checkEquals(ms2.records().iterator(), read.records.records().iterator()); } @@ -225,7 +227,7 @@ public void testReadWhenNoMaxPosition(boolean minOneMessage) throws IOException int maxSize = 1; try (LogSegment seg = createSegment(40)) { MemoryRecords ms = v1Records(50, "hello", "there"); - seg.append(51, RecordBatch.NO_TIMESTAMP, -1L, ms); + seg.append(51, ms); // read at first offset FetchDataInfo read = seg.read(50, maxSize, maxPosition, minOneMessage); @@ -257,9 +259,9 @@ public void testTruncate() throws IOException { long offset = 40; for (int i = 0; i < 30; i++) { MemoryRecords ms1 = v1Records(offset, "hello"); - seg.append(offset, RecordBatch.NO_TIMESTAMP, -1L, ms1); + seg.append(offset, ms1); MemoryRecords ms2 = v1Records(offset + 1, "hello"); - seg.append(offset + 1, RecordBatch.NO_TIMESTAMP, -1L, ms2); + seg.append(offset + 1, ms2); // check that we can read back both messages FetchDataInfo read = seg.read(offset, 10000); @@ -320,7 +322,7 @@ public void testReloadLargestTimestampAndNextOffsetAfterTruncation() throws IOEx try (LogSegment seg = createSegment(40, 2 * v1Records(0, "hello").sizeInBytes() - 1)) { int offset = 40; for (int i = 0; i < numMessages; i++) { - seg.append(offset, offset, offset, v1Records(offset, "hello")); + seg.append(offset, v1Records(offset, "hello")); offset++; } assertEquals(offset, seg.readNextOffset()); @@ -343,7 +345,12 @@ public void testTruncateFull() throws IOException { MockTime time = new MockTime(); try (LogSegment seg = createSegment(40, time)) { - seg.append(41, RecordBatch.NO_TIMESTAMP, -1L, v1Records(40, "hello", "there")); + seg.append(41, + MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 40, Compression.NONE, TimestampType.CREATE_TIME, + List.of( + new SimpleRecord("hello".getBytes()), + new SimpleRecord("there".getBytes()) + ).toArray(new SimpleRecord[0]))); // If the segment is empty after truncation, the create time should be reset time.sleep(500); @@ -355,7 +362,7 @@ public void testTruncateFull() throws IOException { assertFalse(seg.offsetIndex().isFull()); assertNull(seg.read(0, 1024), "Segment should be empty."); - seg.append(41, RecordBatch.NO_TIMESTAMP, -1L, v1Records(40, "hello", "there")); + seg.append(41, v1Records(40, "hello", "there")); } } @@ -368,7 +375,7 @@ public void testFindOffsetByTimestamp() throws IOException { try (LogSegment seg = createSegment(40, messageSize * 2 - 1)) { // Produce some messages for (int i = 40; i < 50; i++) { - seg.append(i, i * 10, i, v1Records(i, "msg" + i)); + seg.append(i, v1Records(i, "msg" + i)); } assertEquals(490, seg.largestTimestamp()); @@ -394,7 +401,7 @@ public void testFindOffsetByTimestamp() throws IOException { public void testNextOffsetCalculation() throws IOException { try (LogSegment seg = createSegment(40)) { assertEquals(40, seg.readNextOffset()); - seg.append(52, RecordBatch.NO_TIMESTAMP, -1L, v1Records(50, "hello", "there", "you")); + seg.append(52, v1Records(50, "hello", "there", "you")); assertEquals(53, seg.readNextOffset()); } } @@ -437,7 +444,7 @@ public void testChangeFileSuffixes() throws IOException { public void testRecoveryFixesCorruptIndex() throws Exception { try (LogSegment seg = createSegment(0)) { for (int i = 0; i < 100; i++) { - seg.append(i, RecordBatch.NO_TIMESTAMP, -1L, v1Records(i, Integer.toString(i))); + seg.append(i, v1Records(i, Integer.toString(i))); } File indexFile = seg.offsetIndexFile(); writeNonsenseToFile(indexFile, 5, (int) indexFile.length()); @@ -460,27 +467,27 @@ public void testRecoverTransactionIndex() throws Exception { long pid2 = 10L; // append transactional records from pid1 - segment.append(101L, RecordBatch.NO_TIMESTAMP, - 100L, MemoryRecords.withTransactionalRecords(100L, Compression.NONE, + segment.append(101L, + MemoryRecords.withTransactionalRecords(100L, Compression.NONE, pid1, producerEpoch, sequence, partitionLeaderEpoch, new SimpleRecord("a".getBytes()), new SimpleRecord("b".getBytes()))); // append transactional records from pid2 - segment.append(103L, RecordBatch.NO_TIMESTAMP, - 102L, MemoryRecords.withTransactionalRecords(102L, Compression.NONE, + segment.append(103L, + MemoryRecords.withTransactionalRecords(102L, Compression.NONE, pid2, producerEpoch, sequence, partitionLeaderEpoch, new SimpleRecord("a".getBytes()), new SimpleRecord("b".getBytes()))); // append non-transactional records - segment.append(105L, RecordBatch.NO_TIMESTAMP, - 104L, MemoryRecords.withRecords(104L, Compression.NONE, + segment.append(105L, + MemoryRecords.withRecords(104L, Compression.NONE, partitionLeaderEpoch, new SimpleRecord("a".getBytes()), new SimpleRecord("b".getBytes()))); // abort the transaction from pid2 - segment.append(106L, RecordBatch.NO_TIMESTAMP, - 106L, endTxnRecords(ControlRecordType.ABORT, pid2, producerEpoch, 106L)); + segment.append(106L, + endTxnRecords(ControlRecordType.ABORT, pid2, producerEpoch, 106L)); // commit the transaction from pid1 - segment.append(107L, RecordBatch.NO_TIMESTAMP, - 107L, endTxnRecords(ControlRecordType.COMMIT, pid1, producerEpoch, 107L)); + segment.append(107L, + endTxnRecords(ControlRecordType.COMMIT, pid1, producerEpoch, 107L)); ProducerStateManager stateManager = newProducerStateManager(); segment.recover(stateManager, mock(LeaderEpochFileCache.class)); @@ -522,16 +529,16 @@ public void testRecoveryRebuildsEpochCache() throws Exception { LeaderEpochCheckpointFile checkpoint = new LeaderEpochCheckpointFile(TestUtils.tempFile(), new LogDirFailureChannel(1)); LeaderEpochFileCache cache = new LeaderEpochFileCache(topicPartition, checkpoint, new MockScheduler(new MockTime())); - seg.append(105L, RecordBatch.NO_TIMESTAMP, 104L, MemoryRecords.withRecords(104L, Compression.NONE, 0, + seg.append(105L, MemoryRecords.withRecords(104L, Compression.NONE, 0, new SimpleRecord("a".getBytes()), new SimpleRecord("b".getBytes()))); - seg.append(107L, RecordBatch.NO_TIMESTAMP, 106L, MemoryRecords.withRecords(106L, Compression.NONE, 1, + seg.append(107L, MemoryRecords.withRecords(106L, Compression.NONE, 1, new SimpleRecord("a".getBytes()), new SimpleRecord("b".getBytes()))); - seg.append(109L, RecordBatch.NO_TIMESTAMP, 108L, MemoryRecords.withRecords(108L, Compression.NONE, 1, + seg.append(109L, MemoryRecords.withRecords(108L, Compression.NONE, 1, new SimpleRecord("a".getBytes()), new SimpleRecord("b".getBytes()))); - seg.append(111L, RecordBatch.NO_TIMESTAMP, 110L, MemoryRecords.withRecords(110L, Compression.NONE, 2, + seg.append(111L, MemoryRecords.withRecords(110L, Compression.NONE, 2, new SimpleRecord("a".getBytes()), new SimpleRecord("b".getBytes()))); seg.recover(newProducerStateManager(), cache); @@ -567,7 +574,7 @@ private MemoryRecords endTxnRecords( public void testRecoveryFixesCorruptTimeIndex() throws IOException { try (LogSegment seg = createSegment(0)) { for (int i = 0; i < 100; i++) { - seg.append(i, i * 10, i, v1Records(i, String.valueOf(i))); + seg.append(i, v1Records(i, String.valueOf(i))); } File timeIndexFile = seg.timeIndexFile(); writeNonsenseToFile(timeIndexFile, 5, (int) timeIndexFile.length()); @@ -590,7 +597,7 @@ public void testRecoveryWithCorruptMessage() throws IOException { for (int ignore = 0; ignore < 10; ignore++) { try (LogSegment seg = createSegment(0)) { for (int i = 0; i < messagesAppended; i++) { - seg.append(i, RecordBatch.NO_TIMESTAMP, -1L, v1Records(i, String.valueOf(i))); + seg.append(i, v1Records(i, String.valueOf(i))); } int offsetToBeginCorruption = TestUtils.RANDOM.nextInt(messagesAppended); // start corrupting somewhere in the middle of the chosen record all the way to the end @@ -627,9 +634,9 @@ public void testCreateWithInitFileSizeAppendMessage() throws IOException { 512 * 1024 * 1024, true, "")) { segments.add(seg); MemoryRecords ms = v1Records(50, "hello", "there"); - seg.append(51, RecordBatch.NO_TIMESTAMP, -1L, ms); + seg.append(51, ms); MemoryRecords ms2 = v1Records(60, "alpha", "beta"); - seg.append(61, RecordBatch.NO_TIMESTAMP, -1L, ms2); + seg.append(61, ms2); FetchDataInfo read = seg.read(55, 200); checkEquals(ms2.records().iterator(), read.records.records().iterator()); } @@ -650,9 +657,9 @@ public void testCreateWithInitFileSizeClearShutdown() throws IOException { try (LogSegment seg = LogSegment.open(tempDir, 40, logConfig, Time.SYSTEM, 512 * 1024 * 1024, true)) { MemoryRecords ms = v1Records(50, "hello", "there"); - seg.append(51, RecordBatch.NO_TIMESTAMP, -1L, ms); + seg.append(51, ms); MemoryRecords ms2 = v1Records(60, "alpha", "beta"); - seg.append(61, RecordBatch.NO_TIMESTAMP, -1L, ms2); + seg.append(61, ms2); FetchDataInfo read = seg.read(55, 200); checkEquals(ms2.records().iterator(), read.records.records().iterator()); long oldSize = seg.log().sizeInBytes(); @@ -690,9 +697,9 @@ public void shouldTruncateEvenIfOffsetPointsToAGapInTheLog() throws IOException // Given two messages with a gap between them (e.g. mid offset compacted away) MemoryRecords ms1 = recordsForTruncateEven(offset, "first message"); - seg.append(offset, RecordBatch.NO_TIMESTAMP, -1L, ms1); + seg.append(offset, ms1); MemoryRecords ms2 = recordsForTruncateEven(offset + 3, "message after gap"); - seg.append(offset + 3, RecordBatch.NO_TIMESTAMP, -1L, ms2); + seg.append(offset + 3, ms2); // When we truncate to an offset without a corresponding log entry seg.truncateTo(offset + 1); @@ -743,7 +750,8 @@ public void testGetFirstBatchTimestamp() throws IOException { try (LogSegment segment = createSegment(1)) { assertEquals(Long.MAX_VALUE, segment.getFirstBatchTimestamp()); - segment.append(1, 1000L, 1, MemoryRecords.withRecords(1, Compression.NONE, new SimpleRecord("one".getBytes()))); + segment.append(1, + MemoryRecords.withRecords(1, Compression.NONE, new SimpleRecord(1000L, "one".getBytes()))); assertEquals(1000L, segment.getFirstBatchTimestamp()); } } @@ -780,6 +788,77 @@ public void testDeleteIfExistsWithGetParentIsNull() throws IOException { } } + @Test + public void testIndexForMultipleBatchesInMemoryRecords() throws IOException { + LogSegment segment = createSegment(0, 1, Time.SYSTEM); + + ByteBuffer buffer1 = ByteBuffer.allocate(1024); + // append first batch to buffer1 + MemoryRecordsBuilder builder = MemoryRecords.builder(buffer1, Compression.NONE, TimestampType.CREATE_TIME, 0); + builder.append(0L, "key1".getBytes(), "value1".getBytes()); + builder.close(); + + // append second batch to buffer1 + builder = MemoryRecords.builder(buffer1, Compression.NONE, TimestampType.CREATE_TIME, 1); + builder.append(1L, "key1".getBytes(), "value1".getBytes()); + builder.close(); + + buffer1.flip(); + MemoryRecords record = MemoryRecords.readableRecords(buffer1); + segment.append(1L, record); + + ByteBuffer buffer2 = ByteBuffer.allocate(1024); + // append first batch to buffer2 + builder = MemoryRecords.builder(buffer2, Compression.NONE, TimestampType.CREATE_TIME, 2); + builder.append(2L, "key1".getBytes(), "value1".getBytes()); + builder.close(); + + buffer2.flip(); + record = MemoryRecords.readableRecords(buffer2); + segment.append(2L, record); + + assertEquals(2, segment.offsetIndex().entries()); + assertEquals(1, segment.offsetIndex().entry(0).offset); + assertEquals(2, segment.offsetIndex().entry(1).offset); + + assertEquals(2, segment.timeIndex().entries()); + assertEquals(new TimestampOffset(1, 1), segment.timeIndex().entry(0)); + assertEquals(new TimestampOffset(2, 2), segment.timeIndex().entry(1)); + } + + @Test + public void testNonMonotonicTimestampForMultipleBatchesInMemoryRecords() throws IOException { + LogSegment segment = createSegment(0, 1, Time.SYSTEM); + + ByteBuffer buffer1 = ByteBuffer.allocate(1024); + // append first batch to buffer1 + MemoryRecordsBuilder builder = MemoryRecords.builder(buffer1, Compression.NONE, TimestampType.CREATE_TIME, 0); + builder.append(1L, "key1".getBytes(), "value1".getBytes()); + builder.close(); + + // append second batch to buffer1 + builder = MemoryRecords.builder(buffer1, Compression.NONE, TimestampType.CREATE_TIME, 1); + builder.append(0L, "key1".getBytes(), "value1".getBytes()); + builder.close(); + + // append third batch to buffer1 + builder = MemoryRecords.builder(buffer1, Compression.NONE, TimestampType.CREATE_TIME, 2); + builder.append(2L, "key1".getBytes(), "value1".getBytes()); + builder.close(); + + buffer1.flip(); + MemoryRecords record = MemoryRecords.readableRecords(buffer1); + segment.append(2L, record); + + assertEquals(2, segment.offsetIndex().entries()); + assertEquals(1, segment.offsetIndex().entry(0).offset); + assertEquals(2, segment.offsetIndex().entry(1).offset); + + assertEquals(2, segment.timeIndex().entries()); + assertEquals(new TimestampOffset(1, 0), segment.timeIndex().entry(0)); + assertEquals(new TimestampOffset(2, 2), segment.timeIndex().entry(1)); + } + private ProducerStateManager newProducerStateManager() throws IOException { return new ProducerStateManager( topicPartition, diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LogValidatorTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogValidatorTest.java index c58b15257e..3bef886cee 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/LogValidatorTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogValidatorTest.java @@ -240,7 +240,6 @@ public void testCreateTimeUpConversionV1ToV2() { } assertEquals(timestamp, validatedResults.maxTimestampMs); - assertEquals(2, validatedResults.shallowOffsetOfMaxTimestamp, "Offset of max timestamp should be the last offset 2."); assertTrue(validatedResults.messageSizeMaybeChanged, "Message size should have been changed"); verifyRecordValidationStats( @@ -287,7 +286,6 @@ public void checkCreateTimeUpConversionFromV0(byte toMagic) { } assertEquals(RecordBatch.NO_TIMESTAMP, validatedResults.maxTimestampMs, "Max timestamp should be " + RecordBatch.NO_TIMESTAMP); - assertEquals(-1, validatedResults.shallowOffsetOfMaxTimestamp); assertTrue(validatedResults.messageSizeMaybeChanged, "Message size should have been changed"); verifyRecordValidationStats(validatedResults.recordValidationStats, 3, records, true); @@ -383,7 +381,6 @@ public void checkRecompression(byte magic) { // Both V2 and V1 have single batch in the validated records when compression is enabled, and hence their shallow // OffsetOfMaxTimestamp is the last offset of the single batch assertEquals(1, iteratorSize(validatedRecords.batches().iterator())); - assertEquals(2, validatingResults.shallowOffsetOfMaxTimestamp); assertTrue(validatingResults.messageSizeMaybeChanged, "Message size should have been changed"); @@ -571,8 +568,6 @@ public void checkCompressed(byte magic) { assertEquals(now + 1, validatedResults.maxTimestampMs, "Max timestamp should be " + (now + 1)); - int expectedShallowOffsetOfMaxTimestamp = 2; - assertEquals(expectedShallowOffsetOfMaxTimestamp, validatedResults.shallowOffsetOfMaxTimestamp, "Shallow offset of max timestamp should be 2"); assertFalse(validatedResults.messageSizeMaybeChanged, "Message size should not have been changed"); verifyRecordValidationStats(validatedResults.recordValidationStats, 0, records, true); @@ -1831,10 +1826,8 @@ public void checkNonCompressed(byte magic) { if (magic >= RecordBatch.MAGIC_VALUE_V2) { assertEquals(1, iteratorSize(records.batches().iterator())); - assertEquals(2, validatingResults.shallowOffsetOfMaxTimestamp); } else { assertEquals(3, iteratorSize(records.batches().iterator())); - assertEquals(1, validatingResults.shallowOffsetOfMaxTimestamp); } assertFalse(validatingResults.messageSizeMaybeChanged, @@ -1908,8 +1901,6 @@ public void checkLogAppendTimeWithoutRecompression(byte magic) { "MessageSet should still valid"); assertEquals(now, validatedResults.maxTimestampMs, "Max timestamp should be " + now); - assertEquals(2, validatedResults.shallowOffsetOfMaxTimestamp, - "The shallow offset of max timestamp should be the last offset 2 if logAppendTime is used"); assertFalse(validatedResults.messageSizeMaybeChanged, "Message size should not have been changed"); @@ -1950,8 +1941,6 @@ public void checkLogAppendTimeWithRecompression(byte targetMagic) { assertTrue(validatedRecords.batches().iterator().next().isValid(), "MessageSet should still valid"); assertEquals(now, validatedResults.maxTimestampMs, String.format("Max timestamp should be %d", now)); - assertEquals(2, validatedResults.shallowOffsetOfMaxTimestamp, - "The shallow offset of max timestamp should be 2 if logAppendTime is used"); assertTrue(validatedResults.messageSizeMaybeChanged, "Message size may have been changed"); @@ -2002,19 +1991,6 @@ public void checkLogAppendTimeNonCompressed(byte magic) { assertFalse(validatedResults.messageSizeMaybeChanged, "Message size should not have been changed"); - int expectedMaxTimestampOffset; - switch (magic) { - case RecordBatch.MAGIC_VALUE_V0: - expectedMaxTimestampOffset = -1; - break; - case RecordBatch.MAGIC_VALUE_V1: - expectedMaxTimestampOffset = 0; - break; - default: - expectedMaxTimestampOffset = 2; - break; - } - assertEquals(expectedMaxTimestampOffset, validatedResults.shallowOffsetOfMaxTimestamp); verifyRecordValidationStats(validatedResults.recordValidationStats, 0, records, false); } From 617c96cea49ca9dda52ccc75ab2b6df56b24294c Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 2 Apr 2025 02:22:53 +0300 Subject: [PATCH 04/72] KAFKA-15931: Cancel RemoteLogReader gracefully (#19331) Backports f24945b519005c0bc7a28db2db7aae6cec158927 to 4.0 Instead of reopening the transaction index, it cancels the RemoteFetchTask without interrupting it--avoiding to close the TransactionIndex channel. This will lead to complete the execution of the remote fetch but ignoring the results. Given that this is considered a rare case, we could live with this. If it becomes a performance issue, it could be optimized. Reviewers: Jun Rao --- core/src/main/scala/kafka/server/DelayedRemoteFetch.scala | 5 +++-- .../integration/kafka/server/DelayedRemoteFetchTest.scala | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala b/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala index 45bfe69844..e6bdce63e6 100644 --- a/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala @@ -87,8 +87,9 @@ class DelayedRemoteFetch(remoteFetchTask: Future[Void], } override def onExpiration(): Unit = { - // cancel the remote storage read task, if it has not been executed yet - val cancelled = remoteFetchTask.cancel(true) + // cancel the remote storage read task, if it has not been executed yet and + // avoid interrupting the task if it is already running as it may force closing opened/cached resources as transaction index. + val cancelled = remoteFetchTask.cancel(false) if (!cancelled) debug(s"Remote fetch task for RemoteStorageFetchInfo: $remoteFetchInfo could not be cancelled and its isDone value is ${remoteFetchTask.isDone}") DelayedRemoteFetchMetrics.expiredRequestMeter.mark() diff --git a/core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala b/core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala index 264f5310c2..b3f032e3db 100644 --- a/core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala +++ b/core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala @@ -200,7 +200,7 @@ class DelayedRemoteFetchTest { delayedRemoteFetch.run() // Check that the task was cancelled and force-completed - verify(remoteFetchTask).cancel(true) + verify(remoteFetchTask).cancel(false) assertTrue(delayedRemoteFetch.isCompleted) // Check that the ExpiresPerSec metric was incremented From b0b4f42f4cf83278b5f9bcc3bcd819d1bb09d767 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Fri, 4 Apr 2025 05:13:22 +0800 Subject: [PATCH 05/72] KAFKA-18067: Add a flag to disable producer reset during active task creator shutting down (#19269) JIRA: KAFKA-18067 Fix producer client double-closing issue in Kafka Streams. During StreamThread shutdown, TaskManager closes first, which closes the producer client. Later, calling `unsubscribe` on the main consumer may trigger the `onPartitionsLost` callback, attempting to reset StreamsProducer when EOS is enabled. This causes an already closed producer to be closed twice while the newly created producer is never closed. In detail: This patch adds a flag to control the producer reset and has a new method to change this flag, which is only invoked in `ActiveTaskCreator#close`. This would guarantee that the disable reset producer will only occur when StreamThread shuts down. Reviewers: Anna Sophie Blee-Goldman , Matthias Sax --- .../processor/internals/ActiveTaskCreator.java | 17 ++++++++++++++++- .../internals/ActiveTaskCreatorTest.java | 17 +++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java index 6c973e096f..93953d815b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java @@ -64,6 +64,7 @@ class ActiveTaskCreator { private final StreamsProducer streamsProducer; private final boolean stateUpdaterEnabled; private final boolean processingThreadsEnabled; + private boolean isClosed = false; ActiveTaskCreator(final TopologyMetadata topologyMetadata, final StreamsConfig applicationConfig, @@ -118,14 +119,27 @@ private Producer producer() { return clientSupplier.getProducer(producerConfig); } + + /** + * When {@link org.apache.kafka.streams.processor.internals.StreamThread} is shutting down, + * subsequent calls to reInitializeProducer() will not recreate + * the producer instance, avoiding resource leak. + */ public void reInitializeProducer() { - streamsProducer.resetProducer(producer()); + if (!isClosed) { + streamsProducer.resetProducer(producer()); + } } StreamsProducer streamsProducer() { return streamsProducer; } + // visible for test + boolean isClosed() { + return isClosed; + } + // TODO: convert to StreamTask when we remove TaskManager#StateMachineTask with mocks public Collection createTasks(final Consumer consumer, final Map> tasksToBeCreated) { @@ -255,6 +269,7 @@ private StreamTask createActiveTask(final TaskId taskId, void close() { try { + isClosed = true; streamsProducer.close(); } catch (final RuntimeException e) { throw new StreamsException("Thread producer encounter error trying to close.", e); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreatorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreatorTest.java index 6a4339a3ed..17eb27bb3d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreatorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreatorTest.java @@ -190,9 +190,26 @@ public void shouldCloseIfEosV2Enabled() { activeTaskCreator.close(); + assertThat(activeTaskCreator.isClosed(), is(true)); assertThat(mockClientSupplier.producers.get(0).closed(), is(true)); } + @Test + public void shouldNotResetProducerAfterDisableRest() { + properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_V2); + mockClientSupplier.setApplicationIdForProducer("appId"); + createTasks(); + assertThat(mockClientSupplier.producers.size(), is(1)); + + activeTaskCreator.close(); + activeTaskCreator.reInitializeProducer(); + // Verifies that disableReset() prevents reInitializeProducer() from creating a new producer instance + // Without disabling reset, the producers collection would contain more than one producer + assertThat("Producer should not be recreated after disabling reset", + mockClientSupplier.producers.size(), + is(1)); + } + // error handling @Test From 2c48809fad31e6773bdca1291fcecced45214d2a Mon Sep 17 00:00:00 2001 From: nilmadhab mondal Date: Fri, 4 Apr 2025 01:22:47 +0200 Subject: [PATCH 06/72] KAFKA-18713: Fix FK Left-Join result race condition (#19005) When a row in a FK-join left table is updated, we should send a "delete subscription with no response" for the old FK to the right hand side, to avoid getting two responses from the right hand side. Only the "new subscription" for the new FK should request a response. If two responses are requested, there is a race condition for which both responses could be processed in the wrong order, leading to an incorrect join result. This PR fixes the "delete subscription" case accordingly, to no request a response. Reviewers: Matthias J. Sax --- ...leKTableForeignKeyJoinIntegrationTest.java | 135 ++++++++++++++---- .../SubscriptionSendProcessorSupplier.java | 2 +- ...SubscriptionSendProcessorSupplierTest.java | 8 +- 3 files changed, 114 insertions(+), 31 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyJoinIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyJoinIntegrationTest.java index b4ac10fdbc..b73da0f8a3 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyJoinIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyJoinIntegrationTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.TestInputTopic; @@ -59,10 +60,13 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static java.util.Arrays.asList; +import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkProperties; +import static org.hamcrest.CoreMatchers.hasItem; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; @@ -182,13 +186,13 @@ public void doJoinFromLeftThenDeleteLeftEntity(final boolean leftJoin, right.pipeInput("rhs3", "rhsValue3", baseTimestamp + 2); // this unreferenced FK won't show up in any results assertThat( - outputTopic.readKeyValuesToMap(), - is(emptyMap()) + outputTopic.readKeyValuesToList(), + is(emptyList()) ); if (rejoin) { assertThat( - rejoinOutputTopic.readKeyValuesToMap(), - is(emptyMap()) + rejoinOutputTopic.readKeyValuesToList(), + is(emptyList()) ); } if (materialized) { @@ -202,27 +206,27 @@ public void doJoinFromLeftThenDeleteLeftEntity(final boolean leftJoin, left.pipeInput("lhs2", "lhsValue2|rhs2", baseTimestamp + 4); { - final Map expected = mkMap( - mkEntry("lhs1", "(lhsValue1|rhs1,rhsValue1)"), - mkEntry("lhs2", "(lhsValue2|rhs2,rhsValue2)") + final List> expected = Arrays.asList( + KeyValue.pair("lhs1", "(lhsValue1|rhs1,rhsValue1)"), + KeyValue.pair("lhs2", "(lhsValue2|rhs2,rhsValue2)") ); assertThat( - outputTopic.readKeyValuesToMap(), + outputTopic.readKeyValuesToList(), is(expected) ); if (rejoin) { assertThat( - rejoinOutputTopic.readKeyValuesToMap(), - is(mkMap( - mkEntry("lhs1", "rejoin((lhsValue1|rhs1,rhsValue1),lhsValue1|rhs1)"), - mkEntry("lhs2", "rejoin((lhsValue2|rhs2,rhsValue2),lhsValue2|rhs2)") + rejoinOutputTopic.readKeyValuesToList(), + is(asList( + KeyValue.pair("lhs1", "rejoin((lhsValue1|rhs1,rhsValue1),lhsValue1|rhs1)"), + KeyValue.pair("lhs2", "rejoin((lhsValue2|rhs2,rhsValue2),lhsValue2|rhs2)") )) ); } if (materialized) { assertThat( asMap(store), - is(expected) + is(expected.stream().collect(Collectors.toMap(kv -> kv.key, kv -> kv.value))) ); } } @@ -231,16 +235,16 @@ public void doJoinFromLeftThenDeleteLeftEntity(final boolean leftJoin, left.pipeInput("lhs3", "lhsValue3|rhs1", baseTimestamp + 5); { assertThat( - outputTopic.readKeyValuesToMap(), - is(mkMap( - mkEntry("lhs3", "(lhsValue3|rhs1,rhsValue1)") + outputTopic.readKeyValuesToList(), + is(List.of( + new KeyValue<>("lhs3", "(lhsValue3|rhs1,rhsValue1)") )) ); if (rejoin) { assertThat( - rejoinOutputTopic.readKeyValuesToMap(), - is(mkMap( - mkEntry("lhs3", "rejoin((lhsValue3|rhs1,rhsValue1),lhsValue3|rhs1)") + rejoinOutputTopic.readKeyValuesToList(), + is(List.of( + new KeyValue<>("lhs3", "rejoin((lhsValue3|rhs1,rhsValue1),lhsValue3|rhs1)") )) ); } @@ -255,21 +259,21 @@ public void doJoinFromLeftThenDeleteLeftEntity(final boolean leftJoin, ); } } + // Now delete one LHS entity such that one delete is propagated down to the output. left.pipeInput("lhs1", (String) null, baseTimestamp + 6); assertThat( - outputTopic.readKeyValuesToMap(), - is(mkMap( - mkEntry("lhs1", null) + outputTopic.readKeyValuesToList(), + is(List.of( + new KeyValue<>("lhs1", null) )) ); if (rejoin) { assertThat( - rejoinOutputTopic.readKeyValuesToMap(), - is(mkMap( - mkEntry("lhs1", null) - )) + rejoinOutputTopic.readKeyValuesToList(), + hasItem( + KeyValue.pair("lhs1", null)) ); } if (materialized) { @@ -284,6 +288,79 @@ public void doJoinFromLeftThenDeleteLeftEntity(final boolean leftJoin, } } + @ParameterizedTest + @MethodSource("testCases") + public void doJoinFromLeftThenUpdateFkThenRevertBack(final boolean leftJoin, + final String optimization, + final boolean materialized, + final boolean rejoin, + final boolean leftVersioned, + final boolean rightVersioned) { + final Properties streamsConfig = getStreamsProperties(optimization); + final Topology topology = getTopology(streamsConfig, materialized ? "store" : null, leftJoin, rejoin, leftVersioned, rightVersioned); + try (final TopologyTestDriver driver = new TopologyTestDriver(topology, streamsConfig)) { + final TestInputTopic right = driver.createInputTopic(RIGHT_TABLE, new StringSerializer(), new StringSerializer()); + final TestInputTopic left = driver.createInputTopic(LEFT_TABLE, new StringSerializer(), new StringSerializer()); + final TestOutputTopic outputTopic = driver.createOutputTopic(OUTPUT, new StringDeserializer(), new StringDeserializer()); + final TestOutputTopic rejoinOutputTopic = rejoin ? driver.createOutputTopic(REJOIN_OUTPUT, new StringDeserializer(), new StringDeserializer()) : null; + final KeyValueStore store = driver.getKeyValueStore("store"); + + // Pre-populate the RHS records. This test is all about what happens when we add/remove LHS records + right.pipeInput("rhs1", "rhsValue1", baseTimestamp); + right.pipeInput("rhs2", "rhsValue2", baseTimestamp + 1); + + assertThat( + outputTopic.readKeyValuesToList(), + is(emptyList()) + ); + if (rejoin) { + assertThat( + rejoinOutputTopic.readKeyValuesToList(), + is(emptyList()) + ); + } + if (materialized) { + assertThat( + asMap(store), + is(emptyMap()) + ); + } + + left.pipeInput("lhs1", "lhsValue1|rhs1", baseTimestamp + 3); + + { + final List> expected = asList( + KeyValue.pair("lhs1", "(lhsValue1|rhs1,rhsValue1)") + ); + assertThat( + outputTopic.readKeyValuesToList(), + is(expected) + ); + } + + // Add another reference to an existing FK + left.pipeInput("lhs1", "lhsValue1|rhs2", baseTimestamp + 5); + { + assertThat( + outputTopic.readKeyValuesToList(), + is(List.of( + new KeyValue<>("lhs1", "(lhsValue1|rhs2,rhsValue2)") + )) + ); + } + + // Now revert back the foreign key to earlier reference + + left.pipeInput("lhs1", "lhsValue1|rhs1", baseTimestamp + 6); + assertThat( + outputTopic.readKeyValuesToList(), + is(List.of( + new KeyValue<>("lhs1", "(lhsValue1|rhs1,rhsValue1)") + )) + ); + } + } + @ParameterizedTest @MethodSource("testCases") public void doJoinFromRightThenDeleteRightEntity(final boolean leftJoin, @@ -792,6 +869,12 @@ protected static Map asMap(final KeyValueStore s return result; } + protected static List> makeList(final KeyValueStore> store) { + final List> result = new LinkedList<>(); + store.all().forEachRemaining(ele -> result.add(new KeyValue<>(ele.key, ele.value.value()))); + return result; + } + protected static Topology getTopology(final Properties streamsConfig, final String queryableStoreName, final boolean leftJoin, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionSendProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionSendProcessorSupplier.java index 10199c242b..9e6f1833be 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionSendProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionSendProcessorSupplier.java @@ -131,7 +131,7 @@ private void leftJoinInstructions(final Record> record) { final KO oldForeignKey = foreignKeyExtractor.extract(record.key(), record.value().oldValue); final KO newForeignKey = record.value().newValue == null ? null : foreignKeyExtractor.extract(record.key(), record.value().newValue); if (oldForeignKey != null && !Arrays.equals(serialize(newForeignKey), serialize(oldForeignKey))) { - forward(record, oldForeignKey, DELETE_KEY_AND_PROPAGATE); + forward(record, oldForeignKey, DELETE_KEY_NO_PROPAGATE); } forward(record, newForeignKey, PROPAGATE_NULL_IF_NO_FK_VAL_AVAILABLE); } else if (record.value().newValue != null) { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionSendProcessorSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionSendProcessorSupplierTest.java index 87366bd533..197f79462f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionSendProcessorSupplierTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionSendProcessorSupplierTest.java @@ -149,7 +149,7 @@ public void leftJoinShouldPropagateChangeOfFKFromNonNullToNullValue() { assertThat(context.forwarded().size(), greaterThan(0)); assertThat( context.forwarded().get(0).record(), - is(new Record<>(fk1, new SubscriptionWrapper<>(hash(leftRecordValue), DELETE_KEY_AND_PROPAGATE, pk, 0), 0)) + is(new Record<>(fk1, new SubscriptionWrapper<>(hash(leftRecordValue), DELETE_KEY_NO_PROPAGATE, pk, 0), 0)) ); } @@ -198,7 +198,7 @@ public void leftJoinShouldPropagateDeletionOfAPrimaryKey() { assertThat(context.forwarded().size(), greaterThan(0)); assertThat( context.forwarded().get(0).record(), - is(new Record<>(fk1, new SubscriptionWrapper<>(null, DELETE_KEY_AND_PROPAGATE, pk, 0), 0)) + is(new Record<>(fk1, new SubscriptionWrapper<>(null, DELETE_KEY_NO_PROPAGATE, pk, 0), 0)) ); } @@ -438,7 +438,7 @@ public void biFunctionLeftJoinShouldPropagateChangeOfFKFromNonNullToNullValue() assertThat(context.forwarded().size(), greaterThan(0)); assertThat( context.forwarded().get(0).record(), - is(new Record<>(compositeKey, new SubscriptionWrapper<>(hash(leftRecordValue), DELETE_KEY_AND_PROPAGATE, pk, 0), 0)) + is(new Record<>(compositeKey, new SubscriptionWrapper<>(hash(leftRecordValue), DELETE_KEY_NO_PROPAGATE, pk, 0), 0)) ); } @@ -491,7 +491,7 @@ public void biFunctionLeftJoinShouldPropagateDeletionOfAPrimaryKey() { assertThat(context.forwarded().size(), greaterThan(0)); assertThat( context.forwarded().get(0).record(), - is(new Record<>(compositeKey, new SubscriptionWrapper<>(null, DELETE_KEY_AND_PROPAGATE, pk, 0), 0)) + is(new Record<>(compositeKey, new SubscriptionWrapper<>(null, DELETE_KEY_NO_PROPAGATE, pk, 0), 0)) ); } From 71c9d83b20eba87d539ce94a82165345c5228469 Mon Sep 17 00:00:00 2001 From: Ayoub Omari Date: Sun, 6 Apr 2025 05:13:31 +0200 Subject: [PATCH 07/72] KAFKA-16407: Fix foreign key INNER join on change of FK from/to a null value (#19303) Fixes both KAFKA-16407 and KAFKA-16434. Summary of existing issues: - We are ignoring new left record when its previous FK value is null - We do not unset foreign key join result when FK becomes null Reviewers: Matthias J. Sax --- .../SubscriptionSendProcessorSupplier.java | 31 ++++----- ...SubscriptionSendProcessorSupplierTest.java | 64 ++++++++++++++++++- 2 files changed, 75 insertions(+), 20 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionSendProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionSendProcessorSupplier.java index 9e6f1833be..0911d26b6f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionSendProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionSendProcessorSupplier.java @@ -142,28 +142,25 @@ private void leftJoinInstructions(final Record> record) { private void defaultJoinInstructions(final Record> record) { if (record.value().oldValue != null) { - final KO oldForeignKey = record.value().oldValue == null ? null : foreignKeyExtractor.extract(record.key(), record.value().oldValue); - if (oldForeignKey == null) { + final KO oldForeignKey = foreignKeyExtractor.extract(record.key(), record.value().oldValue); + final KO newForeignKey = record.value().newValue == null ? null : foreignKeyExtractor.extract(record.key(), record.value().newValue); + + if (oldForeignKey == null && newForeignKey == null) { logSkippedRecordDueToNullForeignKey(); - return; - } - if (record.value().newValue != null) { - final KO newForeignKey = record.value().newValue == null ? null : foreignKeyExtractor.extract(record.key(), record.value().newValue); - if (newForeignKey == null) { - logSkippedRecordDueToNullForeignKey(); - return; - } - if (!Arrays.equals(serialize(newForeignKey), serialize(oldForeignKey))) { - //Different Foreign Key - delete the old key value and propagate the new one. - //Delete it from the oldKey's state store - forward(record, oldForeignKey, DELETE_KEY_NO_PROPAGATE); - } + } else if (oldForeignKey == null) { + forward(record, newForeignKey, PROPAGATE_ONLY_IF_FK_VAL_AVAILABLE); + } else if (newForeignKey == null) { + forward(record, oldForeignKey, DELETE_KEY_AND_PROPAGATE); + } else if (!Arrays.equals(serialize(newForeignKey), serialize(oldForeignKey))) { + //Different Foreign Key - delete the old key value and propagate the new one. + //Delete it from the oldKey's state store + forward(record, oldForeignKey, DELETE_KEY_NO_PROPAGATE); //Add to the newKey's state store. Additionally, propagate null if no FK is found there, //since we must "unset" any output set by the previous FK-join. This is true for both INNER //and LEFT join. forward(record, newForeignKey, PROPAGATE_NULL_IF_NO_FK_VAL_AVAILABLE); - } else { - forward(record, oldForeignKey, DELETE_KEY_AND_PROPAGATE); + } else { // unchanged FK + forward(record, newForeignKey, PROPAGATE_ONLY_IF_FK_VAL_AVAILABLE); } } else if (record.value().newValue != null) { final KO newForeignKey = foreignKeyExtractor.extract(record.key(), record.value().newValue); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionSendProcessorSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionSendProcessorSupplierTest.java index 197f79462f..51691f014b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionSendProcessorSupplierTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionSendProcessorSupplierTest.java @@ -295,10 +295,57 @@ public void innerJoinShouldPropagateNothingWhenOldAndNewFKIsNull() { innerJoinProcessor.process(new Record<>(pk, new Change<>(leftRecordValue, leftRecordValue), 0)); assertThat(context.forwarded(), empty()); + } - // test dropped-records sensors - assertEquals(1.0, getDroppedRecordsTotalMetric(context)); - assertNotEquals(0.0, getDroppedRecordsRateMetric(context)); + @Test + public void innerJoinShouldPropagateChangeFromNullFKToNonNullFK() { + final MockInternalProcessorContext> context = new MockInternalProcessorContext<>(); + innerJoinProcessor.init(context); + context.setRecordMetadata("topic", 0, 0); + + final LeftValue leftRecordValue = new LeftValue(fk1); + + innerJoinProcessor.process(new Record<>(pk, new Change<>(leftRecordValue, new LeftValue(null)), 0)); + + assertThat(context.forwarded().size(), is(1)); + assertThat( + context.forwarded().get(0).record(), + is(new Record<>(fk1, new SubscriptionWrapper<>(hash(leftRecordValue), PROPAGATE_ONLY_IF_FK_VAL_AVAILABLE, pk, 0), 0)) + ); + } + + @Test + public void innerJoinShouldDeleteAndPropagateChangeFromNonNullFKToNullFK() { + final MockInternalProcessorContext> context = new MockInternalProcessorContext<>(); + innerJoinProcessor.init(context); + context.setRecordMetadata("topic", 0, 0); + + final LeftValue leftRecordValue = new LeftValue(null); + + innerJoinProcessor.process(new Record<>(pk, new Change<>(leftRecordValue, new LeftValue(fk1)), 0)); + + assertThat(context.forwarded().size(), is(1)); + assertThat( + context.forwarded().get(0).record(), + is(new Record<>(fk1, new SubscriptionWrapper<>(hash(leftRecordValue), DELETE_KEY_AND_PROPAGATE, pk, 0), 0)) + ); + } + + @Test + public void innerJoinShouldPropagateUnchangedFKOnlyIfFKExistsInRightTable() { + final MockInternalProcessorContext> context = new MockInternalProcessorContext<>(); + innerJoinProcessor.init(context); + context.setRecordMetadata("topic", 0, 0); + + final LeftValue leftRecordValue = new LeftValue(fk1); + + innerJoinProcessor.process(new Record<>(pk, new Change<>(leftRecordValue, leftRecordValue), 0)); + + assertThat(context.forwarded().size(), is(1)); + assertThat( + context.forwarded().get(0).record(), + is(new Record<>(fk1, new SubscriptionWrapper<>(hash(leftRecordValue), PROPAGATE_ONLY_IF_FK_VAL_AVAILABLE, pk, 0), 0)) + ); } @Test @@ -316,6 +363,17 @@ public void innerJoinShouldPropagateDeletionOfPrimaryKey() { ); } + @Test + public void innerJoinShouldNotPropagateDeletionOfPrimaryKeyWhenPreviousFKIsNull() { + final MockInternalProcessorContext> context = new MockInternalProcessorContext<>(); + innerJoinProcessor.init(context); + context.setRecordMetadata("topic", 0, 0); + + innerJoinProcessor.process(new Record<>(pk, new Change<>(null, new LeftValue(null)), 0)); + + assertThat(context.forwarded(), empty()); + } + @Test public void innerJoinShouldPropagateNothingWhenOldAndNewLeftValueIsNull() { final MockInternalProcessorContext> context = new MockInternalProcessorContext<>(); From 8de7b69ced939fdf35f8867c3aeba2d6dac67dd7 Mon Sep 17 00:00:00 2001 From: Xuan-Zhang Gong Date: Sun, 6 Apr 2025 22:08:05 +0800 Subject: [PATCH 08/72] MINOR: small optimization by judgment (#19386) judgments can help avoid unnecessary `segments.sizeInBytes()` loops from https://github.com/apache/kafka/pull/18393/files#r2029925512 Reviewers: PoAn Yang , Chia-Ping Tsai --- .../org/apache/kafka/storage/internals/log/LocalLog.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LocalLog.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LocalLog.java index 817da5c831..c939fffdc5 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LocalLog.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LocalLog.java @@ -470,9 +470,10 @@ public FetchDataInfo read(long startOffset, return maybeHandleIOException( () -> "Exception while reading from " + topicPartition + " in dir " + dir.getParent(), () -> { - logger.trace("Reading maximum {} bytes at offset {} from log with total length {} bytes", - maxLength, startOffset, segments.sizeInBytes()); - + if (logger.isTraceEnabled()) { + logger.trace("Reading maximum {} bytes at offset {} from log with total length {} bytes", + maxLength, startOffset, segments.sizeInBytes()); + } LogOffsetMetadata endOffsetMetadata = nextOffsetMetadata; long endOffset = endOffsetMetadata.messageOffset; Optional segmentOpt = segments.floorSegment(startOffset); From 33fa572a1c3de5630c3dc196ec5ef824863e2c40 Mon Sep 17 00:00:00 2001 From: PoAn Yang Date: Wed, 9 Apr 2025 09:10:25 +0800 Subject: [PATCH 09/72] MINOR: remove transform and through from repartition description (#19291) `transform` and `through` are removed in 4.0. Since users cannot reference them in 4.0 document, it's not good to keep using them as example in `repartition` description. Reviewers: Matthias J. Sax --- docs/streams/developer-guide/dsl-api.html | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/streams/developer-guide/dsl-api.html b/docs/streams/developer-guide/dsl-api.html index b59ac764f3..8e8a36f76c 100644 --- a/docs/streams/developer-guide/dsl-api.html +++ b/docs/streams/developer-guide/dsl-api.html @@ -764,10 +764,10 @@

Manually trigger repartitioning of the stream with desired number of partitions. (details)

- repartition() is similar to through() however Kafka Streams will manage the topic for you. + Kafka Streams will manage the topic for repartition(). Generated topic is treated as internal topic, as a result data will be purged automatically as any other internal repartition topic. In addition, you can specify the desired number of partitions, which allows to easily scale in/out downstream sub-topologies. - repartition() operation always triggers repartitioning of the stream, as a result it can be used with embedded Processor API methods (like transform() et al.) that do not trigger auto repartitioning when key changing operation is performed beforehand. + repartition() operation always triggers repartitioning of the stream, as a result it can be used with embedded Processor API methods (like process() et al.) that do not trigger auto repartitioning when key changing operation is performed beforehand.
KStream<byte[], String> stream = ... ;
 KStream<byte[], String> repartitionedStream = stream.repartition(Repartitioned.numberOfPartitions(10));
From 4844bc3067c579b88c75350366f1f57c0d1f5bec Mon Sep 17 00:00:00 2001 From: Parker Chang Date: Mon, 24 Mar 2025 18:53:49 +0800 Subject: [PATCH 10/72] KAFKA-18984: Reset interval.ms By Using kafka-client-metrics.sh (#19213) kafka-client-metrics.sh cannot reset the interval using `--interval=`. Reviewers: Andrew Schofield --- .../kafka/tools/ClientMetricsCommand.java | 29 ++++++++---- .../kafka/tools/ClientMetricsCommandTest.java | 46 +++++++++++++++++++ 2 files changed, 67 insertions(+), 8 deletions(-) diff --git a/tools/src/main/java/org/apache/kafka/tools/ClientMetricsCommand.java b/tools/src/main/java/org/apache/kafka/tools/ClientMetricsCommand.java index 58c187ba50..abe305068a 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ClientMetricsCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/ClientMetricsCommand.java @@ -119,7 +119,7 @@ public void alterClientMetrics(ClientMetricsCommandOptions opts) throws Exceptio String entityName = opts.hasGenerateNameOption() ? Uuid.randomUuid().toString() : opts.name().get(); Map configsToBeSet = new HashMap<>(); - opts.interval().map(intervalVal -> configsToBeSet.put("interval.ms", intervalVal.toString())); + opts.interval().map(intervalVal -> configsToBeSet.put("interval.ms", intervalVal)); opts.metrics().map(metricslist -> configsToBeSet.put("metrics", String.join(",", metricslist))); opts.match().map(matchlist -> configsToBeSet.put("match", String.join(",", matchlist))); @@ -210,7 +210,7 @@ public static final class ClientMetricsCommandOptions extends CommandDefaultOpti private final OptionSpecBuilder generateNameOpt; - private final ArgumentAcceptingOptionSpec intervalOpt; + private final ArgumentAcceptingOptionSpec intervalOpt; private final ArgumentAcceptingOptionSpec matchOpt; @@ -237,24 +237,25 @@ public ClientMetricsCommandOptions(String[] args) { .describedAs("name") .ofType(String.class); generateNameOpt = parser.accepts("generate-name", "Generate a UUID to use as the name."); - intervalOpt = parser.accepts("interval", "The metrics push interval in milliseconds.") + String nl = System.lineSeparator(); + + intervalOpt = parser.accepts("interval", "The metrics push interval in milliseconds." + nl + "Leave empty to reset the interval.") .withRequiredArg() .describedAs("push interval") - .ofType(java.lang.Integer.class); + .ofType(String.class); - String nl = System.lineSeparator(); String[] matchSelectors = new String[] { "client_id", "client_instance_id", "client_software_name", "client_software_version", "client_source_address", "client_source_port" }; String matchSelectorNames = Arrays.stream(matchSelectors).map(config -> "\t" + config).collect(Collectors.joining(nl)); - matchOpt = parser.accepts("match", "Matching selector 'k1=v1,k2=v2'. The following is a list of valid selector names: " + nl + matchSelectorNames) + matchOpt = parser.accepts("match", "Matching selector 'k1=v1,k2=v2'. The following is a list of valid selector names: " + nl + matchSelectorNames) .withRequiredArg() .describedAs("k1=v1,k2=v2") .ofType(String.class) .withValuesSeparatedBy(','); - metricsOpt = parser.accepts("metrics", "Telemetry metric name prefixes 'm1,m2'.") + metricsOpt = parser.accepts("metrics", "Telemetry metric name prefixes 'm1,m2'.") .withRequiredArg() .describedAs("m1,m2") .ofType(String.class) @@ -329,7 +330,7 @@ public Optional> metrics() { return valuesAsOption(metricsOpt); } - public Optional interval() { + public Optional interval() { return valueAsOption(intervalOpt); } @@ -362,6 +363,18 @@ public void checkArgs() { if (has(alterOpt)) { if ((isNamePresent && has(generateNameOpt)) || (!isNamePresent && !has(generateNameOpt))) throw new IllegalArgumentException("One of --name or --generate-name must be specified with --alter."); + + interval().ifPresent(intervalStr -> { + if (!intervalStr.isEmpty()) { + try { + Integer.parseInt(intervalStr); + } catch (NumberFormatException e) { + throw new IllegalArgumentException( + "Invalid interval value. Enter an integer, or leave empty to reset."); + } + } + + }); } if (has(deleteOpt) && !isNamePresent) diff --git a/tools/src/test/java/org/apache/kafka/tools/ClientMetricsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/ClientMetricsCommandTest.java index 804c70fd07..c58748bf3c 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ClientMetricsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ClientMetricsCommandTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.AdminClientTestUtils; +import org.apache.kafka.clients.admin.AlterConfigOp; import org.apache.kafka.clients.admin.AlterConfigsResult; import org.apache.kafka.clients.admin.Config; import org.apache.kafka.clients.admin.ConfigEntry; @@ -29,11 +30,15 @@ import org.apache.kafka.common.utils.Exit; import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import java.util.Collection; import java.util.Collections; +import java.util.Map; import java.util.concurrent.ExecutionException; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; @@ -114,6 +119,14 @@ public void testOptionsAlterAllOptionsSucceeds() { } + @Test + public void testOptionsAlterInvalidInterval() { + Exception exception = assertThrows(IllegalArgumentException.class, () -> new ClientMetricsCommand.ClientMetricsCommandOptions( + new String[]{"--bootstrap-server", bootstrapServer, "--alter", "--name", clientMetricsName, + "--interval", "abc"})); + assertEquals("Invalid interval value. Enter an integer, or leave empty to reset.", exception.getMessage()); + } + @Test public void testAlter() { Admin adminClient = mock(Admin.class); @@ -156,6 +169,39 @@ public void testAlterGenerateName() { assertTrue(capturedOutput.contains("Altered client metrics config")); } + @Test + public void testAlterResetConfigs() { + Admin adminClient = mock(Admin.class); + ClientMetricsCommand.ClientMetricsService service = new ClientMetricsCommand.ClientMetricsService(adminClient); + + AlterConfigsResult result = AdminClientTestUtils.alterConfigsResult(new ConfigResource(ConfigResource.Type.CLIENT_METRICS, clientMetricsName)); + @SuppressWarnings("unchecked") + final ArgumentCaptor>> configCaptor = ArgumentCaptor.forClass(Map.class); + when(adminClient.incrementalAlterConfigs(configCaptor.capture(), any())).thenReturn(result); + + String capturedOutput = ToolsTestUtils.captureStandardOut(() -> { + try { + service.alterClientMetrics(new ClientMetricsCommand.ClientMetricsCommandOptions( + new String[]{"--bootstrap-server", bootstrapServer, "--alter", + "--name", clientMetricsName, "--metrics", "", + "--interval", "", "--match", ""})); + } catch (Throwable t) { + fail(t); + } + }); + Map> alteredConfigOps = configCaptor.getValue(); + assertNotNull(alteredConfigOps, "alteredConfigOps should not be null"); + assertEquals(1, alteredConfigOps.size(), "Should have exactly one ConfigResource"); + assertEquals(3, alteredConfigOps.values().iterator().next().size(), "Should have exactly 3 operations"); + for (Collection operations : alteredConfigOps.values()) { + for (AlterConfigOp op : operations) { + assertEquals(AlterConfigOp.OpType.DELETE, op.opType(), + "Expected DELETE operation for config: " + op.configEntry().name()); + } + } + assertTrue(capturedOutput.contains("Altered client metrics config for " + clientMetricsName + ".")); + } + @Test public void testDelete() { Admin adminClient = mock(Admin.class); From 4dbe4739bd37fa88154e73a3e44c8cc9c03134b2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Tue, 25 Feb 2025 20:09:19 -0500 Subject: [PATCH 11/72] KAFKA-18723; Better handle invalid records during replication (#18852) For the KRaft implementation there is a race between the network thread, which read bytes in the log segments, and the KRaft driver thread, which truncates the log and appends records to the log. This race can cause the network thread to send corrupted records or inconsistent records. The corrupted records case is handle by catching and logging the CorruptRecordException. The inconsistent records case is handle by only appending record batches who's partition leader epoch is less than or equal to the fetching replica's epoch and the epoch didn't change between the request and response. For the ISR implementation there is also a race between the network thread and the replica fetcher thread, which truncates the log and appends records to the log. This race can cause the network thread send corrupted records or inconsistent records. The replica fetcher thread already handles the corrupted record case. The inconsistent records case is handle by only appending record batches who's partition leader epoch is less than or equal to the leader epoch in the FETCH request. Reviewers: Jun Rao , Alyssa Huang , Chia-Ping Tsai --- build.gradle | 8 + .../common/record/DefaultRecordBatch.java | 3 +- .../kafka/common/record/MemoryRecords.java | 6 +- .../common/record/ArbitraryMemoryRecords.java | 39 ++ .../record/InvalidMemoryRecordsProvider.java | 132 +++++++ .../main/scala/kafka/cluster/Partition.scala | 20 +- .../src/main/scala/kafka/log/UnifiedLog.scala | 137 ++++--- .../scala/kafka/raft/KafkaMetadataLog.scala | 18 +- .../kafka/server/AbstractFetcherThread.scala | 21 +- .../server/ReplicaAlterLogDirsThread.scala | 11 +- .../kafka/server/ReplicaFetcherThread.scala | 11 +- .../kafka/raft/KafkaMetadataLogTest.scala | 95 ++++- .../unit/kafka/cluster/PartitionTest.scala | 108 +++-- .../scala/unit/kafka/log/LogCleanerTest.scala | 22 +- .../unit/kafka/log/LogConcurrencyTest.scala | 11 +- .../scala/unit/kafka/log/LogLoaderTest.scala | 32 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 371 +++++++++++++----- .../server/AbstractFetcherManagerTest.scala | 9 +- .../server/AbstractFetcherThreadTest.scala | 85 +++- .../unit/kafka/server/MockFetcherThread.scala | 35 +- .../server/ReplicaFetcherThreadTest.scala | 33 +- .../kafka/server/ReplicaManagerTest.scala | 9 +- .../ReplicaFetcherThreadBenchmark.java | 8 +- .../PartitionMakeFollowerBenchmark.java | 2 +- .../apache/kafka/raft/KafkaRaftClient.java | 48 ++- .../org/apache/kafka/raft/ReplicatedLog.java | 9 +- .../kafka/raft/KafkaRaftClientFetchTest.java | 152 +++++++ .../java/org/apache/kafka/raft/MockLog.java | 46 ++- .../org/apache/kafka/raft/MockLogTest.java | 125 +++++- 29 files changed, 1302 insertions(+), 304 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/common/record/ArbitraryMemoryRecords.java create mode 100644 clients/src/test/java/org/apache/kafka/common/record/InvalidMemoryRecordsProvider.java create mode 100644 raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientFetchTest.java diff --git a/build.gradle b/build.gradle index 388a85aa85..1c59669c37 100644 --- a/build.gradle +++ b/build.gradle @@ -1110,6 +1110,7 @@ project(':core') { testImplementation project(':test-common:test-common-util') testImplementation libs.bcpkix testImplementation libs.mockitoCore + testImplementation libs.jqwik testImplementation(libs.apacheda) { exclude group: 'xml-apis', module: 'xml-apis' // `mina-core` is a transitive dependency for `apacheds` and `apacheda`. @@ -1304,6 +1305,12 @@ project(':core') { ) } + test { + useJUnitPlatform { + includeEngines 'jqwik', 'junit-jupiter' + } + } + tasks.create(name: "copyDependantTestLibs", type: Copy) { from (configurations.testRuntimeClasspath) { include('*.jar') @@ -1871,6 +1878,7 @@ project(':clients') { testImplementation libs.jacksonJakartarsJsonProvider testImplementation libs.jose4j testImplementation libs.junitJupiter + testImplementation libs.jqwik testImplementation libs.spotbugs testImplementation libs.mockitoCore testImplementation libs.mockitoJunitJupiter // supports MockitoExtension diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java index 912c3490f4..d6e9cc6bd7 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java @@ -159,7 +159,7 @@ public void ensureValid() { /** * Gets the base timestamp of the batch which is used to calculate the record timestamps from the deltas. - * + * * @return The base timestamp */ public long baseTimestamp() { @@ -502,6 +502,7 @@ public static void writeHeader(ByteBuffer buffer, public String toString() { return "RecordBatch(magic=" + magic() + ", offsets=[" + baseOffset() + ", " + lastOffset() + "], " + "sequence=[" + baseSequence() + ", " + lastSequence() + "], " + + "partitionLeaderEpoch=" + partitionLeaderEpoch() + ", " + "isTransactional=" + isTransactional() + ", isControlBatch=" + isControlBatch() + ", " + "compression=" + compressionType() + ", timestampType=" + timestampType() + ", crc=" + checksum() + ")"; } 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 3aee889ade..17334f89c2 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 @@ -33,9 +33,6 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.GatheringByteChannel; @@ -50,7 +47,6 @@ * or one of the {@link #builder(ByteBuffer, byte, Compression, TimestampType, long)} variants. */ public class MemoryRecords extends AbstractRecords { - private static final Logger log = LoggerFactory.getLogger(MemoryRecords.class); public static final MemoryRecords EMPTY = MemoryRecords.readableRecords(ByteBuffer.allocate(0)); private final ByteBuffer buffer; @@ -602,7 +598,7 @@ public static MemoryRecords withRecords(byte magic, long initialOffset, Compress return withRecords(magic, initialOffset, compression, TimestampType.CREATE_TIME, records); } - public static MemoryRecords withRecords(long initialOffset, Compression compression, Integer partitionLeaderEpoch, SimpleRecord... records) { + public static MemoryRecords withRecords(long initialOffset, Compression compression, int partitionLeaderEpoch, SimpleRecord... records) { return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, initialOffset, compression, TimestampType.CREATE_TIME, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, partitionLeaderEpoch, false, records); } diff --git a/clients/src/test/java/org/apache/kafka/common/record/ArbitraryMemoryRecords.java b/clients/src/test/java/org/apache/kafka/common/record/ArbitraryMemoryRecords.java new file mode 100644 index 0000000000..30eec866a6 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/record/ArbitraryMemoryRecords.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.record; + +import net.jqwik.api.Arbitraries; +import net.jqwik.api.Arbitrary; +import net.jqwik.api.ArbitrarySupplier; + +import java.nio.ByteBuffer; +import java.util.Random; + +public final class ArbitraryMemoryRecords implements ArbitrarySupplier { + @Override + public Arbitrary get() { + return Arbitraries.randomValue(ArbitraryMemoryRecords::buildRandomRecords); + } + + private static MemoryRecords buildRandomRecords(Random random) { + int size = random.nextInt(128) + 1; + byte[] bytes = new byte[size]; + random.nextBytes(bytes); + + return MemoryRecords.readableRecords(ByteBuffer.wrap(bytes)); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/record/InvalidMemoryRecordsProvider.java b/clients/src/test/java/org/apache/kafka/common/record/InvalidMemoryRecordsProvider.java new file mode 100644 index 0000000000..0f9446a639 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/record/InvalidMemoryRecordsProvider.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.record; + +import org.apache.kafka.common.errors.CorruptRecordException; + +import org.junit.jupiter.api.extension.ExtensionContext; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.ArgumentsProvider; + +import java.nio.ByteBuffer; +import java.util.Optional; +import java.util.stream.Stream; + +public final class InvalidMemoryRecordsProvider implements ArgumentsProvider { + // Use a baseOffset that's not zero so that it is less likely to match the LEO + private static final long BASE_OFFSET = 1234; + private static final int EPOCH = 4321; + + /** + * Returns a stream of arguments for invalid memory records and the expected exception. + * + * The first object in the {@code Arguments} is a {@code MemoryRecords}. + * + * The second object in the {@code Arguments} is an {@code Optional>} which is + * the expected exception from the log layer. + */ + @Override + public Stream provideArguments(ExtensionContext context) { + return Stream.of( + Arguments.of(MemoryRecords.readableRecords(notEnoughBytes()), Optional.empty()), + Arguments.of(MemoryRecords.readableRecords(recordsSizeTooSmall()), Optional.of(CorruptRecordException.class)), + Arguments.of(MemoryRecords.readableRecords(notEnoughBytesToMagic()), Optional.empty()), + Arguments.of(MemoryRecords.readableRecords(negativeMagic()), Optional.of(CorruptRecordException.class)), + Arguments.of(MemoryRecords.readableRecords(largeMagic()), Optional.of(CorruptRecordException.class)), + Arguments.of(MemoryRecords.readableRecords(lessBytesThanRecordSize()), Optional.empty()) + ); + } + + private static ByteBuffer notEnoughBytes() { + var buffer = ByteBuffer.allocate(Records.LOG_OVERHEAD - 1); + buffer.limit(buffer.capacity()); + + return buffer; + } + + private static ByteBuffer recordsSizeTooSmall() { + var buffer = ByteBuffer.allocate(256); + // Write the base offset + buffer.putLong(BASE_OFFSET); + // Write record size + buffer.putInt(LegacyRecord.RECORD_OVERHEAD_V0 - 1); + buffer.position(0); + buffer.limit(buffer.capacity()); + + return buffer; + } + + private static ByteBuffer notEnoughBytesToMagic() { + var buffer = ByteBuffer.allocate(256); + // Write the base offset + buffer.putLong(BASE_OFFSET); + // Write record size + buffer.putInt(buffer.capacity() - Records.LOG_OVERHEAD); + buffer.position(0); + buffer.limit(Records.HEADER_SIZE_UP_TO_MAGIC - 1); + + return buffer; + } + + private static ByteBuffer negativeMagic() { + var buffer = ByteBuffer.allocate(256); + // Write the base offset + buffer.putLong(BASE_OFFSET); + // Write record size + buffer.putInt(buffer.capacity() - Records.LOG_OVERHEAD); + // Write the epoch + buffer.putInt(EPOCH); + // Write magic + buffer.put((byte) -1); + buffer.position(0); + buffer.limit(buffer.capacity()); + + return buffer; + } + + private static ByteBuffer largeMagic() { + var buffer = ByteBuffer.allocate(256); + // Write the base offset + buffer.putLong(BASE_OFFSET); + // Write record size + buffer.putInt(buffer.capacity() - Records.LOG_OVERHEAD); + // Write the epoch + buffer.putInt(EPOCH); + // Write magic + buffer.put((byte) (RecordBatch.CURRENT_MAGIC_VALUE + 1)); + buffer.position(0); + buffer.limit(buffer.capacity()); + + return buffer; + } + + private static ByteBuffer lessBytesThanRecordSize() { + var buffer = ByteBuffer.allocate(256); + // Write the base offset + buffer.putLong(BASE_OFFSET); + // Write record size + buffer.putInt(buffer.capacity() - Records.LOG_OVERHEAD); + // Write the epoch + buffer.putInt(EPOCH); + // Write magic + buffer.put(RecordBatch.CURRENT_MAGIC_VALUE); + buffer.position(0); + buffer.limit(buffer.capacity() - Records.LOG_OVERHEAD - 1); + + return buffer; + } +} diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 7ec0904cb0..66cc6eaabb 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -1307,27 +1307,35 @@ class Partition(val topicPartition: TopicPartition, } } - private def doAppendRecordsToFollowerOrFutureReplica(records: MemoryRecords, isFuture: Boolean): Option[LogAppendInfo] = { + private def doAppendRecordsToFollowerOrFutureReplica( + records: MemoryRecords, + isFuture: Boolean, + partitionLeaderEpoch: Int + ): Option[LogAppendInfo] = { if (isFuture) { // The read lock is needed to handle race condition if request handler thread tries to // remove future replica after receiving AlterReplicaLogDirsRequest. inReadLock(leaderIsrUpdateLock) { // Note the replica may be undefined if it is removed by a non-ReplicaAlterLogDirsThread before // this method is called - futureLog.map { _.appendAsFollower(records) } + futureLog.map { _.appendAsFollower(records, partitionLeaderEpoch) } } } else { // The lock is needed to prevent the follower replica from being updated while ReplicaAlterDirThread // is executing maybeReplaceCurrentWithFutureReplica() to replace follower replica with the future replica. futureLogLock.synchronized { - Some(localLogOrException.appendAsFollower(records)) + Some(localLogOrException.appendAsFollower(records, partitionLeaderEpoch)) } } } - def appendRecordsToFollowerOrFutureReplica(records: MemoryRecords, isFuture: Boolean): Option[LogAppendInfo] = { + def appendRecordsToFollowerOrFutureReplica( + records: MemoryRecords, + isFuture: Boolean, + partitionLeaderEpoch: Int + ): Option[LogAppendInfo] = { try { - doAppendRecordsToFollowerOrFutureReplica(records, isFuture) + doAppendRecordsToFollowerOrFutureReplica(records, isFuture, partitionLeaderEpoch) } catch { case e: UnexpectedAppendOffsetException => val log = if (isFuture) futureLocalLogOrException else localLogOrException @@ -1345,7 +1353,7 @@ class Partition(val topicPartition: TopicPartition, info(s"Unexpected offset in append to $topicPartition. First offset ${e.firstOffset} is less than log start offset ${log.logStartOffset}." + s" Since this is the first record to be appended to the $replicaName's log, will start the log from offset ${e.firstOffset}.") truncateFullyAndStartAt(e.firstOffset, isFuture) - doAppendRecordsToFollowerOrFutureReplica(records, isFuture) + doAppendRecordsToFollowerOrFutureReplica(records, isFuture, partitionLeaderEpoch) } else throw e } diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index 3c1a157087..e9985f619f 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -698,6 +698,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, * Append this message set to the active segment of the local log, assigning offsets and Partition Leader Epochs * * @param records The records to append + * @param leaderEpoch the epoch of the replica appending * @param origin Declares the origin of the append which affects required validations * @param requestLocal request local instance * @throws KafkaStorageException If the append fails due to an I/O error. @@ -728,14 +729,15 @@ class UnifiedLog(@volatile var logStartOffset: Long, * Append this message set to the active segment of the local log without assigning offsets or Partition Leader Epochs * * @param records The records to append + * @param leaderEpoch the epoch of the replica appending * @throws KafkaStorageException If the append fails due to an I/O error. * @return Information about the appended messages including the first and last offset. */ - def appendAsFollower(records: MemoryRecords): LogAppendInfo = { + def appendAsFollower(records: MemoryRecords, leaderEpoch: Int): LogAppendInfo = { append(records, origin = AppendOrigin.REPLICATION, validateAndAssignOffsets = false, - leaderEpoch = -1, + leaderEpoch = leaderEpoch, requestLocal = None, verificationGuard = VerificationGuard.SENTINEL, // disable to check the validation of record size since the record is already accepted by leader. @@ -1114,63 +1116,85 @@ class UnifiedLog(@volatile var logStartOffset: Long, var shallowOffsetOfMaxTimestamp = -1L var readFirstMessage = false var lastOffsetOfFirstBatch = -1L + var skipRemainingBatches = false records.batches.forEach { batch => if (origin == AppendOrigin.RAFT_LEADER && batch.partitionLeaderEpoch != leaderEpoch) { - throw new InvalidRecordException("Append from Raft leader did not set the batch epoch correctly") + throw new InvalidRecordException( + s"Append from Raft leader did not set the batch epoch correctly, expected $leaderEpoch " + + s"but the batch has ${batch.partitionLeaderEpoch}" + ) } // we only validate V2 and higher to avoid potential compatibility issues with older clients - if (batch.magic >= RecordBatch.MAGIC_VALUE_V2 && origin == AppendOrigin.CLIENT && batch.baseOffset != 0) + if (batch.magic >= RecordBatch.MAGIC_VALUE_V2 && origin == AppendOrigin.CLIENT && batch.baseOffset != 0) { throw new InvalidRecordException(s"The baseOffset of the record batch in the append to $topicPartition should " + s"be 0, but it is ${batch.baseOffset}") - - // update the first offset if on the first message. For magic versions older than 2, we use the last offset - // to avoid the need to decompress the data (the last offset can be obtained directly from the wrapper message). - // For magic version 2, we can get the first offset directly from the batch header. - // When appending to the leader, we will update LogAppendInfo.baseOffset with the correct value. In the follower - // case, validation will be more lenient. - // Also indicate whether we have the accurate first offset or not - if (!readFirstMessage) { - if (batch.magic >= RecordBatch.MAGIC_VALUE_V2) - firstOffset = batch.baseOffset - lastOffsetOfFirstBatch = batch.lastOffset - readFirstMessage = true } - // check that offsets are monotonically increasing - if (lastOffset >= batch.lastOffset) - monotonic = false - - // update the last offset seen - lastOffset = batch.lastOffset - lastLeaderEpoch = batch.partitionLeaderEpoch - - // Check if the message sizes are valid. - val batchSize = batch.sizeInBytes - if (!ignoreRecordSize && batchSize > config.maxMessageSize) { - brokerTopicStats.topicStats(topicPartition.topic).bytesRejectedRate.mark(records.sizeInBytes) - brokerTopicStats.allTopicsStats.bytesRejectedRate.mark(records.sizeInBytes) - throw new RecordTooLargeException(s"The record batch size in the append to $topicPartition is $batchSize bytes " + - s"which exceeds the maximum configured value of ${config.maxMessageSize}.") - } + /* During replication of uncommitted data it is possible for the remote replica to send record batches after it lost + * leadership. This can happen if sending FETCH responses is slow. There is a race between sending the FETCH + * response and the replica truncating and appending to the log. The replicating replica resolves this issue by only + * persisting up to the current leader epoch used in the fetch request. See KAFKA-18723 for more details. + */ + skipRemainingBatches = skipRemainingBatches || hasHigherPartitionLeaderEpoch(batch, origin, leaderEpoch) + if (skipRemainingBatches) { + info( + s"Skipping batch $batch from an origin of $origin because its partition leader epoch " + + s"${batch.partitionLeaderEpoch} is higher than the replica's current leader epoch " + + s"$leaderEpoch" + ) + } else { + // update the first offset if on the first message. For magic versions older than 2, we use the last offset + // to avoid the need to decompress the data (the last offset can be obtained directly from the wrapper message). + // For magic version 2, we can get the first offset directly from the batch header. + // When appending to the leader, we will update LogAppendInfo.baseOffset with the correct value. In the follower + // case, validation will be more lenient. + // Also indicate whether we have the accurate first offset or not + if (!readFirstMessage) { + if (batch.magic >= RecordBatch.MAGIC_VALUE_V2) { + firstOffset = batch.baseOffset + } + lastOffsetOfFirstBatch = batch.lastOffset + readFirstMessage = true + } - // check the validity of the message by checking CRC - if (!batch.isValid) { - brokerTopicStats.allTopicsStats.invalidMessageCrcRecordsPerSec.mark() - throw new CorruptRecordException(s"Record is corrupt (stored crc = ${batch.checksum()}) in topic partition $topicPartition.") - } + // check that offsets are monotonically increasing + if (lastOffset >= batch.lastOffset) { + monotonic = false + } - if (batch.maxTimestamp > maxTimestamp) { - maxTimestamp = batch.maxTimestamp - shallowOffsetOfMaxTimestamp = lastOffset - } + // update the last offset seen + lastOffset = batch.lastOffset + lastLeaderEpoch = batch.partitionLeaderEpoch + + // Check if the message sizes are valid. + val batchSize = batch.sizeInBytes + if (!ignoreRecordSize && batchSize > config.maxMessageSize) { + brokerTopicStats.topicStats(topicPartition.topic).bytesRejectedRate.mark(records.sizeInBytes) + brokerTopicStats.allTopicsStats.bytesRejectedRate.mark(records.sizeInBytes) + throw new RecordTooLargeException(s"The record batch size in the append to $topicPartition is $batchSize bytes " + + s"which exceeds the maximum configured value of ${config.maxMessageSize}.") + } - validBytesCount += batchSize + // check the validity of the message by checking CRC + if (!batch.isValid) { + brokerTopicStats.allTopicsStats.invalidMessageCrcRecordsPerSec.mark() + throw new CorruptRecordException(s"Record is corrupt (stored crc = ${batch.checksum()}) in topic partition $topicPartition.") + } - val batchCompression = CompressionType.forId(batch.compressionType.id) - // sourceCompression is only used on the leader path, which only contains one batch if version is v2 or messages are compressed - if (batchCompression != CompressionType.NONE) - sourceCompression = batchCompression + if (batch.maxTimestamp > maxTimestamp) { + maxTimestamp = batch.maxTimestamp + shallowOffsetOfMaxTimestamp = lastOffset + } + + validBytesCount += batchSize + + val batchCompression = CompressionType.forId(batch.compressionType.id) + // sourceCompression is only used on the leader path, which only contains one batch if version is v2 or messages are compressed + if (batchCompression != CompressionType.NONE) { + sourceCompression = batchCompression + } + } } if (requireOffsetsMonotonic && !monotonic) @@ -1187,6 +1211,25 @@ class UnifiedLog(@volatile var logStartOffset: Long, validBytesCount, lastOffsetOfFirstBatch, Collections.emptyList[RecordError], LeaderHwChange.NONE) } + /** + * Return true if the record batch has a higher leader epoch than the specified leader epoch + * + * @param batch the batch to validate + * @param origin the reason for appending the record batch + * @param leaderEpoch the epoch to compare + * @return true if the append reason is replication and the batch's partition leader epoch is + * greater than the specified leaderEpoch, otherwise false + */ + private def hasHigherPartitionLeaderEpoch( + batch: RecordBatch, + origin: AppendOrigin, + leaderEpoch: Int + ): Boolean = { + origin == AppendOrigin.REPLICATION && + batch.partitionLeaderEpoch() != RecordBatch.NO_PARTITION_LEADER_EPOCH && + batch.partitionLeaderEpoch() > leaderEpoch + } + /** * Trim any invalid bytes from the end of this message set (if there are any) * @@ -1326,7 +1369,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, val asyncOffsetReadFutureHolder = remoteLogManager.get.asyncOffsetRead(topicPartition, targetTimestamp, logStartOffset, leaderEpochCache, () => searchOffsetInLocalLog(targetTimestamp, localLogStartOffset())) - + new OffsetResultHolder(Optional.empty(), Optional.of(asyncOffsetReadFutureHolder)) } else { new OffsetResultHolder(searchOffsetInLocalLog(targetTimestamp, logStartOffset).toJava) diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index bd80c0aca4..997376c471 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -25,6 +25,7 @@ import kafka.raft.KafkaMetadataLog.UnknownReason import kafka.utils.Logging import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.InvalidConfigurationException +import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.record.{MemoryRecords, Records} import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} @@ -89,8 +90,9 @@ final class KafkaMetadataLog private ( } override def appendAsLeader(records: Records, epoch: Int): LogAppendInfo = { - if (records.sizeInBytes == 0) + if (records.sizeInBytes == 0) { throw new IllegalArgumentException("Attempt to append an empty record set") + } handleAndConvertLogAppendInfo( log.appendAsLeader(records.asInstanceOf[MemoryRecords], @@ -101,18 +103,20 @@ final class KafkaMetadataLog private ( ) } - override def appendAsFollower(records: Records): LogAppendInfo = { - if (records.sizeInBytes == 0) + override def appendAsFollower(records: Records, epoch: Int): LogAppendInfo = { + if (records.sizeInBytes == 0) { throw new IllegalArgumentException("Attempt to append an empty record set") + } - handleAndConvertLogAppendInfo(log.appendAsFollower(records.asInstanceOf[MemoryRecords])) + handleAndConvertLogAppendInfo(log.appendAsFollower(records.asInstanceOf[MemoryRecords], epoch)) } private def handleAndConvertLogAppendInfo(appendInfo: internals.log.LogAppendInfo): LogAppendInfo = { - if (appendInfo.firstOffset != UnifiedLog.UnknownOffset) + if (appendInfo.firstOffset == UnifiedLog.UnknownOffset) { + throw new CorruptRecordException(s"Append failed unexpectedly $appendInfo") + } else { new LogAppendInfo(appendInfo.firstOffset, appendInfo.lastOffset) - else - throw new KafkaException(s"Append failed unexpectedly") + } } override def lastFetchedEpoch: Int = { diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index be663d19ec..7a98c83e7f 100755 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -78,9 +78,12 @@ abstract class AbstractFetcherThread(name: String, /* callbacks to be defined in subclass */ // process fetched data - protected def processPartitionData(topicPartition: TopicPartition, - fetchOffset: Long, - partitionData: FetchData): Option[LogAppendInfo] + protected def processPartitionData( + topicPartition: TopicPartition, + fetchOffset: Long, + partitionLeaderEpoch: Int, + partitionData: FetchData + ): Option[LogAppendInfo] protected def truncate(topicPartition: TopicPartition, truncationState: OffsetTruncationState): Unit @@ -333,7 +336,9 @@ abstract class AbstractFetcherThread(name: String, // In this case, we only want to process the fetch response if the partition state is ready for fetch and // the current offset is the same as the offset requested. val fetchPartitionData = sessionPartitions.get(topicPartition) - if (fetchPartitionData != null && fetchPartitionData.fetchOffset == currentFetchState.fetchOffset && currentFetchState.isReadyForFetch) { + if (fetchPartitionData != null && + fetchPartitionData.fetchOffset == currentFetchState.fetchOffset && + currentFetchState.isReadyForFetch) { Errors.forCode(partitionData.errorCode) match { case Errors.NONE => try { @@ -348,10 +353,16 @@ abstract class AbstractFetcherThread(name: String, .setLeaderEpoch(partitionData.divergingEpoch.epoch) .setEndOffset(partitionData.divergingEpoch.endOffset) } else { - // Once we hand off the partition data to the subclass, we can't mess with it any more in this thread + /* Once we hand off the partition data to the subclass, we can't mess with it any more in this thread + * + * When appending batches to the log only append record batches up to the leader epoch when the FETCH + * request was handled. This is done to make sure that logs are not inconsistent because of log + * truncation and append after the FETCH request was handled. See KAFKA-18723 for more details. + */ val logAppendInfoOpt = processPartitionData( topicPartition, currentFetchState.fetchOffset, + fetchPartitionData.currentLeaderEpoch.orElse(currentFetchState.currentLeaderEpoch), partitionData ) diff --git a/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala b/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala index 56492de348..5f5373b364 100644 --- a/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala @@ -66,9 +66,12 @@ class ReplicaAlterLogDirsThread(name: String, } // process fetched data - override def processPartitionData(topicPartition: TopicPartition, - fetchOffset: Long, - partitionData: FetchData): Option[LogAppendInfo] = { + override def processPartitionData( + topicPartition: TopicPartition, + fetchOffset: Long, + partitionLeaderEpoch: Int, + partitionData: FetchData + ): Option[LogAppendInfo] = { val partition = replicaMgr.getPartitionOrException(topicPartition) val futureLog = partition.futureLocalLogOrException val records = toMemoryRecords(FetchResponse.recordsOrFail(partitionData)) @@ -78,7 +81,7 @@ class ReplicaAlterLogDirsThread(name: String, topicPartition, fetchOffset, futureLog.logEndOffset)) val logAppendInfo = if (records.sizeInBytes() > 0) - partition.appendRecordsToFollowerOrFutureReplica(records, isFuture = true) + partition.appendRecordsToFollowerOrFutureReplica(records, isFuture = true, partitionLeaderEpoch) else None diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 7f0c6d41db..4c11301c56 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -98,9 +98,12 @@ class ReplicaFetcherThread(name: String, } // process fetched data - override def processPartitionData(topicPartition: TopicPartition, - fetchOffset: Long, - partitionData: FetchData): Option[LogAppendInfo] = { + override def processPartitionData( + topicPartition: TopicPartition, + fetchOffset: Long, + partitionLeaderEpoch: Int, + partitionData: FetchData + ): Option[LogAppendInfo] = { val logTrace = isTraceEnabled val partition = replicaMgr.getPartitionOrException(topicPartition) val log = partition.localLogOrException @@ -117,7 +120,7 @@ class ReplicaFetcherThread(name: String, .format(log.logEndOffset, topicPartition, records.sizeInBytes, partitionData.highWatermark)) // Append the leader's messages to the log - val logAppendInfo = partition.appendRecordsToFollowerOrFutureReplica(records, isFuture = false) + val logAppendInfo = partition.appendRecordsToFollowerOrFutureReplica(records, isFuture = false, partitionLeaderEpoch) if (logTrace) trace("Follower has replica log end offset %d after appending %d bytes of messages for partition %s" diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index 285560d382..8cca9202eb 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -20,9 +20,12 @@ import kafka.log.UnifiedLog import kafka.server.{KafkaConfig, KafkaRaftServer} import kafka.utils.TestUtils import org.apache.kafka.common.compress.Compression +import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.errors.{InvalidConfigurationException, RecordTooLargeException} import org.apache.kafka.common.protocol import org.apache.kafka.common.protocol.{ObjectSerializationCache, Writable} +import org.apache.kafka.common.record.ArbitraryMemoryRecords +import org.apache.kafka.common.record.InvalidMemoryRecordsProvider import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord} import org.apache.kafka.common.utils.Utils import org.apache.kafka.raft._ @@ -34,7 +37,14 @@ import org.apache.kafka.snapshot.{FileRawSnapshotWriter, RawSnapshotReader, RawS import org.apache.kafka.storage.internals.log.{LogConfig, LogStartOffsetIncrementReason} import org.apache.kafka.test.TestUtils.assertOptional import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.function.Executable import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ArgumentsSource + +import net.jqwik.api.AfterFailureMode +import net.jqwik.api.ForAll +import net.jqwik.api.Property import java.io.File import java.nio.ByteBuffer @@ -109,12 +119,93 @@ final class KafkaMetadataLogTest { classOf[RuntimeException], () => { log.appendAsFollower( - MemoryRecords.withRecords(initialOffset, Compression.NONE, currentEpoch, recordFoo) + MemoryRecords.withRecords(initialOffset, Compression.NONE, currentEpoch, recordFoo), + currentEpoch ) } ) } + @Test + def testEmptyAppendNotAllowed(): Unit = { + val log = buildMetadataLog(tempDir, mockTime) + + assertThrows(classOf[IllegalArgumentException], () => log.appendAsFollower(MemoryRecords.EMPTY, 1)); + assertThrows(classOf[IllegalArgumentException], () => log.appendAsLeader(MemoryRecords.EMPTY, 1)); + } + + @ParameterizedTest + @ArgumentsSource(classOf[InvalidMemoryRecordsProvider]) + def testInvalidMemoryRecords(records: MemoryRecords, expectedException: Optional[Class[Exception]]): Unit = { + val log = buildMetadataLog(tempDir, mockTime) + val previousEndOffset = log.endOffset().offset() + + val action: Executable = () => log.appendAsFollower(records, Int.MaxValue) + if (expectedException.isPresent()) { + assertThrows(expectedException.get, action) + } else { + assertThrows(classOf[CorruptRecordException], action) + } + + assertEquals(previousEndOffset, log.endOffset().offset()) + } + + @Property(tries = 100, afterFailure = AfterFailureMode.SAMPLE_ONLY) + def testRandomRecords( + @ForAll(supplier = classOf[ArbitraryMemoryRecords]) records: MemoryRecords + ): Unit = { + val tempDir = TestUtils.tempDir() + try { + val log = buildMetadataLog(tempDir, mockTime) + val previousEndOffset = log.endOffset().offset() + + assertThrows( + classOf[CorruptRecordException], + () => log.appendAsFollower(records, Int.MaxValue) + ) + + assertEquals(previousEndOffset, log.endOffset().offset()) + } finally { + Utils.delete(tempDir) + } + } + + @Test + def testInvalidLeaderEpoch(): Unit = { + val log = buildMetadataLog(tempDir, mockTime) + val previousEndOffset = log.endOffset().offset() + val epoch = log.lastFetchedEpoch() + 1 + val numberOfRecords = 10 + + val batchWithValidEpoch = MemoryRecords.withRecords( + previousEndOffset, + Compression.NONE, + epoch, + (0 until numberOfRecords).map(number => new SimpleRecord(number.toString.getBytes)): _* + ) + + val batchWithInvalidEpoch = MemoryRecords.withRecords( + previousEndOffset + numberOfRecords, + Compression.NONE, + epoch + 1, + (0 until numberOfRecords).map(number => new SimpleRecord(number.toString.getBytes)): _* + ) + + val buffer = ByteBuffer.allocate(batchWithValidEpoch.sizeInBytes() + batchWithInvalidEpoch.sizeInBytes()) + buffer.put(batchWithValidEpoch.buffer()) + buffer.put(batchWithInvalidEpoch.buffer()) + buffer.flip() + + val records = MemoryRecords.readableRecords(buffer) + + log.appendAsFollower(records, epoch) + + // Check that only the first batch was appended + assertEquals(previousEndOffset + numberOfRecords, log.endOffset().offset()) + // Check that the last fetched epoch matches the first batch + assertEquals(epoch, log.lastFetchedEpoch()) + } + @Test def testCreateSnapshot(): Unit = { val numberOfRecords = 10 @@ -1062,4 +1153,4 @@ object KafkaMetadataLogTest { } dir } -} \ No newline at end of file +} diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index b559189f39..c10f5aab42 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -426,6 +426,7 @@ class PartitionTest extends AbstractPartitionTest { def testMakeFollowerWithWithFollowerAppendRecords(): Unit = { val appendSemaphore = new Semaphore(0) val mockTime = new MockTime() + val prevLeaderEpoch = 0 partition = new Partition( topicPartition, @@ -478,24 +479,38 @@ class PartitionTest extends AbstractPartitionTest { } partition.createLogIfNotExists(isNew = true, isFutureReplica = false, offsetCheckpoints, None) + var partitionState = new LeaderAndIsrRequest.PartitionState() + .setControllerEpoch(0) + .setLeader(2) + .setLeaderEpoch(prevLeaderEpoch) + .setIsr(List[Integer](0, 1, 2, brokerId).asJava) + .setPartitionEpoch(1) + .setReplicas(List[Integer](0, 1, 2, brokerId).asJava) + .setIsNew(false) + assertTrue(partition.makeFollower(partitionState, offsetCheckpoints, None)) val appendThread = new Thread { override def run(): Unit = { - val records = createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes), - new SimpleRecord("k2".getBytes, "v2".getBytes)), - baseOffset = 0) - partition.appendRecordsToFollowerOrFutureReplica(records, isFuture = false) + val records = createRecords( + List( + new SimpleRecord("k1".getBytes, "v1".getBytes), + new SimpleRecord("k2".getBytes, "v2".getBytes) + ), + baseOffset = 0, + partitionLeaderEpoch = prevLeaderEpoch + ) + partition.appendRecordsToFollowerOrFutureReplica(records, isFuture = false, prevLeaderEpoch) } } appendThread.start() TestUtils.waitUntilTrue(() => appendSemaphore.hasQueuedThreads, "follower log append is not called.") - val partitionState = new LeaderAndIsrRequest.PartitionState() + partitionState = new LeaderAndIsrRequest.PartitionState() .setControllerEpoch(0) .setLeader(2) - .setLeaderEpoch(1) + .setLeaderEpoch(prevLeaderEpoch + 1) .setIsr(List[Integer](0, 1, 2, brokerId).asJava) - .setPartitionEpoch(1) + .setPartitionEpoch(2) .setReplicas(List[Integer](0, 1, 2, brokerId).asJava) .setIsNew(false) assertTrue(partition.makeFollower(partitionState, offsetCheckpoints, None)) @@ -535,15 +550,22 @@ class PartitionTest extends AbstractPartitionTest { // Write to the future replica as if the log had been compacted, and do not roll the segment val buffer = ByteBuffer.allocate(1024) - val builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE, - TimestampType.CREATE_TIME, 0L, RecordBatch.NO_TIMESTAMP, 0) + val builder = MemoryRecords.builder( + buffer, + RecordBatch.CURRENT_MAGIC_VALUE, + Compression.NONE, + TimestampType.CREATE_TIME, + 0L, // baseOffset + RecordBatch.NO_TIMESTAMP, + 0 // partitionLeaderEpoch + ) builder.appendWithOffset(2L, new SimpleRecord("k1".getBytes, "v3".getBytes)) builder.appendWithOffset(5L, new SimpleRecord("k2".getBytes, "v6".getBytes)) builder.appendWithOffset(6L, new SimpleRecord("k3".getBytes, "v7".getBytes)) builder.appendWithOffset(7L, new SimpleRecord("k4".getBytes, "v8".getBytes)) val futureLog = partition.futureLocalLogOrException - futureLog.appendAsFollower(builder.build()) + futureLog.appendAsFollower(builder.build(), 0) assertTrue(partition.maybeReplaceCurrentWithFutureReplica()) } @@ -951,6 +973,18 @@ class PartitionTest extends AbstractPartitionTest { def testAppendRecordsAsFollowerBelowLogStartOffset(): Unit = { partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None) val log = partition.localLogOrException + val epoch = 1 + + // Start off as follower + val partitionState = new LeaderAndIsrRequest.PartitionState() + .setControllerEpoch(0) + .setLeader(1) + .setLeaderEpoch(epoch) + .setIsr(List[Integer](0, 1, 2, brokerId).asJava) + .setPartitionEpoch(1) + .setReplicas(List[Integer](0, 1, 2, brokerId).asJava) + .setIsNew(false) + partition.makeFollower(partitionState, offsetCheckpoints, None) val initialLogStartOffset = 5L partition.truncateFullyAndStartAt(initialLogStartOffset, isFuture = false) @@ -960,9 +994,14 @@ class PartitionTest extends AbstractPartitionTest { s"Log start offset after truncate fully and start at $initialLogStartOffset:") // verify that we cannot append records that do not contain log start offset even if the log is empty - assertThrows(classOf[UnexpectedAppendOffsetException], () => + assertThrows( + classOf[UnexpectedAppendOffsetException], // append one record with offset = 3 - partition.appendRecordsToFollowerOrFutureReplica(createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes)), baseOffset = 3L), isFuture = false) + () => partition.appendRecordsToFollowerOrFutureReplica( + createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes)), baseOffset = 3L), + isFuture = false, + partitionLeaderEpoch = epoch + ) ) assertEquals(initialLogStartOffset, log.logEndOffset, s"Log end offset should not change after failure to append") @@ -974,12 +1013,16 @@ class PartitionTest extends AbstractPartitionTest { new SimpleRecord("k2".getBytes, "v2".getBytes), new SimpleRecord("k3".getBytes, "v3".getBytes)), baseOffset = newLogStartOffset) - partition.appendRecordsToFollowerOrFutureReplica(records, isFuture = false) + partition.appendRecordsToFollowerOrFutureReplica(records, isFuture = false, partitionLeaderEpoch = epoch) assertEquals(7L, log.logEndOffset, s"Log end offset after append of 3 records with base offset $newLogStartOffset:") assertEquals(newLogStartOffset, log.logStartOffset, s"Log start offset after append of 3 records with base offset $newLogStartOffset:") // and we can append more records after that - partition.appendRecordsToFollowerOrFutureReplica(createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes)), baseOffset = 7L), isFuture = false) + partition.appendRecordsToFollowerOrFutureReplica( + createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes)), baseOffset = 7L), + isFuture = false, + partitionLeaderEpoch = epoch + ) assertEquals(8L, log.logEndOffset, s"Log end offset after append of 1 record at offset 7:") assertEquals(newLogStartOffset, log.logStartOffset, s"Log start offset not expected to change:") @@ -987,11 +1030,18 @@ class PartitionTest extends AbstractPartitionTest { val records2 = createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes), new SimpleRecord("k2".getBytes, "v2".getBytes)), baseOffset = 3L) - assertThrows(classOf[UnexpectedAppendOffsetException], () => partition.appendRecordsToFollowerOrFutureReplica(records2, isFuture = false)) + assertThrows( + classOf[UnexpectedAppendOffsetException], + () => partition.appendRecordsToFollowerOrFutureReplica(records2, isFuture = false, partitionLeaderEpoch = epoch) + ) assertEquals(8L, log.logEndOffset, s"Log end offset should not change after failure to append") // we still can append to next offset - partition.appendRecordsToFollowerOrFutureReplica(createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes)), baseOffset = 8L), isFuture = false) + partition.appendRecordsToFollowerOrFutureReplica( + createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes)), baseOffset = 8L), + isFuture = false, + partitionLeaderEpoch = epoch + ) assertEquals(9L, log.logEndOffset, s"Log end offset after append of 1 record at offset 8:") assertEquals(newLogStartOffset, log.logStartOffset, s"Log start offset not expected to change:") } @@ -1074,9 +1124,13 @@ class PartitionTest extends AbstractPartitionTest { @Test def testAppendRecordsToFollowerWithNoReplicaThrowsException(): Unit = { - assertThrows(classOf[NotLeaderOrFollowerException], () => - partition.appendRecordsToFollowerOrFutureReplica( - createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes)), baseOffset = 0L), isFuture = false) + assertThrows( + classOf[NotLeaderOrFollowerException], + () => partition.appendRecordsToFollowerOrFutureReplica( + createRecords(List(new SimpleRecord("k1".getBytes, "v1".getBytes)), baseOffset = 0L), + isFuture = false, + partitionLeaderEpoch = 0 + ) ) } @@ -3440,13 +3494,16 @@ class PartitionTest extends AbstractPartitionTest { partition.createLogIfNotExists(isNew = true, isFutureReplica = false, offsetCheckpoints, topicId = topicId) assertTrue(partition.log.isDefined) + val replicas = Seq(brokerId, brokerId + 1) + val isr = replicas + val epoch = 0 partition.makeLeader( new LeaderAndIsrRequest.PartitionState() .setControllerEpoch(0) .setLeader(brokerId) - .setLeaderEpoch(0) - .setIsr(List(brokerId, brokerId + 1).map(Int.box).asJava) - .setReplicas(List(brokerId, brokerId + 1).map(Int.box).asJava) + .setLeaderEpoch(epoch) + .setIsr(isr.map(Int.box).asJava) + .setReplicas(replicas.map(Int.box).asJava) .setPartitionEpoch(1) .setIsNew(true), offsetCheckpoints, @@ -3477,7 +3534,8 @@ class PartitionTest extends AbstractPartitionTest { partition.appendRecordsToFollowerOrFutureReplica( records = records, - isFuture = true + isFuture = true, + partitionLeaderEpoch = epoch ) listener.verify() @@ -3623,9 +3681,9 @@ class PartitionTest extends AbstractPartitionTest { _topicId = topicId, keepPartitionMetadataFile = true) { - override def appendAsFollower(records: MemoryRecords): LogAppendInfo = { + override def appendAsFollower(records: MemoryRecords, epoch: Int): LogAppendInfo = { appendSemaphore.acquire() - val appendInfo = super.appendAsFollower(records) + val appendInfo = super.appendAsFollower(records, epoch) appendInfo } } diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 1be6cfd62d..106a4a78a9 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -1458,7 +1458,7 @@ class LogCleanerTest extends Logging { log.appendAsLeader(TestUtils.singletonRecords(value = v, key = k), leaderEpoch = 0) //0 to Int.MaxValue is Int.MaxValue+1 message, -1 will be the last message of i-th segment val records = messageWithOffset(k, v, (i + 1L) * (Int.MaxValue + 1L) -1 ) - log.appendAsFollower(records) + log.appendAsFollower(records, Int.MaxValue) assertEquals(i + 1, log.numberOfSegments) } @@ -1512,7 +1512,7 @@ class LogCleanerTest extends Logging { // forward offset and append message to next segment at offset Int.MaxValue val records = messageWithOffset("hello".getBytes, "hello".getBytes, Int.MaxValue - 1) - log.appendAsFollower(records) + log.appendAsFollower(records, Int.MaxValue) log.appendAsLeader(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes), leaderEpoch = 0) assertEquals(Int.MaxValue, log.activeSegment.offsetIndex.lastOffset) @@ -1561,14 +1561,14 @@ class LogCleanerTest extends Logging { val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val record1 = messageWithOffset("hello".getBytes, "hello".getBytes, 0) - log.appendAsFollower(record1) + log.appendAsFollower(record1, Int.MaxValue) val record2 = messageWithOffset("hello".getBytes, "hello".getBytes, 1) - log.appendAsFollower(record2) + log.appendAsFollower(record2, Int.MaxValue) log.roll(Some(Int.MaxValue/2)) // starting a new log segment at offset Int.MaxValue/2 val record3 = messageWithOffset("hello".getBytes, "hello".getBytes, Int.MaxValue/2) - log.appendAsFollower(record3) + log.appendAsFollower(record3, Int.MaxValue) val record4 = messageWithOffset("hello".getBytes, "hello".getBytes, Int.MaxValue.toLong + 1) - log.appendAsFollower(record4) + log.appendAsFollower(record4, Int.MaxValue) assertTrue(log.logEndOffset - 1 - log.logStartOffset > Int.MaxValue, "Actual offset range should be > Int.MaxValue") assertTrue(log.logSegments.asScala.last.offsetIndex.lastOffset - log.logStartOffset <= Int.MaxValue, @@ -1882,8 +1882,8 @@ class LogCleanerTest extends Logging { val noDupSetOffset = 50 val noDupSet = noDupSetKeys zip (noDupSetOffset until noDupSetOffset + noDupSetKeys.size) - log.appendAsFollower(invalidCleanedMessage(dupSetOffset, dupSet, codec)) - log.appendAsFollower(invalidCleanedMessage(noDupSetOffset, noDupSet, codec)) + log.appendAsFollower(invalidCleanedMessage(dupSetOffset, dupSet, codec), Int.MaxValue) + log.appendAsFollower(invalidCleanedMessage(noDupSetOffset, noDupSet, codec), Int.MaxValue) log.roll() @@ -1969,7 +1969,7 @@ class LogCleanerTest extends Logging { log.roll(Some(11L)) // active segment record - log.appendAsFollower(messageWithOffset(1015, 1015, 11L)) + log.appendAsFollower(messageWithOffset(1015, 1015, 11L), Int.MaxValue) val (nextDirtyOffset, _) = cleaner.clean(LogToClean(log.topicPartition, log, 0L, log.activeSegment.baseOffset, needCompactionNow = true)) assertEquals(log.activeSegment.baseOffset, nextDirtyOffset, @@ -1988,7 +1988,7 @@ class LogCleanerTest extends Logging { log.roll(Some(30L)) // active segment record - log.appendAsFollower(messageWithOffset(1015, 1015, 30L)) + log.appendAsFollower(messageWithOffset(1015, 1015, 30L), Int.MaxValue) val (nextDirtyOffset, _) = cleaner.clean(LogToClean(log.topicPartition, log, 0L, log.activeSegment.baseOffset, needCompactionNow = true)) assertEquals(log.activeSegment.baseOffset, nextDirtyOffset, @@ -2088,7 +2088,7 @@ class LogCleanerTest extends Logging { private def writeToLog(log: UnifiedLog, keysAndValues: Iterable[(Int, Int)], offsetSeq: Iterable[Long]): Iterable[Long] = { for (((key, value), offset) <- keysAndValues.zip(offsetSeq)) - yield log.appendAsFollower(messageWithOffset(key, value, offset)).lastOffset + yield log.appendAsFollower(messageWithOffset(key, value, offset), Int.MaxValue).lastOffset } private def invalidCleanedMessage(initialOffset: Long, diff --git a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala index d6d2b06650..b106e38165 100644 --- a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala @@ -126,9 +126,14 @@ class LogConcurrencyTest { log.appendAsLeader(TestUtils.records(records), leaderEpoch) log.maybeIncrementHighWatermark(logEndOffsetMetadata) } else { - log.appendAsFollower(TestUtils.records(records, - baseOffset = logEndOffset, - partitionLeaderEpoch = leaderEpoch)) + log.appendAsFollower( + TestUtils.records( + records, + baseOffset = logEndOffset, + partitionLeaderEpoch = leaderEpoch + ), + Int.MaxValue + ) log.updateHighWatermark(logEndOffset) } diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index b327c3b39e..b1bc865158 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -926,17 +926,17 @@ class LogLoaderTest { val set3 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 3, Compression.NONE, 0, new SimpleRecord("v4".getBytes(), "k4".getBytes())) val set4 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 4, Compression.NONE, 0, new SimpleRecord("v5".getBytes(), "k5".getBytes())) //Writes into an empty log with baseOffset 0 - log.appendAsFollower(set1) + log.appendAsFollower(set1, Int.MaxValue) assertEquals(0L, log.activeSegment.baseOffset) //This write will roll the segment, yielding a new segment with base offset = max(1, Integer.MAX_VALUE+2) = Integer.MAX_VALUE+2 - log.appendAsFollower(set2) + log.appendAsFollower(set2, Int.MaxValue) assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) assertTrue(LogFileUtils.producerSnapshotFile(logDir, Integer.MAX_VALUE.toLong + 2).exists) //This will go into the existing log - log.appendAsFollower(set3) + log.appendAsFollower(set3, Int.MaxValue) assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) //This will go into the existing log - log.appendAsFollower(set4) + log.appendAsFollower(set4, Int.MaxValue) assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) log.close() val indexFiles = logDir.listFiles.filter(file => file.getName.contains(".index")) @@ -965,17 +965,17 @@ class LogLoaderTest { new SimpleRecord("v7".getBytes(), "k7".getBytes()), new SimpleRecord("v8".getBytes(), "k8".getBytes())) //Writes into an empty log with baseOffset 0 - log.appendAsFollower(set1) + log.appendAsFollower(set1, Int.MaxValue) assertEquals(0L, log.activeSegment.baseOffset) //This write will roll the segment, yielding a new segment with base offset = max(1, Integer.MAX_VALUE+2) = Integer.MAX_VALUE+2 - log.appendAsFollower(set2) + log.appendAsFollower(set2, Int.MaxValue) assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) assertTrue(LogFileUtils.producerSnapshotFile(logDir, Integer.MAX_VALUE.toLong + 2).exists) //This will go into the existing log - log.appendAsFollower(set3) + log.appendAsFollower(set3, Int.MaxValue) assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) //This will go into the existing log - log.appendAsFollower(set4) + log.appendAsFollower(set4, Int.MaxValue) assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) log.close() val indexFiles = logDir.listFiles.filter(file => file.getName.contains(".index")) @@ -1005,18 +1005,18 @@ class LogLoaderTest { new SimpleRecord("v7".getBytes(), "k7".getBytes()), new SimpleRecord("v8".getBytes(), "k8".getBytes())) //Writes into an empty log with baseOffset 0 - log.appendAsFollower(set1) + log.appendAsFollower(set1, Int.MaxValue) assertEquals(0L, log.activeSegment.baseOffset) //This write will roll the segment, yielding a new segment with base offset = max(1, 3) = 3 - log.appendAsFollower(set2) + log.appendAsFollower(set2, Int.MaxValue) assertEquals(3, log.activeSegment.baseOffset) assertTrue(LogFileUtils.producerSnapshotFile(logDir, 3).exists) //This will also roll the segment, yielding a new segment with base offset = max(5, Integer.MAX_VALUE+4) = Integer.MAX_VALUE+4 - log.appendAsFollower(set3) + log.appendAsFollower(set3, Int.MaxValue) assertEquals(Integer.MAX_VALUE.toLong + 4, log.activeSegment.baseOffset) assertTrue(LogFileUtils.producerSnapshotFile(logDir, Integer.MAX_VALUE.toLong + 4).exists) //This will go into the existing log - log.appendAsFollower(set4) + log.appendAsFollower(set4, Int.MaxValue) assertEquals(Integer.MAX_VALUE.toLong + 4, log.activeSegment.baseOffset) log.close() val indexFiles = logDir.listFiles.filter(file => file.getName.contains(".index")) @@ -1206,16 +1206,16 @@ class LogLoaderTest { val log = createLog(logDir, new LogConfig(new Properties)) val leaderEpochCache = log.leaderEpochCache val firstBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 1, offset = 0) - log.appendAsFollower(records = firstBatch) + log.appendAsFollower(records = firstBatch, Int.MaxValue) val secondBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 2, offset = 1) - log.appendAsFollower(records = secondBatch) + log.appendAsFollower(records = secondBatch, Int.MaxValue) val thirdBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 2, offset = 2) - log.appendAsFollower(records = thirdBatch) + log.appendAsFollower(records = thirdBatch, Int.MaxValue) val fourthBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 3, offset = 3) - log.appendAsFollower(records = fourthBatch) + log.appendAsFollower(records = fourthBatch, Int.MaxValue) assertEquals(java.util.Arrays.asList(new EpochEntry(1, 0), new EpochEntry(2, 1), new EpochEntry(3, 3)), leaderEpochCache.epochEntries) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 9d208055a5..2b9d2d0437 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -48,11 +48,16 @@ import org.apache.kafka.storage.log.metrics.{BrokerTopicMetrics, BrokerTopicStat import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ArgumentsSource import org.junit.jupiter.params.provider.{EnumSource, ValueSource} import org.mockito.ArgumentMatchers import org.mockito.ArgumentMatchers.{any, anyLong} import org.mockito.Mockito.{doAnswer, doThrow, spy} +import net.jqwik.api.AfterFailureMode +import net.jqwik.api.ForAll +import net.jqwik.api.Property + import java.io._ import java.nio.ByteBuffer import java.nio.file.Files @@ -304,7 +309,7 @@ class UnifiedLogTest { assertHighWatermark(3L) // Update high watermark as follower - log.appendAsFollower(records(3L)) + log.appendAsFollower(records(3L), leaderEpoch) log.updateHighWatermark(6L) assertHighWatermark(6L) @@ -589,6 +594,7 @@ class UnifiedLogTest { @Test def testRollSegmentThatAlreadyExists(): Unit = { val logConfig = LogTestUtils.createLogConfig(segmentMs = 1 * 60 * 60L) + val partitionLeaderEpoch = 0 // create a log val log = createLog(logDir, logConfig) @@ -601,16 +607,16 @@ class UnifiedLogTest { // should be able to append records to active segment val records = TestUtils.records( List(new SimpleRecord(mockTime.milliseconds, "k1".getBytes, "v1".getBytes)), - baseOffset = 0L, partitionLeaderEpoch = 0) - log.appendAsFollower(records) + baseOffset = 0L, partitionLeaderEpoch = partitionLeaderEpoch) + log.appendAsFollower(records, partitionLeaderEpoch) assertEquals(1, log.numberOfSegments, "Expect one segment.") assertEquals(0L, log.activeSegment.baseOffset) // make sure we can append more records val records2 = TestUtils.records( List(new SimpleRecord(mockTime.milliseconds + 10, "k2".getBytes, "v2".getBytes)), - baseOffset = 1L, partitionLeaderEpoch = 0) - log.appendAsFollower(records2) + baseOffset = 1L, partitionLeaderEpoch = partitionLeaderEpoch) + log.appendAsFollower(records2, partitionLeaderEpoch) assertEquals(2, log.logEndOffset, "Expect two records in the log") assertEquals(0, LogTestUtils.readLog(log, 0, 1).records.batches.iterator.next().lastOffset) @@ -625,8 +631,8 @@ class UnifiedLogTest { log.activeSegment.offsetIndex.resize(0) val records3 = TestUtils.records( List(new SimpleRecord(mockTime.milliseconds + 12, "k3".getBytes, "v3".getBytes)), - baseOffset = 2L, partitionLeaderEpoch = 0) - log.appendAsFollower(records3) + baseOffset = 2L, partitionLeaderEpoch = partitionLeaderEpoch) + log.appendAsFollower(records3, partitionLeaderEpoch) assertTrue(log.activeSegment.offsetIndex.maxEntries > 1) assertEquals(2, LogTestUtils.readLog(log, 2, 1).records.batches.iterator.next().lastOffset) assertEquals(2, log.numberOfSegments, "Expect two segments.") @@ -800,17 +806,25 @@ class UnifiedLogTest { val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig) val pid = 1L - val epoch = 0.toShort + val producerEpoch = 0.toShort + val partitionLeaderEpoch = 0 val seq = 0 val baseOffset = 23L // create a batch with a couple gaps to simulate compaction - val records = TestUtils.records(producerId = pid, producerEpoch = epoch, sequence = seq, baseOffset = baseOffset, records = List( - new SimpleRecord(mockTime.milliseconds(), "a".getBytes), - new SimpleRecord(mockTime.milliseconds(), "key".getBytes, "b".getBytes), - new SimpleRecord(mockTime.milliseconds(), "c".getBytes), - new SimpleRecord(mockTime.milliseconds(), "key".getBytes, "d".getBytes))) - records.batches.forEach(_.setPartitionLeaderEpoch(0)) + val records = TestUtils.records( + producerId = pid, + producerEpoch = producerEpoch, + sequence = seq, + baseOffset = baseOffset, + records = List( + new SimpleRecord(mockTime.milliseconds(), "a".getBytes), + new SimpleRecord(mockTime.milliseconds(), "key".getBytes, "b".getBytes), + new SimpleRecord(mockTime.milliseconds(), "c".getBytes), + new SimpleRecord(mockTime.milliseconds(), "key".getBytes, "d".getBytes) + ) + ) + records.batches.forEach(_.setPartitionLeaderEpoch(partitionLeaderEpoch)) val filtered = ByteBuffer.allocate(2048) records.filterTo(new RecordFilter(0, 0) { @@ -821,14 +835,18 @@ class UnifiedLogTest { filtered.flip() val filteredRecords = MemoryRecords.readableRecords(filtered) - log.appendAsFollower(filteredRecords) + log.appendAsFollower(filteredRecords, partitionLeaderEpoch) // append some more data and then truncate to force rebuilding of the PID map - val moreRecords = TestUtils.records(baseOffset = baseOffset + 4, records = List( - new SimpleRecord(mockTime.milliseconds(), "e".getBytes), - new SimpleRecord(mockTime.milliseconds(), "f".getBytes))) - moreRecords.batches.forEach(_.setPartitionLeaderEpoch(0)) - log.appendAsFollower(moreRecords) + val moreRecords = TestUtils.records( + baseOffset = baseOffset + 4, + records = List( + new SimpleRecord(mockTime.milliseconds(), "e".getBytes), + new SimpleRecord(mockTime.milliseconds(), "f".getBytes) + ) + ) + moreRecords.batches.forEach(_.setPartitionLeaderEpoch(partitionLeaderEpoch)) + log.appendAsFollower(moreRecords, partitionLeaderEpoch) log.truncateTo(baseOffset + 4) @@ -844,15 +862,23 @@ class UnifiedLogTest { val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig) val pid = 1L - val epoch = 0.toShort + val producerEpoch = 0.toShort + val partitionLeaderEpoch = 0 val seq = 0 val baseOffset = 23L // create an empty batch - val records = TestUtils.records(producerId = pid, producerEpoch = epoch, sequence = seq, baseOffset = baseOffset, records = List( - new SimpleRecord(mockTime.milliseconds(), "key".getBytes, "a".getBytes), - new SimpleRecord(mockTime.milliseconds(), "key".getBytes, "b".getBytes))) - records.batches.forEach(_.setPartitionLeaderEpoch(0)) + val records = TestUtils.records( + producerId = pid, + producerEpoch = producerEpoch, + sequence = seq, + baseOffset = baseOffset, + records = List( + new SimpleRecord(mockTime.milliseconds(), "key".getBytes, "a".getBytes), + new SimpleRecord(mockTime.milliseconds(), "key".getBytes, "b".getBytes) + ) + ) + records.batches.forEach(_.setPartitionLeaderEpoch(partitionLeaderEpoch)) val filtered = ByteBuffer.allocate(2048) records.filterTo(new RecordFilter(0, 0) { @@ -863,14 +889,18 @@ class UnifiedLogTest { filtered.flip() val filteredRecords = MemoryRecords.readableRecords(filtered) - log.appendAsFollower(filteredRecords) + log.appendAsFollower(filteredRecords, partitionLeaderEpoch) // append some more data and then truncate to force rebuilding of the PID map - val moreRecords = TestUtils.records(baseOffset = baseOffset + 2, records = List( - new SimpleRecord(mockTime.milliseconds(), "e".getBytes), - new SimpleRecord(mockTime.milliseconds(), "f".getBytes))) - moreRecords.batches.forEach(_.setPartitionLeaderEpoch(0)) - log.appendAsFollower(moreRecords) + val moreRecords = TestUtils.records( + baseOffset = baseOffset + 2, + records = List( + new SimpleRecord(mockTime.milliseconds(), "e".getBytes), + new SimpleRecord(mockTime.milliseconds(), "f".getBytes) + ) + ) + moreRecords.batches.forEach(_.setPartitionLeaderEpoch(partitionLeaderEpoch)) + log.appendAsFollower(moreRecords, partitionLeaderEpoch) log.truncateTo(baseOffset + 2) @@ -886,17 +916,25 @@ class UnifiedLogTest { val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5) val log = createLog(logDir, logConfig) val pid = 1L - val epoch = 0.toShort + val producerEpoch = 0.toShort + val partitionLeaderEpoch = 0 val seq = 0 val baseOffset = 23L // create a batch with a couple gaps to simulate compaction - val records = TestUtils.records(producerId = pid, producerEpoch = epoch, sequence = seq, baseOffset = baseOffset, records = List( - new SimpleRecord(mockTime.milliseconds(), "a".getBytes), - new SimpleRecord(mockTime.milliseconds(), "key".getBytes, "b".getBytes), - new SimpleRecord(mockTime.milliseconds(), "c".getBytes), - new SimpleRecord(mockTime.milliseconds(), "key".getBytes, "d".getBytes))) - records.batches.forEach(_.setPartitionLeaderEpoch(0)) + val records = TestUtils.records( + producerId = pid, + producerEpoch = producerEpoch, + sequence = seq, + baseOffset = baseOffset, + records = List( + new SimpleRecord(mockTime.milliseconds(), "a".getBytes), + new SimpleRecord(mockTime.milliseconds(), "key".getBytes, "b".getBytes), + new SimpleRecord(mockTime.milliseconds(), "c".getBytes), + new SimpleRecord(mockTime.milliseconds(), "key".getBytes, "d".getBytes) + ) + ) + records.batches.forEach(_.setPartitionLeaderEpoch(partitionLeaderEpoch)) val filtered = ByteBuffer.allocate(2048) records.filterTo(new RecordFilter(0, 0) { @@ -907,7 +945,7 @@ class UnifiedLogTest { filtered.flip() val filteredRecords = MemoryRecords.readableRecords(filtered) - log.appendAsFollower(filteredRecords) + log.appendAsFollower(filteredRecords, partitionLeaderEpoch) val activeProducers = log.activeProducersWithLastSequence assertTrue(activeProducers.contains(pid)) @@ -1337,33 +1375,44 @@ class UnifiedLogTest { // create a log val log = createLog(logDir, new LogConfig(new Properties)) - val epoch: Short = 0 + val producerEpoch: Short = 0 + val partitionLeaderEpoch = 0 val buffer = ByteBuffer.allocate(512) - var builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE, - TimestampType.LOG_APPEND_TIME, 0L, mockTime.milliseconds(), 1L, epoch, 0, false, 0) + var builder = MemoryRecords.builder( + buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE, + TimestampType.LOG_APPEND_TIME, 0L, mockTime.milliseconds(), 1L, producerEpoch, 0, false, + partitionLeaderEpoch + ) builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) builder.close() builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE, - TimestampType.LOG_APPEND_TIME, 1L, mockTime.milliseconds(), 2L, epoch, 0, false, 0) + TimestampType.LOG_APPEND_TIME, 1L, mockTime.milliseconds(), 2L, producerEpoch, 0, false, + partitionLeaderEpoch) builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) builder.close() - builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE, - TimestampType.LOG_APPEND_TIME, 2L, mockTime.milliseconds(), 3L, epoch, 0, false, 0) + builder = MemoryRecords.builder( + buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE, + TimestampType.LOG_APPEND_TIME, 2L, mockTime.milliseconds(), 3L, producerEpoch, 0, false, + partitionLeaderEpoch + ) builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) builder.close() - builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE, - TimestampType.LOG_APPEND_TIME, 3L, mockTime.milliseconds(), 4L, epoch, 0, false, 0) + builder = MemoryRecords.builder( + buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE, + TimestampType.LOG_APPEND_TIME, 3L, mockTime.milliseconds(), 4L, producerEpoch, 0, false, + partitionLeaderEpoch + ) builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) builder.close() buffer.flip() val memoryRecords = MemoryRecords.readableRecords(buffer) - log.appendAsFollower(memoryRecords) + log.appendAsFollower(memoryRecords, partitionLeaderEpoch) log.flush(false) val fetchedData = LogTestUtils.readLog(log, 0, Int.MaxValue) @@ -1382,7 +1431,7 @@ class UnifiedLogTest { def testDuplicateAppendToFollower(): Unit = { val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) - val epoch: Short = 0 + val producerEpoch: Short = 0 val pid = 1L val baseSequence = 0 val partitionLeaderEpoch = 0 @@ -1390,10 +1439,32 @@ class UnifiedLogTest { // this is a bit contrived. to trigger the duplicate case for a follower append, we have to append // a batch with matching sequence numbers, but valid increasing offsets assertEquals(0L, log.logEndOffset) - log.appendAsFollower(MemoryRecords.withIdempotentRecords(0L, Compression.NONE, pid, epoch, baseSequence, - partitionLeaderEpoch, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes))) - log.appendAsFollower(MemoryRecords.withIdempotentRecords(2L, Compression.NONE, pid, epoch, baseSequence, - partitionLeaderEpoch, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes))) + log.appendAsFollower( + MemoryRecords.withIdempotentRecords( + 0L, + Compression.NONE, + pid, + producerEpoch, + baseSequence, + partitionLeaderEpoch, + new SimpleRecord("a".getBytes), + new SimpleRecord("b".getBytes) + ), + partitionLeaderEpoch + ) + log.appendAsFollower( + MemoryRecords.withIdempotentRecords( + 2L, + Compression.NONE, + pid, + producerEpoch, + baseSequence, + partitionLeaderEpoch, + new SimpleRecord("a".getBytes), + new SimpleRecord("b".getBytes) + ), + partitionLeaderEpoch + ) // Ensure that even the duplicate sequences are accepted on the follower. assertEquals(4L, log.logEndOffset) @@ -1406,48 +1477,49 @@ class UnifiedLogTest { val pid1 = 1L val pid2 = 2L - val epoch: Short = 0 + val producerEpoch: Short = 0 val buffer = ByteBuffer.allocate(512) // pid1 seq = 0 var builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE, - TimestampType.LOG_APPEND_TIME, 0L, mockTime.milliseconds(), pid1, epoch, 0) + TimestampType.LOG_APPEND_TIME, 0L, mockTime.milliseconds(), pid1, producerEpoch, 0) builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) builder.close() // pid2 seq = 0 builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE, - TimestampType.LOG_APPEND_TIME, 1L, mockTime.milliseconds(), pid2, epoch, 0) + TimestampType.LOG_APPEND_TIME, 1L, mockTime.milliseconds(), pid2, producerEpoch, 0) builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) builder.close() // pid1 seq = 1 builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE, - TimestampType.LOG_APPEND_TIME, 2L, mockTime.milliseconds(), pid1, epoch, 1) + TimestampType.LOG_APPEND_TIME, 2L, mockTime.milliseconds(), pid1, producerEpoch, 1) builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) builder.close() // pid2 seq = 1 builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE, - TimestampType.LOG_APPEND_TIME, 3L, mockTime.milliseconds(), pid2, epoch, 1) + TimestampType.LOG_APPEND_TIME, 3L, mockTime.milliseconds(), pid2, producerEpoch, 1) builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) builder.close() // // pid1 seq = 1 (duplicate) builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE, - TimestampType.LOG_APPEND_TIME, 4L, mockTime.milliseconds(), pid1, epoch, 1) + TimestampType.LOG_APPEND_TIME, 4L, mockTime.milliseconds(), pid1, producerEpoch, 1) builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) builder.close() buffer.flip() + val epoch = 0 val records = MemoryRecords.readableRecords(buffer) - records.batches.forEach(_.setPartitionLeaderEpoch(0)) + records.batches.forEach(_.setPartitionLeaderEpoch(epoch)) // Ensure that batches with duplicates are accepted on the follower. assertEquals(0L, log.logEndOffset) - log.appendAsFollower(records) + log.appendAsFollower(records, epoch) assertEquals(5L, log.logEndOffset) } @@ -1589,8 +1661,12 @@ class UnifiedLogTest { val records = messageIds.map(id => new SimpleRecord(id.toString.getBytes)) // now test the case that we give the offsets and use non-sequential offsets - for (i <- records.indices) - log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), Compression.NONE, 0, records(i))) + for (i <- records.indices) { + log.appendAsFollower( + MemoryRecords.withRecords(messageIds(i), Compression.NONE, 0, records(i)), + Int.MaxValue + ) + } for (i <- 50 until messageIds.max) { val idx = messageIds.indexWhere(_ >= i) val read = LogTestUtils.readLog(log, i, 100).records.records.iterator.next() @@ -1637,8 +1713,12 @@ class UnifiedLogTest { val records = messageIds.map(id => new SimpleRecord(id.toString.getBytes)) // now test the case that we give the offsets and use non-sequential offsets - for (i <- records.indices) - log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), Compression.NONE, 0, records(i))) + for (i <- records.indices) { + log.appendAsFollower( + MemoryRecords.withRecords(messageIds(i), Compression.NONE, 0, records(i)), + Int.MaxValue + ) + } for (i <- 50 until messageIds.max) { val idx = messageIds.indexWhere(_ >= i) @@ -1662,8 +1742,12 @@ class UnifiedLogTest { val records = messageIds.map(id => new SimpleRecord(id.toString.getBytes)) // now test the case that we give the offsets and use non-sequential offsets - for (i <- records.indices) - log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), Compression.NONE, 0, records(i))) + for (i <- records.indices) { + log.appendAsFollower( + MemoryRecords.withRecords(messageIds(i), Compression.NONE, 0, records(i)), + Int.MaxValue + ) + } for (i <- 50 until messageIds.max) { assertEquals(MemoryRecords.EMPTY, LogTestUtils.readLog(log, i, maxLength = 0, minOneMessage = false).records) @@ -1911,9 +1995,94 @@ class UnifiedLogTest { val log = createLog(logDir, LogTestUtils.createLogConfig(maxMessageBytes = second.sizeInBytes - 1)) - log.appendAsFollower(first) + log.appendAsFollower(first, Int.MaxValue) // the second record is larger then limit but appendAsFollower does not validate the size. - log.appendAsFollower(second) + log.appendAsFollower(second, Int.MaxValue) + } + + @ParameterizedTest + @ArgumentsSource(classOf[InvalidMemoryRecordsProvider]) + def testInvalidMemoryRecords(records: MemoryRecords, expectedException: Optional[Class[Exception]]): Unit = { + val logConfig = LogTestUtils.createLogConfig() + val log = createLog(logDir, logConfig) + val previousEndOffset = log.logEndOffsetMetadata.messageOffset + + if (expectedException.isPresent()) { + assertThrows( + expectedException.get(), + () => log.appendAsFollower(records, Int.MaxValue) + ) + } else { + log.appendAsFollower(records, Int.MaxValue) + } + + assertEquals(previousEndOffset, log.logEndOffsetMetadata.messageOffset) + } + + @Property(tries = 100, afterFailure = AfterFailureMode.SAMPLE_ONLY) + def testRandomRecords( + @ForAll(supplier = classOf[ArbitraryMemoryRecords]) records: MemoryRecords + ): Unit = { + val tempDir = TestUtils.tempDir() + val logDir = TestUtils.randomPartitionLogDir(tempDir) + try { + val logConfig = LogTestUtils.createLogConfig() + val log = createLog(logDir, logConfig) + val previousEndOffset = log.logEndOffsetMetadata.messageOffset + + // Depending on the corruption, unified log sometimes throws and sometimes returns an + // empty set of batches + assertThrows( + classOf[CorruptRecordException], + () => { + val info = log.appendAsFollower(records, Int.MaxValue) + if (info.firstOffset == UnifiedLog.UnknownOffset) { + throw new CorruptRecordException("Unknown offset is test") + } + } + ) + + assertEquals(previousEndOffset, log.logEndOffsetMetadata.messageOffset) + } finally { + Utils.delete(tempDir) + } + } + + @Test + def testInvalidLeaderEpoch(): Unit = { + val logConfig = LogTestUtils.createLogConfig() + val log = createLog(logDir, logConfig) + val previousEndOffset = log.logEndOffsetMetadata.messageOffset + val epoch = log.latestEpoch.getOrElse(0) + 1 + val numberOfRecords = 10 + + val batchWithValidEpoch = MemoryRecords.withRecords( + previousEndOffset, + Compression.NONE, + epoch, + (0 until numberOfRecords).map(number => new SimpleRecord(number.toString.getBytes)): _* + ) + + val batchWithInvalidEpoch = MemoryRecords.withRecords( + previousEndOffset + numberOfRecords, + Compression.NONE, + epoch + 1, + (0 until numberOfRecords).map(number => new SimpleRecord(number.toString.getBytes)): _* + ) + + val buffer = ByteBuffer.allocate(batchWithValidEpoch.sizeInBytes() + batchWithInvalidEpoch.sizeInBytes()) + buffer.put(batchWithValidEpoch.buffer()) + buffer.put(batchWithInvalidEpoch.buffer()) + buffer.flip() + + val records = MemoryRecords.readableRecords(buffer) + + log.appendAsFollower(records, epoch) + + // Check that only the first batch was appended + assertEquals(previousEndOffset + numberOfRecords, log.logEndOffsetMetadata.messageOffset) + // Check that the last fetched epoch matches the first batch + assertEquals(epoch, log.latestEpoch.get) } @Test @@ -2014,7 +2183,7 @@ class UnifiedLogTest { val messages = (0 until numMessages).map { i => MemoryRecords.withRecords(100 + i, Compression.NONE, 0, new SimpleRecord(mockTime.milliseconds + i, i.toString.getBytes())) } - messages.foreach(log.appendAsFollower) + messages.foreach(message => log.appendAsFollower(message, Int.MaxValue)) val timeIndexEntries = log.logSegments.asScala.foldLeft(0) { (entries, segment) => entries + segment.timeIndex.entries } assertEquals(numMessages - 1, timeIndexEntries, s"There should be ${numMessages - 1} time index entries") assertEquals(mockTime.milliseconds + numMessages - 1, log.activeSegment.timeIndex.lastEntry.timestamp, @@ -2158,7 +2327,7 @@ class UnifiedLogTest { // The cache can be updated directly after a leader change. // The new latest offset should reflect the updated epoch. log.assignEpochStartOffset(2, 2L) - + assertEquals(new OffsetResultHolder(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 2L, Optional.of(2))), log.fetchOffsetByTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, Some(remoteLogManager))) } @@ -2426,20 +2595,22 @@ class UnifiedLogTest { def testAppendWithOutOfOrderOffsetsThrowsException(): Unit = { val log = createLog(logDir, new LogConfig(new Properties)) + val epoch = 0 val appendOffsets = Seq(0L, 1L, 3L, 2L, 4L) val buffer = ByteBuffer.allocate(512) for (offset <- appendOffsets) { val builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE, TimestampType.LOG_APPEND_TIME, offset, mockTime.milliseconds(), - 1L, 0, 0, false, 0) + 1L, 0, 0, false, epoch) builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) builder.close() } buffer.flip() val memoryRecords = MemoryRecords.readableRecords(buffer) - assertThrows(classOf[OffsetsOutOfOrderException], () => - log.appendAsFollower(memoryRecords) + assertThrows( + classOf[OffsetsOutOfOrderException], + () => log.appendAsFollower(memoryRecords, epoch) ) } @@ -2454,9 +2625,11 @@ class UnifiedLogTest { for (magic <- magicVals; compressionType <- compressionTypes) { val compression = Compression.of(compressionType).build() val invalidRecord = MemoryRecords.withRecords(magic, compression, new SimpleRecord(1.toString.getBytes)) - assertThrows(classOf[UnexpectedAppendOffsetException], - () => log.appendAsFollower(invalidRecord), - () => s"Magic=$magic, compressionType=$compressionType") + assertThrows( + classOf[UnexpectedAppendOffsetException], + () => log.appendAsFollower(invalidRecord, Int.MaxValue), + () => s"Magic=$magic, compressionType=$compressionType" + ) } } @@ -2477,7 +2650,10 @@ class UnifiedLogTest { magicValue = magic, codec = Compression.of(compressionType).build(), baseOffset = firstOffset) - val exception = assertThrows(classOf[UnexpectedAppendOffsetException], () => log.appendAsFollower(records = batch)) + val exception = assertThrows( + classOf[UnexpectedAppendOffsetException], + () => log.appendAsFollower(records = batch, Int.MaxValue) + ) assertEquals(firstOffset, exception.firstOffset, s"Magic=$magic, compressionType=$compressionType, UnexpectedAppendOffsetException#firstOffset") assertEquals(firstOffset + 2, exception.lastOffset, s"Magic=$magic, compressionType=$compressionType, UnexpectedAppendOffsetException#lastOffset") } @@ -2576,9 +2752,16 @@ class UnifiedLogTest { log.appendAsLeader(TestUtils.records(List(new SimpleRecord("foo".getBytes()))), leaderEpoch = 5) assertEquals(OptionalInt.of(5), log.leaderEpochCache.latestEpoch) - log.appendAsFollower(TestUtils.records(List(new SimpleRecord("foo".getBytes())), - baseOffset = 1L, - magicValue = RecordVersion.V1.value)) + log.appendAsFollower( + TestUtils.records( + List( + new SimpleRecord("foo".getBytes()) + ), + baseOffset = 1L, + magicValue = RecordVersion.V1.value + ), + 5 + ) assertEquals(OptionalInt.empty, log.leaderEpochCache.latestEpoch) } @@ -2934,7 +3117,7 @@ class UnifiedLogTest { //When appending as follower (assignOffsets = false) for (i <- records.indices) - log.appendAsFollower(recordsForEpoch(i)) + log.appendAsFollower(recordsForEpoch(i), i) assertEquals(Some(42), log.latestEpoch) } @@ -3002,7 +3185,7 @@ class UnifiedLogTest { def append(epoch: Int, startOffset: Long, count: Int): Unit = { for (i <- 0 until count) - log.appendAsFollower(createRecords(startOffset + i, epoch)) + log.appendAsFollower(createRecords(startOffset + i, epoch), epoch) } //Given 2 segments, 10 messages per segment @@ -3236,7 +3419,7 @@ class UnifiedLogTest { buffer.flip() - appendAsFollower(log, MemoryRecords.readableRecords(buffer)) + appendAsFollower(log, MemoryRecords.readableRecords(buffer), epoch) val abortedTransactions = LogTestUtils.allAbortedTransactions(log) val expectedTransactions = List( @@ -3320,7 +3503,7 @@ class UnifiedLogTest { appendEndTxnMarkerToBuffer(buffer, pid, epoch, 10L, ControlRecordType.COMMIT, leaderEpoch = 1) buffer.flip() - log.appendAsFollower(MemoryRecords.readableRecords(buffer)) + log.appendAsFollower(MemoryRecords.readableRecords(buffer), epoch) LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 2, leaderEpoch = 1) LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 2, leaderEpoch = 1) @@ -3441,10 +3624,16 @@ class UnifiedLogTest { val log = createLog(logDir, logConfig) // append a few records - appendAsFollower(log, MemoryRecords.withRecords(Compression.NONE, - new SimpleRecord("a".getBytes), - new SimpleRecord("b".getBytes), - new SimpleRecord("c".getBytes)), 5) + appendAsFollower( + log, + MemoryRecords.withRecords( + Compression.NONE, + new SimpleRecord("a".getBytes), + new SimpleRecord("b".getBytes), + new SimpleRecord("c".getBytes) + ), + 5 + ) log.updateHighWatermark(3L) @@ -4508,9 +4697,9 @@ class UnifiedLogTest { builder.close() } - private def appendAsFollower(log: UnifiedLog, records: MemoryRecords, leaderEpoch: Int = 0): Unit = { + private def appendAsFollower(log: UnifiedLog, records: MemoryRecords, leaderEpoch: Int): Unit = { records.batches.forEach(_.setPartitionLeaderEpoch(leaderEpoch)) - log.appendAsFollower(records) + log.appendAsFollower(records, leaderEpoch) } private def createLog(dir: File, diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala index 7528eefc42..d1a05e7d91 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala @@ -328,9 +328,12 @@ class AbstractFetcherManagerTest { fetchBackOffMs = 0, brokerTopicStats = new BrokerTopicStats) { - override protected def processPartitionData(topicPartition: TopicPartition, fetchOffset: Long, partitionData: FetchData): Option[LogAppendInfo] = { - None - } + override protected def processPartitionData( + topicPartition: TopicPartition, + fetchOffset: Long, + partitionLeaderEpoch: Int, + partitionData: FetchData + ): Option[LogAppendInfo] = None override protected def truncate(topicPartition: TopicPartition, truncationState: OffsetTruncationState): Unit = {} diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala index 5f01458ffa..aa25cad89c 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala @@ -631,6 +631,7 @@ class AbstractFetcherThreadTest { @Test def testFollowerFetchOutOfRangeLow(): Unit = { + val leaderEpoch = 4 val partition = new TopicPartition("topic", 0) val mockLeaderEndpoint = new MockLeaderEndPoint(truncateOnFetch = truncateOnFetch, version = version) val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndpoint) @@ -640,14 +641,19 @@ class AbstractFetcherThreadTest { val replicaLog = Seq( mkBatch(baseOffset = 0, leaderEpoch = 0, new SimpleRecord("a".getBytes))) - val replicaState = PartitionState(replicaLog, leaderEpoch = 0, highWatermark = 0L) + val replicaState = PartitionState(replicaLog, leaderEpoch = leaderEpoch, highWatermark = 0L) fetcher.setReplicaState(partition, replicaState) - fetcher.addPartitions(Map(partition -> initialFetchState(topicIds.get(partition.topic), 3L, leaderEpoch = 0))) + fetcher.addPartitions( + Map( + partition -> initialFetchState(topicIds.get(partition.topic), 3L, leaderEpoch = leaderEpoch) + ) + ) val leaderLog = Seq( - mkBatch(baseOffset = 2, leaderEpoch = 4, new SimpleRecord("c".getBytes))) + mkBatch(baseOffset = 2, leaderEpoch = leaderEpoch, new SimpleRecord("c".getBytes)) + ) - val leaderState = PartitionState(leaderLog, leaderEpoch = 0, highWatermark = 2L) + val leaderState = PartitionState(leaderLog, leaderEpoch = leaderEpoch, highWatermark = 2L) fetcher.mockLeader.setLeaderState(partition, leaderState) fetcher.mockLeader.setReplicaPartitionStateCallback(fetcher.replicaPartitionState) @@ -674,6 +680,7 @@ class AbstractFetcherThreadTest { @Test def testRetryAfterUnknownLeaderEpochInLatestOffsetFetch(): Unit = { + val leaderEpoch = 4 val partition = new TopicPartition("topic", 0) val mockLeaderEndPoint = new MockLeaderEndPoint(truncateOnFetch = truncateOnFetch, version = version) { val tries = new AtomicInteger(0) @@ -688,16 +695,18 @@ class AbstractFetcherThreadTest { // The follower begins from an offset which is behind the leader's log start offset val replicaLog = Seq( - mkBatch(baseOffset = 0, leaderEpoch = 0, new SimpleRecord("a".getBytes))) + mkBatch(baseOffset = 0, leaderEpoch = 0, new SimpleRecord("a".getBytes)) + ) - val replicaState = PartitionState(replicaLog, leaderEpoch = 0, highWatermark = 0L) + val replicaState = PartitionState(replicaLog, leaderEpoch = leaderEpoch, highWatermark = 0L) fetcher.setReplicaState(partition, replicaState) - fetcher.addPartitions(Map(partition -> initialFetchState(topicIds.get(partition.topic), 3L, leaderEpoch = 0))) + fetcher.addPartitions(Map(partition -> initialFetchState(topicIds.get(partition.topic), 3L, leaderEpoch = leaderEpoch))) val leaderLog = Seq( - mkBatch(baseOffset = 2, leaderEpoch = 4, new SimpleRecord("c".getBytes))) + mkBatch(baseOffset = 2, leaderEpoch = 4, new SimpleRecord("c".getBytes)) + ) - val leaderState = PartitionState(leaderLog, leaderEpoch = 0, highWatermark = 2L) + val leaderState = PartitionState(leaderLog, leaderEpoch = leaderEpoch, highWatermark = 2L) fetcher.mockLeader.setLeaderState(partition, leaderState) fetcher.mockLeader.setReplicaPartitionStateCallback(fetcher.replicaPartitionState) @@ -715,6 +724,46 @@ class AbstractFetcherThreadTest { assertEquals(leaderState.highWatermark, replicaState.highWatermark) } + @Test + def testReplicateBatchesUpToLeaderEpoch(): Unit = { + val leaderEpoch = 4 + val partition = new TopicPartition("topic", 0) + val mockLeaderEndpoint = new MockLeaderEndPoint(version = version) + val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndpoint) + val fetcher = new MockFetcherThread(mockLeaderEndpoint, mockTierStateMachine, failedPartitions = failedPartitions) + + val replicaState = PartitionState(Seq(), leaderEpoch = leaderEpoch, highWatermark = 0L) + fetcher.setReplicaState(partition, replicaState) + fetcher.addPartitions( + Map( + partition -> initialFetchState(topicIds.get(partition.topic), 3L, leaderEpoch = leaderEpoch) + ) + ) + + val leaderLog = Seq( + mkBatch(baseOffset = 0, leaderEpoch = leaderEpoch - 1, new SimpleRecord("c".getBytes)), + mkBatch(baseOffset = 1, leaderEpoch = leaderEpoch, new SimpleRecord("d".getBytes)), + mkBatch(baseOffset = 2, leaderEpoch = leaderEpoch + 1, new SimpleRecord("e".getBytes)) + ) + + val leaderState = PartitionState(leaderLog, leaderEpoch = leaderEpoch, highWatermark = 0L) + fetcher.mockLeader.setLeaderState(partition, leaderState) + fetcher.mockLeader.setReplicaPartitionStateCallback(fetcher.replicaPartitionState) + + assertEquals(Option(Fetching), fetcher.fetchState(partition).map(_.state)) + assertEquals(0, replicaState.logStartOffset) + assertEquals(List(), replicaState.log.toList) + + TestUtils.waitUntilTrue(() => { + fetcher.doWork() + fetcher.replicaPartitionState(partition).log == fetcher.mockLeader.leaderPartitionState(partition).log.dropRight(1) + }, "Failed to reconcile leader and follower logs up to the leader epoch") + + assertEquals(leaderState.logStartOffset, replicaState.logStartOffset) + assertEquals(leaderState.logEndOffset - 1, replicaState.logEndOffset) + assertEquals(leaderState.highWatermark, replicaState.highWatermark) + } + @Test def testCorruptMessage(): Unit = { val partition = new TopicPartition("topic", 0) @@ -904,11 +953,16 @@ class AbstractFetcherThreadTest { val mockLeaderEndpoint = new MockLeaderEndPoint(truncateOnFetch = truncateOnFetch, version = version) val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndpoint) val fetcherForAppend = new MockFetcherThread(mockLeaderEndpoint, mockTierStateMachine, failedPartitions = failedPartitions) { - override def processPartitionData(topicPartition: TopicPartition, fetchOffset: Long, partitionData: FetchData): Option[LogAppendInfo] = { + override def processPartitionData( + topicPartition: TopicPartition, + fetchOffset: Long, + partitionLeaderEpoch: Int, + partitionData: FetchData + ): Option[LogAppendInfo] = { if (topicPartition == partition1) { throw new KafkaException() } else { - super.processPartitionData(topicPartition, fetchOffset, partitionData) + super.processPartitionData(topicPartition, fetchOffset, partitionLeaderEpoch, partitionData) } } } @@ -1013,9 +1067,14 @@ class AbstractFetcherThreadTest { val mockLeaderEndpoint = new MockLeaderEndPoint(truncateOnFetch = truncateOnFetch, version = version) val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndpoint) val fetcher = new MockFetcherThread(mockLeaderEndpoint, mockTierStateMachine) { - override def processPartitionData(topicPartition: TopicPartition, fetchOffset: Long, partitionData: FetchData): Option[LogAppendInfo] = { + override def processPartitionData( + topicPartition: TopicPartition, + fetchOffset: Long, + partitionLeaderEpoch: Int, + partitionData: FetchData + ): Option[LogAppendInfo] = { processPartitionDataCalls += 1 - super.processPartitionData(topicPartition, fetchOffset, partitionData) + super.processPartitionData(topicPartition, fetchOffset, partitionLeaderEpoch, partitionData) } override def truncate(topicPartition: TopicPartition, truncationState: OffsetTruncationState): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/MockFetcherThread.scala b/core/src/test/scala/unit/kafka/server/MockFetcherThread.scala index 5d50de0409..ff1e919656 100644 --- a/core/src/test/scala/unit/kafka/server/MockFetcherThread.scala +++ b/core/src/test/scala/unit/kafka/server/MockFetcherThread.scala @@ -66,9 +66,12 @@ class MockFetcherThread(val mockLeader: MockLeaderEndPoint, partitions } - override def processPartitionData(topicPartition: TopicPartition, - fetchOffset: Long, - partitionData: FetchData): Option[LogAppendInfo] = { + override def processPartitionData( + topicPartition: TopicPartition, + fetchOffset: Long, + leaderEpochForReplica: Int, + partitionData: FetchData + ): Option[LogAppendInfo] = { val state = replicaPartitionState(topicPartition) if (leader.isTruncationOnFetchSupported && FetchResponse.isDivergingEpoch(partitionData)) { @@ -87,17 +90,24 @@ class MockFetcherThread(val mockLeader: MockLeaderEndPoint, var shallowOffsetOfMaxTimestamp = -1L var lastOffset = state.logEndOffset var lastEpoch: OptionalInt = OptionalInt.empty() + var skipRemainingBatches = false for (batch <- batches) { batch.ensureValid() - if (batch.maxTimestamp > maxTimestamp) { - maxTimestamp = batch.maxTimestamp - shallowOffsetOfMaxTimestamp = batch.baseOffset + + skipRemainingBatches = skipRemainingBatches || hasHigherPartitionLeaderEpoch(batch, leaderEpochForReplica); + if (skipRemainingBatches) { + info(s"Skipping batch $batch because leader epoch is $leaderEpochForReplica") + } else { + if (batch.maxTimestamp > maxTimestamp) { + maxTimestamp = batch.maxTimestamp + shallowOffsetOfMaxTimestamp = batch.baseOffset + } + state.log.append(batch) + state.logEndOffset = batch.nextOffset + lastOffset = batch.lastOffset + lastEpoch = OptionalInt.of(batch.partitionLeaderEpoch) } - state.log.append(batch) - state.logEndOffset = batch.nextOffset - lastOffset = batch.lastOffset - lastEpoch = OptionalInt.of(batch.partitionLeaderEpoch) } state.logStartOffset = partitionData.logStartOffset @@ -115,6 +125,11 @@ class MockFetcherThread(val mockLeader: MockLeaderEndPoint, batches.headOption.map(_.lastOffset).getOrElse(-1))) } + private def hasHigherPartitionLeaderEpoch(batch: RecordBatch, leaderEpoch: Int): Boolean = { + batch.partitionLeaderEpoch() != RecordBatch.NO_PARTITION_LEADER_EPOCH && + batch.partitionLeaderEpoch() > leaderEpoch + } + override def truncate(topicPartition: TopicPartition, truncationState: OffsetTruncationState): Unit = { val state = replicaPartitionState(topicPartition) state.log = state.log.takeWhile { batch => diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala index 6526d6628c..b0ee5a2d14 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala @@ -281,9 +281,22 @@ class ReplicaFetcherThreadTest { val fetchSessionHandler = new FetchSessionHandler(logContext, brokerEndPoint.id) val leader = new RemoteLeaderEndPoint(logContext.logPrefix, mockNetwork, fetchSessionHandler, config, replicaManager, quota, () => MetadataVersion.MINIMUM_VERSION, () => 1) - val thread = new ReplicaFetcherThread("bob", leader, config, failedPartitions, - replicaManager, quota, logContext.logPrefix, () => MetadataVersion.MINIMUM_VERSION) { - override def processPartitionData(topicPartition: TopicPartition, fetchOffset: Long, partitionData: FetchData): Option[LogAppendInfo] = None + val thread = new ReplicaFetcherThread( + "bob", + leader, + config, + failedPartitions, + replicaManager, + quota, + logContext.logPrefix, + () => MetadataVersion.MINIMUM_VERSION + ) { + override def processPartitionData( + topicPartition: TopicPartition, + fetchOffset: Long, + partitionLeaderEpoch: Int, + partitionData: FetchData + ): Option[LogAppendInfo] = None } thread.addPartitions(Map(t1p0 -> initialFetchState(Some(topicId1), initialLEO), t1p1 -> initialFetchState(Some(topicId1), initialLEO))) val partitions = Set(t1p0, t1p1) @@ -379,7 +392,7 @@ class ReplicaFetcherThreadTest { when(replicaManager.getPartitionOrException(t1p0)).thenReturn(partition) when(partition.localLogOrException).thenReturn(log) - when(partition.appendRecordsToFollowerOrFutureReplica(any(), any())).thenReturn(None) + when(partition.appendRecordsToFollowerOrFutureReplica(any(), any(), any())).thenReturn(None) val logContext = new LogContext(s"[ReplicaFetcher replicaId=${config.brokerId}, leaderId=${brokerEndPoint.id}, fetcherId=0] ") @@ -460,7 +473,7 @@ class ReplicaFetcherThreadTest { when(replicaManager.brokerTopicStats).thenReturn(mock(classOf[BrokerTopicStats])) when(partition.localLogOrException).thenReturn(log) - when(partition.appendRecordsToFollowerOrFutureReplica(any(), any())).thenReturn(Some(new LogAppendInfo( + when(partition.appendRecordsToFollowerOrFutureReplica(any(), any(), any())).thenReturn(Some(new LogAppendInfo( -1, 0, OptionalInt.empty, @@ -679,7 +692,7 @@ class ReplicaFetcherThreadTest { val partition: Partition = mock(classOf[Partition]) when(partition.localLogOrException).thenReturn(log) - when(partition.appendRecordsToFollowerOrFutureReplica(any[MemoryRecords], any[Boolean])).thenReturn(appendInfo) + when(partition.appendRecordsToFollowerOrFutureReplica(any[MemoryRecords], any[Boolean], any[Int])).thenReturn(appendInfo) // Capture the argument at the time of invocation. val completeDelayedFetchRequestsArgument = mutable.Buffer.empty[TopicPartition] @@ -710,8 +723,8 @@ class ReplicaFetcherThreadTest { .setRecords(records) .setHighWatermark(highWatermarkReceivedFromLeader) - thread.processPartitionData(tp0, 0, partitionData.setPartitionIndex(0)) - thread.processPartitionData(tp1, 0, partitionData.setPartitionIndex(1)) + thread.processPartitionData(tp0, 0, Int.MaxValue, partitionData.setPartitionIndex(0)) + thread.processPartitionData(tp1, 0, Int.MaxValue, partitionData.setPartitionIndex(1)) verify(replicaManager, times(0)).completeDelayedFetchRequests(any[Seq[TopicPartition]]) thread.doWork() @@ -761,7 +774,7 @@ class ReplicaFetcherThreadTest { when(partition.localLogOrException).thenReturn(log) when(partition.isReassigning).thenReturn(isReassigning) when(partition.isAddingLocalReplica).thenReturn(isReassigning) - when(partition.appendRecordsToFollowerOrFutureReplica(records, isFuture = false)).thenReturn(None) + when(partition.appendRecordsToFollowerOrFutureReplica(records, isFuture = false, Int.MaxValue)).thenReturn(None) val replicaManager: ReplicaManager = mock(classOf[ReplicaManager]) when(replicaManager.getPartitionOrException(any[TopicPartition])).thenReturn(partition) @@ -785,7 +798,7 @@ class ReplicaFetcherThreadTest { .setLastStableOffset(0) .setLogStartOffset(0) .setRecords(records) - thread.processPartitionData(t1p0, 0, partitionData) + thread.processPartitionData(t1p0, 0, Int.MaxValue, partitionData) if (isReassigning) assertEquals(records.sizeInBytes(), brokerTopicStats.allTopicsStats.reassignmentBytesInPerSec.get.count()) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 5c2b61adfb..a09e1f6ebd 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -5248,9 +5248,12 @@ class ReplicaManagerTest { replicaManager.getPartition(topicPartition) match { case HostedPartition.Online(partition) => partition.appendRecordsToFollowerOrFutureReplica( - records = MemoryRecords.withRecords(Compression.NONE, 0, - new SimpleRecord("first message".getBytes)), - isFuture = false + records = MemoryRecords.withRecords( + Compression.NONE, 0, + new SimpleRecord("first message".getBytes) + ), + isFuture = false, + partitionLeaderEpoch = 0 ) case _ => diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java index a6a13a165a..dc2f9857bc 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java @@ -336,8 +336,12 @@ public Option endOffsetForEpoch(TopicPartition topicPartition, i } @Override - public Option processPartitionData(TopicPartition topicPartition, long fetchOffset, - FetchResponseData.PartitionData partitionData) { + public Option processPartitionData( + TopicPartition topicPartition, + long fetchOffset, + int partitionLeaderEpoch, + FetchResponseData.PartitionData partitionData + ) { return Option.empty(); } } diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java index 80505d53d2..7b56b68dd3 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java @@ -134,7 +134,7 @@ public void setup() throws IOException { int initialOffSet = 0; while (true) { MemoryRecords memoryRecords = MemoryRecords.withRecords(initialOffSet, Compression.NONE, 0, simpleRecords); - partition.appendRecordsToFollowerOrFutureReplica(memoryRecords, false); + partition.appendRecordsToFollowerOrFutureReplica(memoryRecords, false, Integer.MAX_VALUE); initialOffSet = initialOffSet + 2; } }); diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 75c90fe385..d1b1d3477c 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.InvalidRecordException; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; @@ -23,6 +24,7 @@ import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.errors.ClusterAuthorizationException; +import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.errors.NotLeaderOrFollowerException; import org.apache.kafka.common.feature.SupportedVersionRange; import org.apache.kafka.common.memory.MemoryPool; @@ -50,6 +52,7 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.DefaultRecordBatch; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.UnalignedMemoryRecords; @@ -93,8 +96,10 @@ import org.slf4j.Logger; import java.net.InetSocketAddress; +import java.nio.ByteBuffer; import java.util.Collection; import java.util.Collections; +import java.util.HexFormat; import java.util.IdentityHashMap; import java.util.Iterator; import java.util.List; @@ -1785,10 +1790,7 @@ private boolean handleFetchResponse( } } } else { - Records records = FetchResponse.recordsOrFail(partitionResponse); - if (records.sizeInBytes() > 0) { - appendAsFollower(records); - } + appendAsFollower(FetchResponse.recordsOrFail(partitionResponse)); OptionalLong highWatermark = partitionResponse.highWatermark() < 0 ? OptionalLong.empty() : OptionalLong.of(partitionResponse.highWatermark()); @@ -1802,10 +1804,31 @@ private boolean handleFetchResponse( } } - private void appendAsFollower( - Records records - ) { - LogAppendInfo info = log.appendAsFollower(records); + private static String convertToHexadecimal(Records records) { + ByteBuffer buffer = ((MemoryRecords) records).buffer(); + byte[] bytes = new byte[Math.min(buffer.remaining(), DefaultRecordBatch.RECORD_BATCH_OVERHEAD)]; + buffer.get(bytes); + + return HexFormat.of().formatHex(bytes); + } + + private void appendAsFollower(Records records) { + if (records.sizeInBytes() == 0) { + // Nothing to do if there are no bytes in the response + return; + } + + try { + var info = log.appendAsFollower(records, quorum.epoch()); + kafkaRaftMetrics.updateFetchedRecords(info.lastOffset - info.firstOffset + 1); + } catch (CorruptRecordException | InvalidRecordException e) { + logger.info( + "Failed to append the records with the batch header '{}' to the log", + convertToHexadecimal(records), + e + ); + } + if (quorum.isVoter() || followersAlwaysFlush) { // the leader only requires that voters have flushed their log before sending a Fetch // request. Because of reconfiguration some observers (that are getting added to the @@ -1817,14 +1840,11 @@ private void appendAsFollower( partitionState.updateState(); OffsetAndEpoch endOffset = endOffset(); - kafkaRaftMetrics.updateFetchedRecords(info.lastOffset - info.firstOffset + 1); kafkaRaftMetrics.updateLogEnd(endOffset); logger.trace("Follower end offset updated to {} after append", endOffset); } - private LogAppendInfo appendAsLeader( - Records records - ) { + private LogAppendInfo appendAsLeader(Records records) { LogAppendInfo info = log.appendAsLeader(records, quorum.epoch()); partitionState.updateState(); @@ -3477,6 +3497,10 @@ private long append(int epoch, List records) { () -> new NotLeaderException("Append failed because the replica is not the current leader") ); + if (records.isEmpty()) { + throw new IllegalArgumentException("Append failed because there are no records"); + } + BatchAccumulator accumulator = leaderState.accumulator(); boolean isFirstAppend = accumulator.isEmpty(); final long offset = accumulator.append(epoch, records, true); diff --git a/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java b/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java index a22f7fd73c..8f5ba31a45 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java +++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java @@ -31,6 +31,8 @@ public interface ReplicatedLog extends AutoCloseable { * be written atomically in a single batch or the call will fail and raise an * exception. * + * @param records records batches to append + * @param epoch the epoch of the replica * @return the metadata information of the appended batch * @throws IllegalArgumentException if the record set is empty * @throws RuntimeException if the batch base offset doesn't match the log end offset @@ -42,11 +44,16 @@ public interface ReplicatedLog extends AutoCloseable { * difference from appendAsLeader is that we do not need to assign the epoch * or do additional validation. * + * The log will append record batches up to and including batches that have a partition + * leader epoch less than or equal to the passed epoch. + * + * @param records records batches to append + * @param epoch the epoch of the replica * @return the metadata information of the appended batch * @throws IllegalArgumentException if the record set is empty * @throws RuntimeException if the batch base offset doesn't match the log end offset */ - LogAppendInfo appendAsFollower(Records records); + LogAppendInfo appendAsFollower(Records records, int epoch); /** * Read a set of records within a range of offsets. diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientFetchTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientFetchTest.java new file mode 100644 index 0000000000..ade509d805 --- /dev/null +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientFetchTest.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.raft; + +import org.apache.kafka.common.compress.Compression; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.ArbitraryMemoryRecords; +import org.apache.kafka.common.record.InvalidMemoryRecordsProvider; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.SimpleRecord; +import org.apache.kafka.server.common.KRaftVersion; + +import net.jqwik.api.AfterFailureMode; +import net.jqwik.api.ForAll; +import net.jqwik.api.Property; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ArgumentsSource; + +import java.nio.ByteBuffer; +import java.util.Optional; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public final class KafkaRaftClientFetchTest { + @Property(tries = 100, afterFailure = AfterFailureMode.SAMPLE_ONLY) + void testRandomRecords( + @ForAll(supplier = ArbitraryMemoryRecords.class) MemoryRecords memoryRecords + ) throws Exception { + testFetchResponseWithInvalidRecord(memoryRecords, Integer.MAX_VALUE); + } + + @ParameterizedTest + @ArgumentsSource(InvalidMemoryRecordsProvider.class) + void testInvalidMemoryRecords(MemoryRecords records, Optional> expectedException) throws Exception { + // CorruptRecordException are handled by the KafkaRaftClient so ignore the expected exception + testFetchResponseWithInvalidRecord(records, Integer.MAX_VALUE); + } + + private static void testFetchResponseWithInvalidRecord(MemoryRecords records, int epoch) throws Exception { + int localId = KafkaRaftClientTest.randomReplicaId(); + ReplicaKey local = KafkaRaftClientTest.replicaKey(localId, true); + ReplicaKey electedLeader = KafkaRaftClientTest.replicaKey(localId + 1, true); + + RaftClientTestContext context = new RaftClientTestContext.Builder( + local.id(), + local.directoryId().get() + ) + .withStartingVoters( + VoterSetTest.voterSet(Stream.of(local, electedLeader)), KRaftVersion.KRAFT_VERSION_1 + ) + .withElectedLeader(epoch, electedLeader.id()) + .withRaftProtocol(RaftClientTestContext.RaftProtocol.KIP_996_PROTOCOL) + .build(); + + context.pollUntilRequest(); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + + long oldLogEndOffset = context.log.endOffset().offset(); + + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse(epoch, electedLeader.id(), records, 0L, Errors.NONE) + ); + + context.client.poll(); + + assertEquals(oldLogEndOffset, context.log.endOffset().offset()); + } + + @Test + void testReplicationOfHigherPartitionLeaderEpoch() throws Exception { + int epoch = 2; + int localId = KafkaRaftClientTest.randomReplicaId(); + ReplicaKey local = KafkaRaftClientTest.replicaKey(localId, true); + ReplicaKey electedLeader = KafkaRaftClientTest.replicaKey(localId + 1, true); + + RaftClientTestContext context = new RaftClientTestContext.Builder( + local.id(), + local.directoryId().get() + ) + .withStartingVoters( + VoterSetTest.voterSet(Stream.of(local, electedLeader)), KRaftVersion.KRAFT_VERSION_1 + ) + .withElectedLeader(epoch, electedLeader.id()) + .withRaftProtocol(RaftClientTestContext.RaftProtocol.KIP_996_PROTOCOL) + .build(); + + context.pollUntilRequest(); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + + long oldLogEndOffset = context.log.endOffset().offset(); + int numberOfRecords = 10; + MemoryRecords batchWithValidEpoch = MemoryRecords.withRecords( + oldLogEndOffset, + Compression.NONE, + epoch, + IntStream + .range(0, numberOfRecords) + .mapToObj(number -> new SimpleRecord(Integer.toString(number).getBytes())) + .toArray(SimpleRecord[]::new) + ); + + MemoryRecords batchWithInvalidEpoch = MemoryRecords.withRecords( + oldLogEndOffset + numberOfRecords, + Compression.NONE, + epoch + 1, + IntStream + .range(0, numberOfRecords) + .mapToObj(number -> new SimpleRecord(Integer.toString(number).getBytes())) + .toArray(SimpleRecord[]::new) + ); + + var buffer = ByteBuffer.allocate(batchWithValidEpoch.sizeInBytes() + batchWithInvalidEpoch.sizeInBytes()); + buffer.put(batchWithValidEpoch.buffer()); + buffer.put(batchWithInvalidEpoch.buffer()); + buffer.flip(); + + MemoryRecords records = MemoryRecords.readableRecords(buffer); + + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse(epoch, electedLeader.id(), records, 0L, Errors.NONE) + ); + + context.client.poll(); + + // Check that only the first batch was appended because the second batch has a greater epoch + assertEquals(oldLogEndOffset + numberOfRecords, context.log.endOffset().offset()); + } +} diff --git a/raft/src/test/java/org/apache/kafka/raft/MockLog.java b/raft/src/test/java/org/apache/kafka/raft/MockLog.java index a7a8e89a88..9fb4724cc0 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockLog.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockLog.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.compress.Compression; +import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.errors.OffsetOutOfRangeException; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecordsBuilder; @@ -279,7 +280,7 @@ private LogEntry buildEntry(Long offset, SimpleRecord record) { @Override public LogAppendInfo appendAsLeader(Records records, int epoch) { - return append(records, OptionalInt.of(epoch)); + return append(records, epoch, true); } private long appendBatch(LogBatch batch) { @@ -292,16 +293,18 @@ private long appendBatch(LogBatch batch) { } @Override - public LogAppendInfo appendAsFollower(Records records) { - return append(records, OptionalInt.empty()); + public LogAppendInfo appendAsFollower(Records records, int epoch) { + return append(records, epoch, false); } - private LogAppendInfo append(Records records, OptionalInt epoch) { - if (records.sizeInBytes() == 0) + private LogAppendInfo append(Records records, int epoch, boolean isLeader) { + if (records.sizeInBytes() == 0) { throw new IllegalArgumentException("Attempt to append an empty record set"); + } long baseOffset = endOffset().offset(); long lastOffset = baseOffset; + boolean hasBatches = false; for (RecordBatch batch : records.batches()) { if (batch.baseOffset() != endOffset().offset()) { /* KafkaMetadataLog throws an kafka.common.UnexpectedAppendOffsetException this is the @@ -314,26 +317,47 @@ private LogAppendInfo append(Records records, OptionalInt epoch) { endOffset().offset() ) ); + } else if (isLeader && epoch != batch.partitionLeaderEpoch()) { + // the partition leader epoch is set and does not match the one set in the batch + throw new RuntimeException( + String.format( + "Epoch %s doesn't match batch leader epoch %s", + epoch, + batch.partitionLeaderEpoch() + ) + ); + } else if (!isLeader && batch.partitionLeaderEpoch() > epoch) { + /* To avoid inconsistent log replication, follower should only append record + * batches with an epoch less than or equal to the leader epoch. There is more + * details on this issue and scenario in KAFKA-18723. + */ + break; } + hasBatches = true; LogBatch logBatch = new LogBatch( - epoch.orElseGet(batch::partitionLeaderEpoch), + batch.partitionLeaderEpoch(), batch.isControlBatch(), buildEntries(batch, Record::offset) ); if (logger.isDebugEnabled()) { - String nodeState = "Follower"; - if (epoch.isPresent()) { - nodeState = "Leader"; - } - logger.debug("{} appending to the log {}", nodeState, logBatch); + logger.debug( + "{} appending to the log {}", + isLeader ? "Leader" : "Follower", + logBatch + ); } appendBatch(logBatch); lastOffset = logBatch.last().offset; } + if (!hasBatches) { + // This emulates the default handling when records doesn't have enough bytes for a batch + throw new CorruptRecordException("Append failed unexpectedly"); + } + return new LogAppendInfo(baseOffset, lastOffset); } diff --git a/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java b/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java index 8306e10325..eca0fe5d3d 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java @@ -19,9 +19,12 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.compress.Compression; +import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.errors.OffsetOutOfRangeException; import org.apache.kafka.common.message.LeaderChangeMessage; +import org.apache.kafka.common.record.ArbitraryMemoryRecords; import org.apache.kafka.common.record.ControlRecordUtils; +import org.apache.kafka.common.record.InvalidMemoryRecordsProvider; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.RecordBatch; @@ -32,9 +35,16 @@ import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.snapshot.RawSnapshotWriter; +import net.jqwik.api.AfterFailureMode; +import net.jqwik.api.ForAll; +import net.jqwik.api.Property; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.function.Executable; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ArgumentsSource; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -44,6 +54,7 @@ import java.util.List; import java.util.Objects; import java.util.Optional; +import java.util.stream.IntStream; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -169,14 +180,17 @@ public void testUnexpectedAppendOffset() { assertThrows( RuntimeException.class, () -> log.appendAsLeader( - MemoryRecords.withRecords(initialOffset, Compression.NONE, currentEpoch, recordFoo), - currentEpoch) + MemoryRecords.withRecords(initialOffset, Compression.NONE, currentEpoch, recordFoo), + currentEpoch + ) ); assertThrows( RuntimeException.class, () -> log.appendAsFollower( - MemoryRecords.withRecords(initialOffset, Compression.NONE, currentEpoch, recordFoo)) + MemoryRecords.withRecords(initialOffset, Compression.NONE, currentEpoch, recordFoo), + currentEpoch + ) ); } @@ -187,7 +201,13 @@ public void testAppendControlRecord() { LeaderChangeMessage messageData = new LeaderChangeMessage().setLeaderId(0); ByteBuffer buffer = ByteBuffer.allocate(256); log.appendAsLeader( - MemoryRecords.withLeaderChangeMessage(initialOffset, 0L, 2, buffer, messageData), + MemoryRecords.withLeaderChangeMessage( + initialOffset, + 0L, + currentEpoch, + buffer, + messageData + ), currentEpoch ); @@ -221,7 +241,10 @@ public void testAppendAsFollower() { } log.truncateToLatestSnapshot(); - log.appendAsFollower(MemoryRecords.withRecords(initialOffset, Compression.NONE, epoch, recordFoo)); + log.appendAsFollower( + MemoryRecords.withRecords(initialOffset, Compression.NONE, epoch, recordFoo), + epoch + ); assertEquals(initialOffset, log.startOffset()); assertEquals(initialOffset + 1, log.endOffset().offset()); @@ -368,10 +391,82 @@ public void testEndOffsetForEpoch() { @Test public void testEmptyAppendNotAllowed() { - assertThrows(IllegalArgumentException.class, () -> log.appendAsFollower(MemoryRecords.EMPTY)); + assertThrows(IllegalArgumentException.class, () -> log.appendAsFollower(MemoryRecords.EMPTY, 1)); assertThrows(IllegalArgumentException.class, () -> log.appendAsLeader(MemoryRecords.EMPTY, 1)); } + @ParameterizedTest + @ArgumentsSource(InvalidMemoryRecordsProvider.class) + void testInvalidMemoryRecords(MemoryRecords records, Optional> expectedException) { + long previousEndOffset = log.endOffset().offset(); + + Executable action = () -> log.appendAsFollower(records, Integer.MAX_VALUE); + if (expectedException.isPresent()) { + assertThrows(expectedException.get(), action); + } else { + assertThrows(CorruptRecordException.class, action); + } + + assertEquals(previousEndOffset, log.endOffset().offset()); + } + + @Property(tries = 100, afterFailure = AfterFailureMode.SAMPLE_ONLY) + void testRandomRecords( + @ForAll(supplier = ArbitraryMemoryRecords.class) MemoryRecords records + ) { + try (MockLog log = new MockLog(topicPartition, topicId, new LogContext())) { + long previousEndOffset = log.endOffset().offset(); + + assertThrows( + CorruptRecordException.class, + () -> log.appendAsFollower(records, Integer.MAX_VALUE) + ); + + assertEquals(previousEndOffset, log.endOffset().offset()); + } + } + + @Test + void testInvalidLeaderEpoch() { + var previousEndOffset = log.endOffset().offset(); + var epoch = log.lastFetchedEpoch() + 1; + var numberOfRecords = 10; + + MemoryRecords batchWithValidEpoch = MemoryRecords.withRecords( + previousEndOffset, + Compression.NONE, + epoch, + IntStream + .range(0, numberOfRecords) + .mapToObj(number -> new SimpleRecord(Integer.toString(number).getBytes())) + .toArray(SimpleRecord[]::new) + ); + + MemoryRecords batchWithInvalidEpoch = MemoryRecords.withRecords( + previousEndOffset + numberOfRecords, + Compression.NONE, + epoch + 1, + IntStream + .range(0, numberOfRecords) + .mapToObj(number -> new SimpleRecord(Integer.toString(number).getBytes())) + .toArray(SimpleRecord[]::new) + ); + + var buffer = ByteBuffer.allocate(batchWithValidEpoch.sizeInBytes() + batchWithInvalidEpoch.sizeInBytes()); + buffer.put(batchWithValidEpoch.buffer()); + buffer.put(batchWithInvalidEpoch.buffer()); + buffer.flip(); + + var records = MemoryRecords.readableRecords(buffer); + + log.appendAsFollower(records, epoch); + + // Check that only the first batch was appended + assertEquals(previousEndOffset + numberOfRecords, log.endOffset().offset()); + // Check that the last fetched epoch matches the first batch + assertEquals(epoch, log.lastFetchedEpoch()); + } + @Test public void testReadOutOfRangeOffset() { final long initialOffset = 5L; @@ -383,12 +478,19 @@ public void testReadOutOfRangeOffset() { } log.truncateToLatestSnapshot(); - log.appendAsFollower(MemoryRecords.withRecords(initialOffset, Compression.NONE, epoch, recordFoo)); + log.appendAsFollower( + MemoryRecords.withRecords(initialOffset, Compression.NONE, epoch, recordFoo), + epoch + ); - assertThrows(OffsetOutOfRangeException.class, () -> log.read(log.startOffset() - 1, - Isolation.UNCOMMITTED)); - assertThrows(OffsetOutOfRangeException.class, () -> log.read(log.endOffset().offset() + 1, - Isolation.UNCOMMITTED)); + assertThrows( + OffsetOutOfRangeException.class, + () -> log.read(log.startOffset() - 1, Isolation.UNCOMMITTED) + ); + assertThrows( + OffsetOutOfRangeException.class, + () -> log.read(log.endOffset().offset() + 1, Isolation.UNCOMMITTED) + ); } @Test @@ -958,6 +1060,7 @@ private void appendAsLeader(Collection records, int epoch) { MemoryRecords.withRecords( log.endOffset().offset(), Compression.NONE, + epoch, records.toArray(new SimpleRecord[records.size()]) ), epoch From 952c8a5e940a0d604c2c9b86cbc2c29c420c1318 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Wed, 26 Mar 2025 00:14:01 +0800 Subject: [PATCH 12/72] KAFKA-18991: FetcherThread should match leader epochs between fetch request and fetch state (#19223) This PR fixes a potential issue where the `FetchResponse` returns `divergingEndOffsets` with an older leader epoch. This can lead to committed records being removed from the follower's log, potentially causing data loss. In detail: `processFetchRequest` gets the requested leader epoch of partition data by `topicPartition` and compares it with the leader epoch of the current fetch state. If they don't match, the response is ignored. Reviewers: Jun Rao --- .../kafka/server/AbstractFetcherThread.scala | 12 ++++++--- .../server/AbstractFetcherThreadTest.scala | 26 ++++++++++++++++++- 2 files changed, 33 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index 7a98c83e7f..0fd9c9333d 100755 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -307,7 +307,8 @@ abstract class AbstractFetcherThread(name: String, } } - private def processFetchRequest(sessionPartitions: util.Map[TopicPartition, FetchRequest.PartitionData], + // visible for testing + private[server] def processFetchRequest(sessionPartitions: util.Map[TopicPartition, FetchRequest.PartitionData], fetchRequest: FetchRequest.Builder): Unit = { val partitionsWithError = mutable.Set[TopicPartition]() val divergingEndOffsets = mutable.Map.empty[TopicPartition, EpochEndOffset] @@ -333,11 +334,14 @@ abstract class AbstractFetcherThread(name: String, responseData.foreachEntry { (topicPartition, partitionData) => Option(partitionStates.stateValue(topicPartition)).foreach { currentFetchState => // It's possible that a partition is removed and re-added or truncated when there is a pending fetch request. - // In this case, we only want to process the fetch response if the partition state is ready for fetch and - // the current offset is the same as the offset requested. + // In this case, we only want to process the fetch response if: + // - the partition state is ready for fetch + // - the current offset is the same as the offset requested + // - the current leader epoch is the same as the leader epoch requested val fetchPartitionData = sessionPartitions.get(topicPartition) if (fetchPartitionData != null && fetchPartitionData.fetchOffset == currentFetchState.fetchOffset && + fetchPartitionData.currentLeaderEpoch.map[Boolean](_ == currentFetchState.currentLeaderEpoch).orElse(true) && currentFetchState.isReadyForFetch) { Errors.forCode(partitionData.errorCode) match { case Errors.NONE => @@ -362,7 +366,7 @@ abstract class AbstractFetcherThread(name: String, val logAppendInfoOpt = processPartitionData( topicPartition, currentFetchState.fetchOffset, - fetchPartitionData.currentLeaderEpoch.orElse(currentFetchState.currentLeaderEpoch), + currentFetchState.currentLeaderEpoch, partitionData ) diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala index aa25cad89c..174dbaafaa 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala @@ -1162,4 +1162,28 @@ class AbstractFetcherThreadTest { assertTrue(fetcher.fetchState(unknownPartition).isEmpty) } -} + @Test + def testIgnoreFetchResponseWhenLeaderEpochChanged(): Unit = { + val newEpoch = 1 + val initEpoch = 0 + + val partition = new TopicPartition("topic", 0) + val mockLeaderEndpoint = new MockLeaderEndPoint(version = version) + val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndpoint) + val fetcher = new MockFetcherThread(mockLeaderEndpoint, mockTierStateMachine) + val replicaState = PartitionState(leaderEpoch = newEpoch) + fetcher.setReplicaState(partition, replicaState) + val initFetchState = initialFetchState(topicIds.get(partition.topic), 0L, leaderEpoch = newEpoch) + fetcher.addPartitions(Map(partition -> initFetchState)) + + val batch = mkBatch(baseOffset = 0L, leaderEpoch = initEpoch, new SimpleRecord("a".getBytes)) + val leaderState = PartitionState(Seq(batch), leaderEpoch = initEpoch, highWatermark = 1L) + fetcher.mockLeader.setLeaderState(partition, leaderState) + + val partitionData = Map(partition -> new FetchRequest.PartitionData(Uuid.randomUuid(), 0, 0, 1048576, Optional.of(initEpoch), Optional.of(initEpoch))).asJava + val fetchRequestOpt = FetchRequest.Builder.forReplica(0, 0, initEpoch, 0, Int.MaxValue, partitionData) + + fetcher.processFetchRequest(partitionData, fetchRequestOpt) + assertEquals(0, replicaState.logEndOffset, "FetchResponse should be ignored when leader epoch does not match") + } +} \ No newline at end of file From 83f6a1d7e6dfce4a78e1192a8fecf523b39ddaab Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Wed, 9 Apr 2025 12:52:15 -0400 Subject: [PATCH 13/72] KAFKA-18991; Missing change for cherry-pick --- .../scala/unit/kafka/server/AbstractFetcherThreadTest.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala index 174dbaafaa..7a555b400a 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala @@ -32,6 +32,7 @@ import org.junit.jupiter.api.Assumptions.assumeTrue import org.junit.jupiter.api.{BeforeEach, Test} import kafka.server.FetcherThreadTestUtils.{initialFetchState, mkBatch} +import java.util.Optional; import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.ArrayBuffer import scala.collection.{Map, Set} @@ -1186,4 +1187,4 @@ class AbstractFetcherThreadTest { fetcher.processFetchRequest(partitionData, fetchRequestOpt) assertEquals(0, replicaState.logEndOffset, "FetchResponse should be ignored when leader epoch does not match") } -} \ No newline at end of file +} From de27409e307649595fe9ce541c8c2fc23351471e Mon Sep 17 00:00:00 2001 From: Florian Hussonnois Date: Wed, 9 Apr 2025 22:17:38 +0200 Subject: [PATCH 14/72] KAFKA-18962: Fix onBatchRestored call in GlobalStateManagerImpl (#19188) Call the StateRestoreListener#onBatchRestored with numRestored and not the totalRestored when reprocessing state See: https://issues.apache.org/jira/browse/KAFKA-18962 Reviewers: Anna Sophie Blee-Goldman , Matthias Sax --- checkstyle/suppressions.xml | 2 +- .../kafka/clients/consumer/MockConsumer.java | 39 ++++++++++++++++--- .../clients/consumer/MockConsumerTest.java | 30 ++++++++++++++ .../internals/GlobalStateManagerImpl.java | 4 +- .../internals/GlobalStateManagerImplTest.java | 25 ++++++++++-- 5 files changed, 89 insertions(+), 11 deletions(-) diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index e48b71c450..a690852930 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -105,7 +105,7 @@ files="(AbstractRequest|AbstractResponse|KerberosLogin|WorkerSinkTaskTest|TransactionManagerTest|SenderTest|KafkaAdminClient|ConsumerCoordinatorTest|KafkaAdminClientTest).java"/> + files="(AbstractMembershipManager|ConsumerCoordinator|BufferPool|MetricName|Node|ConfigDef|RecordBatch|SslFactory|SslTransportLayer|MetadataResponse|KerberosLogin|Selector|Sender|Serdes|TokenInformation|Agent|PluginUtils|MiniTrogdorCluster|TasksRequest|KafkaProducer|AbstractStickyAssignor|Authorizer|FetchSessionHandler|RecordAccumulator|Shell|MockConsumer).java"/> diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java index b9e6980669..c92ce6ec19 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -36,6 +36,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -79,6 +80,8 @@ public class MockConsumer implements Consumer { private Uuid clientInstanceId; private int injectTimeoutExceptionCounter; + private long maxPollRecords = Long.MAX_VALUE; + private final List addedMetrics = new ArrayList<>(); /** @@ -275,14 +278,22 @@ public synchronized ConsumerRecords poll(final Duration timeout) { // update the consumed offset final Map>> results = new HashMap<>(); final Map nextOffsetAndMetadata = new HashMap<>(); - final List toClear = new ArrayList<>(); + long numPollRecords = 0L; + + final Iterator>>> partitionsIter = this.records.entrySet().iterator(); + while (partitionsIter.hasNext() && numPollRecords < this.maxPollRecords) { + Map.Entry>> entry = partitionsIter.next(); - for (Map.Entry>> entry : this.records.entrySet()) { if (!subscriptions.isPaused(entry.getKey())) { - final List> recs = entry.getValue(); - for (final ConsumerRecord rec : recs) { + final Iterator> recIterator = entry.getValue().iterator(); + while (recIterator.hasNext()) { + if (numPollRecords >= this.maxPollRecords) { + break; + } long position = subscriptions.position(entry.getKey()).offset; + final ConsumerRecord rec = recIterator.next(); + if (beginningOffsets.get(entry.getKey()) != null && beginningOffsets.get(entry.getKey()) > position) { throw new OffsetOutOfRangeException(Collections.singletonMap(entry.getKey(), position)); } @@ -294,13 +305,17 @@ public synchronized ConsumerRecords poll(final Duration timeout) { rec.offset() + 1, rec.leaderEpoch(), leaderAndEpoch); subscriptions.position(entry.getKey(), newPosition); nextOffsetAndMetadata.put(entry.getKey(), new OffsetAndMetadata(rec.offset() + 1, rec.leaderEpoch(), "")); + numPollRecords++; + recIterator.remove(); } } - toClear.add(entry.getKey()); + + if (entry.getValue().isEmpty()) { + partitionsIter.remove(); + } } } - toClear.forEach(records::remove); return new ConsumerRecords<>(results, nextOffsetAndMetadata); } @@ -314,6 +329,18 @@ public synchronized void addRecord(ConsumerRecord record) { recs.add(record); } + /** + * Sets the maximum number of records returned in a single call to {@link #poll(Duration)}. + * + * @param maxPollRecords the max.poll.records. + */ + public synchronized void setMaxPollRecords(long maxPollRecords) { + if (this.maxPollRecords < 1) { + throw new IllegalArgumentException("MaxPollRecords must be strictly superior to 0"); + } + this.maxPollRecords = maxPollRecords; + } + public synchronized void setPollException(KafkaException exception) { this.pollException = exception; } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java index 21cee3183b..647976b1d1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java @@ -32,6 +32,7 @@ import java.util.Iterator; import java.util.List; import java.util.Optional; +import java.util.stream.IntStream; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -202,4 +203,33 @@ public void testRe2JPatternSubscription() { assertThrows(IllegalStateException.class, () -> consumer.subscribe(List.of("topic1"))); } + @Test + public void shouldReturnMaxPollRecords() { + TopicPartition partition = new TopicPartition("test", 0); + consumer.assign(Collections.singleton(partition)); + consumer.updateBeginningOffsets(Collections.singletonMap(partition, 0L)); + + IntStream.range(0, 10).forEach(offset -> { + consumer.addRecord(new ConsumerRecord<>("test", 0, offset, null, null)); + }); + + consumer.setMaxPollRecords(2L); + + ConsumerRecords records; + + records = consumer.poll(Duration.ofMillis(1)); + assertEquals(2, records.count()); + + records = consumer.poll(Duration.ofMillis(1)); + assertEquals(2, records.count()); + + consumer.setMaxPollRecords(Long.MAX_VALUE); + + records = consumer.poll(Duration.ofMillis(1)); + assertEquals(6, records.count()); + + records = consumer.poll(Duration.ofMillis(1)); + assertTrue(records.isEmpty()); + } + } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java index 53064da373..2bf65c31d7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java @@ -300,6 +300,7 @@ private void reprocessState(final List topicPartitions, currentDeadline = NO_DEADLINE; } + long batchRestoreCount = 0; for (final ConsumerRecord record : records.records(topicPartition)) { final ProcessorRecordContext recordContext = new ProcessorRecordContext( @@ -318,6 +319,7 @@ private void reprocessState(final List topicPartitions, record.timestamp(), record.headers())); restoreCount++; + batchRestoreCount++; } } catch (final Exception deserializationException) { // while Java distinguishes checked vs unchecked exceptions, other languages @@ -341,7 +343,7 @@ private void reprocessState(final List topicPartitions, offset = getGlobalConsumerOffset(topicPartition); - stateRestoreListener.onBatchRestored(topicPartition, storeName, offset, restoreCount); + stateRestoreListener.onBatchRestored(topicPartition, storeName, offset, batchRestoreCount); } stateRestoreListener.onRestoreEnd(topicPartition, storeName, restoreCount); checkpointFileCache.put(topicPartition, offset); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java index e28ef673f0..758f8cd500 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java @@ -355,16 +355,35 @@ public void shouldRestoreRecordsUpToHighwatermark() { assertEquals(2, stateRestoreCallback.restored.size()); } + @Test + public void shouldListenForRestoreEventsWhenReprocessing() { + setUpReprocessing(); + + initializeConsumer(6, 1, t1); + consumer.setMaxPollRecords(2L); + + stateManager.initialize(); + stateManager.registerStore(store1, stateRestoreCallback, null); + + assertThat(stateRestoreListener.numBatchRestored, equalTo(2L)); + assertThat(stateRestoreListener.restoreStartOffset, equalTo(1L)); + assertThat(stateRestoreListener.restoreEndOffset, equalTo(7L)); + assertThat(stateRestoreListener.totalNumRestored, equalTo(6L)); + } + @Test public void shouldListenForRestoreEvents() { - initializeConsumer(5, 1, t1); + initializeConsumer(6, 1, t1); + consumer.setMaxPollRecords(2L); + stateManager.initialize(); stateManager.registerStore(store1, stateRestoreCallback, null); + assertThat(stateRestoreListener.numBatchRestored, equalTo(2L)); assertThat(stateRestoreListener.restoreStartOffset, equalTo(1L)); - assertThat(stateRestoreListener.restoreEndOffset, equalTo(6L)); - assertThat(stateRestoreListener.totalNumRestored, equalTo(5L)); + assertThat(stateRestoreListener.restoreEndOffset, equalTo(7L)); + assertThat(stateRestoreListener.totalNumRestored, equalTo(6L)); assertThat(stateRestoreListener.storeNameCalledStates.get(RESTORE_START), equalTo(store1.name())); From 143fcb1d7c71dcfa16a008f85c9e7be3101beee2 Mon Sep 17 00:00:00 2001 From: Azhar Ahmed Date: Sun, 13 Apr 2025 20:08:49 -0700 Subject: [PATCH 15/72] KAFKA-19071: Fix doc for remote.storage.enable (#19345) As of 3.9, Kafka allows disabling remote storage on a topic after it was enabled. It allows subsequent enabling and disabling too. However the documentation says otherwise and needs to be corrected. Doc: https://kafka.apache.org/39/documentation/#topicconfigs_remote.storage.enable Reviewers: Luke Chen , PoAn Yang , Ken Huang --- .../kafka/common/config/TopicConfig.java | 5 +-- .../kafka/admin/RemoteTopicCrudTest.scala | 31 ++++++++++++++++++- 2 files changed, 33 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 10d6024163..181e6d72bc 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -80,8 +80,9 @@ public class TopicConfig { "Moreover, it triggers the rolling of new segment if the retention.ms condition is satisfied."; public static final String REMOTE_LOG_STORAGE_ENABLE_CONFIG = "remote.storage.enable"; - public static final String REMOTE_LOG_STORAGE_ENABLE_DOC = "To enable tiered storage for a topic, set this configuration as true. " + - "You can not disable this config once it is enabled. It will be provided in future versions."; + public static final String REMOTE_LOG_STORAGE_ENABLE_DOC = "To enable tiered storage for a topic, set this configuration to true. " + + "To disable tiered storage for a topic that has it enabled, set this configuration to false. " + + "When disabling, you must also set remote.log.delete.on.disable to true."; public static final String LOCAL_LOG_RETENTION_MS_CONFIG = "local.retention.ms"; public static final String LOCAL_LOG_RETENTION_MS_DOC = "The number of milliseconds to keep the local log segment before it gets deleted. " + diff --git a/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala b/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala index d89a83c775..4f66dd9e31 100644 --- a/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala +++ b/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala @@ -441,7 +441,36 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { AlterConfigOp.OpType.SET), )) assertThrowsException(classOf[InvalidConfigurationException], - () => admin.incrementalAlterConfigs(configs).all().get(), "Disabling remote storage feature on the topic level is not supported.") + () => admin.incrementalAlterConfigs(configs).all().get(), "It is invalid to disable remote storage without deleting remote data. " + + "If you want to keep the remote data and turn to read only, please set `remote.storage.enable=true,remote.log.copy.disable=true`. " + + "If you want to disable remote storage and delete all remote data, please set `remote.storage.enable=false,remote.log.delete.on.disable=true`.") + } + + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testUpdateTopicConfigWithDisablingRemoteStorageWithDeleteOnDisable(quorum: String): Unit = { + val admin = createAdminClient() + val topicConfig = new Properties + topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + TestUtils.createTopicWithAdmin(admin, testTopicName, brokers, controllerServers, numPartitions, numReplicationFactor, + topicConfig = topicConfig) + + val configs = new util.HashMap[ConfigResource, util.Collection[AlterConfigOp]]() + configs.put(new ConfigResource(ConfigResource.Type.TOPIC, testTopicName), + util.Arrays.asList( + new AlterConfigOp(new ConfigEntry(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "false"), + AlterConfigOp.OpType.SET), + new AlterConfigOp(new ConfigEntry(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, "true"), + AlterConfigOp.OpType.SET) + )) + admin.incrementalAlterConfigs(configs).all().get() + + val newProps = new Properties() + configs.get(new ConfigResource(ConfigResource.Type.TOPIC, testTopicName)).forEach { op => + newProps.setProperty(op.configEntry().name(), op.configEntry().value()) + } + + verifyRemoteLogTopicConfigs(newProps) } @ParameterizedTest From f98dec9440c77449e2194024afdd13374ddebfdb Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Wed, 16 Apr 2025 12:57:44 +0100 Subject: [PATCH 16/72] KAFKA-19147: Start authorizer before group coordinator to ensure coordinator authorizes regex topics (#19488) [KAFKA-18813](https://issues.apache.org/jira/browse/KAFKA-18813) added `Topic:Describe` authorization of topics matching regex patterns to the group coordinator since it was difficult to authorize these in the broker when processing consumer heartbeats using the new protocol. But group coordinator is started in `BrokerServer` before the authorizer is created. And hence group coordinator doesn't have an authorizer and never performs authorization. As a result, topics that are not authorized for `Describe` may be assigned to consumers. This potentially leaks information about topic existence, topic id and partition count to users who are not authorized to describe a topic. This PR starts authorizer earlier to ensure that authorization is performed by the group coordinator. Also adds integration tests for verification. Note that we still have a second issue when members have different permissions. If regex is resolved by a member with permission to more topics, unauthorized topics may be assigned to members with lower permissions. In this case, we still return assignment containing topic id and partitions to the member without `Topic:Describe` access. This is not addressed by this PR, but an integration test that illustrates the issue has been added so that we can verify when the issue is fixed. Reviewers: David Jacot --- .../scala/kafka/server/BrokerServer.scala | 8 +- .../kafka/api/AuthorizerIntegrationTest.scala | 120 +++++++++++++++++- 2 files changed, 121 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 6d2d108908..4fdd3a0779 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -358,6 +358,10 @@ class BrokerServer( tokenManager = new DelegationTokenManager(config, tokenCache, time) tokenManager.startup() + // Create and initialize an authorizer if one is configured. + authorizer = config.createNewAuthorizer() + authorizer.foreach(_.configure(config.originals)) + /* initializing the groupConfigManager */ groupConfigManager = new GroupConfigManager(config.groupCoordinatorConfig.extractGroupConfigMap(config.shareGroupConfig)) @@ -412,10 +416,6 @@ class BrokerServer( logManager.readBrokerEpochFromCleanShutdownFiles() ) - // Create and initialize an authorizer if one is configured. - authorizer = config.createNewAuthorizer() - authorizer.foreach(_.configure(config.originals)) - // The FetchSessionCache is divided into config.numIoThreads shards, each responsible // for Math.max(1, shardNum * sessionIdRange) <= sessionId < (shardNum + 1) * sessionIdRange val sessionIdRange = Int.MaxValue / NumFetchSessionCacheShards diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index fc74344c86..671386a282 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -37,7 +37,8 @@ import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProt import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity import org.apache.kafka.common.message.ListOffsetsRequestData.{ListOffsetsPartition, ListOffsetsTopic} import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.{OffsetForLeaderPartition, OffsetForLeaderTopic, OffsetForLeaderTopicCollection} -import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AlterPartitionReassignmentsRequestData, AlterReplicaLogDirsRequestData, ConsumerGroupDescribeRequestData, ConsumerGroupHeartbeatRequestData, CreateAclsRequestData, CreatePartitionsRequestData, CreateTopicsRequestData, DeleteAclsRequestData, DeleteGroupsRequestData, DeleteRecordsRequestData, DeleteTopicsRequestData, DescribeClusterRequestData, DescribeConfigsRequestData, DescribeGroupsRequestData, DescribeLogDirsRequestData, DescribeProducersRequestData, DescribeTransactionsRequestData, FetchResponseData, FindCoordinatorRequestData, HeartbeatRequestData, IncrementalAlterConfigsRequestData, JoinGroupRequestData, ListPartitionReassignmentsRequestData, ListTransactionsRequestData, MetadataRequestData, OffsetCommitRequestData, ProduceRequestData, SyncGroupRequestData, WriteTxnMarkersRequestData} +import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AlterPartitionReassignmentsRequestData, AlterReplicaLogDirsRequestData, ConsumerGroupDescribeRequestData, ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData, CreateAclsRequestData, CreatePartitionsRequestData, CreateTopicsRequestData, DeleteAclsRequestData, DeleteGroupsRequestData, DeleteRecordsRequestData, DeleteTopicsRequestData, DescribeClusterRequestData, DescribeConfigsRequestData, DescribeGroupsRequestData, DescribeLogDirsRequestData, DescribeProducersRequestData, DescribeTransactionsRequestData, FetchResponseData, FindCoordinatorRequestData, HeartbeatRequestData, IncrementalAlterConfigsRequestData, JoinGroupRequestData, ListPartitionReassignmentsRequestData, ListTransactionsRequestData, MetadataRequestData, OffsetCommitRequestData, ProduceRequestData, SyncGroupRequestData, WriteTxnMarkersRequestData} +import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, SimpleRecord} import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData @@ -2546,6 +2547,118 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testConsumerGroupHeartbeaWithRegex(quorum: String): Unit = { + createTopicWithBrokerPrincipal(topic) + val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) + addAndVerifyAcls(Set(allowAllOpsAcl), groupResource) + addAndVerifyAcls(Set(allowAllOpsAcl), topicResource) + + val response = sendAndReceiveFirstRegexHeartbeat(Uuid.randomUuid.toString, listenerName) + sendAndReceiveRegexHeartbeat(response, listenerName, Some(1)) + } + + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testConsumerGroupHeartbeaWithRegexWithoutTopicDescribeAcl(quorum: String): Unit = { + createTopicWithBrokerPrincipal(topic) + val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) + addAndVerifyAcls(Set(allowAllOpsAcl), groupResource) + + val response = sendAndReceiveFirstRegexHeartbeat(Uuid.randomUuid.toString, listenerName) + sendAndReceiveRegexHeartbeat(response, listenerName, None) + } + + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testConsumerGroupHeartbeaWithRegexWithDifferentMemberAcls(quorum: String): Unit = { + createTopicWithBrokerPrincipal(topic, numPartitions = 2) + val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) + addAndVerifyAcls(Set(allowAllOpsAcl), groupResource) + + // Member on inter-broker listener has all access and is assigned the matching topic + var member1Response = sendAndReceiveFirstRegexHeartbeat("memberWithAllAccess", interBrokerListenerName) + member1Response = sendAndReceiveRegexHeartbeat(member1Response, interBrokerListenerName, Some(2)) + + // Member on client listener has no topic describe access, but is assigned a partition of the + // unauthorized topic. This is leaking unauthorized topic metadata to member2. Simply filtering out + // the topic from the assignment in the response is not sufficient since different assignment states + // in the broker and client can lead to other issues. This needs to be fixed properly by using + // member permissions while computing assignments. + var member2Response = sendAndReceiveFirstRegexHeartbeat("memberWithLimitedAccess", listenerName) + member1Response = sendAndReceiveRegexHeartbeat(member1Response, interBrokerListenerName, Some(1)) + member1Response = sendAndReceiveRegexHeartbeat(member1Response, interBrokerListenerName, None, fullRequest = true) + member2Response = sendAndReceiveRegexHeartbeat(member2Response, listenerName, Some(1)) + + // Create another topic and send heartbeats on member1 to trigger regex refresh + createTopicWithBrokerPrincipal("topic2", numPartitions = 2) + TestUtils.retry(15000) { + member1Response = sendAndReceiveRegexHeartbeat(member1Response, interBrokerListenerName, Some(2)) + } + // This is leaking unauthorized topic metadata to member2. + member2Response = sendAndReceiveRegexHeartbeat(member2Response, listenerName, Some(2)) + + // Create another topic and send heartbeats on member2 to trigger regex refresh + createTopicWithBrokerPrincipal("topic3", numPartitions = 2) + TestUtils.retry(15000) { + member2Response = sendAndReceiveRegexHeartbeat(member2Response, listenerName, Some(0), fullRequest = true) + } + // This removes all topics from member1 since member2's permissions were used to refresh regex. + sendAndReceiveRegexHeartbeat(member1Response, interBrokerListenerName, Some(0), fullRequest = true) + } + + private def sendAndReceiveFirstRegexHeartbeat(memberId: String, + listenerName: ListenerName): ConsumerGroupHeartbeatResponseData = { + val request = new ConsumerGroupHeartbeatRequest.Builder( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(group) + .setMemberId(memberId) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(5 * 60 * 1000) + .setTopicPartitions(Collections.emptyList()) + .setSubscribedTopicRegex("^top.*")).build() + val resource = Set[ResourceType](GROUP, TOPIC) + val response = sendRequestAndVerifyResponseError(request, resource, isAuthorized = true, listenerName = listenerName) + .data.asInstanceOf[ConsumerGroupHeartbeatResponseData] + assertEquals(Errors.NONE.code, response.errorCode, s"Unexpected response $response") + assertEquals(0, response.assignment.topicPartitions.size, s"Unexpected assignment $response") + response + } + + private def sendAndReceiveRegexHeartbeat(lastResponse: ConsumerGroupHeartbeatResponseData, + listenerName: ListenerName, + expectedAssignmentSize: Option[Int], + fullRequest: Boolean = false): ConsumerGroupHeartbeatResponseData = { + var data = new ConsumerGroupHeartbeatRequestData() + .setGroupId(group) + .setMemberId(lastResponse.memberId) + .setMemberEpoch(lastResponse.memberEpoch) + if (fullRequest) { + val partitions = Option(lastResponse.assignment).map(_.topicPartitions.asScala.map(p => + new ConsumerGroupHeartbeatRequestData.TopicPartitions() + .setTopicId(p.topicId) + .setPartitions(p.partitions) + )).getOrElse(List()) + data = data + .setTopicPartitions(partitions.asJava) + .setSubscribedTopicRegex("^top.*") + } + val request = new ConsumerGroupHeartbeatRequest.Builder(data).build() + val resource = Set[ResourceType](GROUP, TOPIC) + val response = sendRequestAndVerifyResponseError(request, resource, isAuthorized = true, listenerName = listenerName) + .data.asInstanceOf[ConsumerGroupHeartbeatResponseData] + assertEquals(Errors.NONE.code, response.errorCode, s"Unexpected response $response") + expectedAssignmentSize match { + case Some(size) => + assertNotNull(response.assignment, s"Unexpected assignment $response") + assertEquals(size, response.assignment.topicPartitions.asScala.map(_.partitions.size).sum, s"Unexpected assignment $response") + case None => + assertNull(response.assignment, s"Unexpected assignment $response") + } + response + } + private def createConsumerGroupToDescribe(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) @@ -2650,9 +2763,10 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { resources: Set[ResourceType], isAuthorized: Boolean, topicExists: Boolean = true, - topicNames: Map[Uuid, String] = getTopicNames()): AbstractResponse = { + topicNames: Map[Uuid, String] = getTopicNames(), + listenerName: ListenerName = listenerName): AbstractResponse = { val apiKey = request.apiKey - val response = connectAndReceive[AbstractResponse](request) + val response = connectAndReceive[AbstractResponse](request, listenerName = listenerName) val error = requestKeyToError(topicNames, request.version())(apiKey).asInstanceOf[AbstractResponse => Errors](response) val authorizationErrors = resources.flatMap { resourceType => From 8a515da2c824e6e10c00f938aa9d941076e9cd82 Mon Sep 17 00:00:00 2001 From: Hong-Yi Chen <131829440+apalan60@users.noreply.github.com> Date: Thu, 17 Apr 2025 11:35:43 +0800 Subject: [PATCH 17/72] KAFKA-19054: StreamThread exception handling with SHUTDOWN_APPLICATION may trigger a tight loop with MANY logs (#19394) Under the `SHUTDOWN_APPLICATION` configuration in Kafka Streams, a tight loop in the shutdown process can flood logs with repeated messages. This PR introduces a check to ensure that the shutdown log is emitted only once every 10 seconds, thereby preventing log flooding. Reviewers: PoAn Yang , Matthias J. Sax --- ...msUncaughtExceptionHandlerIntegrationTest.java | 15 ++++++++++++--- .../streams/processor/internals/StreamThread.java | 11 +++++++++-- 2 files changed, 21 insertions(+), 5 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java index 91ae7748f0..b6da54c11b 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java @@ -23,6 +23,8 @@ import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.utils.LogCaptureAppender; +import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; @@ -39,10 +41,12 @@ import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.internals.StreamThread; import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder; import org.apache.kafka.test.TestUtils; +import org.apache.logging.log4j.Level; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; @@ -334,11 +338,14 @@ private void testShutdownApplication(final int numThreads) throws Exception { properties.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, numThreads); final Topology topology = builder.build(); - - try (final KafkaStreams kafkaStreams1 = new KafkaStreams(topology, properties); - final KafkaStreams kafkaStreams2 = new KafkaStreams(topology, properties)) { + final MockTime time = new MockTime(0L); + + try (final KafkaStreams kafkaStreams1 = new KafkaStreams(topology, properties, time); + final KafkaStreams kafkaStreams2 = new KafkaStreams(topology, properties, time); + final LogCaptureAppender logCaptureAppender = LogCaptureAppender.createAndRegister()) { kafkaStreams1.setUncaughtExceptionHandler(exception -> SHUTDOWN_APPLICATION); kafkaStreams2.setUncaughtExceptionHandler(exception -> SHUTDOWN_APPLICATION); + logCaptureAppender.setClassLogger(StreamThread.class, Level.WARN); startApplicationAndWaitUntilRunning(asList(kafkaStreams1, kafkaStreams2)); @@ -346,6 +353,8 @@ private void testShutdownApplication(final int numThreads) throws Exception { waitForApplicationState(asList(kafkaStreams1, kafkaStreams2), KafkaStreams.State.ERROR, DEFAULT_DURATION); assertThat(processorValueCollector.size(), equalTo(1)); + assertThat("Shutdown warning log message should be exported exactly once", + logCaptureAppender.getMessages("WARN").stream().filter(msg -> msg.contains("Detected that shutdown was requested")).count(), equalTo(1L)); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index f505fab155..2a83f0b612 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -346,6 +346,7 @@ public boolean isStartingRunningOrPartitionAssigned() { // These are used to signal from outside the stream thread, but the variables themselves are internal to the thread private final AtomicLong cacheResizeSize = new AtomicLong(-1L); private final AtomicBoolean leaveGroupRequested = new AtomicBoolean(false); + private final AtomicLong lastShutdownWarningTimestamp = new AtomicLong(0L); private final boolean eosEnabled; private final boolean stateUpdaterEnabled; private final boolean processingThreadsEnabled; @@ -869,8 +870,14 @@ public void setStreamsUncaughtExceptionHandler(final BiConsumer= 10_000L) { + if (lastShutdownWarningTimestamp.compareAndSet(lastLogged, now)) { + log.warn("Detected that shutdown was requested. " + + "All clients in this app will now begin to shutdown"); + } + } mainConsumer.enforceRebalance("Shutdown requested"); } } From 56743b37f6bffb0386b08d8f4fdc5a2cd30f0140 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Thu, 17 Apr 2025 14:29:21 +0530 Subject: [PATCH 18/72] MINOR: Supress stdout when checking Log4j 1.x configuration compatibility mode (#19502) when using log41 config, we are printing addtional line like below. This PR is to fix that. --- bin/kafka-run-class.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 8bd1b17623..3463389d3c 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -225,7 +225,7 @@ if [ -z "$KAFKA_LOG4J_OPTS" ]; then (( WINDOWS_OS_FORMAT )) && LOG4J_DIR=$(cygpath --path --mixed "${LOG4J_DIR}") KAFKA_LOG4J_OPTS="-Dlog4j2.configurationFile=${LOG4J_DIR}" else - if echo "$KAFKA_LOG4J_OPTS" | grep -E "log4j\.[^[:space:]]+(\.properties|\.xml)$"; then + if echo "$KAFKA_LOG4J_OPTS" | grep -E "log4j\.[^[:space:]]+(\.properties|\.xml)$" >/dev/null; then # Enable Log4j 1.x configuration compatibility mode for Log4j 2 export LOG4J_COMPATIBILITY=true echo DEPRECATED: A Log4j 1.x configuration file has been detected, which is no longer recommended. >&2 From 3901c8519eed751fd1ba659b4f12b61087f8ed4f Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 22 Apr 2025 05:04:22 -0400 Subject: [PATCH 19/72] KAFKA-19166: Fix RC tag in release script (#19518) The release script was pushing the RC tag off of a temporary branch that was never merged back into the release branch. This meant that our RC and release tags were detached from the rest of the repository. This patch changes the release script to merge the RC tag back into the release branch and pushes both the tag and the branch. Reviewers: Luke Chen --- release/git.py | 8 ++++++-- release/release.py | 4 +++- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/release/git.py b/release/git.py index 61de830ece..9cb106df2f 100644 --- a/release/git.py +++ b/release/git.py @@ -128,8 +128,12 @@ def create_tag(tag, **kwargs): cmd(f"Creating git tag {tag}", ["git", "tag", "-a", tag, "-m", tag], **kwargs) -def push_tag(tag, remote=push_remote_name, **kwargs): +def push_ref(ref, remote=push_remote_name, **kwargs): __defaults(kwargs) - cmd("Pushing tag {tag} to {remote}", f"git push {remote} {tag}") + cmd(f"Pushing ref {ref} to {remote}", f"git push {remote} {ref}") +def merge_ref(ref, **kwargs): + __defaults(kwargs) + cmd(f"Merging ref {ref}", f"git merge {ref}") + diff --git a/release/release.py b/release/release.py index b3a428f651..d0cba6f178 100644 --- a/release/release.py +++ b/release/release.py @@ -297,6 +297,7 @@ def delete_gitrefs(): git.commit(f"Bump version to {release_version}") git.create_tag(rc_tag) git.switch_branch(starting_branch) +git.merge_ref(rc_tag) # Note that we don't use tempfile here because mkdtemp causes problems with being able to determine the absolute path to a file. # Instead we rely on a fixed path @@ -367,7 +368,8 @@ def delete_gitrefs(): print(templates.deploy_instructions()) confirm_or_fail("Have you successfully deployed the artifacts?") confirm_or_fail(f"Ok to push RC tag {rc_tag}?") -git.push_tag(rc_tag) +git.push_ref(rc_tag) +git.push_ref(starting_branch) # Move back to starting branch and clean out the temporary release branch (e.g. 1.0.0) we used to generate everything git.reset_hard_head() From 0297ba2c67e42b11cfaa932d9d3f201e63c3a2a1 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Thu, 24 Apr 2025 12:43:35 -0700 Subject: [PATCH 20/72] KAFKA-19192; Old bootstrap checkpoint files cause problems updated servers (#19545) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Old bootstrap.metadata files cause problems with server that include KAFKA-18601. When the server tries to read the bootstrap.checkpoint file, it will fail if the metadata.version is older than 3.3-IV3 (feature level 7). This causes problems when these clusters are upgraded. This PR makes it possible to represent older MVs in BootstrapMetadata objects without causing an exception. An exception is thrown only if we attempt to access the BootstrapMetadata. This ensures that only the code path in which we start with an empty metadata log checks that the metadata version is 7 or newer. Reviewers: José Armando García Sancio , Ismael Juma , PoAn Yang , Liu Zeyu , Alyssa Huang --- .../kafka/server/KRaftClusterTest.scala | 46 +++++++++++++++++++ .../metadata/bootstrap/BootstrapMetadata.java | 37 ++++++++------- .../bootstrap/BootstrapMetadataTest.java | 12 ++--- .../common/test/KafkaClusterTestKit.java | 23 ++++++++-- 4 files changed, 88 insertions(+), 30 deletions(-) diff --git a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala index 17a75080ba..4fe4fb48cd 100644 --- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala +++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala @@ -65,6 +65,7 @@ import java.util.{Collections, Optional, OptionalLong, Properties} import scala.collection.{Seq, mutable} import scala.concurrent.duration.{FiniteDuration, MILLISECONDS, SECONDS} import scala.jdk.CollectionConverters._ +import scala.util.Using @Timeout(120) @Tag("integration") @@ -1621,6 +1622,51 @@ class KRaftClusterTest { } } + /** + * Test that once a cluster is formatted, a bootstrap.metadata file that contains an unsupported + * MetadataVersion is not a problem. This is a regression test for KAFKA-19192. + */ + @Test + def testOldBootstrapMetadataFile(): Unit = { + val baseDirectory = TestUtils.tempDir().toPath() + Using.resource(new KafkaClusterTestKit.Builder( + new TestKitNodes.Builder(). + setNumBrokerNodes(1). + setNumControllerNodes(1). + setBaseDirectory(baseDirectory). + build()). + setDeleteOnClose(false). + build() + ) { cluster => + cluster.format() + cluster.startup() + cluster.waitForReadyBrokers() + } + val oldBootstrapMetadata = BootstrapMetadata.fromRecords( + util.Arrays.asList( + new ApiMessageAndVersion( + new FeatureLevelRecord(). + setName(MetadataVersion.FEATURE_NAME). + setFeatureLevel(1), + 0.toShort) + ), + "oldBootstrapMetadata") + // Re-create the cluster using the same directory structure as above. + // Since we do not need to use the bootstrap metadata, the fact that + // it specifies an obsolete metadata.version should not be a problem. + Using.resource(new KafkaClusterTestKit.Builder( + new TestKitNodes.Builder(). + setNumBrokerNodes(1). + setNumControllerNodes(1). + setBaseDirectory(baseDirectory). + setBootstrapMetadata(oldBootstrapMetadata). + build()).build() + ) { cluster => + cluster.startup() + cluster.waitForReadyBrokers() + } + } + @Test def testIncreaseNumIoThreads(): Unit = { val cluster = new KafkaClusterTestKit.Builder( diff --git a/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadata.java b/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadata.java index 2f6d8a08c0..375c60f64d 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadata.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadata.java @@ -37,7 +37,7 @@ */ public class BootstrapMetadata { private final List records; - private final MetadataVersion metadataVersion; + private final short metadataVersionLevel; private final String source; public static BootstrapMetadata fromVersions( @@ -67,7 +67,7 @@ public static BootstrapMetadata fromVersions( setFeatureLevel(level), (short) 0)); } } - return new BootstrapMetadata(records, metadataVersion, source); + return new BootstrapMetadata(records, metadataVersion.featureLevel(), source); } public static BootstrapMetadata fromVersion(MetadataVersion metadataVersion, String source) { @@ -75,29 +75,28 @@ public static BootstrapMetadata fromVersion(MetadataVersion metadataVersion, Str new ApiMessageAndVersion(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). setFeatureLevel(metadataVersion.featureLevel()), (short) 0)); - return new BootstrapMetadata(records, metadataVersion, source); + return new BootstrapMetadata(records, metadataVersion.featureLevel(), source); } public static BootstrapMetadata fromRecords(List records, String source) { - MetadataVersion metadataVersion = null; + Optional metadataVersionLevel = Optional.empty(); for (ApiMessageAndVersion record : records) { - Optional version = recordToMetadataVersion(record.message()); - if (version.isPresent()) { - metadataVersion = version.get(); + Optional level = recordToMetadataVersionLevel(record.message()); + if (level.isPresent()) { + metadataVersionLevel = level; } } - if (metadataVersion == null) { + if (metadataVersionLevel.isEmpty()) { throw new RuntimeException("No FeatureLevelRecord for " + MetadataVersion.FEATURE_NAME + " was found in the bootstrap metadata from " + source); } - return new BootstrapMetadata(records, metadataVersion, source); + return new BootstrapMetadata(records, metadataVersionLevel.get(), source); } - public static Optional recordToMetadataVersion(ApiMessage record) { - if (record instanceof FeatureLevelRecord) { - FeatureLevelRecord featureLevel = (FeatureLevelRecord) record; + public static Optional recordToMetadataVersionLevel(ApiMessage record) { + if (record instanceof FeatureLevelRecord featureLevel) { if (featureLevel.name().equals(MetadataVersion.FEATURE_NAME)) { - return Optional.of(MetadataVersion.fromFeatureLevel(featureLevel.featureLevel())); + return Optional.of(featureLevel.featureLevel()); } } return Optional.empty(); @@ -105,11 +104,11 @@ public static Optional recordToMetadataVersion(ApiMessage recor BootstrapMetadata( List records, - MetadataVersion metadataVersion, + short metadataVersionLevel, String source ) { this.records = Objects.requireNonNull(records); - this.metadataVersion = metadataVersion; + this.metadataVersionLevel = metadataVersionLevel; Objects.requireNonNull(source); this.source = source; } @@ -119,7 +118,7 @@ public List records() { } public MetadataVersion metadataVersion() { - return metadataVersion; + return MetadataVersion.fromFeatureLevel(metadataVersionLevel); } public String source() { @@ -167,7 +166,7 @@ public BootstrapMetadata copyWithFeatureRecord(String featureName, short level) @Override public int hashCode() { - return Objects.hash(records, metadataVersion, source); + return Objects.hash(records, metadataVersionLevel, source); } @Override @@ -175,14 +174,14 @@ public boolean equals(Object o) { if (o == null || !o.getClass().equals(this.getClass())) return false; BootstrapMetadata other = (BootstrapMetadata) o; return Objects.equals(records, other.records) && - metadataVersion.equals(other.metadataVersion) && + metadataVersionLevel == other.metadataVersionLevel && source.equals(other.source); } @Override public String toString() { return "BootstrapMetadata(records=" + records.toString() + - ", metadataVersion=" + metadataVersion + + ", metadataVersionLevel=" + metadataVersionLevel + ", source=" + source + ")"; } diff --git a/metadata/src/test/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadataTest.java b/metadata/src/test/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadataTest.java index 549032989b..20688b502c 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadataTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadataTest.java @@ -56,13 +56,13 @@ public void testFromVersion() { new ApiMessageAndVersion(new FeatureLevelRecord(). setName(FEATURE_NAME). setFeatureLevel((short) 7), (short) 0)), - IBP_3_3_IV3, "foo"), + IBP_3_3_IV3.featureLevel(), "foo"), BootstrapMetadata.fromVersion(IBP_3_3_IV3, "foo")); } @Test public void testFromRecordsList() { - assertEquals(new BootstrapMetadata(SAMPLE_RECORDS1, IBP_3_3_IV3, "bar"), + assertEquals(new BootstrapMetadata(SAMPLE_RECORDS1, IBP_3_3_IV3.featureLevel(), "bar"), BootstrapMetadata.fromRecords(SAMPLE_RECORDS1, "bar")); } @@ -133,10 +133,10 @@ public void testFeatureLevelForFeature() { @Test public void testFromRecordsListWithOldMetadataVersion() { - RuntimeException exception = assertThrows(RuntimeException.class, - () -> BootstrapMetadata.fromRecords(RECORDS_WITH_OLD_METADATA_VERSION, "quux")); + BootstrapMetadata bootstrapMetadata = BootstrapMetadata.fromRecords(RECORDS_WITH_OLD_METADATA_VERSION, "quux"); assertEquals("No MetadataVersion with feature level 1. Valid feature levels are from " + MetadataVersion.MINIMUM_VERSION.featureLevel() - + " to " + MetadataVersion.latestTesting().featureLevel() + ".", - exception.getMessage()); + + " to " + MetadataVersion.latestTesting().featureLevel() + ".", + assertThrows(RuntimeException.class, + () -> bootstrapMetadata.metadataVersion()).getMessage()); } } diff --git a/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java index 097f8c3e26..2096366544 100644 --- a/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java +++ b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java @@ -114,6 +114,7 @@ public static class Builder { private final String controllerListenerName; private final String brokerSecurityProtocol; private final String controllerSecurityProtocol; + private boolean deleteOnClose; public Builder(TestKitNodes nodes) { this.nodes = nodes; @@ -121,6 +122,7 @@ public Builder(TestKitNodes nodes) { this.controllerListenerName = nodes.controllerListenerName().value(); this.brokerSecurityProtocol = nodes.brokerListenerProtocol().name; this.controllerSecurityProtocol = nodes.controllerListenerProtocol().name; + this.deleteOnClose = true; } public Builder setConfigProp(String key, Object value) { @@ -206,6 +208,11 @@ private void setSecurityProtocolProps(Map props, String security } } + public Builder setDeleteOnClose(boolean deleteOnClose) { + this.deleteOnClose = deleteOnClose; + return this; + } + public KafkaClusterTestKit build() throws Exception { Map controllers = new HashMap<>(); Map brokers = new HashMap<>(); @@ -308,7 +315,8 @@ public KafkaClusterTestKit build() throws Exception { baseDirectory, faultHandlerFactory, socketFactoryManager, - jaasFile == null ? Optional.empty() : Optional.of(jaasFile)); + Optional.ofNullable(jaasFile), + deleteOnClose); } private String listeners(int node) { @@ -353,6 +361,7 @@ private static void setupNodeDirectories(File baseDirectory, private final PreboundSocketFactoryManager socketFactoryManager; private final String controllerListenerName; private final Optional jaasFile; + private final boolean deleteOnClose; private KafkaClusterTestKit( TestKitNodes nodes, @@ -361,7 +370,8 @@ private KafkaClusterTestKit( File baseDirectory, SimpleFaultHandlerFactory faultHandlerFactory, PreboundSocketFactoryManager socketFactoryManager, - Optional jaasFile + Optional jaasFile, + boolean deleteOnClose ) { /* Number of threads = Total number of brokers + Total number of controllers + Total number of Raft Managers @@ -378,6 +388,7 @@ private KafkaClusterTestKit( this.socketFactoryManager = socketFactoryManager; this.controllerListenerName = nodes.controllerListenerName().value(); this.jaasFile = jaasFile; + this.deleteOnClose = deleteOnClose; } public void format() throws Exception { @@ -640,9 +651,11 @@ public void close() throws Exception { } waitForAllFutures(futureEntries); futureEntries.clear(); - Utils.delete(baseDirectory); - if (jaasFile.isPresent()) { - Utils.delete(jaasFile.get()); + if (deleteOnClose) { + Utils.delete(baseDirectory); + if (jaasFile.isPresent()) { + Utils.delete(jaasFile.get()); + } } } catch (Exception e) { for (Entry> entry : futureEntries) { From 0832c2ceb132850d93220154d2afb496963eebc6 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Mon, 28 Apr 2025 15:33:20 +0200 Subject: [PATCH 21/72] KAFKA-19195: Only send the right group ID subset to each GC shard (#19555) Cherry-picked from [e79f5f0](https://github.com/apache/kafka/commit/e79f5f0f651aba4a797da8234814bf40458beb19) If a share or consumer group is described, all group IDs sent to all shards of the group coordinator. This change fixes it. It tested in the unit tests, since it's somewhat inconvenient to test the passed read operation lambda. --- .../group/GroupCoordinatorService.java | 4 +-- .../group/GroupCoordinatorServiceTest.java | 33 +++++++++++++++---- 2 files changed, 29 insertions(+), 8 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java index de830076bf..893b82fa7a 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java @@ -647,7 +647,7 @@ public CompletableFuture> runtime.scheduleReadOperation( "consumer-group-describe", topicPartition, - (coordinator, lastCommittedOffset) -> coordinator.consumerGroupDescribe(groupIds, lastCommittedOffset) + (coordinator, lastCommittedOffset) -> coordinator.consumerGroupDescribe(groupList, lastCommittedOffset) ).exceptionally(exception -> handleOperationException( "consumer-group-describe", groupList, @@ -698,7 +698,7 @@ public CompletableFuture> shareGroupDescribe( runtime.scheduleReadOperation( "share-group-describe", topicPartition, - (coordinator, lastCommittedOffset) -> coordinator.shareGroupDescribe(groupIds, lastCommittedOffset) + (coordinator, lastCommittedOffset) -> coordinator.shareGroupDescribe(groupList, lastCommittedOffset) ).exceptionally(exception -> handleOperationException( "share-group-describe", groupList, diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java index 1974a796d4..6d1120c5c9 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java @@ -82,6 +82,7 @@ import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.NullSource; import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.ArgumentCaptor; import org.mockito.ArgumentMatchers; import java.net.InetAddress; @@ -1415,6 +1416,10 @@ public void testConsumerGroupDescribe() throws InterruptedException, ExecutionEx int partitionCount = 2; service.startup(() -> partitionCount); + @SuppressWarnings("unchecked") + ArgumentCaptor>> readOperationCaptor = + ArgumentCaptor.forClass(CoordinatorRuntime.CoordinatorReadOperation.class); + ConsumerGroupDescribeResponseData.DescribedGroup describedGroup1 = new ConsumerGroupDescribeResponseData.DescribedGroup() .setGroupId("group-id-1"); ConsumerGroupDescribeResponseData.DescribedGroup describedGroup2 = new ConsumerGroupDescribeResponseData.DescribedGroup() @@ -1427,14 +1432,14 @@ public void testConsumerGroupDescribe() throws InterruptedException, ExecutionEx when(runtime.scheduleReadOperation( ArgumentMatchers.eq("consumer-group-describe"), ArgumentMatchers.eq(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0)), - ArgumentMatchers.any() + readOperationCaptor.capture() )).thenReturn(CompletableFuture.completedFuture(Collections.singletonList(describedGroup1))); - CompletableFuture describedGroupFuture = new CompletableFuture<>(); + CompletableFuture> describedGroupFuture = new CompletableFuture<>(); when(runtime.scheduleReadOperation( ArgumentMatchers.eq("consumer-group-describe"), ArgumentMatchers.eq(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 1)), - ArgumentMatchers.any() + readOperationCaptor.capture() )).thenReturn(describedGroupFuture); CompletableFuture> future = @@ -1443,6 +1448,12 @@ public void testConsumerGroupDescribe() throws InterruptedException, ExecutionEx assertFalse(future.isDone()); describedGroupFuture.complete(Collections.singletonList(describedGroup2)); assertEquals(expectedDescribedGroups, future.get()); + + // Validate that the captured read operations, on the first and the second partition + GroupCoordinatorShard shard = mock(GroupCoordinatorShard.class); + readOperationCaptor.getAllValues().forEach(x -> x.generateResponse(shard, 100)); + verify(shard).consumerGroupDescribe(List.of("group-id-2"), 100); + verify(shard).consumerGroupDescribe(List.of("group-id-1"), 100); } @Test @@ -2282,6 +2293,10 @@ public void testShareGroupDescribe() throws InterruptedException, ExecutionExcep int partitionCount = 2; service.startup(() -> partitionCount); + @SuppressWarnings("unchecked") + ArgumentCaptor>> readOperationCaptor = + ArgumentCaptor.forClass(CoordinatorRuntime.CoordinatorReadOperation.class); + ShareGroupDescribeResponseData.DescribedGroup describedGroup1 = new ShareGroupDescribeResponseData.DescribedGroup() .setGroupId("share-group-id-1"); ShareGroupDescribeResponseData.DescribedGroup describedGroup2 = new ShareGroupDescribeResponseData.DescribedGroup() @@ -2294,14 +2309,14 @@ public void testShareGroupDescribe() throws InterruptedException, ExecutionExcep when(runtime.scheduleReadOperation( ArgumentMatchers.eq("share-group-describe"), ArgumentMatchers.eq(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0)), - ArgumentMatchers.any() + readOperationCaptor.capture() )).thenReturn(CompletableFuture.completedFuture(Collections.singletonList(describedGroup1))); - CompletableFuture describedGroupFuture = new CompletableFuture<>(); + CompletableFuture> describedGroupFuture = new CompletableFuture<>(); when(runtime.scheduleReadOperation( ArgumentMatchers.eq("share-group-describe"), ArgumentMatchers.eq(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 1)), - ArgumentMatchers.any() + readOperationCaptor.capture() )).thenReturn(describedGroupFuture); CompletableFuture> future = @@ -2310,6 +2325,12 @@ public void testShareGroupDescribe() throws InterruptedException, ExecutionExcep assertFalse(future.isDone()); describedGroupFuture.complete(Collections.singletonList(describedGroup2)); assertEquals(expectedDescribedGroups, future.get()); + + // Validate that the captured read operations, on the first and the second partition + GroupCoordinatorShard shard = mock(GroupCoordinatorShard.class); + readOperationCaptor.getAllValues().forEach(x -> x.generateResponse(shard, 100)); + verify(shard).shareGroupDescribe(List.of("share-group-id-2"), 100); + verify(shard).shareGroupDescribe(List.of("share-group-id-1"), 100); } @Test From c2068878c920af06a23e8079be41d20ceb59e2f1 Mon Sep 17 00:00:00 2001 From: Kamal Chandraprakash Date: Sun, 4 May 2025 18:53:16 +0530 Subject: [PATCH 22/72] KAFKA-19131: Adjust remote storage reader thread maximum pool size to avoid illegal argument (#19629) The remote storage reader thread pool use same count for both maximum and core size. If users adjust the pool size larger than original value, it throws `IllegalArgumentException`. Updated both value to fix the issue. cherry-pick PR: #19532 cherry-pick commit: https://github.com/apache/kafka/commit/965743c35bdf2d009456676e50024b767b2a9318 --------- Signed-off-by: PoAn Yang Reviewers: Chia-Ping Tsai , PoAn Yang Co-authored-by: PoAn Yang --- .../java/kafka/log/remote/RemoteLogManager.java | 14 +++++++++++++- .../kafka/log/remote/RemoteLogManagerTest.java | 11 ++++++++++- 2 files changed, 23 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index b5f9e408c9..5fbc4693e4 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -303,8 +303,15 @@ public void resizeExpirationThreadPool(int newSize) { public void resizeReaderThreadPool(int newSize) { int currentSize = remoteStorageReaderThreadPool.getCorePoolSize(); + int currentMaximumSize = remoteStorageReaderThreadPool.getMaximumPoolSize(); LOGGER.info("Updating remote reader thread pool size from {} to {}", currentSize, newSize); - remoteStorageReaderThreadPool.setCorePoolSize(newSize); + if (newSize > currentMaximumSize) { + remoteStorageReaderThreadPool.setMaximumPoolSize(newSize); + remoteStorageReaderThreadPool.setCorePoolSize(newSize); + } else { + remoteStorageReaderThreadPool.setCorePoolSize(newSize); + remoteStorageReaderThreadPool.setMaximumPoolSize(newSize); + } } private void removeMetrics() { @@ -313,6 +320,11 @@ private void removeMetrics() { remoteStorageReaderThreadPool.removeMetrics(); } + // Visible for testing + int readerThreadPoolSize() { + return remoteStorageReaderThreadPool.getCorePoolSize(); + } + /** * Returns the timeout for the RLM Tasks to wait for the quota to be available */ diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index 7f536c9872..3943ff9289 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -1156,7 +1156,7 @@ void testRemoteLogManagerRemoteMetrics() throws Exception { safeLongYammerMetricValue("RemoteLogSizeComputationTime,topic=" + leaderTopic), safeLongYammerMetricValue("RemoteLogSizeComputationTime"))); remoteLogSizeComputationTimeLatch.countDown(); - + TestUtils.waitForCondition( () -> 0 == safeLongYammerMetricValue("RemoteCopyLagBytes") && 0 == safeLongYammerMetricValue("RemoteCopyLagBytes,topic=" + leaderTopic), String.format("Expected to find 0 for RemoteCopyLagBytes metric value, but found %d for topic 'Leader' and %d for all topics.", @@ -3713,6 +3713,15 @@ public void testRLMOpsWhenMetadataIsNotReady() throws InterruptedException { verifyNoMoreInteractions(remoteStorageManager); } + @Test + void testUpdateRemoteStorageReaderThreads() { + assertEquals(10, remoteLogManager.readerThreadPoolSize()); + remoteLogManager.resizeReaderThreadPool(6); + assertEquals(6, remoteLogManager.readerThreadPoolSize()); + remoteLogManager.resizeReaderThreadPool(12); + assertEquals(12, remoteLogManager.readerThreadPoolSize()); + } + private void appendRecordsToFile(File file, int nRecords, int nRecordsPerBatch) throws IOException { byte magic = RecordBatch.CURRENT_MAGIC_VALUE; Compression compression = Compression.NONE; From 1f856d437d1554bd1c741cfd0787b371c185c5fe Mon Sep 17 00:00:00 2001 From: Luke Chen Date: Tue, 6 May 2025 09:42:52 +0900 Subject: [PATCH 23/72] MINOR: exclude error_prone_annotations lib from caffeine dependency (#19638) In https://github.com/apache/kafka/pull/16578 , we tried to exclude both `checker-qual` and `error_prone_annotations`, but when excluding `error_prone_annotations`, the compilation failed. So in the end, we only excluded `checker-qual` and shipped `error_prone_annotations.jar` to users. In Kafka v4.0.0, thanks to jdk 8 removal, we upgraded caffeine to the latest v3.1.8, instead of v2.x.x, and now, we can successfully pass the compilation without error after excluding `error_prone_annotations` from `caffeine`. Reviewers: Chia-Ping Tsai , Ken Huang --- LICENSE-binary | 1 - build.gradle | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/LICENSE-binary b/LICENSE-binary index 030f62b967..380af15b5c 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -212,7 +212,6 @@ License Version 2.0: - commons-lang3-3.12.0 - commons-logging-1.3.2 - commons-validator-1.9.0 -- error_prone_annotations-2.14.0 - jackson-annotations-2.16.2 - jackson-core-2.16.2 - jackson-databind-2.16.2 diff --git a/build.gradle b/build.gradle index 1c59669c37..243f181c0b 100644 --- a/build.gradle +++ b/build.gradle @@ -2267,6 +2267,7 @@ project(':storage') { implementation project(':transaction-coordinator') implementation(libs.caffeine) { exclude group: 'org.checkerframework', module: 'checker-qual' + exclude group: 'com.google.errorprone', module: 'error_prone_annotations' } implementation libs.slf4jApi implementation libs.jacksonDatabind From cf3c177936e8cc2963ac053840c2ba0965b03808 Mon Sep 17 00:00:00 2001 From: Sean Quah Date: Mon, 12 May 2025 08:32:17 +0100 Subject: [PATCH 24/72] KAFKA-19160;KAFKA-19164; Improve performance of fetching stable offsets (#19497) When fetching stable offsets in the group coordinator, we iterate over all requested partitions. For each partition, we iterate over the group's ongoing transactions to check if there is a pending transactional offset commit for that partition. This can get slow when there are a large number of partitions and a large number of pending transactions. Instead, maintain a list of pending transactions per partition to speed up lookups. Reviewers: Shaan, Dongnuo Lyu , Chia-Ping Tsai , David Jaco --- .../group/OffsetMetadataManager.java | 278 ++++++++++++++---- .../TransactionalOffsetFetchBenchmark.java | 142 +++++++++ 2 files changed, 355 insertions(+), 65 deletions(-) create mode 100644 jmh-benchmarks/src/main/java/org/apache/kafka/jmh/coordinator/TransactionalOffsetFetchBenchmark.java diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java index 5e0cf7589e..0922590f79 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java @@ -65,6 +65,7 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; import java.util.function.Consumer; import static org.apache.kafka.common.requests.OffsetFetchResponse.INVALID_OFFSET; @@ -91,37 +92,37 @@ public static class Builder { private GroupCoordinatorConfig config = null; private GroupCoordinatorMetricsShard metrics = null; - Builder withLogContext(LogContext logContext) { + public Builder withLogContext(LogContext logContext) { this.logContext = logContext; return this; } - Builder withSnapshotRegistry(SnapshotRegistry snapshotRegistry) { + public Builder withSnapshotRegistry(SnapshotRegistry snapshotRegistry) { this.snapshotRegistry = snapshotRegistry; return this; } - Builder withTime(Time time) { + public Builder withTime(Time time) { this.time = time; return this; } - Builder withGroupMetadataManager(GroupMetadataManager groupMetadataManager) { + public Builder withGroupMetadataManager(GroupMetadataManager groupMetadataManager) { this.groupMetadataManager = groupMetadataManager; return this; } - Builder withGroupCoordinatorConfig(GroupCoordinatorConfig config) { + public Builder withGroupCoordinatorConfig(GroupCoordinatorConfig config) { this.config = config; return this; } - Builder withMetadataImage(MetadataImage metadataImage) { + public Builder withMetadataImage(MetadataImage metadataImage) { this.metadataImage = metadataImage; return this; } - Builder withGroupCoordinatorMetricsShard(GroupCoordinatorMetricsShard metrics) { + public Builder withGroupCoordinatorMetricsShard(GroupCoordinatorMetricsShard metrics) { this.metrics = metrics; return this; } @@ -196,9 +197,167 @@ public OffsetMetadataManager build() { private final TimelineHashMap pendingTransactionalOffsets; /** - * The open transactions (producer ids) keyed by group. + * The open transactions (producer ids) by group id, topic name and partition id. */ - private final TimelineHashMap> openTransactionsByGroup; + private final OpenTransactions openTransactions; + + /** + * Tracks open transactions (producer ids) by group id, topic name and partition id. + * It is the responsiblity of the caller to update {@link #pendingTransactionalOffsets}. + */ + private class OpenTransactions { + /** + * The open transactions (producer ids) keyed by group id, topic name and partition id. + * Tracks whether partitions have any pending transactional offsets that have not been deleted. + * + * Values in each level of the map will never be empty collections. + */ + private final TimelineHashMap>>> openTransactionsByGroup; + + private OpenTransactions() { + this.openTransactionsByGroup = new TimelineHashMap<>(snapshotRegistry, 0); + } + + /** + * Adds a producer id to the open transactions for the given group and topic partition. + * + * @param groupId The group id. + * @param topic The topic name. + * @param partition The partition. + * @param producerId The producer id. + * @return {@code true} if the partition did not already have a pending offset from the producer id. + */ + private boolean add(String groupId, String topic, int partition, long producerId) { + return openTransactionsByGroup + .computeIfAbsent(groupId, __ -> new TimelineHashMap<>(snapshotRegistry, 1)) + .computeIfAbsent(topic, __ -> new TimelineHashMap<>(snapshotRegistry, 1)) + .computeIfAbsent(partition, __ -> new TimelineHashSet<>(snapshotRegistry, 1)) + .add(producerId); + } + + /** + * Clears all open transactions for the given group and topic partition. + * + * @param groupId The group id. + * @param topic The topic name. + * @param partition The partition. + */ + private void clear(String groupId, String topic, int partition) { + TimelineHashMap>> openTransactionsByTopic = + openTransactionsByGroup.get(groupId); + if (openTransactionsByTopic == null) return; + + TimelineHashMap> openTransactionsByPartition = openTransactionsByTopic.get(topic); + if (openTransactionsByPartition == null) return; + + openTransactionsByPartition.remove(partition); + + if (openTransactionsByPartition.isEmpty()) { + openTransactionsByTopic.remove(topic); + if (openTransactionsByTopic.isEmpty()) { + openTransactionsByGroup.remove(groupId); + } + } + } + + /** + * Returns {@code true} if the given group has any pending transactional offsets. + * + * @param groupId The group id. + * @return {@code true} if the given group has any pending transactional offsets. + */ + private boolean contains(String groupId) { + return openTransactionsByGroup.containsKey(groupId); + } + + /** + * Returns {@code true} if the given group has any pending transactional offsets for the given topic and partition. + * + * @param groupId The group id. + * @param topic The topic name. + * @param partition The partition. + * @return {@code true} if the given group has any pending transactional offsets for the given topic and partition. + */ + private boolean contains(String groupId, String topic, int partition) { + TimelineHashSet openTransactions = get(groupId, topic, partition); + return openTransactions != null; + } + + /** + * Performs the given action for each partition with a pending transactional offset for the given group. + * + * @param groupId The group id. + * @param action The action to be performed for each partition with a pending transactional offset. + */ + private void forEachTopicPartition(String groupId, BiConsumer action) { + TimelineHashMap>> openTransactionsByTopic = + openTransactionsByGroup.get(groupId); + if (openTransactionsByTopic == null) return; + + openTransactionsByTopic.forEach((topic, openTransactionsByPartition) -> { + openTransactionsByPartition.forEach((partition, producerIds) -> { + action.accept(topic, partition); + }); + }); + } + + /** + * Performs the given action for each producer id with a pending transactional offset for the given group and topic partition. + * + * @param groupId The group id. + * @param topic The topic name. + * @param partition The partition. + * @param action The action to be performed for each producer id with a pending transactional offset. + */ + private void forEach(String groupId, String topic, int partition, Consumer action) { + TimelineHashSet openTransactions = get(groupId, topic, partition); + if (openTransactions == null) return; + + openTransactions.forEach(action); + } + + /** + * Gets the set of producer ids with pending transactional offsets for the given group and topic partition. + * + * @param groupId The group id. + * @param topic The topic name. + * @param partition The partition. + * @return The set of producer ids with pending transactional offsets for the given group and topic partition. + */ + private TimelineHashSet get(String groupId, String topic, int partition) { + TimelineHashMap>> openTransactionsByTopic = + openTransactionsByGroup.get(groupId); + if (openTransactionsByTopic == null) return null; + + TimelineHashMap> openTransactionsByPartition = openTransactionsByTopic.get(topic); + if (openTransactionsByPartition == null) return null; + + return openTransactionsByPartition.get(partition); + } + + /** + * Removes a producer id from the open transactions for the given group and topic partition. + * + * @param groupId The group id. + * @param topic The topic name. + * @param partition The partition. + * @param producerId The producer id. + * @return {@code true} if the group and topic partition had a pending transactional offset from the producer id. + */ + private boolean remove(String groupId, String topic, int partition, long producerId) { + TimelineHashSet openTransactions = get(groupId, topic, partition); + if (openTransactions == null) return false; + + boolean removed = openTransactions.remove(producerId); + + if (openTransactions.isEmpty()) { + // Re-use the clean up in clear. + clear(groupId, topic, partition); + } + + return removed; + } + } private class Offsets { /** @@ -283,7 +442,7 @@ private OffsetAndMetadata remove( this.metrics = metrics; this.offsets = new Offsets(); this.pendingTransactionalOffsets = new TimelineHashMap<>(snapshotRegistry, 0); - this.openTransactionsByGroup = new TimelineHashMap<>(snapshotRegistry, 0); + this.openTransactions = new OpenTransactions(); } /** @@ -653,26 +812,12 @@ public int deleteAllOffsets( // Delete all the pending transactional offsets too. Here we only write a tombstone // if the topic-partition was not in the main storage because we don't need to write // two consecutive tombstones. - TimelineHashSet openTransactions = openTransactionsByGroup.get(groupId); - if (openTransactions != null) { - openTransactions.forEach(producerId -> { - Offsets pendingOffsets = pendingTransactionalOffsets.get(producerId); - if (pendingOffsets != null) { - TimelineHashMap> pendingGroupOffsets = - pendingOffsets.offsetsByGroup.get(groupId); - if (pendingGroupOffsets != null) { - pendingGroupOffsets.forEach((topic, offsetsByPartition) -> { - offsetsByPartition.keySet().forEach(partition -> { - if (!hasCommittedOffset(groupId, topic, partition)) { - records.add(GroupCoordinatorRecordHelpers.newOffsetCommitTombstoneRecord(groupId, topic, partition)); - numDeletedOffsets.getAndIncrement(); - } - }); - }); - } - } - }); - } + openTransactions.forEachTopicPartition(groupId, (topic, partition) -> { + if (!hasCommittedOffset(groupId, topic, partition)) { + records.add(GroupCoordinatorRecordHelpers.newOffsetCommitTombstoneRecord(groupId, topic, partition)); + numDeletedOffsets.getAndIncrement(); + } + }); return numDeletedOffsets.get(); } @@ -688,17 +833,7 @@ boolean hasPendingTransactionalOffsets( String topic, int partition ) { - final TimelineHashSet openTransactions = openTransactionsByGroup.get(groupId); - if (openTransactions == null) return false; - - for (Long producerId : openTransactions) { - Offsets offsets = pendingTransactionalOffsets.get(producerId); - if (offsets != null && offsets.get(groupId, topic, partition) != null) { - return true; - } - } - - return false; + return openTransactions.contains(groupId, topic, partition); } /** @@ -739,6 +874,11 @@ public OffsetFetchResponseData.OffsetFetchResponseGroup fetchOffsets( final List topicResponses = new ArrayList<>(request.topics().size()); final TimelineHashMap> groupOffsets = failAllPartitions ? null : offsets.offsetsByGroup.get(request.groupId(), lastCommittedOffset); + // We inline the lookups from hasPendingTransactionalOffsets here, to avoid repeating string + // comparisons of group ids and topic names for every partition. They're only used when the + // client has requested stable offsets. + final TimelineHashMap>> openTransactionsByTopic = + requireStable ? openTransactions.openTransactionsByGroup.get(request.groupId(), lastCommittedOffset) : null; request.topics().forEach(topic -> { final OffsetFetchResponseData.OffsetFetchResponseTopics topicResponse = @@ -747,12 +887,16 @@ public OffsetFetchResponseData.OffsetFetchResponseGroup fetchOffsets( final TimelineHashMap topicOffsets = groupOffsets == null ? null : groupOffsets.get(topic.name(), lastCommittedOffset); + final TimelineHashMap> openTransactionsByPartition = + (requireStable && openTransactionsByTopic != null) ? openTransactionsByTopic.get(topic.name(), lastCommittedOffset) : null; topic.partitionIndexes().forEach(partitionIndex -> { final OffsetAndMetadata offsetAndMetadata = topicOffsets == null ? null : topicOffsets.get(partitionIndex, lastCommittedOffset); - if (requireStable && hasPendingTransactionalOffsets(request.groupId(), topic.name(), partitionIndex)) { + if (requireStable && + openTransactionsByPartition != null && + openTransactionsByPartition.containsKey(partitionIndex, lastCommittedOffset)) { topicResponse.partitions().add(new OffsetFetchResponseData.OffsetFetchResponsePartitions() .setPartitionIndex(partitionIndex) .setErrorCode(Errors.UNSTABLE_OFFSET_COMMIT.code()) @@ -805,11 +949,18 @@ public OffsetFetchResponseData.OffsetFetchResponseGroup fetchAllOffsets( final List topicResponses = new ArrayList<>(); final TimelineHashMap> groupOffsets = offsets.offsetsByGroup.get(request.groupId(), lastCommittedOffset); + // We inline the lookups from hasPendingTransactionalOffsets here, to avoid repeating string + // comparisons of group ids and topic names for every partition. They're only used when the + // client has requested stable offsets. + final TimelineHashMap>> openTransactionsByTopic = + requireStable ? openTransactions.openTransactionsByGroup.get(request.groupId(), lastCommittedOffset) : null; if (groupOffsets != null) { groupOffsets.entrySet(lastCommittedOffset).forEach(topicEntry -> { final String topic = topicEntry.getKey(); final TimelineHashMap topicOffsets = topicEntry.getValue(); + final TimelineHashMap> openTransactionsByPartition = + (requireStable && openTransactionsByTopic != null) ? openTransactionsByTopic.get(topic, lastCommittedOffset) : null; final OffsetFetchResponseData.OffsetFetchResponseTopics topicResponse = new OffsetFetchResponseData.OffsetFetchResponseTopics().setName(topic); @@ -819,7 +970,9 @@ public OffsetFetchResponseData.OffsetFetchResponseGroup fetchAllOffsets( final int partition = partitionEntry.getKey(); final OffsetAndMetadata offsetAndMetadata = partitionEntry.getValue(); - if (requireStable && hasPendingTransactionalOffsets(request.groupId(), topic, partition)) { + if (requireStable && + openTransactionsByPartition != null && + openTransactionsByPartition.containsKey(partition, lastCommittedOffset)) { topicResponse.partitions().add(new OffsetFetchResponseData.OffsetFetchResponsePartitions() .setPartitionIndex(partition) .setErrorCode(Errors.UNSTABLE_OFFSET_COMMIT.code()) @@ -888,8 +1041,8 @@ public boolean cleanupExpiredOffsets(String groupId, List rec }); metrics.record(OFFSET_EXPIRED_SENSOR_NAME, expiredPartitions.size()); - // We don't want to remove the group if there are ongoing transactions. - return allOffsetsExpired.get() && !openTransactionsByGroup.containsKey(groupId); + // We don't want to remove the group if there are ongoing transactions with undeleted offsets. + return allOffsetsExpired.get() && !openTransactions.contains(groupId); } /** @@ -1010,9 +1163,7 @@ public void replay( partition, OffsetAndMetadata.fromRecord(recordOffset, value) ); - openTransactionsByGroup - .computeIfAbsent(groupId, __ -> new TimelineHashSet<>(snapshotRegistry, 1)) - .add(producerId); + openTransactions.add(groupId, topic, partition, producerId); } } else { if (offsets.remove(groupId, topic, partition) != null) { @@ -1020,15 +1171,13 @@ public void replay( } // Remove all the pending offset commits related to the tombstone. - TimelineHashSet openTransactions = openTransactionsByGroup.get(groupId); - if (openTransactions != null) { - openTransactions.forEach(openProducerId -> { - Offsets pendingOffsets = pendingTransactionalOffsets.get(openProducerId); - if (pendingOffsets != null) { - pendingOffsets.remove(groupId, topic, partition); - } - }); - } + openTransactions.forEach(groupId, topic, partition, openProducerId -> { + Offsets pendingOffsets = pendingTransactionalOffsets.get(openProducerId); + if (pendingOffsets != null) { + pendingOffsets.remove(groupId, topic, partition); + } + }); + openTransactions.clear(groupId, topic, partition); } } @@ -1039,6 +1188,7 @@ public void replay( * @param result The result of the transaction. * @throws RuntimeException if the transaction can not be completed. */ + @SuppressWarnings("NPathComplexity") public void replayEndTransactionMarker( long producerId, TransactionResult result @@ -1051,14 +1201,12 @@ public void replayEndTransactionMarker( return; } - pendingOffsets.offsetsByGroup.keySet().forEach(groupId -> { - TimelineHashSet openTransactions = openTransactionsByGroup.get(groupId); - if (openTransactions != null) { - openTransactions.remove(producerId); - if (openTransactions.isEmpty()) { - openTransactionsByGroup.remove(groupId); - } - } + pendingOffsets.offsetsByGroup.forEach((groupId, topicOffsets) -> { + topicOffsets.forEach((topic, partitionOffsets) -> { + partitionOffsets.keySet().forEach(partitionId -> { + openTransactions.remove(groupId, topic, partitionId, producerId); + }); + }); }); if (result == TransactionResult.COMMIT) { diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/coordinator/TransactionalOffsetFetchBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/coordinator/TransactionalOffsetFetchBenchmark.java new file mode 100644 index 0000000000..b1986f639a --- /dev/null +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/coordinator/TransactionalOffsetFetchBenchmark.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.jmh.coordinator; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.OffsetFetchRequestData; +import org.apache.kafka.common.metadata.TopicRecord; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.coordinator.group.Group; +import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; +import org.apache.kafka.coordinator.group.GroupMetadataManager; +import org.apache.kafka.coordinator.group.OffsetMetadataManager; +import org.apache.kafka.coordinator.group.generated.OffsetCommitKey; +import org.apache.kafka.coordinator.group.generated.OffsetCommitValue; +import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard; +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.image.MetadataProvenance; +import org.apache.kafka.timeline.SnapshotRegistry; + +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 5) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.MILLISECONDS) +public class TransactionalOffsetFetchBenchmark { + private static final Time TIME = Time.SYSTEM; + + @Param({"4000"}) + private int partitionCount; + + @Param({"4000"}) + private int transactionCount; + + private static final String GROUP_ID = "my-group-id"; + private static final String TOPIC_NAME = "my-topic-name"; + + private OffsetMetadataManager offsetMetadataManager; + + /** A list of partition indexes from 0 to partitionCount - 1. */ + private List partitionIndexes; + + @Setup(Level.Trial) + public void setup() { + LogContext logContext = new LogContext(); + MetadataDelta delta = new MetadataDelta(MetadataImage.EMPTY); + delta.replay(new TopicRecord() + .setTopicId(Uuid.randomUuid()) + .setName(TOPIC_NAME)); + MetadataImage image = delta.apply(MetadataProvenance.EMPTY); + + SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext); + + GroupMetadataManager groupMetadataManager = mock(GroupMetadataManager.class); + Group group = mock(Group.class); + when(groupMetadataManager.group(anyString(), anyLong())).thenReturn(group); + + offsetMetadataManager = new OffsetMetadataManager.Builder() + .withLogContext(logContext) + .withSnapshotRegistry(snapshotRegistry) + .withTime(TIME) + .withGroupMetadataManager(groupMetadataManager) + .withGroupCoordinatorConfig(mock(GroupCoordinatorConfig.class)) + .withMetadataImage(image) + .withGroupCoordinatorMetricsShard(mock(GroupCoordinatorMetricsShard.class)) + .build(); + + for (int i = 0; i < transactionCount; i++) { + snapshotRegistry.idempotentCreateSnapshot(i); + offsetMetadataManager.replay( + i, + 3193600 + i, + new OffsetCommitKey() + .setGroup(GROUP_ID) + .setTopic(TOPIC_NAME) + .setPartition(i), + new OffsetCommitValue() + .setOffset(100) + ); + } + + partitionIndexes = new ArrayList<>(); + for (int i = 0; i < partitionCount; i++) { + partitionIndexes.add(i); + } + } + + @Benchmark + @Threads(1) + @OutputTimeUnit(TimeUnit.MILLISECONDS) + public void run() { + offsetMetadataManager.fetchOffsets( + new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId(GROUP_ID) + .setTopics(List.of( + new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName(TOPIC_NAME) + .setPartitionIndexes(partitionIndexes) + )), + Long.MAX_VALUE + ); + } +} From 5015183c1c38740900552e8cf1c0e491c038b2df Mon Sep 17 00:00:00 2001 From: ChickenchickenLove Date: Tue, 13 May 2025 00:01:29 +0900 Subject: [PATCH 25/72] KAFKA-19242: Fix commit bugs caused by race condition during rebalancing. (#19631) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### Motivation While investigating “events skipped in group rebalancing” ([spring‑projects/spring‑kafka#3703](https://github.com/spring-projects/spring-kafka/issues/3703)) I discovered a race condition between - the main poll/commit thread, and - the consumer‑coordinator heartbeat thread. If the main thread enters `ConsumerCoordinator.sendOffsetCommitRequest()` while the heartbeat thread is finishing a rebalance (`SyncGroupResponseHandler.handle()`), the group state transitions in the following order: ``` COMPLETING_REBALANCE → (race window) → STABLE ``` Because we read the state twice without a lock: 1. `generationIfStable()` returns `null` (state still `COMPLETING_REBALANCE`), 2. the heartbeat thread flips the state to `STABLE`, 3. the main thread re‑checks with `rebalanceInProgress()` and wrongly decides that a rebalance is still active, 4. a spurious `CommitFailedException` is returned even though the commit could succeed. For more details, please refer to sequence diagram below. image ### Impact - The exception is semantically wrong: the consumer is in a stable group, but reports failure. - Frameworks and applications that rely on the semantics of `CommitFailedException` and `RetryableCommitException` (for example `Spring Kafka`) take the wrong code path, which can ultimately skip the events and break “at‑most‑once” guarantees. ### Fix We enlarge the synchronized block in `ConsumerCoordinator.sendOffsetCommitRequest()` so that the consumer group state is examined atomically with respect to the heartbeat thread: ### Jira https://issues.apache.org/jira/browse/KAFKA-19242 https: //github.com/spring-projects/spring-kafka/issues/3703 Signed-off-by: chickenchickenlove Reviewers: David Jacot --- .../internals/ConsumerCoordinator.java | 36 ++++++++++--------- 1 file changed, 19 insertions(+), 17 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index 584a03736f..8d54c871bb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -1263,23 +1263,25 @@ RequestFuture sendOffsetCommitRequest(final Map Date: Mon, 12 May 2025 20:17:30 +0200 Subject: [PATCH 26/72] MINOR: Fix version in 4.0 branch (#19686) This patch fixes the version used in the `4.0` branch. It should be `4.0.1` instead of `4.1.0`. Reviewers: Chia-Ping Tsai --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index 46bfc41b7e..04ea428dd3 100644 --- a/gradle.properties +++ b/gradle.properties @@ -23,7 +23,7 @@ group=org.apache.kafka # - streams/quickstart/pom.xml # - streams/quickstart/java/src/main/resources/archetype-resources/pom.xml # - streams/quickstart/java/pom.xml -version=4.1.0-SNAPSHOT +version=4.0.1-SNAPSHOT scalaVersion=2.13.15 # Adding swaggerVersion in gradle.properties to have a single version in place for swagger swaggerVersion=2.2.25 From 48f75616d7c8d756117f885b01384af8e1b5cef1 Mon Sep 17 00:00:00 2001 From: Sean Quah Date: Tue, 13 May 2025 13:10:26 +0100 Subject: [PATCH 27/72] KAFKA-19163: Avoid deleting groups with pending transactional offsets (#19496) When a group has pending transactional offsets but no committed offsets, we can accidentally delete it while cleaning up expired offsets. Add a check to avoid this case. Reviewers: David Jacot --- .../group/OffsetMetadataManager.java | 5 +-- .../group/OffsetMetadataManagerTest.java | 36 +++++++++++++++++++ 2 files changed, 39 insertions(+), 2 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java index 0922590f79..b92bf1354d 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java @@ -1001,13 +1001,14 @@ public OffsetFetchResponseData.OffsetFetchResponseGroup fetchAllOffsets( * @param groupId The group id. * @param records The list of records to populate with offset commit tombstone records. * - * @return True if no offsets exist or if all offsets expired, false otherwise. + * @return True if no offsets exist after expiry and no pending transactional offsets exist, + * false otherwise. */ public boolean cleanupExpiredOffsets(String groupId, List records) { TimelineHashMap> offsetsByTopic = offsets.offsetsByGroup.get(groupId); if (offsetsByTopic == null) { - return true; + return !openTransactions.contains(groupId); } // We expect the group to exist. diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java index a0150c3bda..f8fbc4f7c7 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java @@ -2591,6 +2591,42 @@ public void testCleanupExpiredOffsetsWithPendingTransactionalOffsets() { assertEquals(Collections.emptyList(), records); } + @Test + public void testCleanupExpiredOffsetsWithPendingTransactionalOffsetsOnly() { + GroupMetadataManager groupMetadataManager = mock(GroupMetadataManager.class); + Group group = mock(Group.class); + + OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder() + .withGroupMetadataManager(groupMetadataManager) + .withOffsetsRetentionMinutes(1) + .build(); + + long commitTimestamp = context.time.milliseconds(); + + context.commitOffset("group-id", "foo", 0, 100L, 0, commitTimestamp); + context.commitOffset(10L, "group-id", "foo", 1, 101L, 0, commitTimestamp + 500); + + context.time.sleep(Duration.ofMinutes(1).toMillis()); + + when(groupMetadataManager.group("group-id")).thenReturn(group); + when(group.offsetExpirationCondition()).thenReturn(Optional.of( + new OffsetExpirationConditionImpl(offsetAndMetadata -> offsetAndMetadata.commitTimestampMs))); + when(group.isSubscribedToTopic("foo")).thenReturn(false); + + // foo-0 is expired, but the group is not deleted beacuse it has pending transactional offset commits. + List expectedRecords = List.of( + GroupCoordinatorRecordHelpers.newOffsetCommitTombstoneRecord("group-id", "foo", 0) + ); + List records = new ArrayList<>(); + assertFalse(context.cleanupExpiredOffsets("group-id", records)); + assertEquals(expectedRecords, records); + + // No offsets are expired, and the group is still not deleted because it has pending transactional offset commits. + records = new ArrayList<>(); + assertFalse(context.cleanupExpiredOffsets("group-id", records)); + assertEquals(List.of(), records); + } + private static OffsetFetchResponseData.OffsetFetchResponsePartitions mkOffsetPartitionResponse( int partition, long offset, From d64a97099d992ba9ad559ae52b57573e3b4601a2 Mon Sep 17 00:00:00 2001 From: Sean Quah Date: Tue, 13 May 2025 16:01:14 +0100 Subject: [PATCH 28/72] KAFKA-18688: Fix uniform homogeneous assignor stability (#19677) When the number of partitions is not divisible by the number of members, some members will end up with one more partition than others. Previously, we required these to be the members at the start of the iteration order, which meant that partitions could be reassigned even when the previous assignment was already balanced. Allow any member to have the extra partition, so that we do not move partitions around when the previous assignment is already balanced. Before the PR ``` Benchmark (assignmentType) (assignorType) (isRackAware) (memberCount) (partitionsToMemberRatio) (subscriptionType) (topicCount) Mode Cnt Score Error Units ServerSideAssignorBenchmark.doAssignment FULL RANGE false 10000 50 HOMOGENEOUS 1000 avgt 2 26.175 ms/op ServerSideAssignorBenchmark.doAssignment FULL RANGE false 10000 50 HETEROGENEOUS 1000 avgt 2 123.955 ms/op ServerSideAssignorBenchmark.doAssignment INCREMENTAL RANGE false 10000 50 HOMOGENEOUS 1000 avgt 2 24.408 ms/op ServerSideAssignorBenchmark.doAssignment INCREMENTAL RANGE false 10000 50 HETEROGENEOUS 1000 avgt 2 114.873 ms/op ``` After the PR ``` Benchmark (assignmentType) (assignorType) (isRackAware) (memberCount) (partitionsToMemberRatio) (subscriptionType) (topicCount) Mode Cnt Score Error Units ServerSideAssignorBenchmark.doAssignment FULL RANGE false 10000 50 HOMOGENEOUS 1000 avgt 2 24.259 ms/op ServerSideAssignorBenchmark.doAssignment FULL RANGE false 10000 50 HETEROGENEOUS 1000 avgt 2 118.513 ms/op ServerSideAssignorBenchmark.doAssignment INCREMENTAL RANGE false 10000 50 HOMOGENEOUS 1000 avgt 2 24.636 ms/op ServerSideAssignorBenchmark.doAssignment INCREMENTAL RANGE false 10000 50 HETEROGENEOUS 1000 avgt 2 115.503 ms/op ``` Reviewers: David Jacot --- .../UniformHomogeneousAssignmentBuilder.java | 22 ++++- ...OptimizedUniformAssignmentBuilderTest.java | 86 +++++++++++++++++-- 2 files changed, 98 insertions(+), 10 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformHomogeneousAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformHomogeneousAssignmentBuilder.java index 8e7fe209c8..3cd9bf6618 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformHomogeneousAssignmentBuilder.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformHomogeneousAssignmentBuilder.java @@ -153,8 +153,13 @@ public GroupAssignment build() throws PartitionAssignorException { * This method ensures that the original assignment is not copied if it is not * altered. */ + @SuppressWarnings({"CyclomaticComplexity", "NPathComplexity"}) private void maybeRevokePartitions() { + int memberCount = groupSpec.memberIds().size(); + int memberIndex = -1; for (String memberId : groupSpec.memberIds()) { + memberIndex++; + Map> oldAssignment = groupSpec.memberAssignment(memberId).partitions(); Map> newAssignment = null; @@ -165,9 +170,10 @@ private void maybeRevokePartitions() { } int quota = minimumMemberQuota; + boolean quotaHasExtraPartition = false; if (remainingMembersToGetAnExtraPartition > 0) { quota++; - remainingMembersToGetAnExtraPartition--; + quotaHasExtraPartition = true; } for (Map.Entry> topicPartitions : oldAssignment.entrySet()) { @@ -209,10 +215,24 @@ private void maybeRevokePartitions() { } } + if (quota > 0 && + quotaHasExtraPartition && + memberCount - memberIndex > remainingMembersToGetAnExtraPartition) { + // Give up the extra partition quota for another member to claim, + // unless this member is one of the last remainingMembersToGetAnExtraPartition + // members in the list and must take the extra partition. + quota--; + quotaHasExtraPartition = false; + } + if (quota > 0) { unfilledMembers.add(new MemberWithRemainingQuota(memberId, quota)); } + if (quotaHasExtraPartition) { + remainingMembersToGetAnExtraPartition--; + } + if (newAssignment == null) { targetAssignment.put(memberId, new MemberAssignmentImpl(oldAssignment)); } else { diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java index 39f3dc1f5d..5ee0edbf78 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java @@ -162,11 +162,10 @@ public void testFirstAssignmentTwoMembersTwoTopicsNoMemberRacks() { Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(memberA, mkAssignment( - mkTopicAssignment(topic1Uuid, 0), mkTopicAssignment(topic3Uuid, 0, 1) )); expectedAssignment.put(memberB, mkAssignment( - mkTopicAssignment(topic1Uuid, 1, 2) + mkTopicAssignment(topic1Uuid, 0, 1, 2) )); GroupSpec groupSpec = new GroupSpecImpl( @@ -219,15 +218,15 @@ public void testFirstAssignmentNumMembersGreaterThanTotalNumPartitions() { // Topic 3 has 2 partitions but three members subscribed to it - one of them should not get an assignment. Map>> expectedAssignment = new HashMap<>(); - expectedAssignment.put(memberA, mkAssignment( + expectedAssignment.put(memberA, + Collections.emptyMap() + ); + expectedAssignment.put(memberB, mkAssignment( mkTopicAssignment(topic3Uuid, 0) )); - expectedAssignment.put(memberB, mkAssignment( + expectedAssignment.put(memberC, mkAssignment( mkTopicAssignment(topic3Uuid, 1) )); - expectedAssignment.put(memberC, - Collections.emptyMap() - ); GroupSpec groupSpec = new GroupSpecImpl( members, @@ -383,11 +382,11 @@ public void testReassignmentWhenPartitionsAreAddedForTwoMembersTwoTopics() { Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(memberA, mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 2, 3), + mkTopicAssignment(topic1Uuid, 0, 2), mkTopicAssignment(topic2Uuid, 0, 3, 4) )); expectedAssignment.put(memberB, mkAssignment( - mkTopicAssignment(topic1Uuid, 1, 4, 5), + mkTopicAssignment(topic1Uuid, 1, 3, 4, 5), mkTopicAssignment(topic2Uuid, 1, 2) )); @@ -604,6 +603,75 @@ public void testReassignmentWhenOneSubscriptionRemovedAfterInitialAssignmentWith checkValidityAndBalance(members, computedAssignment); } + @Test + public void testReassignmentStickinessWhenAlreadyBalanced() { + Map topicMetadata = new HashMap<>(); + topicMetadata.put(topic1Uuid, new TopicMetadata( + topic1Uuid, + topic1Name, + 5 + )); + + // A TreeMap ensures that memberA is first in the iteration order. + Map members = new TreeMap<>(); + + // Two members must have extra partitions. In the previous assignment, they were members A + // and C. + members.put(memberA, new MemberSubscriptionAndAssignmentImpl( + Optional.empty(), + Optional.empty(), + Set.of(topic1Uuid), + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 0, 3) + )) + )); + + members.put(memberB, new MemberSubscriptionAndAssignmentImpl( + Optional.empty(), + Optional.empty(), + Set.of(topic1Uuid, topic2Uuid), + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 1) + )) + )); + + members.put(memberC, new MemberSubscriptionAndAssignmentImpl( + Optional.empty(), + Optional.empty(), + Set.of(topic1Uuid, topic2Uuid), + new Assignment(mkAssignment( + mkTopicAssignment(topic1Uuid, 2, 4) + )) + )); + + // Members A and C should keep their partitions. + Map>> expectedAssignment = new HashMap<>(); + expectedAssignment.put(memberA, mkAssignment( + mkTopicAssignment(topic1Uuid, 0, 3) + )); + expectedAssignment.put(memberB, mkAssignment( + mkTopicAssignment(topic1Uuid, 1) + )); + expectedAssignment.put(memberC, mkAssignment( + mkTopicAssignment(topic1Uuid, 2, 4) + )); + + GroupSpec groupSpec = new GroupSpecImpl( + members, + HOMOGENEOUS, + invertedTargetAssignment(members) + ); + SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(topicMetadata); + + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); + + assertAssignment(expectedAssignment, computedAssignment); + checkValidityAndBalance(members, computedAssignment); + } + /** * Verifies that the given assignment is valid with respect to the given subscriptions. * Validity requirements: From f99db0804e52644b3e05aedb96d9209246f3fe60 Mon Sep 17 00:00:00 2001 From: Kuan-Po Tseng Date: Wed, 14 May 2025 14:07:32 +0800 Subject: [PATCH 29/72] KAFKA-19275 client-state and thread-state metrics are always "Unavailable" (#19712) Fix the issue where JMC is unable to correctly display client-state and thread-state metrics. The root cause is that these two metrics directly return the `State` class to JMX. If the user has not set up the RMI server, JMC or other monitoring tools will be unable to interpret the `State` class. To resolve this, we should return a string representation of the state instead of the State class in these two metrics. Reviewers: Luke Chen , Ken Huang , Chia-Ping Tsai --- .../integration/MetricsIntegrationTest.java | 14 +++++++------- .../org/apache/kafka/streams/KafkaStreams.java | 2 +- .../streams/internals/metrics/ClientMetrics.java | 3 +-- .../streams/processor/internals/StreamThread.java | 2 +- .../processor/internals/metrics/ThreadMetrics.java | 3 +-- .../internals/metrics/ClientMetricsTest.java | 2 +- .../internals/metrics/ThreadMetricsTest.java | 2 +- 7 files changed, 13 insertions(+), 15 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/MetricsIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/MetricsIntegrationTest.java index c6dc962d6d..a56723abc3 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/MetricsIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/MetricsIntegrationTest.java @@ -272,7 +272,7 @@ private void startApplication() throws InterruptedException { kafkaStreams = new KafkaStreams(topology, streamsConfiguration); verifyAliveStreamThreadsMetric(); - verifyStateMetric(State.CREATED); + verifyStateMetric(State.CREATED.name()); verifyTopologyDescriptionMetric(topology.describe().toString()); verifyApplicationIdMetric(); @@ -283,7 +283,7 @@ private void startApplication() throws InterruptedException { () -> "Kafka Streams application did not reach state RUNNING in " + timeout + " ms"); verifyAliveStreamThreadsMetric(); - verifyStateMetric(State.RUNNING); + verifyStateMetric(State.RUNNING.name()); } private void produceRecordsForTwoSegments(final Duration segmentInterval) { @@ -357,7 +357,7 @@ public void shouldAddMetricsOnAllLevels() throws Exception { .to(STREAM_OUTPUT_4); startApplication(); - verifyStateMetric(State.RUNNING); + verifyStateMetric(State.RUNNING.name()); checkClientLevelMetrics(); checkThreadLevelMetrics(); checkTaskLevelMetrics(); @@ -392,7 +392,7 @@ public void shouldAddMetricsForWindowStoreAndSuppressionBuffer() throws Exceptio produceRecordsForClosingWindow(windowSize); startApplication(); - verifyStateMetric(State.RUNNING); + verifyStateMetric(State.RUNNING.name()); checkWindowStoreAndSuppressionBufferMetrics(); @@ -421,7 +421,7 @@ public void shouldAddMetricsForSessionStore() throws Exception { startApplication(); - verifyStateMetric(State.RUNNING); + verifyStateMetric(State.RUNNING.name()); checkSessionStoreMetrics(); @@ -439,14 +439,14 @@ private void verifyAliveStreamThreadsMetric() { assertThat(metricsList.get(0).metricValue(), is(NUM_THREADS)); } - private void verifyStateMetric(final State state) { + private void verifyStateMetric(final String state) { final List metricsList = new ArrayList(kafkaStreams.metrics().values()).stream() .filter(m -> m.metricName().name().equals(STATE) && m.metricName().group().equals(STREAM_CLIENT_NODE_METRICS)) .collect(Collectors.toList()); assertThat(metricsList.size(), is(1)); assertThat(metricsList.get(0).metricValue(), is(state)); - assertThat(metricsList.get(0).metricValue().toString(), is(state.toString())); + assertThat(metricsList.get(0).metricValue().toString(), is(state)); } private void verifyTopologyDescriptionMetric(final String topologyDescription) { diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index 00e9ede261..1fbf307426 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -983,7 +983,7 @@ private KafkaStreams(final TopologyMetadata topologyMetadata, ClientMetrics.addCommitIdMetric(streamsMetrics); ClientMetrics.addApplicationIdMetric(streamsMetrics, applicationConfigs.getString(StreamsConfig.APPLICATION_ID_CONFIG)); ClientMetrics.addTopologyDescriptionMetric(streamsMetrics, (metricsConfig, now) -> this.topologyMetadata.topologyDescriptionString()); - ClientMetrics.addStateMetric(streamsMetrics, (metricsConfig, now) -> state); + ClientMetrics.addStateMetric(streamsMetrics, (metricsConfig, now) -> state.name()); ClientMetrics.addClientStateTelemetryMetric(streamsMetrics, (metricsConfig, now) -> state.ordinal()); ClientMetrics.addClientRecordingLevelMetric(streamsMetrics, calculateMetricsRecordingLevel()); threads = Collections.synchronizedList(new LinkedList<>()); diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/ClientMetrics.java b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/ClientMetrics.java index 22e09042e1..21bac269d5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/ClientMetrics.java +++ b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/ClientMetrics.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.metrics.Gauge; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.Sensor.RecordingLevel; -import org.apache.kafka.streams.KafkaStreams.State; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.slf4j.Logger; @@ -118,7 +117,7 @@ public static void addTopologyDescriptionMetric(final StreamsMetricsImpl streams } public static void addStateMetric(final StreamsMetricsImpl streamsMetrics, - final Gauge stateProvider) { + final Gauge stateProvider) { streamsMetrics.addClientLevelMutableMetric( STATE, STATE_DESCRIPTION, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 2a83f0b612..20d49b44db 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -628,7 +628,7 @@ public StreamThread(final Time time, ThreadMetrics.addThreadStateMetric( threadId, streamsMetrics, - (metricConfig, now) -> this.state()); + (metricConfig, now) -> this.state().name()); ThreadMetrics.addThreadBlockedTimeMetric( threadId, new StreamThreadTotalBlockedTime( diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetrics.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetrics.java index bddd27c590..b45bde5ddd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetrics.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetrics.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.metrics.Gauge; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.Sensor.RecordingLevel; -import org.apache.kafka.streams.processor.internals.StreamThread; import org.apache.kafka.streams.processor.internals.StreamThreadTotalBlockedTime; import java.util.Collections; @@ -313,7 +312,7 @@ public static void addThreadStateTelemetryMetric(final String processId, public static void addThreadStateMetric(final String threadId, final StreamsMetricsImpl streamsMetrics, - final Gauge threadStateProvider) { + final Gauge threadStateProvider) { streamsMetrics.addThreadLevelMutableMetric( STATE, THREAD_STATE_DESCRIPTION, diff --git a/streams/src/test/java/org/apache/kafka/streams/internals/metrics/ClientMetricsTest.java b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/ClientMetricsTest.java index 21e65ce892..9142835b92 100644 --- a/streams/src/test/java/org/apache/kafka/streams/internals/metrics/ClientMetricsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/ClientMetricsTest.java @@ -89,7 +89,7 @@ public void shouldAddTopologyDescriptionMetric() { public void shouldAddStateMetric() { final String name = "state"; final String description = "The state of the Kafka Streams client"; - final Gauge stateProvider = (config, now) -> State.RUNNING; + final Gauge stateProvider = (config, now) -> State.RUNNING.name(); setUpAndVerifyMutableMetric( name, description, diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetricsTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetricsTest.java index a24a250c9e..87891ab389 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetricsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetricsTest.java @@ -435,7 +435,7 @@ public void shouldAddThreadStateTelemetryMetric() { @Test public void shouldAddThreadStateJmxMetric() { - final Gauge threadStateProvider = (streamsMetrics, startTime) -> StreamThread.State.RUNNING; + final Gauge threadStateProvider = (streamsMetrics, startTime) -> StreamThread.State.RUNNING.name(); ThreadMetrics.addThreadStateMetric( THREAD_ID, streamsMetrics, From d7d7876989e31da54797e96e1960010f27d2c97b Mon Sep 17 00:00:00 2001 From: David Jacot Date: Thu, 15 May 2025 19:04:38 +0200 Subject: [PATCH 30/72] KAFKA-19274; Group Coordinator Shards are not unloaded when `__consumer_offsets` topic is deleted (#19713) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Group Coordinator Shards are not unloaded when `__consumer_offsets` topic is deleted. The unloading is scheduled but it is ignored because the epoch is equal to the current epoch: ``` [2025-05-13 08:46:00,883] INFO [GroupCoordinator id=1] Scheduling unloading of metadata for __consumer_offsets-0 with epoch OptionalInt[0] (org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime) [2025-05-13 08:46:00,883] INFO [GroupCoordinator id=1] Scheduling unloading of metadata for __consumer_offsets-1 with epoch OptionalInt[0] (org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime) [2025-05-13 08:46:00,883] INFO [GroupCoordinator id=1] Ignored unloading metadata for __consumer_offsets-0 in epoch OptionalInt[0] since current epoch is 0. (org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime) [2025-05-13 08:46:00,883] INFO [GroupCoordinator id=1] Ignored unloading metadata for __consumer_offsets-1 in epoch OptionalInt[0] since current epoch is 0. (org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime) ``` This patch fixes the issue by not setting the leader epoch in this case. The coordinator expects the leader epoch to be incremented when the resignation code is called. When the topic is deleted, the epoch is not incremented. Therefore, we must not use it. Note that this is aligned with deleted partitions are handled too. Reviewers: Dongnuo Lyu , José Armando García Sancio --- .../metadata/BrokerMetadataPublisher.scala | 7 +- .../api/GroupCoordinatorIntegrationTest.scala | 59 +++++++++++++++- .../BrokerMetadataPublisherTest.scala | 68 ++++++++++++++++++- 3 files changed, 129 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala index 1985f04348..4a7ae84c2d 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala @@ -250,6 +250,11 @@ class BrokerMetadataPublisher( /** * Update the coordinator of local replica changes: election and resignation. * + * When the topic is deleted or a partition of the topic is deleted, {@param resignation} + * callback must be called with {@code None}. The coordinator expects the leader epoch to be + * incremented when the {@param resignation} callback is called but the leader epoch + * is not incremented when a topic is deleted. + * * @param image latest metadata image * @param delta metadata delta from the previous image and the latest image * @param topicName name of the topic associated with the coordinator @@ -270,7 +275,7 @@ class BrokerMetadataPublisher( if (topicsDelta.topicWasDeleted(topicName)) { topicsDelta.image.getTopic(topicName).partitions.entrySet.forEach { entry => if (entry.getValue.leader == brokerId) { - resignation(entry.getKey, Some(entry.getValue.leaderEpoch)) + resignation(entry.getKey, None) } } } diff --git a/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala index cbd69baedc..8d1399a1eb 100644 --- a/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala @@ -17,8 +17,8 @@ import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, Typ import kafka.utils.TestUtils import org.apache.kafka.clients.admin.{Admin, ConsumerGroupDescription} import org.apache.kafka.clients.consumer.{Consumer, GroupProtocol, OffsetAndMetadata} -import org.apache.kafka.common.errors.GroupIdNotFoundException -import org.apache.kafka.common.{ConsumerGroupState, GroupType, KafkaFuture, TopicPartition} +import org.apache.kafka.common.errors.{GroupIdNotFoundException, UnknownTopicOrPartitionException} +import org.apache.kafka.common.{ConsumerGroupState, GroupType, KafkaFuture, TopicCollection, TopicPartition} import org.junit.jupiter.api.Assertions._ import scala.jdk.CollectionConverters._ @@ -27,11 +27,12 @@ import org.apache.kafka.common.record.CompressionType import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.server.config.ServerConfigs +import org.apache.kafka.test.{TestUtils => JTestUtils} import org.junit.jupiter.api.Timeout import java.time.Duration import java.util.Collections -import java.util.concurrent.TimeUnit +import java.util.concurrent.{ExecutionException, TimeUnit} @Timeout(120) class GroupCoordinatorIntegrationTest(cluster: ClusterInstance) { @@ -278,6 +279,58 @@ class GroupCoordinatorIntegrationTest(cluster: ClusterInstance) { } } + @ClusterTest( + types = Array(Type.KRAFT), + serverProperties = Array( + new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"), + new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1") + ) + ) + def testRecreatingConsumerOffsetsTopic(): Unit = { + withAdmin { admin => + TestUtils.createTopicWithAdminRaw( + admin = admin, + topic = "foo", + numPartitions = 3 + ) + + withConsumer(groupId = "group", groupProtocol = GroupProtocol.CONSUMER) { consumer => + consumer.subscribe(List("foo").asJava) + TestUtils.waitUntilTrue(() => { + consumer.poll(Duration.ofMillis(50)) + consumer.assignment().asScala.nonEmpty + }, msg = "Consumer did not get an non empty assignment") + } + + admin + .deleteTopics(TopicCollection.ofTopicNames(List(Topic.GROUP_METADATA_TOPIC_NAME).asJava)) + .all() + .get() + + TestUtils.waitUntilTrue(() => { + try { + admin + .describeTopics(TopicCollection.ofTopicNames(List(Topic.GROUP_METADATA_TOPIC_NAME).asJava)) + .topicNameValues() + .get(Topic.GROUP_METADATA_TOPIC_NAME) + .get(JTestUtils.DEFAULT_MAX_WAIT_MS, TimeUnit.MILLISECONDS) + false + } catch { + case e: ExecutionException => + e.getCause.isInstanceOf[UnknownTopicOrPartitionException] + } + }, msg = s"${Topic.GROUP_METADATA_TOPIC_NAME} was not deleted") + + withConsumer(groupId = "group", groupProtocol = GroupProtocol.CONSUMER) { consumer => + consumer.subscribe(List("foo").asJava) + TestUtils.waitUntilTrue(() => { + consumer.poll(Duration.ofMillis(50)) + consumer.assignment().asScala.nonEmpty + }, msg = "Consumer did not get an non empty assignment") + } + } + } + private def rollAndCompactConsumerOffsets(): Unit = { val tp = new TopicPartition("__consumer_offsets", 0) val broker = cluster.brokers.asScala.head._2 diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala index a166368a5a..c5f4d3187e 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala @@ -20,15 +20,18 @@ package kafka.server.metadata import kafka.coordinator.transaction.TransactionCoordinator import java.util.Collections.{singleton, singletonList, singletonMap} -import java.util.Properties +import java.util.{OptionalInt, Properties} import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} import kafka.log.LogManager import kafka.server.{BrokerServer, KafkaConfig, ReplicaManager} import kafka.utils.TestUtils import org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, ConfigEntry, NewTopic} +import org.apache.kafka.common.Uuid import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.config.ConfigResource.Type.BROKER +import org.apache.kafka.common.internals.Topic +import org.apache.kafka.common.metadata.{PartitionRecord, RemoveTopicRecord, TopicRecord} import org.apache.kafka.common.test.{KafkaClusterTestKit, TestKitNodes} import org.apache.kafka.common.utils.Exit import org.apache.kafka.coordinator.group.GroupCoordinator @@ -179,6 +182,69 @@ class BrokerMetadataPublisherTest { } } + @Test + def testGroupCoordinatorTopicDeletion(): Unit = { + val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(0)) + val metadataCache = new KRaftMetadataCache(0, () => KRaftVersion.KRAFT_VERSION_1) + val logManager = mock(classOf[LogManager]) + val replicaManager = mock(classOf[ReplicaManager]) + val groupCoordinator = mock(classOf[GroupCoordinator]) + val faultHandler = mock(classOf[FaultHandler]) + + val metadataPublisher = new BrokerMetadataPublisher( + config, + metadataCache, + logManager, + replicaManager, + groupCoordinator, + mock(classOf[TransactionCoordinator]), + Some(mock(classOf[ShareCoordinator])), + mock(classOf[DynamicConfigPublisher]), + mock(classOf[DynamicClientQuotaPublisher]), + mock(classOf[DynamicTopicClusterQuotaPublisher]), + mock(classOf[ScramPublisher]), + mock(classOf[DelegationTokenPublisher]), + mock(classOf[AclPublisher]), + faultHandler, + faultHandler + ) + + val topicId = Uuid.randomUuid() + var delta = new MetadataDelta(MetadataImage.EMPTY) + delta.replay(new TopicRecord() + .setName(Topic.GROUP_METADATA_TOPIC_NAME) + .setTopicId(topicId) + ) + delta.replay(new PartitionRecord() + .setTopicId(topicId) + .setPartitionId(0) + .setLeader(config.brokerId) + ) + delta.replay(new PartitionRecord() + .setTopicId(topicId) + .setPartitionId(1) + .setLeader(config.brokerId) + ) + val image = delta.apply(MetadataProvenance.EMPTY) + + delta = new MetadataDelta(image) + delta.replay(new RemoveTopicRecord() + .setTopicId(topicId) + ) + + metadataPublisher.onMetadataUpdate(delta, delta.apply(MetadataProvenance.EMPTY), + LogDeltaManifest.newBuilder() + .provenance(MetadataProvenance.EMPTY) + .leaderAndEpoch(LeaderAndEpoch.UNKNOWN) + .numBatches(1) + .elapsedNs(100) + .numBytes(42) + .build()) + + verify(groupCoordinator).onResignation(0, OptionalInt.empty()) + verify(groupCoordinator).onResignation(1, OptionalInt.empty()) + } + @Test def testNewImagePushedToGroupCoordinator(): Unit = { val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(0)) From 62c6697ac98dd361b7fd669b2b0776c00cc619f2 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Thu, 15 May 2025 18:34:07 -0700 Subject: [PATCH 31/72] KAFKA-19208: KStream-GlobalKTable join should not drop left-null-key record (#19580) Reviewers: Lucas Brutschy --- .../internals/KStreamKTableJoinProcessor.java | 2 +- .../KStreamGlobalKTableJoinTest.java | 112 +++++++++++--- .../KStreamGlobalKTableLeftJoinTest.java | 140 ++++++++++++++---- .../internals/KStreamKTableJoinTest.java | 71 +++++++-- .../internals/KStreamKTableLeftJoinTest.java | 62 ++++++++ 5 files changed, 322 insertions(+), 65 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java index e81877c99e..17f1ceb22f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java @@ -150,7 +150,7 @@ private boolean maybeDropRecord(final Record record) { // furthermore, on left/outer joins 'null' in ValueJoiner#apply() indicates a missing record -- // thus, to be consistent and to avoid ambiguous null semantics, null values are ignored final K2 mappedKey = keyMapper.apply(record.key(), record.value()); - if (leftJoin && record.key() == null && record.value() != null) { + if (leftJoin && mappedKey == null && record.value() != null) { return false; } if (mappedKey == null || record.value() == null) { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoinTest.java index cd9126f03e..6085bdacf6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoinTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.kstream.internals; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; @@ -46,6 +47,10 @@ import java.util.Properties; import java.util.Set; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; import static org.junit.jupiter.api.Assertions.assertEquals; public class KStreamGlobalKTableJoinTest { @@ -89,7 +94,7 @@ private void init(final Optional versionedStoreHistoryRetentionMs) { } keyMapper = (key, value) -> { final String[] tokens = value.split(","); - // Value is comma delimited. If second token is present, it's the key to the global ktable. + // Value is comma-delimited. If second token is present, it's the key to the global ktable. // If not present, use null to indicate no match return tokens.length > 1 ? tokens[1] : null; }; @@ -169,8 +174,10 @@ public void shouldNotJoinOnGlobalTableUpdates() { // push all four items to the primary stream. this should produce two items. pushToStream(4, "X", true, false); - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "X0,FKey0+Y0", 2), - new KeyValueTimestamp<>(1, "X1,FKey1+Y1", 3)); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "X0,FKey0+Y0", 2), + new KeyValueTimestamp<>(1, "X1,FKey1+Y1", 3) + ); // push all items to the globalTable. this should not produce any item @@ -180,10 +187,12 @@ public void shouldNotJoinOnGlobalTableUpdates() { // push all four items to the primary stream. this should produce four items. pushToStream(4, "X", true, false); - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "X0,FKey0+YY0", 6), - new KeyValueTimestamp<>(1, "X1,FKey1+YY1", 7), - new KeyValueTimestamp<>(2, "X2,FKey2+YY2", 8), - new KeyValueTimestamp<>(3, "X3,FKey3+YY3", 9)); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "X0,FKey0+YY0", 6), + new KeyValueTimestamp<>(1, "X1,FKey1+YY1", 7), + new KeyValueTimestamp<>(2, "X2,FKey2+YY2", 8), + new KeyValueTimestamp<>(3, "X3,FKey3+YY3", 9) + ); // push all items to the globalTable. this should not produce any item @@ -202,8 +211,10 @@ public void shouldJoinOnlyIfMatchFoundOnStreamUpdates() { // push all four items to the primary stream. this should produce two items. pushToStream(4, "X", true, false); - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "X0,FKey0+Y0", 0), - new KeyValueTimestamp<>(1, "X1,FKey1+Y1", 1)); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "X0,FKey0+Y0", 0), + new KeyValueTimestamp<>(1, "X1,FKey1+Y1", 1) + ); } @@ -218,10 +229,12 @@ public void shouldClearGlobalTableEntryOnNullValueUpdates() { // push all four items to the primary stream. this should produce four items. pushToStream(4, "X", true, false); - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "X0,FKey0+Y0", 0), - new KeyValueTimestamp<>(1, "X1,FKey1+Y1", 1), - new KeyValueTimestamp<>(2, "X2,FKey2+Y2", 2), - new KeyValueTimestamp<>(3, "X3,FKey3+Y3", 3)); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "X0,FKey0+Y0", 0), + new KeyValueTimestamp<>(1, "X1,FKey1+Y1", 1), + new KeyValueTimestamp<>(2, "X2,FKey2+Y2", 2), + new KeyValueTimestamp<>(3, "X3,FKey3+Y3", 3) + ); // push two items with null to the globalTable as deletes. this should not produce any item. @@ -231,8 +244,10 @@ public void shouldClearGlobalTableEntryOnNullValueUpdates() { // push all four items to the primary stream. this should produce two items. pushToStream(4, "XX", true, false); - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(2, "XX2,FKey2+Y2", 6), - new KeyValueTimestamp<>(3, "XX3,FKey3+Y3", 7)); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(2, "XX2,FKey2+Y2", 6), + new KeyValueTimestamp<>(3, "XX3,FKey3+Y3", 7) + ); } @Test @@ -248,10 +263,54 @@ public void shouldNotJoinOnNullKeyMapperValues() { pushToStream(4, "XXX", false, false); processor.checkAndClearProcessResult(EMPTY); + + assertThat( + driver.metrics().get( + new MetricName( + "dropped-records-total", + "stream-task-metrics", + "", + mkMap( + mkEntry("thread-id", Thread.currentThread().getName()), + mkEntry("task-id", "0_0") + ) + )) + .metricValue(), + is(4.0) + ); + } + + @Test + public void shouldNotJoinOnNullKeyMapperValuesWithNullKeys() { + // push all items to the globalTable. this should not produce any item + + pushToGlobalTable(4, "Y"); + processor.checkAndClearProcessResult(EMPTY); + + // push all four items to the primary stream with no foreign key, resulting in null keyMapper values. + // this should not produce any item. + + pushToStream(4, "XXX", false, true); + processor.checkAndClearProcessResult(EMPTY); + + assertThat( + driver.metrics().get( + new MetricName( + "dropped-records-total", + "stream-task-metrics", + "", + mkMap( + mkEntry("thread-id", Thread.currentThread().getName()), + mkEntry("task-id", "0_0") + ) + )) + .metricValue(), + is(4.0) + ); } @Test - public void shouldJoinOnNullKeyWithNonNullKeyMapperValues() { + public void shouldJoinOnNullKey() { // push two items to the globalTable. this should not produce any item. pushToGlobalTable(2, "Y"); @@ -260,7 +319,24 @@ public void shouldJoinOnNullKeyWithNonNullKeyMapperValues() { // push all four items to the primary stream. this should produce two items. pushToStream(4, "X", true, true); - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(null, "X0,FKey0+Y0", 0), - new KeyValueTimestamp<>(1, "X1,FKey1+Y1", 1)); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(null, "X0,FKey0+Y0", 0), + new KeyValueTimestamp<>(1, "X1,FKey1+Y1", 1) + ); + + assertThat( + driver.metrics().get( + new MetricName( + "dropped-records-total", + "stream-task-metrics", + "", + mkMap( + mkEntry("thread-id", Thread.currentThread().getName()), + mkEntry("task-id", "0_0") + ) + )) + .metricValue(), + is(0.0) + ); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableLeftJoinTest.java index b0a44c36e2..6f49917a01 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableLeftJoinTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.kstream.internals; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; @@ -46,6 +47,10 @@ import java.util.Properties; import java.util.Set; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; import static org.junit.jupiter.api.Assertions.assertEquals; public class KStreamGlobalKTableLeftJoinTest { @@ -89,7 +94,7 @@ private void init(final Optional versionedStoreHistoryRetentionMs) { } keyMapper = (key, value) -> { final String[] tokens = value.split(","); - // Value is comma delimited. If second token is present, it's the key to the global ktable. + // Value is comma-delimited. If second token is present, it's the key to the global ktable. // If not present, use null to indicate no match return tokens.length > 1 ? tokens[1] : null; }; @@ -150,8 +155,10 @@ public void shouldNotJoinWithEmptyGlobalTableOnStreamUpdates() { // push two items to the primary stream. the globalTable is empty pushToStream(2, "X", true, false); - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "X0,FKey0+null", 0), - new KeyValueTimestamp<>(1, "X1,FKey1+null", 1)); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "X0,FKey0+null", 0), + new KeyValueTimestamp<>(1, "X1,FKey1+null", 1) + ); } @Test @@ -160,8 +167,10 @@ public void shouldNotJoinOnGlobalTableUpdates() { // push two items to the primary stream. the globalTable is empty pushToStream(2, "X", true, false); - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "X0,FKey0+null", 0), - new KeyValueTimestamp<>(1, "X1,FKey1+null", 1)); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "X0,FKey0+null", 0), + new KeyValueTimestamp<>(1, "X1,FKey1+null", 1) + ); // push two items to the globalTable. this should not produce any item. @@ -171,10 +180,12 @@ public void shouldNotJoinOnGlobalTableUpdates() { // push all four items to the primary stream. this should produce four items. pushToStream(4, "X", true, false); - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "X0,FKey0+Y0", 2), - new KeyValueTimestamp<>(1, "X1,FKey1+Y1", 3), - new KeyValueTimestamp<>(2, "X2,FKey2+null", 4), - new KeyValueTimestamp<>(3, "X3,FKey3+null", 5)); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "X0,FKey0+Y0", 2), + new KeyValueTimestamp<>(1, "X1,FKey1+Y1", 3), + new KeyValueTimestamp<>(2, "X2,FKey2+null", 4), + new KeyValueTimestamp<>(3, "X3,FKey3+null", 5) + ); // push all items to the globalTable. this should not produce any item @@ -184,10 +195,12 @@ public void shouldNotJoinOnGlobalTableUpdates() { // push all four items to the primary stream. this should produce four items. pushToStream(4, "X", true, false); - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "X0,FKey0+YY0", 6), - new KeyValueTimestamp<>(1, "X1,FKey1+YY1", 7), - new KeyValueTimestamp<>(2, "X2,FKey2+YY2", 8), - new KeyValueTimestamp<>(3, "X3,FKey3+YY3", 9)); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "X0,FKey0+YY0", 6), + new KeyValueTimestamp<>(1, "X1,FKey1+YY1", 7), + new KeyValueTimestamp<>(2, "X2,FKey2+YY2", 8), + new KeyValueTimestamp<>(3, "X3,FKey3+YY3", 9) + ); // push all items to the globalTable. this should not produce any item @@ -206,10 +219,12 @@ public void shouldJoinRegardlessIfMatchFoundOnStreamUpdates() { // push all four items to the primary stream. this should produce four items. pushToStream(4, "X", true, false); - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "X0,FKey0+Y0", 0), - new KeyValueTimestamp<>(1, "X1,FKey1+Y1", 1), - new KeyValueTimestamp<>(2, "X2,FKey2+null", 2), - new KeyValueTimestamp<>(3, "X3,FKey3+null", 3)); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "X0,FKey0+Y0", 0), + new KeyValueTimestamp<>(1, "X1,FKey1+Y1", 1), + new KeyValueTimestamp<>(2, "X2,FKey2+null", 2), + new KeyValueTimestamp<>(3, "X3,FKey3+null", 3) + ); } @@ -224,10 +239,12 @@ public void shouldClearGlobalTableEntryOnNullValueUpdates() { // push all four items to the primary stream. this should produce four items. pushToStream(4, "X", true, false); - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "X0,FKey0+Y0", 0), - new KeyValueTimestamp<>(1, "X1,FKey1+Y1", 1), - new KeyValueTimestamp<>(2, "X2,FKey2+Y2", 2), - new KeyValueTimestamp<>(3, "X3,FKey3+Y3", 3)); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "X0,FKey0+Y0", 0), + new KeyValueTimestamp<>(1, "X1,FKey1+Y1", 1), + new KeyValueTimestamp<>(2, "X2,FKey2+Y2", 2), + new KeyValueTimestamp<>(3, "X3,FKey3+Y3", 3) + ); // push two items with null to the globalTable as deletes. this should not produce any item. @@ -236,15 +253,17 @@ public void shouldClearGlobalTableEntryOnNullValueUpdates() { // push all four items to the primary stream. this should produce four items. - pushToStream(4, "XX", true, false); - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "XX0,FKey0+null", 4), - new KeyValueTimestamp<>(1, "XX1,FKey1+null", 5), - new KeyValueTimestamp<>(2, "XX2,FKey2+Y2", 6), - new KeyValueTimestamp<>(3, "XX3,FKey3+Y3", 7)); + pushToStream(4, "X", true, false); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "X0,FKey0+null", 4), + new KeyValueTimestamp<>(1, "X1,FKey1+null", 5), + new KeyValueTimestamp<>(2, "X2,FKey2+Y2", 6), + new KeyValueTimestamp<>(3, "X3,FKey3+Y3", 7) + ); } @Test - public void shouldNotJoinOnNullKeyMapperValues() { + public void shouldJoinOnNullKeyMapperValues() { // push all items to the globalTable. this should not produce any item @@ -255,11 +274,66 @@ public void shouldNotJoinOnNullKeyMapperValues() { // this should not produce any item. pushToStream(4, "XXX", false, false); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "XXX0+null", 0), + new KeyValueTimestamp<>(1, "XXX1+null", 1), + new KeyValueTimestamp<>(2, "XXX2+null", 2), + new KeyValueTimestamp<>(3, "XXX3+null", 3) + ); + + assertThat( + driver.metrics().get( + new MetricName( + "dropped-records-total", + "stream-task-metrics", + "", + mkMap( + mkEntry("thread-id", Thread.currentThread().getName()), + mkEntry("task-id", "0_0") + ) + )) + .metricValue(), + is(0.0) + ); + } + + @Test + public void shouldJoinOnNullKeyMapperValuesWithNullKeys() { + + // push all items to the globalTable. this should not produce any item + + pushToGlobalTable(4, "Y"); processor.checkAndClearProcessResult(EMPTY); + + // push all four items to the primary stream with no foreign key, resulting in null keyMapper values. + // this should not produce any item. + + pushToStream(4, "XXX", false, true); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(null, "XXX0+null", 0), + new KeyValueTimestamp<>(1, "XXX1+null", 1), + new KeyValueTimestamp<>(2, "XXX2+null", 2), + new KeyValueTimestamp<>(3, "XXX3+null", 3) + ); + + assertThat( + driver.metrics().get( + new MetricName( + "dropped-records-total", + "stream-task-metrics", + "", + mkMap( + mkEntry("thread-id", Thread.currentThread().getName()), + mkEntry("task-id", "0_0") + ) + )) + .metricValue(), + is(0.0) + ); } @Test - public void shouldJoinOnNullKeyWithNonNullKeyMapperValues() { + public void shouldJoinOnNullKey() { // push four items to the globalTable. this should not produce any item. pushToGlobalTable(4, "Y"); @@ -268,9 +342,11 @@ public void shouldJoinOnNullKeyWithNonNullKeyMapperValues() { // push all four items to the primary stream. this should produce four items. pushToStream(4, "X", true, true); - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(null, "X0,FKey0+Y0", 0), - new KeyValueTimestamp<>(1, "X1,FKey1+Y1", 1), - new KeyValueTimestamp<>(2, "X2,FKey2+Y2", 2), - new KeyValueTimestamp<>(3, "X3,FKey3+Y3", 3)); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(null, "X0,FKey0+Y0", 0), + new KeyValueTimestamp<>(1, "X1,FKey1+Y1", 1), + new KeyValueTimestamp<>(2, "X2,FKey2+Y2", 2), + new KeyValueTimestamp<>(3, "X3,FKey3+Y3", 3) + ); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java index c455271598..1419fd716c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.kstream.internals; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; @@ -52,6 +53,8 @@ import java.util.Random; import java.util.Set; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; import static org.hamcrest.CoreMatchers.hasItem; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; @@ -354,8 +357,10 @@ public void shouldNotJoinOnTableUpdates() { // push all four items to the primary stream. this should produce two items. pushToStream(4, "X"); - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "X0+Y0", 0), - new KeyValueTimestamp<>(1, "X1+Y1", 1)); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "X0+Y0", 0), + new KeyValueTimestamp<>(1, "X1+Y1", 1) + ); // push all items to the table. this should not produce any item pushToTable(4, "YY"); @@ -363,10 +368,12 @@ public void shouldNotJoinOnTableUpdates() { // push all four items to the primary stream. this should produce four items. pushToStream(4, "X"); - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "X0+YY0", 0), - new KeyValueTimestamp<>(1, "X1+YY1", 1), - new KeyValueTimestamp<>(2, "X2+YY2", 2), - new KeyValueTimestamp<>(3, "X3+YY3", 3)); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "X0+YY0", 0), + new KeyValueTimestamp<>(1, "X1+YY1", 1), + new KeyValueTimestamp<>(2, "X2+YY2", 2), + new KeyValueTimestamp<>(3, "X3+YY3", 3) + ); // push all items to the table. this should not produce any item pushToTable(4, "YYY"); @@ -381,8 +388,10 @@ public void shouldJoinOnlyIfMatchFoundOnStreamUpdates() { // push all four items to the primary stream. this should produce two items. pushToStream(4, "X"); - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "X0+Y0", 0), - new KeyValueTimestamp<>(1, "X1+Y1", 1)); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "X0+Y0", 0), + new KeyValueTimestamp<>(1, "X1+Y1", 1) + ); } @Test @@ -393,10 +402,12 @@ public void shouldClearTableEntryOnNullValueUpdates() { // push all four items to the primary stream. this should produce four items. pushToStream(4, "X"); - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "X0+Y0", 0), - new KeyValueTimestamp<>(1, "X1+Y1", 1), - new KeyValueTimestamp<>(2, "X2+Y2", 2), - new KeyValueTimestamp<>(3, "X3+Y3", 3)); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "X0+Y0", 0), + new KeyValueTimestamp<>(1, "X1+Y1", 1), + new KeyValueTimestamp<>(2, "X2+Y2", 2), + new KeyValueTimestamp<>(3, "X3+Y3", 3) + ); // push two items with null to the table as deletes. this should not produce any item. pushNullValueToTable(); @@ -404,8 +415,10 @@ public void shouldClearTableEntryOnNullValueUpdates() { // push all four items to the primary stream. this should produce two items. pushToStream(4, "XX"); - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(2, "XX2+Y2", 2), - new KeyValueTimestamp<>(3, "XX3+Y3", 3)); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(2, "XX2+Y2", 2), + new KeyValueTimestamp<>(3, "XX3+Y3", 3) + ); } @Test @@ -420,6 +433,21 @@ public void shouldLogAndMeterWhenSkippingNullLeftKey() { hasItem("Skipping record due to null join key or value. topic=[streamTopic] partition=[0] " + "offset=[0]")); } + + assertThat( + driver.metrics().get( + new MetricName( + "dropped-records-total", + "stream-task-metrics", + "", + mkMap( + mkEntry("thread-id", Thread.currentThread().getName()), + mkEntry("task-id", "0_0") + ) + )) + .metricValue(), + is(1.0) + ); } @Test @@ -435,6 +463,21 @@ public void shouldLogAndMeterWhenSkippingNullLeftValue() { + "offset=[0]") ); } + + assertThat( + driver.metrics().get( + new MetricName( + "dropped-records-total", + "stream-task-metrics", + "", + mkMap( + mkEntry("thread-id", Thread.currentThread().getName()), + mkEntry("task-id", "0_0") + ) + )) + .metricValue(), + is(1.0) + ); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java index 42ff6aa8d9..25eafd3043 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java @@ -16,9 +16,11 @@ */ package org.apache.kafka.streams.kstream.internals; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.utils.LogCaptureAppender; import org.apache.kafka.streams.KeyValueTimestamp; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.TestInputTopic; @@ -45,7 +47,10 @@ import java.util.Random; import java.util.Set; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; public class KStreamKTableLeftJoinTest { private static final KeyValueTimestamp[] EMPTY = new KeyValueTimestamp[0]; @@ -199,4 +204,61 @@ public void shouldClearTableEntryOnNullValueUpdates() { new KeyValueTimestamp<>(3, "XX3+Y3", 3)); } + @Test + public void shouldNotDropLeftNullKey() { + // push all four items to the table. this should not produce any item. + pushToTable(1, "Y"); + processor.checkAndClearProcessResult(EMPTY); + + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(KStreamKTableJoin.class)) { + final TestInputTopic inputTopic = + driver.createInputTopic(streamTopic, new IntegerSerializer(), new StringSerializer()); + inputTopic.pipeInput(null, "A", 0); + + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(null, "A+null", 0)); + + assertTrue(appender.getMessages().isEmpty()); + } + + assertEquals( + 0.0, + driver.metrics().get( + new MetricName( + "dropped-records-total", + "stream-task-metrics", + "", + mkMap( + mkEntry("thread-id", Thread.currentThread().getName()), + mkEntry("task-id", "0_0") + ) + )) + .metricValue() + ); + } + + @Test + public void shouldLogAndMeterWhenSkippingNullLeftValue() { + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(KStreamKTableJoin.class)) { + final TestInputTopic inputTopic = + driver.createInputTopic(streamTopic, new IntegerSerializer(), new StringSerializer()); + inputTopic.pipeInput(1, null); + + assertTrue(appender.getMessages().contains("Skipping record due to null join key or value. topic=[streamTopic] partition=[0] offset=[0]")); + } + + assertEquals( + 1.0, + driver.metrics().get( + new MetricName( + "dropped-records-total", + "stream-task-metrics", + "", + mkMap( + mkEntry("thread-id", Thread.currentThread().getName()), + mkEntry("task-id", "0_0") + ) + )) + .metricValue() + ); + } } From 923086dba246297c916ee43331459ca2ba6a595f Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Thu, 15 May 2025 21:37:04 -0700 Subject: [PATCH 32/72] KAFKA-19171: Kafka Streams crashes with UnsupportedOperationException (#19507) This PR fixes a regression bug introduced with KAFKA-17203. We need to pass in mutable collections into `closeTaskClean(...)`. Reviewers: Chia-Ping Tsai , Bruno Cadonna , Lucas Brutschy --- .../streams/processor/internals/TaskManager.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java index 7b360a7606..8e51c4215e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java @@ -513,8 +513,14 @@ private void handleTasksWithStateUpdater(final Map> private void handleTasksPendingInitialization() { // All tasks pending initialization are not part of the usual bookkeeping + + final Set tasksToCloseDirty = new HashSet<>(); + for (final Task task : tasks.drainPendingTasksToInit()) { - closeTaskClean(task, Collections.emptySet(), Collections.emptyMap()); + closeTaskClean(task, tasksToCloseDirty, new HashMap<>()); + } + for (final Task task : tasksToCloseDirty) { + closeTaskDirty(task, false); } } @@ -1245,7 +1251,6 @@ private void closeRunningTasksDirty() { private void removeLostActiveTasksFromStateUpdaterAndPendingTasksToInit() { if (stateUpdater != null) { final Map> futures = new LinkedHashMap<>(); - final Map failedTasksDuringCleanClose = new HashMap<>(); final Set tasksToCloseClean = new HashSet<>(tasks.drainPendingActiveTasksToInit()); final Set tasksToCloseDirty = new HashSet<>(); for (final Task restoringTask : stateUpdater.tasks()) { @@ -1256,7 +1261,7 @@ private void removeLostActiveTasksFromStateUpdaterAndPendingTasksToInit() { addToTasksToClose(futures, tasksToCloseClean, tasksToCloseDirty); for (final Task task : tasksToCloseClean) { - closeTaskClean(task, tasksToCloseDirty, failedTasksDuringCleanClose); + closeTaskClean(task, tasksToCloseDirty, new HashMap<>()); } for (final Task task : tasksToCloseDirty) { closeTaskDirty(task, false); From 14fd498ed0ef499979b6fa46b6553db5c9f22d96 Mon Sep 17 00:00:00 2001 From: Andy Li Date: Wed, 21 May 2025 01:26:51 +0900 Subject: [PATCH 33/72] MINOR: API Responses missing latest version in Kafka protocol guide (#19769) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### Issue: API Responses missing latest version in [Kafka protocol guide](https://kafka.apache.org/protocol.html) #### For example: These are missing: - ApiVersions Response (Version: 4) — Only versions 0–3 are documented, though version 4 of the request is included. - DescribeTopicPartitions Response — Not listed at all. - Fetch Response (Version: 17) — Only versions 4–16 are documented, though version 17 of the request is included. #### After the fix: docs/generated/protocol_messages.html image Reviewers: dengziming , Ken Huang , Chia-Ping Tsai --- .../main/java/org/apache/kafka/common/protocol/Protocol.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index 237948f61c..c23aa1782d 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -208,7 +208,7 @@ public static String toHtml() { // Responses b.append("Responses:
\n"); Schema[] responses = key.messageType.responseSchemas(); - for (int version = key.oldestVersion(); version < key.latestVersion(); version++) { + for (int version = key.oldestVersion(); version <= key.latestVersion(); version++) { Schema schema = responses[version]; if (schema == null) throw new IllegalStateException("Unexpected null schema for " + key + " with version " + version); From 3170e1130c4464705f4d636c5bb5a0d313c5ba96 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Mon, 12 May 2025 13:23:18 -0700 Subject: [PATCH 34/72] KAFKA-18345; Prevent livelocked elections (#19658) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit At the retry limit binaryExponentialElectionBackoffMs it becomes statistically likely that the exponential backoff returned electionBackoffMaxMs. This is an issue as multiple replicas can get stuck starting elections at the same cadence. This change fixes that by added a random jitter to the max election backoff. Reviewers: José Armando García Sancio , TaiJuWu , Yung --- .../apache/kafka/raft/KafkaRaftClient.java | 21 +++---- .../java/org/apache/kafka/raft/RaftUtil.java | 15 +++++ .../kafka/raft/KafkaRaftClientTest.java | 4 +- .../org/apache/kafka/raft/MockableRandom.java | 5 ++ .../org/apache/kafka/raft/RaftUtilTest.java | 62 +++++++++++++++++++ 5 files changed, 92 insertions(+), 15 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index d1b1d3477c..2bf2036909 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -166,7 +166,8 @@ */ @SuppressWarnings({ "ClassDataAbstractionCoupling", "ClassFanOutComplexity", "ParameterNumber", "NPathComplexity" }) public final class KafkaRaftClient implements RaftClient { - private static final int RETRY_BACKOFF_BASE_MS = 100; + // visible for testing + static final int RETRY_BACKOFF_BASE_MS = 50; private static final int MAX_NUMBER_OF_BATCHES = 10; public static final int MAX_FETCH_WAIT_MS = 500; public static final int MAX_BATCH_SIZE_BYTES = 8 * 1024 * 1024; @@ -1027,7 +1028,12 @@ private void maybeHandleElectionLoss(NomineeState state, long currentTimeMs) { // replica has failed multiple elections in succession. candidate.startBackingOff( currentTimeMs, - binaryExponentialElectionBackoffMs(candidate.retries()) + RaftUtil.binaryExponentialElectionBackoffMs( + quorumConfig.electionBackoffMaxMs(), + RETRY_BACKOFF_BASE_MS, + candidate.retries(), + random + ) ); } } else if (state instanceof ProspectiveState prospective) { @@ -1045,17 +1051,6 @@ private void maybeHandleElectionLoss(NomineeState state, long currentTimeMs) { } } - private int binaryExponentialElectionBackoffMs(int retries) { - if (retries <= 0) { - throw new IllegalArgumentException("Retries " + retries + " should be larger than zero"); - } - // upper limit exponential co-efficients at 20 to avoid overflow - return Math.min( - RETRY_BACKOFF_BASE_MS * random.nextInt(2 << Math.min(20, retries - 1)), - quorumConfig.electionBackoffMaxMs() - ); - } - private int strictExponentialElectionBackoffMs(int positionInSuccessors, int totalNumSuccessors) { if (positionInSuccessors == 0) { return 0; diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java b/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java index 12c48955b3..fea9846aa1 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java @@ -48,6 +48,7 @@ import java.util.Collections; import java.util.List; import java.util.Optional; +import java.util.Random; import java.util.function.Consumer; import java.util.function.UnaryOperator; import java.util.stream.Collectors; @@ -765,4 +766,18 @@ static boolean hasValidTopicPartition(DescribeQuorumRequestData data, TopicParti data.topics().get(0).partitions().size() == 1 && data.topics().get(0).partitions().get(0).partitionIndex() == topicPartition.partition(); } + + static int binaryExponentialElectionBackoffMs(int backoffMaxMs, int backoffBaseMs, int retries, Random random) { + if (retries <= 0) { + throw new IllegalArgumentException("Retries " + retries + " should be larger than zero"); + } + // Takes minimum of the following: + // 1. exponential backoff calculation (maxes out at 102.4 seconds) + // 2. configurable electionBackoffMaxMs + jitter + // The jitter is added to prevent livelock of elections. + return Math.min( + backoffBaseMs * random.nextInt(1, 2 << Math.min(10, retries - 1)), + backoffMaxMs + random.nextInt(backoffBaseMs) + ); + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 79c255efbc..b1205bb1cd 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -1817,7 +1817,7 @@ public void testCandidateBackoffElection(boolean withKip853Rpc) throws Exception context.client.poll(); assertTrue(candidate.isBackingOff()); assertEquals( - context.electionBackoffMaxMs, + context.electionBackoffMaxMs + exponentialFactor, candidate.remainingBackoffMs(context.time.milliseconds()) ); @@ -1826,7 +1826,7 @@ public void testCandidateBackoffElection(boolean withKip853Rpc) throws Exception // Even though candidacy was rejected, local replica will backoff for jitter period // before transitioning to prospective and starting a new election. - context.time.sleep(context.electionBackoffMaxMs - 1); + context.time.sleep(context.electionBackoffMaxMs + exponentialFactor - 1); context.client.poll(); context.assertVotedCandidate(epoch, localId); diff --git a/raft/src/test/java/org/apache/kafka/raft/MockableRandom.java b/raft/src/test/java/org/apache/kafka/raft/MockableRandom.java index b487b16067..45cfd568d8 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockableRandom.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockableRandom.java @@ -48,4 +48,9 @@ public void mockNextInt(int returnValue) { public int nextInt(int bound) { return nextIntFunction.apply(bound).orElse(super.nextInt(bound)); } + + @Override + public int nextInt(int origin, int bound) { + return nextIntFunction.apply(bound).orElse(super.nextInt(bound)); + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftUtilTest.java b/raft/src/test/java/org/apache/kafka/raft/RaftUtilTest.java index 681bcaae8d..81485adca6 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftUtilTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftUtilTest.java @@ -58,16 +58,24 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; import java.net.InetSocketAddress; import java.nio.ByteBuffer; import java.util.Collections; +import java.util.List; +import java.util.Random; import java.util.stream.Stream; import static java.util.Collections.singletonList; import static java.util.Collections.singletonMap; +import static org.apache.kafka.raft.KafkaRaftClient.RETRY_BACKOFF_BASE_MS; +import static org.apache.kafka.raft.RaftUtil.binaryExponentialElectionBackoffMs; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; public class RaftUtilTest { @@ -569,6 +577,60 @@ public void testSingletonDescribeQuorumResponseForAllVersion(final short version assertEquals(expectedJson, json.toString()); } + @ParameterizedTest + @ValueSource(ints = {1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13}) + public void testExponentialBoundOfExponentialElectionBackoffMs(int retries) { + Random mockedRandom = Mockito.mock(Random.class); + int electionBackoffMaxMs = 1000; + + // test the bound of the method's first call to random.nextInt + binaryExponentialElectionBackoffMs(electionBackoffMaxMs, RETRY_BACKOFF_BASE_MS, retries, mockedRandom); + ArgumentCaptor nextIntCaptor = ArgumentCaptor.forClass(Integer.class); + Mockito.verify(mockedRandom).nextInt(Mockito.eq(1), nextIntCaptor.capture()); + int actualBound = nextIntCaptor.getValue(); + int expectedBound = (int) (2 * Math.pow(2, retries - 1)); + // after the 10th retry, the bound of the first call to random.nextInt will remain capped to + // (RETRY_BACKOFF_BASE_MS * 2 << 10)=2048 to prevent overflow + if (retries > 10) { + expectedBound = 2048; + } + assertEquals(expectedBound, actualBound, "Incorrect bound for retries=" + retries); + } + + // test that the return value of the method is capped to QUORUM_ELECTION_BACKOFF_MAX_MS_CONFIG + jitter + // any exponential >= (1000 + jitter)/(RETRY_BACKOFF_BASE_MS)=21 will result in this cap + @ParameterizedTest + @ValueSource(ints = {1, 2, 20, 21, 22, 2048}) + public void testExponentialElectionBackoffMsIsCapped(int exponential) { + Random mockedRandom = Mockito.mock(Random.class); + int electionBackoffMaxMs = 1000; + // this is the max bound of the method's first call to random.nextInt + int firstNextIntMaxBound = 2048; + + int jitterMs = 50; + Mockito.when(mockedRandom.nextInt(1, firstNextIntMaxBound)).thenReturn(exponential); + Mockito.when(mockedRandom.nextInt(RETRY_BACKOFF_BASE_MS)).thenReturn(jitterMs); + + int returnedBackoffMs = binaryExponentialElectionBackoffMs(electionBackoffMaxMs, RETRY_BACKOFF_BASE_MS, 11, mockedRandom); + + // verify nextInt was called on both expected bounds + ArgumentCaptor nextIntCaptor = ArgumentCaptor.forClass(Integer.class); + Mockito.verify(mockedRandom).nextInt(Mockito.eq(1), nextIntCaptor.capture()); + Mockito.verify(mockedRandom).nextInt(nextIntCaptor.capture()); + List allCapturedBounds = nextIntCaptor.getAllValues(); + assertEquals(firstNextIntMaxBound, allCapturedBounds.get(0)); + assertEquals(RETRY_BACKOFF_BASE_MS, allCapturedBounds.get(1)); + + // finally verify the backoff returned is capped to electionBackoffMaxMs + jitterMs + int backoffValueCap = electionBackoffMaxMs + jitterMs; + if (exponential < 21) { + assertEquals(RETRY_BACKOFF_BASE_MS * exponential, returnedBackoffMs); + assertTrue(returnedBackoffMs < backoffValueCap); + } else { + assertEquals(backoffValueCap, returnedBackoffMs); + } + } + private Records createRecords() { ByteBuffer allocate = ByteBuffer.allocate(1024); From e9c5069be78cb00d2af4f45c803907660b157726 Mon Sep 17 00:00:00 2001 From: Dongnuo Lyu <139248811+dongnuo123@users.noreply.github.com> Date: Tue, 27 May 2025 05:25:57 -0400 Subject: [PATCH 35/72] KAFKA-18687: Setting the subscriptionMetadata during conversion to consumer group (#19790) When a consumer protocol static member replaces an existing member in a classic group, it's not necessary to recompute the assignment. However, it happens anyway. In [ConsumerGroup.fromClassicGroup](https://github.com/apache/kafka/blob/0ff4dafb7de4e24ddb7961d52e50e728f2eee4eb/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java#L1140), we don't set the group's subscriptionMetadata. Later in the consumer group heartbeat, we [call updateSubscriptionMetadata](https://github.com/apache/kafka/blob/0ff4dafb7de4e24ddb7961d52e50e728f2eee4eb/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java#L1748), which [notices that the group's subscriptionMetadata needs an update](https://github.com/apache/kafka/blob/0ff4dafb7de4e24ddb7961d52e50e728f2eee4eb/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java#L2757) and bumps the epoch. Since the epoch is bumped, we [recompute the assignment](https://github.com/apache/kafka/blob/0ff4dafb7de4e24ddb7961d52e50e728f2eee4eb/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java#L1766). As a fix, this patch sets the subscriptionMetadata in ConsumerGroup.fromClassicGroup. Reviewers: Sean Quah , David Jacot --- .../group/GroupMetadataManager.java | 3 +- .../group/modern/consumer/ConsumerGroup.java | 15 ++++- .../group/GroupMetadataManagerTest.java | 66 ++++++++----------- .../modern/consumer/ConsumerGroupTest.java | 8 ++- 4 files changed, 49 insertions(+), 43 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index d53586efe4..d228c7538d 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java @@ -1115,7 +1115,8 @@ ConsumerGroup convertToConsumerGroup(ClassicGroup classicGroup, List diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index 6375038c66..cb4d93d5d1 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java @@ -9815,6 +9815,10 @@ memberId2, new MemberAssignmentImpl(mkAssignment( // Create the new consumer group with member 1. GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember1), + GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of( + fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 1), + barTopicName, new TopicMetadata(barTopicId, barTopicName, 1) + )), GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 0), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, expectedMember1.assignedPartitions()), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 0), @@ -9823,12 +9827,6 @@ memberId2, new MemberAssignmentImpl(mkAssignment( // Member 2 joins the new consumer group. GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember2), - // The subscription metadata hasn't been updated during the conversion, so a new one is computed. - GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of( - fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 1), - barTopicName, new TopicMetadata(barTopicId, barTopicName, 1) - )), - // Newly joining member 2 bumps the group epoch. A new target assignment is computed. GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 1), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId2, assignor.targetPartitions(memberId2)), @@ -10030,6 +10028,11 @@ memberId3, new MemberAssignmentImpl(mkAssignment( GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember1), GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember2), + GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of( + fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 2), + barTopicName, new TopicMetadata(barTopicId, barTopicName, 1) + )), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 0), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, expectedMember1.assignedPartitions()), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId2, expectedMember2.assignedPartitions()), @@ -10042,12 +10045,6 @@ memberId3, new MemberAssignmentImpl(mkAssignment( // Member 3 joins the new consumer group. GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember3), - // The subscription metadata hasn't been updated during the conversion, so a new one is computed. - GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of( - fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 2), - barTopicName, new TopicMetadata(barTopicId, barTopicName, 1) - )), - // Newly joining member 3 bumps the group epoch. A new target assignment is computed. GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 1), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, assignor.targetPartitions(memberId1)), @@ -10235,7 +10232,7 @@ public void testConsumerGroupHeartbeatToClassicGroupFromExistingStaticMember() { ); group.transitionTo(PREPARING_REBALANCE); - group.transitionTo(COMPLETING_REBALANCE); + group.initNextGeneration(); group.transitionTo(STABLE); context.replay(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, assignments)); @@ -10257,8 +10254,8 @@ public void testConsumerGroupHeartbeatToClassicGroupFromExistingStaticMember() { ConsumerGroupMember expectedClassicMember = new ConsumerGroupMember.Builder(memberId) .setInstanceId(instanceId) - .setMemberEpoch(0) - .setPreviousMemberEpoch(0) + .setMemberEpoch(group.generationId()) + .setPreviousMemberEpoch(group.generationId()) .setClientId(DEFAULT_CLIENT_ID) .setClientHost(DEFAULT_CLIENT_ADDRESS.toString()) .setSubscribedTopicNames(List.of(fooTopicName)) @@ -10294,7 +10291,7 @@ public void testConsumerGroupHeartbeatToClassicGroupFromExistingStaticMember() { .build(); ConsumerGroupMember expectedFinalConsumerMember = new ConsumerGroupMember.Builder(expectedReplacingConsumerMember) - .setMemberEpoch(1) + .setMemberEpoch(group.generationId()) .setServerAssignorName(NoOpPartitionAssignor.NAME) .setRebalanceTimeoutMs(5000) .setClassicMemberMetadata(null) @@ -10306,9 +10303,10 @@ public void testConsumerGroupHeartbeatToClassicGroupFromExistingStaticMember() { // Create the new consumer group with the static member. GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedClassicMember), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 0), + GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of(fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 1))), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, group.generationId()), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId, expectedClassicMember.assignedPartitions()), - GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 0), + GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, group.generationId()), GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedClassicMember), // Remove the static member because the rejoining member replaces it. @@ -10321,17 +10319,10 @@ public void testConsumerGroupHeartbeatToClassicGroupFromExistingStaticMember() { GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, newMemberId, mkAssignment(mkTopicAssignment(fooTopicId, 0))), GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedReplacingConsumerMember), - // The static member rejoins the new consumer group. + // The static member rejoins the new consumer group with the same instance id and + // takes the assignment of the previous member. No new target assignment is computed. GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedFinalConsumerMember), - // The subscription metadata hasn't been updated during the conversion, so a new one is computed. - GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of(fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 1))), - - // Newly joining static member bumps the group epoch. A new target assignment is computed. - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 1), - GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, newMemberId, mkAssignment(mkTopicAssignment(fooTopicId, 0))), - GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 1), - // The newly created static member takes the assignment from the existing member. // Bump its member epoch and transition to STABLE. GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedFinalConsumerMember) @@ -10432,6 +10423,10 @@ public void testConsumerGroupHeartbeatToClassicGroupWithEmptyAssignmentMember() // Create the new consumer group with member 1. GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember1), + GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of( + fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 1), + barTopicName, new TopicMetadata(barTopicId, barTopicName, 1) + )), GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 1), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, expectedMember1.assignedPartitions()), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 1), @@ -10440,12 +10435,6 @@ public void testConsumerGroupHeartbeatToClassicGroupWithEmptyAssignmentMember() // Member 2 joins the new consumer group. GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember2), - // The subscription metadata hasn't been updated during the conversion, so a new one is computed. - GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of( - fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 1), - barTopicName, new TopicMetadata(barTopicId, barTopicName, 1) - )), - // Newly joining member 2 bumps the group epoch. A new target assignment is computed. GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 2), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId2, Collections.emptyMap()), @@ -10817,6 +10806,11 @@ memberId3, new MemberAssignmentImpl(mkAssignment( GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember1), GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember2), + GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of( + fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 2), + barTopicName, new TopicMetadata(barTopicId, barTopicName, 1) + )), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 1), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, expectedMember1.assignedPartitions()), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId2, expectedMember2.assignedPartitions()), @@ -10829,12 +10823,6 @@ memberId3, new MemberAssignmentImpl(mkAssignment( // Member 3 joins the new consumer group. GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember3), - // The subscription metadata hasn't been updated during the conversion, so a new one is computed. - GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of( - fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 2), - barTopicName, new TopicMetadata(barTopicId, barTopicName, 1) - )), - // Newly joining member 3 bumps the group epoch. A new target assignment is computed. GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 2), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, assignor.targetPartitions(memberId1)), diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupTest.java index 886820e551..8153a84d39 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupTest.java @@ -1533,7 +1533,8 @@ public void testFromClassicGroup() { new SnapshotRegistry(logContext), mock(GroupCoordinatorMetricsShard.class), classicGroup, - metadataImage.topics() + metadataImage.topics(), + metadataImage.cluster() ); ConsumerGroup expectedConsumerGroup = new ConsumerGroup( @@ -1546,6 +1547,10 @@ public void testFromClassicGroup() { expectedConsumerGroup.updateTargetAssignment(memberId, new Assignment(mkAssignment( mkTopicAssignment(fooTopicId, 0) ))); + expectedConsumerGroup.setSubscriptionMetadata(Map.of( + fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 1), + barTopicName, new TopicMetadata(barTopicId, barTopicName, 1) + )); expectedConsumerGroup.updateMember(new ConsumerGroupMember.Builder(memberId) .setMemberEpoch(classicGroup.generationId()) .setState(MemberState.STABLE) @@ -1577,6 +1582,7 @@ public void testFromClassicGroup() { assertEquals(expectedConsumerGroup.groupEpoch(), consumerGroup.groupEpoch()); assertEquals(expectedConsumerGroup.state(), consumerGroup.state()); assertEquals(expectedConsumerGroup.preferredServerAssignor(), consumerGroup.preferredServerAssignor()); + assertEquals(Map.copyOf(expectedConsumerGroup.subscriptionMetadata()), Map.copyOf(consumerGroup.subscriptionMetadata())); assertEquals(expectedConsumerGroup.members(), consumerGroup.members()); } From ded76530667c66f192c32d61a5366903dd5e881d Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Tue, 27 May 2025 15:37:29 -0700 Subject: [PATCH 36/72] MINOR: Fix some Request toString methods (#19655) (#19689) Reviewers: Colin P. McCabe ``` Conflicts: clients/src/main/java/org/apache/kafka/common/requests/AlterUserScramCredentialsRequest.java - import statement clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsRequest.java - import statement core/src/test/scala/unit/kafka/server/KafkaApisTest.scala - different logging and metadatacache instantiation ``` Cherry-Picked-From: 042be5b9ac3f4be83069b18f17fda382e07d6539 Cherry-Picked-By: Alyssa Huang Cherry-Picked-At: Mon May 12 11:01:47 2025 -0700 --- checkstyle/import-control.xml | 2 ++ .../common/requests/AbstractRequest.java | 2 +- .../AlterUserScramCredentialsRequest.java | 16 +++++++++++++++ .../IncrementalAlterConfigsRequest.java | 16 +++++++++++++++ .../kafka/network/RequestChannelTest.scala | 16 ++++++++++++--- ...AlterUserScramCredentialsRequestTest.scala | 14 +++++++++++-- .../unit/kafka/server/KafkaApisTest.scala | 20 ++++++++++++++++--- 7 files changed, 77 insertions(+), 9 deletions(-) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index dc674ab997..ab6ebff743 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -194,6 +194,8 @@ + + diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java index 38da78efb1..4ae02c5145 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -136,7 +136,7 @@ public String toString(boolean verbose) { } @Override - public final String toString() { + public String toString() { return toString(true); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterUserScramCredentialsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterUserScramCredentialsRequest.java index 1ca7ea77aa..c779f6d9a8 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterUserScramCredentialsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterUserScramCredentialsRequest.java @@ -17,10 +17,14 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData; +import org.apache.kafka.common.message.AlterUserScramCredentialsRequestDataJsonConverter; import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ByteBufferAccessor; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; + import java.nio.ByteBuffer; import java.util.List; import java.util.Set; @@ -82,4 +86,16 @@ public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { .collect(Collectors.toList()); return new AlterUserScramCredentialsResponse(new AlterUserScramCredentialsResponseData().setResults(results)); } + + // Do not print salt or saltedPassword + @Override + public String toString() { + JsonNode json = AlterUserScramCredentialsRequestDataJsonConverter.write(data, version()).deepCopy(); + + for (JsonNode upsertion : json.get("upsertions")) { + ((ObjectNode) upsertion).put("salt", ""); + ((ObjectNode) upsertion).put("saltedPassword", ""); + } + return AlterUserScramCredentialsRequestDataJsonConverter.read(json, version()).toString(); + } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsRequest.java index 222097502b..5454083775 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsRequest.java @@ -21,11 +21,15 @@ import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData; import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData.AlterConfigsResource; +import org.apache.kafka.common.message.IncrementalAlterConfigsRequestDataJsonConverter; import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData; import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData.AlterConfigsResourceResponse; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ByteBufferAccessor; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; + import java.nio.ByteBuffer; import java.util.Collection; import java.util.Map; @@ -107,4 +111,16 @@ public AbstractResponse getErrorResponse(final int throttleTimeMs, final Throwab } return new IncrementalAlterConfigsResponse(response); } + + // It is not safe to print all config values + @Override + public String toString() { + JsonNode json = IncrementalAlterConfigsRequestDataJsonConverter.write(data, version()).deepCopy(); + for (JsonNode resource : json.get("resources")) { + for (JsonNode config : resource.get("configs")) { + ((ObjectNode) config).put("value", "REDACTED"); + } + } + return IncrementalAlterConfigsRequestDataJsonConverter.read(json, version()).toString(); + } } diff --git a/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala b/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala index 1ab0f0ae8e..ecea412e98 100644 --- a/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala @@ -131,11 +131,21 @@ class RequestChannelTest { op: OpType, entries: Map[String, String], expectedValues: Map[String, String]): Unit = { - val alterConfigs = request(incrementalAlterConfigs(resource, entries, op)) - val loggableAlterConfigs = alterConfigs.loggableRequest.asInstanceOf[IncrementalAlterConfigsRequest] + val alterConfigs = incrementalAlterConfigs(resource, entries, op) + val alterConfigsString = alterConfigs.toString + entries.foreach { entry => + if (!alterConfigsString.contains(entry._1)) { + fail("Config names should be in the request string") + } + if (entry._2 != null && alterConfigsString.contains(entry._2)) { + fail("Config values should not be in the request string") + } + } + val req = request(alterConfigs) + val loggableAlterConfigs = req.loggableRequest.asInstanceOf[IncrementalAlterConfigsRequest] val loggedConfig = loggableAlterConfigs.data.resources.find(resource.`type`.id, resource.name).configs assertEquals(expectedValues, toMap(loggedConfig)) - val alterConfigsDesc = RequestConvertToJson.requestDesc(alterConfigs.header, alterConfigs.requestLog.toJava, alterConfigs.isForwarded).toString + val alterConfigsDesc = RequestConvertToJson.requestDesc(req.header, req.requestLog.toJava, req.isForwarded).toString assertFalse(alterConfigsDesc.contains(sensitiveValue), s"Sensitive config logged $alterConfigsDesc") } diff --git a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala index ced7887351..4ebe65ec9a 100644 --- a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala @@ -244,6 +244,9 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest { // create a bunch of credentials val request1_0 = new AlterUserScramCredentialsRequest.Builder( new AlterUserScramCredentialsRequestData() + .setDeletions(util.Arrays.asList( + new AlterUserScramCredentialsRequestData.ScramCredentialDeletion() + .setName(user2).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`))) .setUpsertions(util.Arrays.asList( new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion() .setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`) @@ -251,10 +254,15 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest { .setSalt(saltBytes) .setSaltedPassword(saltedPasswordBytes), ))).build() + assertEquals("AlterUserScramCredentialsRequestData(" + + "deletions=[ScramCredentialDeletion(name='" + user2 + "', mechanism=" + ScramMechanism.SCRAM_SHA_256.`type` + ")], " + + "upsertions=[ScramCredentialUpsertion(name='" + user1 + "', mechanism=" + ScramMechanism.SCRAM_SHA_256.`type` + + ", iterations=4096, salt=[], saltedPassword=[])])", request1_0.toString) val results1_0 = sendAlterUserScramCredentialsRequest(request1_0).data.results - assertEquals(1, results1_0.size) - checkNoErrorsAlteringCredentials(results1_0) + assertEquals(2, results1_0.size) + assertEquals(1, results1_0.asScala.count(_.errorCode == Errors.RESOURCE_NOT_FOUND.code())) checkUserAppearsInAlterResults(results1_0, user1) + checkUserAppearsInAlterResults(results1_0, user2) // When creating credentials, do not update the same user more than once per request val request1_1 = new AlterUserScramCredentialsRequest.Builder( @@ -276,6 +284,8 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest { .setSalt(saltBytes) .setSaltedPassword(saltedPasswordBytes), ))).build() + assertFalse(request1_1.toString.contains(saltBytes)) + assertFalse(request1_1.toString.contains(saltedPasswordBytes)) val results1_1 = sendAlterUserScramCredentialsRequest(request1_1).data.results assertEquals(3, results1_1.size) checkNoErrorsAlteringCredentials(results1_1) diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 20a030714f..bb1f4105ba 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -9710,7 +9710,12 @@ class KafkaApisTest extends Logging { @Test def testEmptyIncrementalAlterConfigsRequestWithKRaft(): Unit = { - val request = buildRequest(new IncrementalAlterConfigsRequest(new IncrementalAlterConfigsRequestData(), 1.toShort)) + val alterConfigsRequest = new IncrementalAlterConfigsRequest(new IncrementalAlterConfigsRequestData(), 1.toShort) + assertEquals( + "IncrementalAlterConfigsRequestData(resources=[], validateOnly=false)", + alterConfigsRequest.toString + ) + val request = buildRequest(alterConfigsRequest) metadataCache = MetadataCache.kRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) when(clientRequestQuotaManager.maybeRecordAndGetThrottleTimeMs(any[RequestChannel.Request](), any[Long])).thenReturn(0) @@ -9722,7 +9727,7 @@ class KafkaApisTest extends Logging { @Test def testLog4jIncrementalAlterConfigsRequestWithKRaft(): Unit = { - val request = buildRequest(new IncrementalAlterConfigsRequest(new IncrementalAlterConfigsRequestData(). + val alterConfigsRequest = new IncrementalAlterConfigsRequest(new IncrementalAlterConfigsRequestData(). setValidateOnly(true). setResources(new IAlterConfigsResourceCollection(asList(new IAlterConfigsResource(). setResourceName(brokerId.toString). @@ -9730,7 +9735,16 @@ class KafkaApisTest extends Logging { setConfigs(new IAlterableConfigCollection(asList(new IAlterableConfig(). setName(Log4jController.ROOT_LOGGER). setValue("TRACE")).iterator()))).iterator())), - 1.toShort)) + 1.toShort) + assertEquals( + "IncrementalAlterConfigsRequestData(resources=[" + + "AlterConfigsResource(resourceType=" + BROKER_LOGGER.id() + ", " + + "resourceName='"+ brokerId + "', " + + "configs=[AlterableConfig(name='" + Log4jController.ROOT_LOGGER + "', configOperation=0, value='REDACTED')])], " + + "validateOnly=true)", + alterConfigsRequest.toString + ) + val request = buildRequest(alterConfigsRequest) metadataCache = MetadataCache.kRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) when(clientRequestQuotaManager.maybeRecordAndGetThrottleTimeMs(any[RequestChannel.Request](), any[Long])).thenReturn(0) From 1cc14f6343462716c0add9e358ca99cba74ec37a Mon Sep 17 00:00:00 2001 From: Okada Haruki Date: Mon, 9 Jun 2025 13:24:26 +0900 Subject: [PATCH 37/72] KAFKA-19334 MetadataShell execution unintentionally deletes lock file (#19817) ## Summary - MetadataShell may deletes lock file unintentionally when it exists or fails to acquire lock. If there's running server, this causes unexpected result as below: * MetadataShell succeeds on 2nd run unexpectedly * Even worse, LogManager/RaftManager's lock also no longer work from concurrent Kafka process startup Reviewers: TengYao Chi # Conflicts: # shell/src/test/java/org/apache/kafka/shell/MetadataShellIntegrationTest.java --- .../apache/kafka/server/util/FileLock.java | 8 ++++++++ .../org/apache/kafka/shell/MetadataShell.java | 6 +++--- .../shell/MetadataShellIntegrationTest.java | 20 +++++++++++-------- 3 files changed, 23 insertions(+), 11 deletions(-) diff --git a/server-common/src/main/java/org/apache/kafka/server/util/FileLock.java b/server-common/src/main/java/org/apache/kafka/server/util/FileLock.java index 4f55b4aebc..b06f239183 100644 --- a/server-common/src/main/java/org/apache/kafka/server/util/FileLock.java +++ b/server-common/src/main/java/org/apache/kafka/server/util/FileLock.java @@ -91,4 +91,12 @@ public synchronized void destroy() throws IOException { } channel.close(); } + + /** + * Unlock the file and close the associated FileChannel + */ + public synchronized void unlockAndClose() throws IOException { + unlock(); + channel.close(); + } } diff --git a/shell/src/main/java/org/apache/kafka/shell/MetadataShell.java b/shell/src/main/java/org/apache/kafka/shell/MetadataShell.java index 0242e34983..5600aa5e5e 100644 --- a/shell/src/main/java/org/apache/kafka/shell/MetadataShell.java +++ b/shell/src/main/java/org/apache/kafka/shell/MetadataShell.java @@ -128,7 +128,7 @@ static FileLock takeDirectoryLock(File directory) throws IOException { "directory before proceeding."); } } catch (Throwable e) { - fileLock.destroy(); + fileLock.unlockAndClose(); throw e; } return fileLock; @@ -232,9 +232,9 @@ public void close() { Utils.closeQuietly(snapshotFileReader, "raftManager"); if (fileLock != null) { try { - fileLock.destroy(); + fileLock.unlockAndClose(); } catch (Exception e) { - log.error("Error destroying fileLock", e); + log.error("Error cleaning up fileLock", e); } finally { fileLock = null; } diff --git a/shell/src/test/java/org/apache/kafka/shell/MetadataShellIntegrationTest.java b/shell/src/test/java/org/apache/kafka/shell/MetadataShellIntegrationTest.java index 970075d959..a81642d745 100644 --- a/shell/src/test/java/org/apache/kafka/shell/MetadataShellIntegrationTest.java +++ b/shell/src/test/java/org/apache/kafka/shell/MetadataShellIntegrationTest.java @@ -33,7 +33,7 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.NoSuchFileException; -import java.util.Collections; +import java.util.List; import java.util.concurrent.ExecutionException; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -91,18 +91,22 @@ public void testLock(boolean canLock) throws Exception { if (canLock) { assertEquals(NoSuchFileException.class, assertThrows(ExecutionException.class, - () -> env.shell.run(Collections.emptyList())). + () -> env.shell.run(List.of())). getCause().getClass()); } else { FileLock fileLock = new FileLock(new File(env.tempDir, ".lock")); try { fileLock.lock(); - assertEquals("Unable to lock " + env.tempDir.getAbsolutePath() + - ". Please ensure that no broker or controller process is using this " + - "directory before proceeding.", - assertThrows(RuntimeException.class, - () -> env.shell.run(Collections.emptyList())). - getMessage()); + // We had a bug where the shell can lock the directory unintentionally + // at the 2nd run, so we check that it fails (See KAFKA-19334) + for (int i = 0; i < 2; i++) { + assertEquals("Unable to lock " + env.tempDir.getAbsolutePath() + + ". Please ensure that no broker or controller process is using this " + + "directory before proceeding.", + assertThrows(RuntimeException.class, + () -> env.shell.run(List.of())). + getMessage()); + } } finally { fileLock.destroy(); } From 00a1b1e8ceb1653bbd725c93cabc836df60657c0 Mon Sep 17 00:00:00 2001 From: Luke Chen Date: Wed, 11 Jun 2025 15:27:22 +0800 Subject: [PATCH 38/72] Bump the commons-beanutils for CVE-2025-48734. Since `commons-validator` hasn't had new release with newer `commons-beanutils` versions, we manually bump it in kafka. Reviewers: Mickael Maison --- LICENSE-binary | 4 ++-- build.gradle | 2 ++ gradle/dependencies.gradle | 2 ++ 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index 380af15b5c..8ceda84ebe 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -206,11 +206,11 @@ This project bundles some components that are also licensed under the Apache License Version 2.0: - caffeine-3.1.1 -- commons-beanutils-1.9.4 +- commons-beanutils-1.11.0 - commons-collections-3.2.2 - commons-digester-2.1 - commons-lang3-3.12.0 -- commons-logging-1.3.2 +- commons-logging-1.3.5 - commons-validator-1.9.0 - jackson-annotations-2.16.2 - jackson-core-2.16.2 diff --git a/build.gradle b/build.gradle index 243f181c0b..05d38aface 100644 --- a/build.gradle +++ b/build.gradle @@ -197,6 +197,8 @@ allprojects { // ensure we have a single version in the classpath despite transitive dependencies libs.scalaLibrary, libs.scalaReflect, + // Workaround before `commons-validator` has new release. See KAFKA-19359. + libs.commonsBeanutils, libs.jacksonAnnotations ) } diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 66eca369aa..5debdea5a0 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -60,6 +60,7 @@ versions += [ caffeine: "3.1.1", bndlib: "7.0.0", checkstyle: project.hasProperty('checkstyleVersion') ? checkstyleVersion : "10.20.2", + commonsBeanutils: "1.11.0", commonsValidator: "1.9.0", classgraph: "4.8.173", gradle: "8.10.2", @@ -148,6 +149,7 @@ libs += [ bndlib:"biz.aQute.bnd:biz.aQute.bndlib:$versions.bndlib", caffeine: "com.github.ben-manes.caffeine:caffeine:$versions.caffeine", classgraph: "io.github.classgraph:classgraph:$versions.classgraph", + commonsBeanutils: "commons-beanutils:commons-beanutils:$versions.commonsBeanutils", commonsValidator: "commons-validator:commons-validator:$versions.commonsValidator", jacksonAnnotations: "com.fasterxml.jackson.core:jackson-annotations:$versions.jackson", jacksonDatabind: "com.fasterxml.jackson.core:jackson-databind:$versions.jackson", From 254c1fa519b37bddc2488e80979406129fd56808 Mon Sep 17 00:00:00 2001 From: Apoorv Mittal Date: Thu, 12 Jun 2025 22:52:50 +0100 Subject: [PATCH 39/72] MINOR: Fixing client telemetry validate request (#19959) Minor fix to correct the validate condition for GetTelemetryRequests. Added respective tests as well. Reviewers: Andrew Schofield --- .../kafka/server/ClientMetricsManager.java | 2 +- .../server/ClientMetricsManagerTest.java | 110 +++++++++++++++++- 2 files changed, 110 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/apache/kafka/server/ClientMetricsManager.java b/server/src/main/java/org/apache/kafka/server/ClientMetricsManager.java index ba26c9b4da..dcd17a3ecc 100644 --- a/server/src/main/java/org/apache/kafka/server/ClientMetricsManager.java +++ b/server/src/main/java/org/apache/kafka/server/ClientMetricsManager.java @@ -392,7 +392,7 @@ private void validateGetRequest(GetTelemetrySubscriptionsRequest request, ClientMetricsInstance clientInstance, long timestamp) { if (!clientInstance.maybeUpdateGetRequestTimestamp(timestamp) && (clientInstance.lastKnownError() != Errors.UNKNOWN_SUBSCRIPTION_ID - || clientInstance.lastKnownError() != Errors.UNSUPPORTED_COMPRESSION_TYPE)) { + && clientInstance.lastKnownError() != Errors.UNSUPPORTED_COMPRESSION_TYPE)) { clientMetricsStats.recordThrottleCount(clientInstance.clientInstanceId()); String msg = String.format("Request from the client [%s] arrived before the next push interval time", request.data().clientInstanceId()); diff --git a/server/src/test/java/org/apache/kafka/server/ClientMetricsManagerTest.java b/server/src/test/java/org/apache/kafka/server/ClientMetricsManagerTest.java index 4cb4053505..ad6134a468 100644 --- a/server/src/test/java/org/apache/kafka/server/ClientMetricsManagerTest.java +++ b/server/src/test/java/org/apache/kafka/server/ClientMetricsManagerTest.java @@ -826,7 +826,6 @@ public void testPushTelemetrySubscriptionIdInvalid() throws Exception { assertEquals((double) 0, getMetric(ClientMetricsManager.ClientMetricsStats.PLUGIN_ERROR + "-count").metricValue()); assertEquals(Double.NaN, getMetric(ClientMetricsManager.ClientMetricsStats.PLUGIN_EXPORT_TIME + "-avg").metricValue()); assertEquals(Double.NaN, getMetric(ClientMetricsManager.ClientMetricsStats.PLUGIN_EXPORT_TIME + "-max").metricValue()); - } @Test @@ -1124,6 +1123,115 @@ public void testPushTelemetryPluginException() throws Exception { } } + @Test + public void testGetTelemetrySubscriptionAfterPushTelemetryUnknownSubscriptionSucceeds() throws Exception { + clientMetricsManager.updateSubscription("sub-1", ClientMetricsTestUtils.defaultTestProperties()); + assertEquals(1, clientMetricsManager.subscriptions().size()); + + GetTelemetrySubscriptionsRequest subscriptionsRequest = new GetTelemetrySubscriptionsRequest.Builder( + new GetTelemetrySubscriptionsRequestData(), true).build(); + + GetTelemetrySubscriptionsResponse subscriptionsResponse = clientMetricsManager.processGetTelemetrySubscriptionRequest( + subscriptionsRequest, ClientMetricsTestUtils.requestContext()); + + Properties properties = new Properties(); + properties.put("interval.ms", "100"); + clientMetricsManager.updateSubscription("sub-2", properties); + assertEquals(2, clientMetricsManager.subscriptions().size()); + + PushTelemetryRequest request = new Builder( + new PushTelemetryRequestData() + .setClientInstanceId(subscriptionsResponse.data().clientInstanceId()) + .setSubscriptionId(subscriptionsResponse.data().subscriptionId()) + .setCompressionType(CompressionType.NONE.id) + .setMetrics(ByteBuffer.wrap("test-data".getBytes(StandardCharsets.UTF_8))), true).build(); + + PushTelemetryResponse response = clientMetricsManager.processPushTelemetryRequest( + request, ClientMetricsTestUtils.requestContext()); + + assertEquals(Errors.UNKNOWN_SUBSCRIPTION_ID, response.error()); + ClientMetricsInstance instance = clientMetricsManager.clientInstance(subscriptionsResponse.data().clientInstanceId()); + assertNotNull(instance); + assertEquals(Errors.UNKNOWN_SUBSCRIPTION_ID, instance.lastKnownError()); + + subscriptionsRequest = new GetTelemetrySubscriptionsRequest.Builder( + new GetTelemetrySubscriptionsRequestData().setClientInstanceId(subscriptionsResponse.data().clientInstanceId()), true).build(); + subscriptionsResponse = clientMetricsManager.processGetTelemetrySubscriptionRequest( + subscriptionsRequest, ClientMetricsTestUtils.requestContext()); + assertEquals(Errors.NONE, subscriptionsResponse.error()); + } + + @Test + public void testGetTelemetrySubscriptionAfterPushTelemetryUnknownCompressionSucceeds() throws Exception { + clientMetricsManager.updateSubscription("sub-1", ClientMetricsTestUtils.defaultTestProperties()); + assertEquals(1, clientMetricsManager.subscriptions().size()); + + GetTelemetrySubscriptionsRequest subscriptionsRequest = new GetTelemetrySubscriptionsRequest.Builder( + new GetTelemetrySubscriptionsRequestData(), true).build(); + + GetTelemetrySubscriptionsResponse subscriptionsResponse = clientMetricsManager.processGetTelemetrySubscriptionRequest( + subscriptionsRequest, ClientMetricsTestUtils.requestContext()); + + PushTelemetryRequest request = new Builder( + new PushTelemetryRequestData() + .setClientInstanceId(subscriptionsResponse.data().clientInstanceId()) + .setSubscriptionId(subscriptionsResponse.data().subscriptionId()) + .setCompressionType((byte) 10) // // Invalid compression type + .setMetrics(ByteBuffer.wrap("test-data".getBytes(StandardCharsets.UTF_8))), true).build(); + + PushTelemetryResponse response = clientMetricsManager.processPushTelemetryRequest( + request, ClientMetricsTestUtils.requestContext()); + + assertEquals(Errors.UNSUPPORTED_COMPRESSION_TYPE, response.error()); + ClientMetricsInstance instance = clientMetricsManager.clientInstance(subscriptionsResponse.data().clientInstanceId()); + assertNotNull(instance); + assertEquals(Errors.UNSUPPORTED_COMPRESSION_TYPE, instance.lastKnownError()); + + subscriptionsRequest = new GetTelemetrySubscriptionsRequest.Builder( + new GetTelemetrySubscriptionsRequestData().setClientInstanceId(subscriptionsResponse.data().clientInstanceId()), true).build(); + subscriptionsResponse = clientMetricsManager.processGetTelemetrySubscriptionRequest( + subscriptionsRequest, ClientMetricsTestUtils.requestContext()); + assertEquals(Errors.NONE, subscriptionsResponse.error()); + } + + @Test + public void testGetTelemetrySubscriptionAfterPushTelemetryBytesExceptionFails() throws Exception { + try ( + Metrics kafkaMetrics = new Metrics(); + ClientMetricsManager clientMetricsManager = new ClientMetricsManager(clientMetricsReceiverPlugin, 1, time, kafkaMetrics) + ) { + GetTelemetrySubscriptionsRequest subscriptionsRequest = new GetTelemetrySubscriptionsRequest.Builder( + new GetTelemetrySubscriptionsRequestData(), true).build(); + + GetTelemetrySubscriptionsResponse subscriptionsResponse = clientMetricsManager.processGetTelemetrySubscriptionRequest( + subscriptionsRequest, ClientMetricsTestUtils.requestContext()); + + byte[] metrics = "ab".getBytes(StandardCharsets.UTF_8); + assertEquals(2, metrics.length); + + PushTelemetryRequest request = new PushTelemetryRequest.Builder( + new PushTelemetryRequestData() + .setClientInstanceId(subscriptionsResponse.data().clientInstanceId()) + .setSubscriptionId(subscriptionsResponse.data().subscriptionId()) + .setMetrics(ByteBuffer.wrap(metrics)), true).build(); + + // Set the max bytes 1 to force the error. + PushTelemetryResponse response = clientMetricsManager.processPushTelemetryRequest( + request, ClientMetricsTestUtils.requestContext()); + + assertEquals(Errors.TELEMETRY_TOO_LARGE, response.error()); + ClientMetricsInstance instance = clientMetricsManager.clientInstance(subscriptionsResponse.data().clientInstanceId()); + assertNotNull(instance); + assertEquals(Errors.TELEMETRY_TOO_LARGE, instance.lastKnownError()); + + subscriptionsRequest = new GetTelemetrySubscriptionsRequest.Builder( + new GetTelemetrySubscriptionsRequestData().setClientInstanceId(subscriptionsResponse.data().clientInstanceId()), true).build(); + subscriptionsResponse = clientMetricsManager.processGetTelemetrySubscriptionRequest( + subscriptionsRequest, ClientMetricsTestUtils.requestContext()); + assertEquals(Errors.THROTTLING_QUOTA_EXCEEDED, subscriptionsResponse.error()); + } + } + @Test public void testCacheEviction() throws Exception { Properties properties = new Properties(); From c6b44b5d66f05c5a1ff792e9bcb43b7d9cfff267 Mon Sep 17 00:00:00 2001 From: Ritika Reddy <98577846+rreddy-22@users.noreply.github.com> Date: Sat, 14 Jun 2025 11:40:00 -0700 Subject: [PATCH 40/72] Cherry Pick KAFKA-19367 to 4.0 (#19958) [https://github.com/apache/kafka/commit/0b2e410d61970e66c6f73a18c75028df0a871777](url) Bug fix in 4.0 **Conflicts:** - The Transaction Coordinator had some conflicts, mainly with the transaction states. Ex: ongoing in 4.0 is TransactionState.ONGOING in 4.1. - The TransactionCoordinatorTest file had conflicts w.r.t the 2PC changes from KIP-939 in 4.1 and the above mentioned state changes Reviewers: Justine Olshan , Artem Livshits --- .../transaction/TransactionCoordinator.scala | 6 +- .../TransactionCoordinatorTest.scala | 134 ++++++++++++++++++ 2 files changed, 136 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala index e0019f0d77..064de12a4a 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala @@ -802,11 +802,9 @@ class TransactionCoordinator(txnConfig: TransactionConfig, } if (nextState == PrepareAbort && isEpochFence) { - // We should clear the pending state to make way for the transition to PrepareAbort and also bump - // the epoch in the transaction metadata we are about to append. + // We should clear the pending state to make way for the transition to PrepareAbort txnMetadata.pendingState = None - txnMetadata.producerEpoch = producerEpoch - txnMetadata.lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH + // For TV2+, don't manually set the epoch - let prepareAbortOrCommit handle it naturally. } nextProducerIdOrErrors.flatMap { diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala index e5b48d9246..bbca105fd4 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala @@ -1165,6 +1165,140 @@ class TransactionCoordinatorTest { any()) } + @Test + def shouldNotCauseEpochOverflowWhenInitPidDuringOngoingTxnV2(): Unit = { + // When InitProducerId is called with an ongoing transaction at epoch 32766 (Short.MaxValue - 1), + // it should not cause an epoch overflow by incrementing twice. + // The only true increment happens in prepareAbortOrCommit + val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, + (Short.MaxValue - 1).toShort, (Short.MaxValue - 2).toShort, txnTimeoutMs, Ongoing, partitions, time.milliseconds(), time.milliseconds(), TV_2) + + when(transactionManager.validateTransactionTimeoutMs(anyInt())) + .thenReturn(true) + when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) + .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) + when(transactionManager.transactionVersionLevel()).thenReturn(TV_2) + + // Capture the transition metadata to verify epoch increments + val capturedTxnTransitMetadata: ArgumentCaptor[TxnTransitMetadata] = ArgumentCaptor.forClass(classOf[TxnTransitMetadata]) + when(transactionManager.appendTransactionToLog( + ArgumentMatchers.eq(transactionalId), + ArgumentMatchers.eq(coordinatorEpoch), + capturedTxnTransitMetadata.capture(), + capturedErrorsCallback.capture(), + any(), + any()) + ).thenAnswer(invocation => { + val transitMetadata = invocation.getArgument[TxnTransitMetadata](2) + // Simulate the metadata update that would happen in the real appendTransactionToLog + txnMetadata.completeTransitionTo(transitMetadata) + capturedErrorsCallback.getValue.apply(Errors.NONE) + }) + + // Handle InitProducerId with ongoing transaction at epoch 32766 + coordinator.handleInitProducerId( + transactionalId, + txnTimeoutMs, + None, + initProducerIdMockCallback + ) + + // Verify that the epoch did not overflow (should be Short.MaxValue = 32767, not negative) + assertEquals(Short.MaxValue, txnMetadata.producerEpoch) + assertEquals(PrepareAbort, txnMetadata.state) + + verify(transactionManager).validateTransactionTimeoutMs(anyInt()) + verify(transactionManager, times(3)).getTransactionState(ArgumentMatchers.eq(transactionalId)) + verify(transactionManager).appendTransactionToLog( + ArgumentMatchers.eq(transactionalId), + ArgumentMatchers.eq(coordinatorEpoch), + any[TxnTransitMetadata], + any(), + any(), + any()) + } + + @Test + def shouldHandleTimeoutAtEpochOverflowBoundaryCorrectlyTV2(): Unit = { + // Test the scenario where we have an ongoing transaction at epoch 32766 (Short.MaxValue - 1) + // and the producer crashes/times out. This test verifies that the timeout handling + // correctly manages the epoch overflow scenario without causing failures. + + val epochAtMaxBoundary = (Short.MaxValue - 1).toShort // 32766 + val now = time.milliseconds() + + // Create transaction metadata at the epoch boundary that would cause overflow IFF double-incremented + val txnMetadata = new TransactionMetadata( + transactionalId = transactionalId, + producerId = producerId, + previousProducerId = RecordBatch.NO_PRODUCER_ID, + nextProducerId = RecordBatch.NO_PRODUCER_ID, + producerEpoch = epochAtMaxBoundary, + lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, + txnTimeoutMs = txnTimeoutMs, + state = Ongoing, + topicPartitions = partitions, + txnStartTimestamp = now, + txnLastUpdateTimestamp = now, + clientTransactionVersion = TV_2 + ) + assertTrue(txnMetadata.isProducerEpochExhausted) + + // Mock the transaction manager to return our test transaction as timed out + when(transactionManager.timedOutTransactions()) + .thenReturn(List(TransactionalIdAndProducerIdEpoch(transactionalId, producerId, epochAtMaxBoundary))) + when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) + .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) + when(transactionManager.transactionVersionLevel()).thenReturn(TV_2) + + // Mock the append operation to simulate successful write and update the metadata + when(transactionManager.appendTransactionToLog( + ArgumentMatchers.eq(transactionalId), + ArgumentMatchers.eq(coordinatorEpoch), + any[TxnTransitMetadata], + capturedErrorsCallback.capture(), + any(), + any()) + ).thenAnswer(invocation => { + val transitMetadata = invocation.getArgument[TxnTransitMetadata](2) + // Simulate the metadata update that would happen in the real appendTransactionToLog + txnMetadata.completeTransitionTo(transitMetadata) + capturedErrorsCallback.getValue.apply(Errors.NONE) + }) + + // Track the actual behavior + var callbackInvoked = false + var resultError: Errors = null + var resultProducerId: Long = -1 + var resultEpoch: Short = -1 + + def checkOnEndTransactionComplete(txnIdAndPidEpoch: TransactionalIdAndProducerIdEpoch) + (error: Errors, newProducerId: Long, newProducerEpoch: Short): Unit = { + callbackInvoked = true + resultError = error + resultProducerId = newProducerId + resultEpoch = newProducerEpoch + } + + // Execute the timeout abort process + coordinator.abortTimedOutTransactions(checkOnEndTransactionComplete) + + assertTrue(callbackInvoked, "Callback should have been invoked") + assertEquals(Errors.NONE, resultError, "Expected no errors in the callback") + assertEquals(producerId, resultProducerId, "Expected producer ID to match") + assertEquals(Short.MaxValue, resultEpoch, "Expected producer epoch to be Short.MaxValue (32767) single epoch bump") + + // Verify the transaction metadata was correctly updated to the final epoch + assertEquals(Short.MaxValue, txnMetadata.producerEpoch, + s"Expected transaction metadata producer epoch to be ${Short.MaxValue} " + + s"after timeout handling, but was ${txnMetadata.producerEpoch}" + ) + + // Verify the basic flow was attempted + verify(transactionManager).timedOutTransactions() + verify(transactionManager, atLeast(1)).getTransactionState(ArgumentMatchers.eq(transactionalId)) + } + @Test def testInitProducerIdWithNoLastProducerData(): Unit = { // If the metadata doesn't include the previous producer data (for example, if it was written to the log by a broker From 26e5a53906ba04721b9f5435632d3c8b4a6a8ece Mon Sep 17 00:00:00 2001 From: Apoorv Mittal Date: Mon, 16 Jun 2025 10:11:18 +0100 Subject: [PATCH 41/72] HOTFIX: Correcting build after cherry-pick (#19969) Fixing build after cherrypicking: ``` commit 254c1fa519b37bddc2488e80979406129fd56808 Author: Apoorv Mittal Date: Thu Jun 12 22:52:50 2025 +0100 MINOR: Fixing client telemetry validate request (#19959) Minor fix to correct the validate condition for GetTelemetryRequests. Added respective tests as well. Reviewers: Andrew Schofield ``` Reviewers: Manikumar Reddy --- .../org/apache/kafka/server/ClientMetricsManagerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/org/apache/kafka/server/ClientMetricsManagerTest.java b/server/src/test/java/org/apache/kafka/server/ClientMetricsManagerTest.java index ad6134a468..55eed0c197 100644 --- a/server/src/test/java/org/apache/kafka/server/ClientMetricsManagerTest.java +++ b/server/src/test/java/org/apache/kafka/server/ClientMetricsManagerTest.java @@ -1125,7 +1125,7 @@ public void testPushTelemetryPluginException() throws Exception { @Test public void testGetTelemetrySubscriptionAfterPushTelemetryUnknownSubscriptionSucceeds() throws Exception { - clientMetricsManager.updateSubscription("sub-1", ClientMetricsTestUtils.defaultTestProperties()); + clientMetricsManager.updateSubscription("sub-1", ClientMetricsTestUtils.defaultProperties()); assertEquals(1, clientMetricsManager.subscriptions().size()); GetTelemetrySubscriptionsRequest subscriptionsRequest = new GetTelemetrySubscriptionsRequest.Builder( @@ -1163,7 +1163,7 @@ public void testGetTelemetrySubscriptionAfterPushTelemetryUnknownSubscriptionSuc @Test public void testGetTelemetrySubscriptionAfterPushTelemetryUnknownCompressionSucceeds() throws Exception { - clientMetricsManager.updateSubscription("sub-1", ClientMetricsTestUtils.defaultTestProperties()); + clientMetricsManager.updateSubscription("sub-1", ClientMetricsTestUtils.defaultProperties()); assertEquals(1, clientMetricsManager.subscriptions().size()); GetTelemetrySubscriptionsRequest subscriptionsRequest = new GetTelemetrySubscriptionsRequest.Builder( From c8b8adf3c1d5fc7ea6256846c4a82c0482ebf0e5 Mon Sep 17 00:00:00 2001 From: Ritika Reddy <98577846+rreddy-22@users.noreply.github.com> Date: Mon, 23 Jun 2025 15:15:36 -0700 Subject: [PATCH 42/72] KAFKA-19367: Follow up bug fix (#19991) This is a follow up to [https://github.com/apache/kafka/pull/19910](https://github.com/apache/kafka/pull/url) The coordinator failed to write an epoch fence transition for producer jt142 to the transaction log with error COORDINATOR_NOT_AVAILABLE. The epoch was increased to 2 but not returned to the client (kafka.coordinator.transaction.TransactionCoordinator) -- as we don't bump the epoch with this change, we should also update the message to not say "increased" and remove the epochAndMetadata.transactionMetadata.hasFailedEpochFence = true line In the test, the expected behavior is: First append transaction to the log fails with COORDINATOR_NOT_AVAILABLE (epoch 1) We try init_pid again, this time the SINGLE epoch bump succeeds, and the following things happen simultaneously (epoch 2) -> Transition to COMPLETE_ABORT -> Return CONCURRENT_TRANSACTION error to the client The client retries, and there is another epoch bump; state transitions to EMPTY (epoch 3) Reviewers: Justine Olshan --- .../transaction/TransactionCoordinator.scala | 6 +- .../TransactionCoordinatorTest.scala | 144 ++++++++++++++++++ 2 files changed, 147 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala index 064de12a4a..fdccf5a020 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala @@ -960,10 +960,10 @@ class TransactionCoordinator(txnConfig: TransactionConfig, case Some(epochAndMetadata) => if (epochAndMetadata.coordinatorEpoch == coordinatorEpoch) { - // This was attempted epoch fence that failed, so mark this state on the metadata - epochAndMetadata.transactionMetadata.hasFailedEpochFence = true + // For TV2, we allow re-bumping the epoch on retry, since we don't complete the epoch bump. + // Therefore, we don't set hasFailedEpochFence = true. warn(s"The coordinator failed to write an epoch fence transition for producer $transactionalId to the transaction log " + - s"with error $error. The epoch was increased to ${newMetadata.producerEpoch} but not returned to the client") + s"with error $error") } } } diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala index bbca105fd4..3adec5c029 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala @@ -33,6 +33,7 @@ import org.junit.jupiter.params.provider.ValueSource import org.mockito.ArgumentMatchers.{any, anyInt} import org.mockito.Mockito._ import org.mockito.{ArgumentCaptor, ArgumentMatchers} +import org.mockito.Mockito.doAnswer import scala.collection.mutable import scala.jdk.CollectionConverters._ @@ -1805,4 +1806,147 @@ class TransactionCoordinatorTest { else producerEpoch } + + @Test + def testTV2AllowsEpochReBumpingAfterFailedWrite(): Unit = { + // Test the complete TV2 flow: failed write → epoch fence → abort → retry with epoch bump + // This demonstrates that TV2 allows epoch re-bumping after failed writes (unlike TV1) + val producerEpoch = 1.toShort + val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, + producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, Ongoing, partitions, time.milliseconds(), time.milliseconds(), TV_2) + + when(transactionManager.validateTransactionTimeoutMs(anyInt())) + .thenReturn(true) + when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) + .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) + when(transactionManager.transactionVersionLevel()).thenReturn(TV_2) + + // First attempt fails with COORDINATOR_NOT_AVAILABLE + when(transactionManager.appendTransactionToLog( + ArgumentMatchers.eq(transactionalId), + ArgumentMatchers.eq(coordinatorEpoch), + any(), + any(), + any(), + any() + )).thenAnswer(invocation => { + val callback = invocation.getArgument[Errors => Unit](3) + + // Simulate the real TransactionStateManager behavior: reset pendingState on failure + // since handleInitProducerId doesn't provide a custom retryOnError function + txnMetadata.pendingState = None + + // For TV2, hasFailedEpochFence is NOT set to true, allowing epoch bumps on retry + // The epoch remains at its original value (1) since completeTransitionTo was never called + + callback.apply(Errors.COORDINATOR_NOT_AVAILABLE) + }) + + coordinator.handleInitProducerId( + transactionalId, + txnTimeoutMs, + None, + initProducerIdMockCallback + ) + assertEquals(InitProducerIdResult(-1, -1, Errors.COORDINATOR_NOT_AVAILABLE), result) + + // After the first failed attempt, the state should be: + // - hasFailedEpochFence = false (NOT set for TV2) + // - pendingState = None (reset by TransactionStateManager) + // - producerEpoch = 1 (unchanged since completeTransitionTo was never called) + // - transaction still ONGOING + + // Second attempt: Should abort the ongoing transaction + reset(transactionManager) + when(transactionManager.validateTransactionTimeoutMs(anyInt())) + .thenReturn(true) + when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) + .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) + when(transactionManager.transactionVersionLevel()).thenReturn(TV_2) + + // Mock the appendTransactionToLog to succeed for the endTransaction call + when(transactionManager.appendTransactionToLog( + ArgumentMatchers.eq(transactionalId), + ArgumentMatchers.eq(coordinatorEpoch), + any(), + any(), + any(), + any() + )).thenAnswer(invocation => { + val newMetadata = invocation.getArgument[TxnTransitMetadata](2) + val callback = invocation.getArgument[Errors => Unit](3) + + // Complete the transition and call the callback with success + txnMetadata.completeTransitionTo(newMetadata) + callback.apply(Errors.NONE) + }) + + // Mock the transactionMarkerChannelManager to simulate the second write (PREPARE_ABORT -> COMPLETE_ABORT) + doAnswer(invocation => { + val newMetadata = invocation.getArgument[TxnTransitMetadata](3) + // Simulate the completion of transaction markers and the second write + // This would normally happen asynchronously after markers are sent + txnMetadata.completeTransitionTo(newMetadata) // This transitions to COMPLETE_ABORT + txnMetadata.pendingState = None + + null + }).when(transactionMarkerChannelManager).addTxnMarkersToSend( + ArgumentMatchers.eq(coordinatorEpoch), + ArgumentMatchers.eq(TransactionResult.ABORT), + ArgumentMatchers.eq(txnMetadata), + any() + ) + + coordinator.handleInitProducerId( + transactionalId, + txnTimeoutMs, + None, + initProducerIdMockCallback + ) + + // The second attempt should return CONCURRENT_TRANSACTIONS (this is intentional) + assertEquals(InitProducerIdResult(-1, -1, Errors.CONCURRENT_TRANSACTIONS), result) + + // The transactionMarkerChannelManager mock should have completed the transition to COMPLETE_ABORT + // Verify that hasFailedEpochFence was never set to true for TV2, allowing future epoch bumps + assertFalse(txnMetadata.hasFailedEpochFence) + + // Third attempt: Client retries after CONCURRENT_TRANSACTIONS + reset(transactionManager) + when(transactionManager.validateTransactionTimeoutMs(anyInt())) + .thenReturn(true) + when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) + .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) + when(transactionManager.transactionVersionLevel()).thenReturn(TV_2) + + when(transactionManager.appendTransactionToLog( + ArgumentMatchers.eq(transactionalId), + ArgumentMatchers.eq(coordinatorEpoch), + any(), + any(), + any(), + any() + )).thenAnswer(invocation => { + val newMetadata = invocation.getArgument[TxnTransitMetadata](2) + val callback = invocation.getArgument[Errors => Unit](3) + + // Complete the transition and call the callback with success + txnMetadata.completeTransitionTo(newMetadata) + callback.apply(Errors.NONE) + }) + + coordinator.handleInitProducerId( + transactionalId, + txnTimeoutMs, + None, + initProducerIdMockCallback + ) + + // The third attempt should succeed with epoch 3 (2 + 1) + // This demonstrates that TV2 allows epoch re-bumping after failed writes + assertEquals(InitProducerIdResult(producerId, 3.toShort, Errors.NONE), result) + + // Final verification that hasFailedEpochFence was never set to true for TV2 + assertFalse(txnMetadata.hasFailedEpochFence) + } } From 9fcfe546d1313b77ad5dc6b10be3095fe6d32569 Mon Sep 17 00:00:00 2001 From: Okada Haruki Date: Wed, 25 Jun 2025 01:35:53 +0900 Subject: [PATCH 43/72] KAFKA-19407 Fix potential IllegalStateException when appending to timeIndex (#19972) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## Summary - Fix potential race condition in LogSegment#readMaxTimestampAndOffsetSoFar(), which may result in non-monotonic offsets and causes replication to stop. - See https://issues.apache.org/jira/browse/KAFKA-19407 for the details how it happen. Reviewers: Vincent PÉRICART , Jun Rao , Chia-Ping Tsai --- .../storage/internals/log/LogSegment.java | 18 ++++++-- .../storage/internals/log/LogSegmentTest.java | 45 +++++++++++++++++++ 2 files changed, 60 insertions(+), 3 deletions(-) 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 b388af1f79..e07d8a2d6a 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 @@ -93,6 +93,9 @@ public class LogSegment implements Closeable { // NOTED: the offset is the last offset of batch having the max timestamp. private volatile TimestampOffset maxTimestampAndOffsetSoFar = TimestampOffset.UNKNOWN; + // Lock for maxTimestampAndOffsetSoFar to ensure that it will be initialized only once + private final Object maxTimestampAndOffsetLock = new Object(); + private long created; /* the number of bytes since we last added an entry in the offset index */ @@ -177,7 +180,7 @@ public void resizeIndexes(int size) throws IOException { public void sanityCheck(boolean timeIndexFileNewlyCreated) throws IOException { if (offsetIndexFile().exists()) { // Resize the time index file to 0 if it is newly created. - if (timeIndexFileNewlyCreated) + if (timeIndexFileNewlyCreated) timeIndex().resize(0); // Sanity checks for time index and offset index are skipped because // we will recover the segments above the recovery point in recoverLog() @@ -192,8 +195,17 @@ public void sanityCheck(boolean timeIndexFileNewlyCreated) throws IOException { * the time index). */ public TimestampOffset readMaxTimestampAndOffsetSoFar() throws IOException { - if (maxTimestampAndOffsetSoFar == TimestampOffset.UNKNOWN) - maxTimestampAndOffsetSoFar = timeIndex().lastEntry(); + if (maxTimestampAndOffsetSoFar == TimestampOffset.UNKNOWN) { + // As stated in LogSegment class javadoc, this class is not thread-safe so basically we assume that + // methods are called within UnifiedLog#lock. + // However, there's exceptional paths where this method can be called outside of the lock, + // so we need lock here to prevent multiple threads trying to modify maxTimestampAndOffsetSoFar + synchronized (maxTimestampAndOffsetLock) { + if (maxTimestampAndOffsetSoFar == TimestampOffset.UNKNOWN) { + maxTimestampAndOffsetSoFar = timeIndex().lastEntry(); + } + } + } return maxTimestampAndOffsetSoFar; } 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 b798378f1a..64064e2bee 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 @@ -42,6 +42,7 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.ValueSource; @@ -59,6 +60,10 @@ import java.util.Map; import java.util.Optional; import java.util.OptionalLong; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -67,7 +72,11 @@ import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.clearInvocations; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; public class LogSegmentTest { @@ -859,6 +868,42 @@ public void testNonMonotonicTimestampForMultipleBatchesInMemoryRecords() throws assertEquals(new TimestampOffset(2, 2), segment.timeIndex().entry(1)); } + @Test + @Timeout(30) + public void testConcurrentAccessToMaxTimestampSoFar() throws Exception { + int numThreads = 16; + ExecutorService executor = Executors.newFixedThreadPool(numThreads); + TimeIndex mockTimeIndex = mock(TimeIndex.class); + when(mockTimeIndex.lastEntry()).thenReturn(new TimestampOffset(RecordBatch.NO_TIMESTAMP, 0L)); + + try { + // to reproduce race, we iterate test for certain duration + long remainingDurationNanos = Duration.ofSeconds(1).toNanos(); + while (remainingDurationNanos > 0) { + long t0 = System.nanoTime(); + clearInvocations(mockTimeIndex); + try (LogSegment seg = spy(LogTestUtils.createSegment(0, logDir, 10, Time.SYSTEM))) { + when(seg.timeIndex()).thenReturn(mockTimeIndex); + List> futures = new ArrayList<>(); + for (int i = 0; i < numThreads; i++) { + futures.add(executor.submit(() -> assertDoesNotThrow(seg::maxTimestampSoFar))); + } + for (Future future : futures) { + future.get(); + } + // timeIndex.lastEntry should be called once if no race + verify(mockTimeIndex, times(1)).lastEntry(); + + long elapsedNanos = System.nanoTime() - t0; + remainingDurationNanos -= elapsedNanos; + } + } + } finally { + executor.shutdown(); + executor.awaitTermination(Long.MAX_VALUE, TimeUnit.SECONDS); + } + } + private ProducerStateManager newProducerStateManager() throws IOException { return new ProducerStateManager( topicPartition, From d426d650414d25dcda3a4af9aac98a819224f7e7 Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Wed, 25 Jun 2025 02:34:13 +0800 Subject: [PATCH 44/72] MINOR: fix reassign command bug (#20003) see https://github.com/apache/kafka/blob/9570c67b8c4ed1a4c3888511adad58d9b3a8bc0f/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala#L1208 During the rewrite for [KAFKA-14595](https://github.com/apache/kafka/pull/13247), the relevant condition was omitted. Reviewers: Chia-Ping Tsai --- .../reassign/ReassignPartitionsCommand.java | 4 +- .../ReassignPartitionsCommandTest.java | 38 ++++++++++++++++++- 2 files changed, 39 insertions(+), 3 deletions(-) diff --git a/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java b/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java index d790fb5586..7ffd1d558a 100644 --- a/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java @@ -1240,7 +1240,7 @@ static Entry, Set> cancelAssignment(A Set targetPartsSet = targetParts.stream().map(t -> t.getKey()).collect(Collectors.toSet()); Set curReassigningParts = new HashSet<>(); adminClient.listPartitionReassignments(targetPartsSet).reassignments().get().forEach((part, reassignment) -> { - if (reassignment.addingReplicas().isEmpty() || !reassignment.removingReplicas().isEmpty()) + if (!reassignment.addingReplicas().isEmpty() || !reassignment.removingReplicas().isEmpty()) curReassigningParts.add(part); }); if (!curReassigningParts.isEmpty()) { @@ -1409,7 +1409,7 @@ static ReassignPartitionsCommandOptions validateAndParseArgs(String[] args) { } OptionSpec action = allActions.get(0); - + if (opts.options.has(opts.bootstrapServerOpt) && opts.options.has(opts.bootstrapControllerOpt)) CommandLineUtils.printUsageAndExit(opts.parser, "Please don't specify both --bootstrap-server and --bootstrap-controller"); else if (!opts.options.has(opts.bootstrapServerOpt) && !opts.options.has(opts.bootstrapControllerOpt)) diff --git a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java index d7f6f937d9..3d6cce5c37 100644 --- a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java @@ -315,7 +315,7 @@ public void testCancellationWithBootstrapController() throws Exception { } @ClusterTest - public void testCancellationWithAddingReplicaInIsr() throws Exception { + public void testCancellationWithAddingAndRemovingReplicaInIsr() throws Exception { createTopics(); TopicPartition foo0 = new TopicPartition("foo", 0); produceMessages(foo0.topic(), foo0.partition(), 200); @@ -350,6 +350,42 @@ public void testCancellationWithAddingReplicaInIsr() throws Exception { verifyReplicaDeleted(new TopicPartitionReplica(foo0.topic(), foo0.partition(), 4)); } + @ClusterTest + public void testCancellationWithAddingReplicaInIsr() throws Exception { + createTopics(); + TopicPartition foo0 = new TopicPartition("foo", 0); + produceMessages(foo0.topic(), foo0.partition(), 200); + + // The reassignment will bring replicas 3 and 4 into the replica set. + String assignment = "{\"version\":1,\"partitions\":" + + "[{\"topic\":\"foo\",\"partition\":0,\"replicas\":[0,1,2,3,4],\"log_dirs\":[\"any\",\"any\",\"any\",\"any\",\"any\"]}" + + "]}"; + + // We will throttle replica 4 so that only replica 3 joins the ISR + setReplicationThrottleForPartitions(foo0); + + // Execute the assignment and wait for replica 3 (only) to join the ISR + runExecuteAssignment(false, assignment, -1L, -1L); + try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { + TestUtils.waitForCondition( + () -> { + Set isr = admin.describeTopics(Collections.singleton(foo0.topic())) + .allTopicNames().get().get(foo0.topic()).partitions().stream() + .filter(p -> p.partition() == foo0.partition()) + .flatMap(p -> p.isr().stream()) + .map(Node::id).collect(Collectors.toSet()); + return isr.containsAll(Arrays.asList(0, 1, 2, 3)); + }, + "Timed out while waiting for replica 3 to join the ISR" + ); + } + + // Now cancel the assignment and verify that the partition is removed from cancelled replicas + assertEquals(new AbstractMap.SimpleImmutableEntry<>(singleton(foo0), Collections.emptySet()), runCancelAssignment(assignment, true, true)); + verifyReplicaDeleted(new TopicPartitionReplica(foo0.topic(), foo0.partition(), 3)); + verifyReplicaDeleted(new TopicPartitionReplica(foo0.topic(), foo0.partition(), 4)); + } + /** * Test moving partitions between directories. */ From 52a5b88512ec652f592bca286bd13982b83e3334 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Tue, 24 Jun 2025 11:24:50 -0700 Subject: [PATCH 45/72] KAFKA-19411: Fix deleteAcls bug which allows more deletions than max records per user op (#19974) If there are more deletion filters after we initially hit the `MAX_RECORDS_PER_USER_OP` bound, we will add an additional deletion record ontop of that for each additional filter. The current error message returned to the client is not useful either, adding logic so client doesn't just get `UNKNOWN_SERVER_EXCEPTION` with no details returned. --- .../kafka/controller/AclControlManager.java | 13 +++-- .../controller/AclControlManagerTest.java | 58 +++++++++++++++++++ 2 files changed, 67 insertions(+), 4 deletions(-) diff --git a/metadata/src/main/java/org/apache/kafka/controller/AclControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/AclControlManager.java index ca324ab578..4771973428 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/AclControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/AclControlManager.java @@ -176,6 +176,10 @@ ControllerResult> deleteAcls(List filter validateFilter(filter); AclDeleteResult result = deleteAclsForFilter(filter, records); results.add(result); + } catch (BoundedListTooLongException e) { + // we do not return partial results here because the fact that only a portion of the deletions + // succeeded can be easily missed due to response size. instead fail the entire response + throw new InvalidRequestException(e.getMessage(), e); } catch (Throwable e) { results.add(new AclDeleteResult(ApiError.fromThrowable(e).exception())); } @@ -191,13 +195,14 @@ AclDeleteResult deleteAclsForFilter(AclBindingFilter filter, StandardAcl acl = entry.getValue(); AclBinding binding = acl.toBinding(); if (filter.matches(binding)) { - deleted.add(new AclBindingDeleteResult(binding)); - records.add(new ApiMessageAndVersion( - new RemoveAccessControlEntryRecord().setId(id), (short) 0)); - if (records.size() > MAX_RECORDS_PER_USER_OP) { + // check size limitation first before adding additional records + if (records.size() >= MAX_RECORDS_PER_USER_OP) { throw new BoundedListTooLongException("Cannot remove more than " + MAX_RECORDS_PER_USER_OP + " acls in a single delete operation."); } + deleted.add(new AclBindingDeleteResult(binding)); + records.add(new ApiMessageAndVersion( + new RemoveAccessControlEntryRecord().setId(id), (short) 0)); } } return new AclDeleteResult(deleted); diff --git a/metadata/src/test/java/org/apache/kafka/controller/AclControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/AclControlManagerTest.java index 30210fe015..e93dd9ad1e 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/AclControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/AclControlManagerTest.java @@ -48,6 +48,7 @@ import org.apache.kafka.server.authorizer.AuthorizationResult; import org.apache.kafka.server.authorizer.AuthorizerServerInfo; import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.mutable.BoundedListTooLongException; import org.apache.kafka.timeline.SnapshotRegistry; import org.junit.jupiter.api.Test; @@ -71,6 +72,7 @@ import static org.apache.kafka.common.resource.PatternType.LITERAL; import static org.apache.kafka.common.resource.PatternType.MATCH; import static org.apache.kafka.common.resource.ResourceType.TOPIC; +import static org.apache.kafka.controller.QuorumController.MAX_RECORDS_PER_USER_OP; import static org.apache.kafka.metadata.authorizer.StandardAclWithIdTest.TEST_ACLS; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -366,4 +368,60 @@ public void testDeleteDedupe() { assertEquals(id, ((RemoveAccessControlEntryRecord) deleteAclResultsBothFilters.records().get(0).message()).id()); assertEquals(2, deleteAclResultsBothFilters.response().size()); } + + @Test + public void testDeleteExceedsMaxRecords() { + AclControlManager manager = new AclControlManager.Builder().build(); + MockClusterMetadataAuthorizer authorizer = new MockClusterMetadataAuthorizer(); + authorizer.loadSnapshot(manager.idToAcl()); + + List firstCreate = new ArrayList<>(); + List secondCreate = new ArrayList<>(); + + // create MAX_RECORDS_PER_USER_OP + 2 ACLs + for (int i = 0; i < MAX_RECORDS_PER_USER_OP + 2; i++) { + StandardAclWithId acl = new StandardAclWithId(Uuid.randomUuid(), + new StandardAcl( + ResourceType.TOPIC, + "mytopic_" + i, + PatternType.LITERAL, + "User:alice", + "127.0.0.1", + AclOperation.READ, + AclPermissionType.ALLOW)); + + // split acl creations between two create requests + if (i % 2 == 0) { + firstCreate.add(acl.toBinding()); + } else { + secondCreate.add(acl.toBinding()); + } + } + ControllerResult> firstCreateResult = manager.createAcls(firstCreate); + assertEquals((MAX_RECORDS_PER_USER_OP / 2) + 1, firstCreateResult.response().size()); + for (AclCreateResult result : firstCreateResult.response()) { + assertTrue(result.exception().isEmpty()); + } + + ControllerResult> secondCreateResult = manager.createAcls(secondCreate); + assertEquals((MAX_RECORDS_PER_USER_OP / 2) + 1, secondCreateResult.response().size()); + for (AclCreateResult result : secondCreateResult.response()) { + assertTrue(result.exception().isEmpty()); + } + + RecordTestUtils.replayAll(manager, firstCreateResult.records()); + RecordTestUtils.replayAll(manager, secondCreateResult.records()); + assertFalse(manager.idToAcl().isEmpty()); + + ArrayList filters = new ArrayList<>(); + for (int i = 0; i < MAX_RECORDS_PER_USER_OP + 2; i++) { + filters.add(new AclBindingFilter( + new ResourcePatternFilter(ResourceType.TOPIC, "mytopic_" + i, PatternType.LITERAL), + AccessControlEntryFilter.ANY)); + } + + Exception exception = assertThrows(InvalidRequestException.class, () -> manager.deleteAcls(filters)); + assertEquals(BoundedListTooLongException.class, exception.getCause().getClass()); + assertEquals("Cannot remove more than " + MAX_RECORDS_PER_USER_OP + " acls in a single delete operation.", exception.getCause().getMessage()); + } } From 7e51a2a43bac271ae6780511c10d9a6014d0a2c1 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Tue, 24 Jun 2025 16:23:25 -0700 Subject: [PATCH 46/72] KAFKA-19294: Fix BrokerLifecycleManager RPC timeouts (#19745) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Previously, we could wait for up to half of the broker session timeout for an RPC to complete, and then delay by up to half of the broker session timeout. When taken together, these two delays could lead to brokers erroneously missing heartbeats. This change removes exponential backoff for heartbeats sent from the broker to the controller. The load caused by heartbeats is not heavy, and controllers can easily time out heartbeats when the queue length is too long. Additionally, we now set the maximum RPC time to the length of the broker period. This minimizes the impact of heavy load. Reviewers: José Armando García Sancio , David Arthur --- .../kafka/server/BrokerLifecycleManager.scala | 21 +++---------------- .../scala/kafka/server/BrokerServer.scala | 2 +- 2 files changed, 4 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala index 36c666b646..fdcbbf5964 100644 --- a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala +++ b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala @@ -28,7 +28,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{BrokerHeartbeatRequest, BrokerHeartbeatResponse, BrokerRegistrationRequest, BrokerRegistrationResponse} import org.apache.kafka.metadata.{BrokerState, VersionRange} import org.apache.kafka.queue.EventQueue.DeadlineFunction -import org.apache.kafka.common.utils.{ExponentialBackoff, LogContext, Time} +import org.apache.kafka.common.utils.{LogContext, Time} import org.apache.kafka.queue.{EventQueue, KafkaEventQueue} import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, NodeToControllerChannelManager} @@ -93,18 +93,6 @@ class BrokerLifecycleManager( private val initialTimeoutNs = MILLISECONDS.toNanos(config.initialRegistrationTimeoutMs.longValue()) - /** - * The exponential backoff to use for resending communication. - */ - private val resendExponentialBackoff = - new ExponentialBackoff(100, 2, config.brokerSessionTimeoutMs.toLong / 2, 0.02) - - /** - * The number of times we've tried and failed to communicate. This variable can only be - * read or written from the BrokerToControllerRequestThread. - */ - private var failedAttempts = 0L - /** * The broker incarnation ID. This ID uniquely identifies each time we start the broker */ @@ -449,7 +437,6 @@ class BrokerLifecycleManager( val message = response.responseBody().asInstanceOf[BrokerRegistrationResponse] val errorCode = Errors.forCode(message.data().errorCode()) if (errorCode == Errors.NONE) { - failedAttempts = 0 _brokerEpoch = message.data().brokerEpoch() registered = true initialRegistrationSucceeded = true @@ -523,7 +510,6 @@ class BrokerLifecycleManager( val errorCode = Errors.forCode(message.data().errorCode()) if (errorCode == Errors.NONE) { val responseData = message.data() - failedAttempts = 0 currentOfflineDirs.foreach(cur => offlineDirs.put(cur, true)) _state match { case BrokerState.STARTING => @@ -586,10 +572,9 @@ class BrokerLifecycleManager( } private def scheduleNextCommunicationAfterFailure(): Unit = { - val delayMs = resendExponentialBackoff.backoff(failedAttempts) - failedAttempts = failedAttempts + 1 nextSchedulingShouldBeImmediate = false // never immediately reschedule after a failure - scheduleNextCommunication(NANOSECONDS.convert(delayMs, MILLISECONDS)) + scheduleNextCommunication(NANOSECONDS.convert( + config.brokerHeartbeatIntervalMs.longValue() , MILLISECONDS)) } private def scheduleNextCommunicationAfterSuccess(): Unit = { diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 4fdd3a0779..85612c58d4 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -405,7 +405,7 @@ class BrokerServer( config, "heartbeat", s"broker-${config.nodeId}-", - config.brokerSessionTimeoutMs / 2 // KAFKA-14392 + config.brokerHeartbeatIntervalMs ) lifecycleManager.start( () => sharedServer.loader.lastAppliedOffset(), From 46e843da9faa8670a100c1bba3166ac76989886d Mon Sep 17 00:00:00 2001 From: Calvin Liu <83986057+CalvinConfluent@users.noreply.github.com> Date: Tue, 24 Jun 2025 19:59:50 -0700 Subject: [PATCH 47/72] KAFKA-19383: Handle the deleted topics when applying ClearElrRecord (#20034) https://issues.apache.org/jira/browse/KAFKA-19383 When applying the ClearElrRecord, it may pick up the topicId in the image without checking if the topic has been deleted. This can cause the creation of a new TopicRecord with an old topic ID. Reviewers: Alyssa Huang , Artem Livshits , Colin P. McCabe No conflicts. --- .../org/apache/kafka/image/TopicsDelta.java | 33 ++- .../apache/kafka/image/TopicsImageTest.java | 223 ++++++++++++++++++ 2 files changed, 248 insertions(+), 8 deletions(-) diff --git a/metadata/src/main/java/org/apache/kafka/image/TopicsDelta.java b/metadata/src/main/java/org/apache/kafka/image/TopicsDelta.java index ec5cee135b..a1f3cb2f52 100644 --- a/metadata/src/main/java/org/apache/kafka/image/TopicsDelta.java +++ b/metadata/src/main/java/org/apache/kafka/image/TopicsDelta.java @@ -96,26 +96,43 @@ public void replay(PartitionChangeRecord record) { topicDelta.replay(record); } + private void maybeReplayClearElrRecord(Uuid topicId, ClearElrRecord record) { + // Only apply the record if the topic is not deleted. + if (!deletedTopicIds.contains(topicId)) { + TopicDelta topicDelta = getOrCreateTopicDelta(topicId); + topicDelta.replay(record); + } + } + + // When replaying the ClearElrRecord, we need to first find the latest topic ID associated with the topic(s) because + // multiple topic IDs for the same topic in a TopicsDelta is possible in the event of topic deletion and recreation. + // Second, we should not add the topicDelta if the given topic ID has been deleted. So that we don't leak the + // deleted topic ID. public void replay(ClearElrRecord record) { if (!record.topicName().isEmpty()) { - Uuid topicId; - if (image.getTopic(record.topicName()) != null) { - topicId = image.getTopic(record.topicName()).id(); - } else { + Uuid topicId = null; + // CreatedTopics contains the latest topic IDs. It should be checked first in case the topic is deleted and + // created in the same batch. + if (createdTopics.containsKey(record.topicName())) { topicId = createdTopics.get(record.topicName()); + } else if (image.getTopic(record.topicName()) != null) { + topicId = image.getTopic(record.topicName()).id(); } + if (topicId == null) { throw new RuntimeException("Unable to clear elr for topic with name " + record.topicName() + ": no such topic found."); } - TopicDelta topicDelta = getOrCreateTopicDelta(topicId); - topicDelta.replay(record); + + maybeReplayClearElrRecord(topicId, record); } else { // Update all the existing topics image.topicsById().forEach((topicId, image) -> { - TopicDelta topicDelta = getOrCreateTopicDelta(topicId); - topicDelta.replay(record); + maybeReplayClearElrRecord(topicId, record); }); + createdTopicIds().forEach((topicId -> { + maybeReplayClearElrRecord(topicId, record); + })); } } diff --git a/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java b/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java index c868ebdfee..9e6666cc5b 100644 --- a/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java @@ -429,6 +429,229 @@ public void testClearElrRecords() { assertEquals(0, image.getTopic(barId).partitions().get(0).lastKnownElr.length); } + @Test + public void testClearElrRecordOnNonExistingTopic() { + TopicsImage image = TopicsImage.EMPTY; + + List topicRecords = new ArrayList<>(); + topicRecords.addAll(List.of( + new ApiMessageAndVersion( + new ClearElrRecord().setTopicName("foo"), + CLEAR_ELR_RECORD.highestSupportedVersion() + )) + ); + TopicsDelta delta = new TopicsDelta(image); + assertThrows(RuntimeException.class, () -> RecordTestUtils.replayAll(delta, topicRecords)); + } + + @Test + public void testClearElrRecords_All_ForDeletedTopics() { + Uuid fooId = Uuid.fromString("0hHJ3X5ZQ-CFfQ5xgpj90w"); + Uuid fooId2 = Uuid.randomUuid(); + Uuid barId = Uuid.fromString("f62ptyETTjet8SL5ZeREiw"); + Uuid barId2 = Uuid.randomUuid(); + + List topics = new ArrayList<>(); + topics.add( + newTopicImage( + "foo", + fooId, + newPartition(new int[] {0, 1, 2, 3}) + ) + ); + TopicsImage image = new TopicsImage(newTopicsByIdMap(topics), + newTopicsByNameMap(topics)); + + List topicRecords = new ArrayList<>(); + topicRecords.add( + new ApiMessageAndVersion( + new PartitionRecord().setTopicId(fooId). + setPartitionId(0). + setLeader(0). + setIsr(List.of(1, 2, 3)), + PARTITION_RECORD.highestSupportedVersion() + ) + ); + + TopicsDelta delta = new TopicsDelta(image); + RecordTestUtils.replayAll(delta, topicRecords); + image = delta.apply(); + + topicRecords = new ArrayList<>(); + /* Test the following: + 1. Topic foo is deleted and created in the same delta, the clear elr applies on the new topic + 2. Topic bar is created, deleted, then created in the same delta, the clear elr applies on the new topic + */ + topicRecords.addAll(List.of( + new ApiMessageAndVersion( + new RemoveTopicRecord().setTopicId(fooId), + REMOVE_TOPIC_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new TopicRecord().setTopicId(fooId2). + setName("foo"), + TOPIC_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new PartitionRecord().setTopicId(fooId2).setPartitionId(0). + setIsr(List.of(0, 1)). + setEligibleLeaderReplicas(List.of(2)). + setLastKnownElr(List.of(3)), + PARTITION_CHANGE_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new TopicRecord().setTopicId(barId). + setName("bar"), + TOPIC_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new PartitionRecord().setTopicId(barId). + setPartitionId(0). + setLeader(0). + setIsr(List.of(1, 2, 3)), + PARTITION_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new RemoveTopicRecord().setTopicId(barId), + REMOVE_TOPIC_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new TopicRecord().setTopicId(barId2). + setName("bar"), + TOPIC_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new PartitionRecord().setTopicId(barId2). + setPartitionId(0). + setLeader(0). + setIsr(List.of(1)). + setEligibleLeaderReplicas(List.of(2)). + setLastKnownElr(List.of(3)), + PARTITION_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new ClearElrRecord(), + CLEAR_ELR_RECORD.highestSupportedVersion() + )) + ); + delta = new TopicsDelta(image); + RecordTestUtils.replayAll(delta, topicRecords); + image = delta.apply(); + assertEquals(2, image.topicsById().size()); + assertEquals(2, image.topicsByName().size()); + + assertEquals(0, image.getTopic(fooId2).partitions().get(0).elr.length); + assertEquals(0, image.getTopic(fooId2).partitions().get(0).lastKnownElr.length); + assertEquals(0, image.getTopic(barId2).partitions().get(0).elr.length); + assertEquals(0, image.getTopic(barId2).partitions().get(0).lastKnownElr.length); + } + + @Test + public void testClearElrRecords_Single_ForDeletedTopics() { + Uuid fooId = Uuid.fromString("0hHJ3X5ZQ-CFfQ5xgpj90w"); + Uuid fooId2 = Uuid.randomUuid(); + Uuid barId = Uuid.fromString("f62ptyETTjet8SL5ZeREiw"); + Uuid barId2 = Uuid.randomUuid(); + + List topics = new ArrayList<>(); + topics.add( + newTopicImage( + "foo", + fooId, + newPartition(new int[] {0, 1, 2, 3}) + ) + ); + TopicsImage image = new TopicsImage(newTopicsByIdMap(topics), + newTopicsByNameMap(topics)); + + List topicRecords = new ArrayList<>(); + topicRecords.add( + new ApiMessageAndVersion( + new PartitionRecord().setTopicId(fooId). + setPartitionId(0). + setLeader(0). + setIsr(List.of(1, 2, 3)), + PARTITION_RECORD.highestSupportedVersion() + ) + ); + + TopicsDelta delta = new TopicsDelta(image); + RecordTestUtils.replayAll(delta, topicRecords); + image = delta.apply(); + + topicRecords = new ArrayList<>(); + /* Test the following: + 1. Topic foo is deleted and created in the same delta, the clear elr applies on the new topic + 2. Topic bar is created, deleted, then created in the same delta, the clear elr applies on the new topic + */ + topicRecords.addAll(List.of( + new ApiMessageAndVersion( + new RemoveTopicRecord().setTopicId(fooId), + REMOVE_TOPIC_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new TopicRecord().setTopicId(fooId2). + setName("foo"), + TOPIC_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new PartitionRecord().setTopicId(fooId2).setPartitionId(0). + setIsr(List.of(0, 1)). + setEligibleLeaderReplicas(List.of(2)). + setLastKnownElr(List.of(3)), + PARTITION_CHANGE_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new TopicRecord().setTopicId(barId). + setName("bar"), + TOPIC_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new PartitionRecord().setTopicId(barId). + setPartitionId(0). + setLeader(0). + setIsr(List.of(1, 2, 3)), + PARTITION_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new RemoveTopicRecord().setTopicId(barId), + REMOVE_TOPIC_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new TopicRecord().setTopicId(barId2). + setName("bar"), + TOPIC_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new PartitionRecord().setTopicId(barId2). + setPartitionId(0). + setLeader(0). + setIsr(List.of(1)). + setEligibleLeaderReplicas(List.of(2)). + setLastKnownElr(List.of(3)), + PARTITION_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new ClearElrRecord().setTopicName("foo"), + CLEAR_ELR_RECORD.highestSupportedVersion() + ), + new ApiMessageAndVersion( + new ClearElrRecord().setTopicName("bar"), + CLEAR_ELR_RECORD.highestSupportedVersion() + )) + ); + delta = new TopicsDelta(image); + RecordTestUtils.replayAll(delta, topicRecords); + image = delta.apply(); + assertEquals(2, image.topicsById().size()); + assertEquals(2, image.topicsByName().size()); + + assertEquals(0, image.getTopic(fooId2).partitions().get(0).elr.length); + assertEquals(0, image.getTopic(fooId2).partitions().get(0).lastKnownElr.length); + assertEquals(0, image.getTopic(barId2).partitions().get(0).elr.length); + assertEquals(0, image.getTopic(barId2).partitions().get(0).lastKnownElr.length); + } + @Test public void testClearElrRecordForNonExistTopic() { TopicsImage image = new TopicsImage(newTopicsByIdMap(Collections.emptyList()), From 15ec0536654f36bea73945f594f640b2592d689e Mon Sep 17 00:00:00 2001 From: Logan Zhu Date: Wed, 25 Jun 2025 19:19:30 +0800 Subject: [PATCH 48/72] KAFKA-18656 Backport KAFKA-18597 to 4.0 (#20026) Backport of [KAFKA-18597](https://github.com/apache/kafka/pull/18627) to the 4.0 branch. Reviewers: Chia-Ping Tsai --- .../src/main/scala/kafka/log/LogCleaner.scala | 19 +- .../scala/unit/kafka/log/LogCleanerTest.scala | 168 +++++++++++++++--- 2 files changed, 153 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 5d7ee51896..e1f4d4afa2 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -117,14 +117,14 @@ class LogCleaner(initialConfig: CleanerConfig, /** * @param f to compute the result - * @return the max value (int value) or 0 if there is no cleaner + * @return the max value or 0 if there is no cleaner */ - private[log] def maxOverCleanerThreads(f: CleanerThread => Double): Int = - cleaners.map(f).maxOption.getOrElse(0.0d).toInt + private[log] def maxOverCleanerThreads(f: CleanerThread => Double): Double = + cleaners.map(f).maxOption.getOrElse(0.0d) /* a metric to track the maximum utilization of any thread's buffer in the last cleaning */ metricsGroup.newGauge(MaxBufferUtilizationPercentMetricName, - () => maxOverCleanerThreads(_.lastStats.bufferUtilization) * 100) + () => (maxOverCleanerThreads(_.lastStats.bufferUtilization) * 100).toInt) /* a metric to track the recopy rate of each thread's last cleaning */ metricsGroup.newGauge(CleanerRecopyPercentMetricName, () => { @@ -134,12 +134,12 @@ class LogCleaner(initialConfig: CleanerConfig, }) /* a metric to track the maximum cleaning time for the last cleaning from each thread */ - metricsGroup.newGauge(MaxCleanTimeMetricName, () => maxOverCleanerThreads(_.lastStats.elapsedSecs)) + metricsGroup.newGauge(MaxCleanTimeMetricName, () => maxOverCleanerThreads(_.lastStats.elapsedSecs).toInt) // a metric to track delay between the time when a log is required to be compacted // as determined by max compaction lag and the time of last cleaner run. metricsGroup.newGauge(MaxCompactionDelayMetricsName, - () => maxOverCleanerThreads(_.lastPreCleanStats.maxCompactionDelayMs.toDouble) / 1000) + () => (maxOverCleanerThreads(_.lastPreCleanStats.maxCompactionDelayMs.toDouble) / 1000).toInt) metricsGroup.newGauge(DeadThreadCountMetricName, () => deadThreadCount) @@ -523,10 +523,11 @@ object LogCleaner { } - private val MaxBufferUtilizationPercentMetricName = "max-buffer-utilization-percent" + // Visible for test. + private[log] val MaxBufferUtilizationPercentMetricName = "max-buffer-utilization-percent" private val CleanerRecopyPercentMetricName = "cleaner-recopy-percent" - private val MaxCleanTimeMetricName = "max-clean-time-secs" - private val MaxCompactionDelayMetricsName = "max-compaction-delay-secs" + private[log] val MaxCleanTimeMetricName = "max-clean-time-secs" + private[log] val MaxCompactionDelayMetricsName = "max-compaction-delay-secs" private val DeadThreadCountMetricName = "DeadThreadCount" // package private for testing private[log] val MetricNames = Set( diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 106a4a78a9..9ca3916319 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -18,6 +18,7 @@ package kafka.log import kafka.common._ +import kafka.log.LogCleaner.{MaxBufferUtilizationPercentMetricName, MaxCleanTimeMetricName, MaxCompactionDelayMetricsName} import kafka.server.KafkaConfig import kafka.utils.{CoreUtils, Logging, Pool, TestUtils} import org.apache.kafka.common.TopicPartition @@ -2048,42 +2049,159 @@ class LogCleanerTest extends Logging { } @Test - def testMaxOverCleanerThreads(): Unit = { - val logCleaner = new LogCleaner(new CleanerConfig(true), + def testMaxBufferUtilizationPercentMetric(): Unit = { + val logCleaner = new LogCleaner( + new CleanerConfig(true), logDirs = Array(TestUtils.tempDir(), TestUtils.tempDir()), logs = new Pool[TopicPartition, UnifiedLog](), logDirFailureChannel = new LogDirFailureChannel(1), - time = time) + time = time + ) + + def assertMaxBufferUtilizationPercent(expected: Int): Unit = { + val gauge = logCleaner.metricsGroup.newGauge(MaxBufferUtilizationPercentMetricName, + () => (logCleaner.maxOverCleanerThreads(_.lastStats.bufferUtilization) * 100).toInt) + assertEquals(expected, gauge.value()) + } + + try { + // No CleanerThreads + assertMaxBufferUtilizationPercent(0) + + val cleaners = logCleaner.cleaners + + val cleaner1 = new logCleaner.CleanerThread(1) + cleaner1.lastStats = new CleanerStats(time) + cleaner1.lastStats.bufferUtilization = 0.75 + cleaners += cleaner1 + + val cleaner2 = new logCleaner.CleanerThread(2) + cleaner2.lastStats = new CleanerStats(time) + cleaner2.lastStats.bufferUtilization = 0.85 + cleaners += cleaner2 + + val cleaner3 = new logCleaner.CleanerThread(3) + cleaner3.lastStats = new CleanerStats(time) + cleaner3.lastStats.bufferUtilization = 0.65 + cleaners += cleaner3 + + // expect the gauge value to reflect the maximum bufferUtilization + assertMaxBufferUtilizationPercent(85) + + // Update bufferUtilization and verify the gauge value updates + cleaner1.lastStats.bufferUtilization = 0.9 + assertMaxBufferUtilizationPercent(90) + + // All CleanerThreads have the same bufferUtilization + cleaners.foreach(_.lastStats.bufferUtilization = 0.5) + assertMaxBufferUtilizationPercent(50) + } finally { + logCleaner.shutdown() + } + } + + @Test + def testMaxCleanTimeMetric(): Unit = { + val logCleaner = new LogCleaner( + new CleanerConfig(true), + logDirs = Array(TestUtils.tempDir(), TestUtils.tempDir()), + logs = new Pool[TopicPartition, UnifiedLog](), + logDirFailureChannel = new LogDirFailureChannel(1), + time = time + ) + + def assertMaxCleanTime(expected: Int): Unit = { + val gauge = logCleaner.metricsGroup.newGauge(MaxCleanTimeMetricName, + () => logCleaner.maxOverCleanerThreads(_.lastStats.elapsedSecs).toInt) + assertEquals(expected, gauge.value()) + } - val cleaners = logCleaner.cleaners + try { + // No CleanerThreads + assertMaxCleanTime(0) - val cleaner1 = new logCleaner.CleanerThread(1) - cleaner1.lastStats = new CleanerStats(time) - cleaner1.lastStats.bufferUtilization = 0.75 - cleaners += cleaner1 + val cleaners = logCleaner.cleaners - val cleaner2 = new logCleaner.CleanerThread(2) - cleaner2.lastStats = new CleanerStats(time) - cleaner2.lastStats.bufferUtilization = 0.85 - cleaners += cleaner2 + val cleaner1 = new logCleaner.CleanerThread(1) + cleaner1.lastStats = new CleanerStats(time) + cleaner1.lastStats.endTime = cleaner1.lastStats.startTime + 1_000L + cleaners += cleaner1 - val cleaner3 = new logCleaner.CleanerThread(3) - cleaner3.lastStats = new CleanerStats(time) - cleaner3.lastStats.bufferUtilization = 0.65 - cleaners += cleaner3 + val cleaner2 = new logCleaner.CleanerThread(2) + cleaner2.lastStats = new CleanerStats(time) + cleaner2.lastStats.endTime = cleaner2.lastStats.startTime + 2_000L + cleaners += cleaner2 - assertEquals(0, logCleaner.maxOverCleanerThreads(_.lastStats.bufferUtilization)) + val cleaner3 = new logCleaner.CleanerThread(3) + cleaner3.lastStats = new CleanerStats(time) + cleaner3.lastStats.endTime = cleaner3.lastStats.startTime + 3_000L + cleaners += cleaner3 - cleaners.clear() + // expect the gauge value to reflect the maximum cleanTime + assertMaxCleanTime(3) - cleaner1.lastStats.bufferUtilization = 5d - cleaners += cleaner1 - cleaner2.lastStats.bufferUtilization = 6d - cleaners += cleaner2 - cleaner3.lastStats.bufferUtilization = 7d - cleaners += cleaner3 + // Update cleanTime and verify the gauge value updates + cleaner1.lastStats.endTime = cleaner1.lastStats.startTime + 4_000L + assertMaxCleanTime(4) - assertEquals(7, logCleaner.maxOverCleanerThreads(_.lastStats.bufferUtilization)) + // All CleanerThreads have the same cleanTime + cleaners.foreach(cleaner => cleaner.lastStats.endTime = cleaner.lastStats.startTime + 1_500L) + assertMaxCleanTime(1) + } finally { + logCleaner.shutdown() + } + } + + @Test + def testMaxCompactionDelayMetrics(): Unit = { + val logCleaner = new LogCleaner( + new CleanerConfig(true), + logDirs = Array(TestUtils.tempDir(), TestUtils.tempDir()), + logs = new Pool[TopicPartition, UnifiedLog](), + logDirFailureChannel = new LogDirFailureChannel(1), + time = time + ) + + def assertMaxCompactionDelay(expected: Int): Unit = { + val gauge = logCleaner.metricsGroup.newGauge(MaxCompactionDelayMetricsName, + () => (logCleaner.maxOverCleanerThreads(_.lastPreCleanStats.maxCompactionDelayMs.toDouble) / 1000).toInt) + assertEquals(expected, gauge.value()) + } + + try { + // No CleanerThreads + assertMaxCompactionDelay(0) + + val cleaners = logCleaner.cleaners + + val cleaner1 = new logCleaner.CleanerThread(1) + cleaner1.lastStats = new CleanerStats(time) + cleaner1.lastPreCleanStats.maxCompactionDelayMs = 1_000L + cleaners += cleaner1 + + val cleaner2 = new logCleaner.CleanerThread(2) + cleaner2.lastStats = new CleanerStats(time) + cleaner2.lastPreCleanStats.maxCompactionDelayMs = 2_000L + cleaners += cleaner2 + + val cleaner3 = new logCleaner.CleanerThread(3) + cleaner3.lastStats = new CleanerStats(time) + cleaner3.lastPreCleanStats.maxCompactionDelayMs = 3_000L + cleaners += cleaner3 + + // expect the gauge value to reflect the maximum CompactionDelay + assertMaxCompactionDelay(3) + + // Update CompactionDelay and verify the gauge value updates + cleaner1.lastPreCleanStats.maxCompactionDelayMs = 4_000L + assertMaxCompactionDelay(4) + + // All CleanerThreads have the same CompactionDelay + cleaners.foreach(_.lastPreCleanStats.maxCompactionDelayMs = 1_500L) + assertMaxCompactionDelay(1) + } finally { + logCleaner.shutdown() + } } private def writeToLog(log: UnifiedLog, keysAndValues: Iterable[(Int, Int)], offsetSeq: Iterable[Long]): Iterable[Long] = { From 6351bc05aafc8ba480e9f85ab702e67e48416953 Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Wed, 25 Jun 2025 17:56:58 +0100 Subject: [PATCH 49/72] MINOR: Fix response for consumer group describe with empty group id (#20030) (#20036) ConsumerGroupDescribe with an empty group id returns a response containing `null` groupId in a non-nullable field. Since the response cannot be serialized, this results in UNKNOWN_SERVER_ERROR being returned to the client. This PR sets the group id in the response to an empty string instead and adds request tests for empty group id. Reviewers: David Jacot , Chia-Ping Tsai --- .../ConsumerGroupDescribeRequestTest.scala | 14 +++++++ .../ConsumerGroupHeartbeatRequestTest.scala | 42 +++++++++++++++++++ .../server/DeleteGroupsRequestTest.scala | 4 +- .../server/DescribeGroupsRequestTest.scala | 9 +++- .../kafka/server/HeartbeatRequestTest.scala | 9 ++++ .../kafka/server/JoinGroupRequestTest.scala | 11 +++++ .../kafka/server/OffsetFetchRequestTest.scala | 33 +++++++++++++++ .../kafka/server/SyncGroupRequestTest.scala | 11 +++++ .../group/GroupCoordinatorService.java | 6 +-- .../group/GroupCoordinatorServiceTest.java | 10 ++--- 10 files changed, 137 insertions(+), 12 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestTest.scala b/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestTest.scala index f6831ca8e3..709c3aa9dc 100644 --- a/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestTest.scala @@ -211,6 +211,20 @@ class ConsumerGroupDescribeRequestTest(cluster: ClusterInstance) extends GroupCo ) assertEquals(expected, actual) + + val unknownGroupResponse = consumerGroupDescribe( + groupIds = List("grp-unknown"), + includeAuthorizedOperations = true, + version = version.toShort, + ) + assertEquals(Errors.GROUP_ID_NOT_FOUND.code, unknownGroupResponse.head.errorCode()) + + val emptyGroupResponse = consumerGroupDescribe( + groupIds = List(""), + includeAuthorizedOperations = true, + version = version.toShort, + ) + assertEquals(Errors.INVALID_GROUP_ID.code, emptyGroupResponse.head.errorCode()) } } finally { admin.close() diff --git a/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala b/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala index e94bcbc56a..b6428540b1 100644 --- a/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala @@ -301,6 +301,48 @@ class ConsumerGroupHeartbeatRequestTest(cluster: ClusterInstance) { } } + @ClusterTest + def testEmptyConsumerGroupId(): Unit = { + val admin = cluster.admin() + + // Creates the __consumer_offsets topics because it won't be created automatically + // in this test because it does not use FindCoordinator API. + try { + TestUtils.createOffsetsTopicWithAdmin( + admin = admin, + brokers = cluster.brokers.values().asScala.toSeq, + controllers = cluster.controllers().values().asScala.toSeq + ) + + // Heartbeat request to join the group. Note that the member subscribes + // to an nonexistent topic. + val consumerGroupHeartbeatRequest = new ConsumerGroupHeartbeatRequest.Builder( + new ConsumerGroupHeartbeatRequestData() + .setGroupId("") + .setMemberId(Uuid.randomUuid().toString) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(5 * 60 * 1000) + .setSubscribedTopicNames(List("foo").asJava) + .setTopicPartitions(List.empty.asJava), + true + ).build() + + // Send the request until receiving a successful response. There is a delay + // here because the group coordinator is loaded in the background. + var consumerGroupHeartbeatResponse: ConsumerGroupHeartbeatResponse = null + TestUtils.waitUntilTrue(() => { + consumerGroupHeartbeatResponse = connectAndReceive(consumerGroupHeartbeatRequest) + consumerGroupHeartbeatResponse.data.errorCode == Errors.INVALID_REQUEST.code + }, msg = s"Did not receive the expected error. Last response $consumerGroupHeartbeatResponse.") + + // Verify the response. + assertEquals(Errors.INVALID_REQUEST.code, consumerGroupHeartbeatResponse.data.errorCode) + assertEquals("GroupId can't be empty.", consumerGroupHeartbeatResponse.data.errorMessage) + } finally { + admin.close() + } + } + @ClusterTest def testConsumerGroupHeartbeatWithEmptySubscription(): Unit = { val admin = cluster.admin() diff --git a/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala index f9b9e9c946..a17c22ea33 100644 --- a/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala @@ -99,8 +99,8 @@ class DeleteGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinator ) deleteGroups( - groupIds = List("grp-non-empty", "grp"), - expectedErrors = List(Errors.NON_EMPTY_GROUP, Errors.NONE), + groupIds = List("grp-non-empty", "grp", ""), + expectedErrors = List(Errors.NON_EMPTY_GROUP, Errors.NONE, Errors.GROUP_ID_NOT_FOUND), version = version.toShort ) diff --git a/core/src/test/scala/unit/kafka/server/DescribeGroupsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeGroupsRequestTest.scala index 4f1ba4b9b2..1223ad6efa 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeGroupsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeGroupsRequestTest.scala @@ -104,10 +104,15 @@ class DescribeGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinat .setGroupId("grp-unknown") .setGroupState(ClassicGroupState.DEAD.toString) // Return DEAD group when the group does not exist. .setErrorCode(if (version >= 6) Errors.GROUP_ID_NOT_FOUND.code() else Errors.NONE.code()) - .setErrorMessage(if (version >= 6) "Group grp-unknown not found." else null) + .setErrorMessage(if (version >= 6) "Group grp-unknown not found." else null), + new DescribedGroup() + .setGroupId("") + .setGroupState(ClassicGroupState.DEAD.toString) // Return DEAD group when the group does not exist. + .setErrorCode(if (version >= 6) Errors.GROUP_ID_NOT_FOUND.code() else Errors.NONE.code()) + .setErrorMessage(if (version >= 6) "Group not found." else null) ), describeGroups( - groupIds = List("grp-1", "grp-2", "grp-unknown"), + groupIds = List("grp-1", "grp-2", "grp-unknown", ""), version = version.toShort ) ) diff --git a/core/src/test/scala/unit/kafka/server/HeartbeatRequestTest.scala b/core/src/test/scala/unit/kafka/server/HeartbeatRequestTest.scala index 332c01aeeb..ad9f77c9fe 100644 --- a/core/src/test/scala/unit/kafka/server/HeartbeatRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/HeartbeatRequestTest.scala @@ -190,6 +190,15 @@ class HeartbeatRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBas expectedError = Errors.UNKNOWN_MEMBER_ID, version = version.toShort ) + + // Heartbeat with empty group id. + heartbeat( + groupId = "", + memberId = leaderMemberId, + generationId = -1, + expectedError = Errors.INVALID_GROUP_ID, + version = version.toShort + ) } } } diff --git a/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala b/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala index f77c2fc1bf..3e8fad1ed4 100644 --- a/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala @@ -149,6 +149,17 @@ class JoinGroupRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBas ) ) + // Join with an empty group id. + verifyJoinGroupResponseDataEquals( + new JoinGroupResponseData() + .setErrorCode(Errors.INVALID_GROUP_ID.code) + .setProtocolName(if (version >= 7) null else ""), + sendJoinRequest( + groupId = "", + version = version.toShort + ) + ) + // Join with an inconsistent protocolType. verifyJoinGroupResponseDataEquals( new JoinGroupResponseData() diff --git a/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala index a504ecdeea..3518f50b4c 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala @@ -269,6 +269,39 @@ class OffsetFetchRequestTest(cluster: ClusterInstance) extends GroupCoordinatorB ) ) + // Fetch with empty group id. + assertEquals( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId("") + .setTopics(List( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName("foo") + .setPartitions(List( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(-1L), + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(1) + .setCommittedOffset(-1L), + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(5) + .setCommittedOffset(-1L) + ).asJava) + ).asJava), + fetchOffsets( + groupId = "", + memberId = memberId, + memberEpoch = memberEpoch, + partitions = List( + new TopicPartition("foo", 0), + new TopicPartition("foo", 1), + new TopicPartition("foo", 5) // This one does not exist. + ), + requireStable = requireStable, + version = version.toShort + ) + ) + // Fetch with stale member epoch. assertEquals( new OffsetFetchResponseData.OffsetFetchResponseGroup() diff --git a/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala b/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala index 3a53fbf144..f89defd3f1 100644 --- a/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala @@ -76,6 +76,17 @@ class SyncGroupRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBas version = version.toShort ) + // Sync with empty group id. + verifySyncGroupWithOldProtocol( + groupId = "", + memberId = "member-id", + generationId = -1, + expectedProtocolType = null, + expectedProtocolName = null, + expectedError = Errors.INVALID_GROUP_ID, + version = version.toShort + ) + val metadata = ConsumerProtocol.serializeSubscription( new ConsumerPartitionAssignor.Subscription(Collections.singletonList("foo")) ).array diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java index 893b82fa7a..447bcb0055 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java @@ -636,7 +636,7 @@ public CompletableFuture> } else { futures.add(CompletableFuture.completedFuture(Collections.singletonList( new ConsumerGroupDescribeResponseData.DescribedGroup() - .setGroupId(null) + .setGroupId("") .setErrorCode(Errors.INVALID_GROUP_ID.code()) ))); } @@ -687,7 +687,7 @@ public CompletableFuture> shareGroupDescribe( } else { futures.add(CompletableFuture.completedFuture(Collections.singletonList( new ShareGroupDescribeResponseData.DescribedGroup() - .setGroupId(null) + .setGroupId("") .setErrorCode(Errors.INVALID_GROUP_ID.code()) ))); } @@ -736,7 +736,7 @@ public CompletableFuture> descri if (groupId == null) { futures.add(CompletableFuture.completedFuture(Collections.singletonList( new DescribeGroupsResponseData.DescribedGroup() - .setGroupId(null) + .setGroupId("") .setErrorCode(Errors.INVALID_GROUP_ID.code()) ))); } else { diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java index 6d1120c5c9..90057cfab8 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java @@ -1038,7 +1038,7 @@ public void testDescribeGroupsInvalidGroupId() throws Exception { .setGroupId(""); List expectedDescribedGroups = Arrays.asList( new DescribeGroupsResponseData.DescribedGroup() - .setGroupId(null) + .setGroupId("") .setErrorCode(Errors.INVALID_GROUP_ID.code()), describedGroup ); @@ -1470,11 +1470,11 @@ public void testConsumerGroupDescribeInvalidGroupId() throws ExecutionException, service.startup(() -> partitionCount); ConsumerGroupDescribeResponseData.DescribedGroup describedGroup = new ConsumerGroupDescribeResponseData.DescribedGroup() - .setGroupId(null) + .setGroupId("") .setErrorCode(Errors.INVALID_GROUP_ID.code()); List expectedDescribedGroups = Arrays.asList( new ConsumerGroupDescribeResponseData.DescribedGroup() - .setGroupId(null) + .setGroupId("") .setErrorCode(Errors.INVALID_GROUP_ID.code()), describedGroup ); @@ -2347,11 +2347,11 @@ public void testShareGroupDescribeInvalidGroupId() throws ExecutionException, In service.startup(() -> partitionCount); ShareGroupDescribeResponseData.DescribedGroup describedGroup = new ShareGroupDescribeResponseData.DescribedGroup() - .setGroupId(null) + .setGroupId("") .setErrorCode(Errors.INVALID_GROUP_ID.code()); List expectedDescribedGroups = Arrays.asList( new ShareGroupDescribeResponseData.DescribedGroup() - .setGroupId(null) + .setGroupId("") .setErrorCode(Errors.INVALID_GROUP_ID.code()), describedGroup ); From 4ce6f5cb923ff0ae736bd2c364767af0b3477c97 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Thu, 26 Jun 2025 10:06:46 -0700 Subject: [PATCH 50/72] MINOR: Improve ProcessorContext JavaDocs (#20042) Clarify that state stores are sharded, and shards cannot be shared across Processors. Reviewers: Lucas Brutschy --- .../apache/kafka/streams/processor/ProcessorContext.java | 7 +++++++ .../kafka/streams/processor/api/ProcessingContext.java | 6 ++++++ 2 files changed, 13 insertions(+) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java index 3d057c5ce2..53d9e7bf41 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java @@ -101,6 +101,13 @@ void register(final StateStore store, /** * Get the state store given the store name. * + *

The returned state store represent one shard of the overall state, which belongs to the current task. + * The returned shard of the state store may only be used by the current + * {@link org.apache.kafka.streams.kstream.Transformer}, {@link org.apache.kafka.streams.kstream.ValueTransformer}, + * or {@link org.apache.kafka.streams.kstream.ValueTransformerWithKey} instance. + * Sharing a shard across different transformers (ie, from different "sibling" tasks; same sub-topology but different + * partition) may lead to data corruption and/or data loss. + * * @param name The store name * @param The type or interface of the store to return * @return The state store instance diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/api/ProcessingContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/api/ProcessingContext.java index 78aa2c8489..7f6874c8c1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/api/ProcessingContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/api/ProcessingContext.java @@ -107,6 +107,12 @@ public interface ProcessingContext { /** * Get the state store given the store name. * + *

The returned state store represent one shard of the overall state, which belongs to the current task. + * The returned shard of the state store may only be used by the current {@link Processor} or + * {@link FixedKeyProcessor} instance. + * Sharing a shard across different processors (ie, from different "sibling" tasks; same sub-topology but different + * partition) may lead to data corruption and/or data loss. + * * @param name The store name * @param The type or interface of the store to return * @return The state store instance From 45327fd597b68ed4651361ecad913f7c719fffeb Mon Sep 17 00:00:00 2001 From: Gaurav Narula Date: Fri, 4 Jul 2025 22:00:10 +0100 Subject: [PATCH 51/72] KAFKA-18035: Backport TransactionsTest testBumpTransactionalEpochWithTV2Disabled failed on trunk (#20102) Backports the flakyness fix in #18451 to 4.0 branch > Sometimes we didn't get into abortable state before aborting, so the epoch didn't get bumped. Now we force abortable state with an attempt to send before aborting so the epoch bump occurs as expected. > > Reviewers: Jeff Kim Reviewers: Chia-Ping Tsai Co-authored-by: Justine Olshan --- .../kafka/api/TransactionsTest.scala | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala index 7473766812..c531f62595 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala @@ -21,7 +21,7 @@ import kafka.utils.TestUtils.{consumeRecords, waitUntilTrue} import kafka.utils.{TestInfoUtils, TestUtils} import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.errors.{ConcurrentTransactionsException, InvalidProducerEpochException, ProducerFencedException, TimeoutException} import org.apache.kafka.common.test.api.Flaky import org.apache.kafka.coordinator.group.GroupCoordinatorConfig @@ -738,6 +738,19 @@ class TransactionsTest extends IntegrationTestHarness { restartDeadBrokers() org.apache.kafka.test.TestUtils.assertFutureThrows(failedFuture, classOf[TimeoutException]) + // Ensure the producer transitions to abortable_error state. + TestUtils.waitUntilTrue(() => { + var failed = false + try { + producer.send(TestUtils.producerRecordWithExpectedTransactionStatus(testTopic, 0, "3", "3", willBeCommitted = false)) + } catch { + case e: Exception => + if (e.isInstanceOf[KafkaException]) + failed = true + } + failed + }, "The send request never failed as expected.") + assertThrows(classOf[KafkaException], () => producer.send(TestUtils.producerRecordWithExpectedTransactionStatus(testTopic, 0, "3", "3", willBeCommitted = false))) producer.abortTransaction() producer.beginTransaction() @@ -760,7 +773,7 @@ class TransactionsTest extends IntegrationTestHarness { producerStateEntry = brokers(partitionLeader).logManager.getLog(new TopicPartition(testTopic, 0)).get.producerStateManager.activeProducers.get(producerId) assertNotNull(producerStateEntry) - assertTrue(producerStateEntry.producerEpoch > initialProducerEpoch) + assertTrue(producerStateEntry.producerEpoch > initialProducerEpoch, "InitialProduceEpoch: " + initialProducerEpoch + " ProducerStateEntry: " + producerStateEntry) } finally { producer.close(Duration.ZERO) } From 8433ac4d31aa4154b7a7fe3b290c4e2d8d847068 Mon Sep 17 00:00:00 2001 From: Gaurav Narula Date: Sat, 5 Jul 2025 20:05:34 +0100 Subject: [PATCH 52/72] KAFKA-19221 Propagate IOException on LogSegment#close (#20072) Log segment closure results in right sizing the segment on disk along with the associated index files. This is specially important for TimeIndexes where a failure to right size may eventually cause log roll failures leading to under replication and log cleaner failures. This change uses `Utils.closeAll` which propagates exceptions, resulting in an "unclean" shutdown. That would then cause the broker to attempt to recover the log segment and the index on next startup, thereby avoiding the failures described above. Reviewers: Omnia Ibrahim , Jun Rao , Chia-Ping Tsai --- .../kafka/common/record/FileRecords.java | 4 ++ .../server/LogManagerIntegrationTest.java | 66 +++++++++++++++++++ .../storage/internals/log/AbstractIndex.java | 4 +- .../storage/internals/log/LogSegment.java | 5 +- .../storage/internals/log/LogSegments.java | 4 +- .../internals/log/TransactionIndex.java | 2 +- .../internals/log/LogSegmentsTest.java | 23 ++++++- .../internals/log/OffsetIndexTest.java | 1 - 8 files changed, 99 insertions(+), 10 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java index 64dd73de41..ba5cb556ce 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java @@ -201,6 +201,10 @@ public void flush() throws IOException { * Close this record set */ public void close() throws IOException { + if (!channel.isOpen()) { + return; + } + flush(); trim(); channel.close(); diff --git a/core/src/test/java/kafka/server/LogManagerIntegrationTest.java b/core/src/test/java/kafka/server/LogManagerIntegrationTest.java index 3d38628394..752f56fa41 100644 --- a/core/src/test/java/kafka/server/LogManagerIntegrationTest.java +++ b/core/src/test/java/kafka/server/LogManagerIntegrationTest.java @@ -34,9 +34,11 @@ import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.Type; +import org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler; import org.apache.kafka.storage.internals.checkpoint.PartitionMetadataFile; import org.apache.kafka.test.TestUtils; +import java.io.File; import java.io.IOException; import java.time.Duration; import java.util.ArrayList; @@ -59,6 +61,70 @@ public LogManagerIntegrationTest(ClusterInstance cluster) { this.cluster = cluster; } + @ClusterTest(types = {Type.KRAFT}) + public void testIOExceptionOnLogSegmentCloseResultsInRecovery() throws IOException, InterruptedException, ExecutionException { + try (Admin admin = cluster.admin()) { + admin.createTopics(List.of(new NewTopic("foo", 1, (short) 1))).all().get(); + } + cluster.waitForTopic("foo", 1); + + // Produce some data into the topic + Map producerConfigs = Map.of( + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers(), + ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName(), + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName() + ); + + try (Producer producer = new KafkaProducer<>(producerConfigs)) { + producer.send(new ProducerRecord<>("foo", 0, null, "bar")).get(); + producer.flush(); + } + + var broker = cluster.brokers().get(0); + + File timeIndexFile = broker.logManager() + .getLog(new TopicPartition("foo", 0), false).get() + .activeSegment() + .timeIndexFile(); + + // Set read only so that we throw an IOException on shutdown + assertTrue(timeIndexFile.exists()); + assertTrue(timeIndexFile.setReadOnly()); + + broker.shutdown(); + + assertEquals(1, broker.config().logDirs().size()); + String logDir = broker.config().logDirs().head(); + CleanShutdownFileHandler cleanShutdownFileHandler = new CleanShutdownFileHandler(logDir); + assertFalse(cleanShutdownFileHandler.exists(), "Did not expect the clean shutdown file to exist"); + + // Ensure we have a corrupt index on broker shutdown + long maxIndexSize = broker.config().logIndexSizeMaxBytes(); + long expectedIndexSize = 12 * (maxIndexSize / 12); + assertEquals(expectedIndexSize, timeIndexFile.length()); + + // Allow write permissions before startup + assertTrue(timeIndexFile.setWritable(true)); + + broker.startup(); + // make sure there is no error during load logs + assertTrue(cluster.firstFatalException().isEmpty()); + try (Admin admin = cluster.admin()) { + TestUtils.waitForCondition(() -> { + List partitionInfos = admin.describeTopics(List.of("foo")) + .topicNameValues().get("foo").get().partitions(); + return partitionInfos.get(0).leader().id() == 0; + }, "Partition does not have a leader assigned"); + } + + // Ensure that sanity check does not fail + broker.logManager() + .getLog(new TopicPartition("foo", 0), false).get() + .activeSegment() + .timeIndex() + .sanityCheck(); + } + @ClusterTest(types = {Type.KRAFT, Type.CO_KRAFT}, brokers = 3) public void testRestartBrokerNoErrorIfMissingPartitionMetadata() throws IOException, ExecutionException, InterruptedException { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/AbstractIndex.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/AbstractIndex.java index 9c7c645c4a..46ceb4801a 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/AbstractIndex.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/AbstractIndex.java @@ -263,7 +263,9 @@ public boolean deleteIfExists() throws IOException { public void trimToValidSize() throws IOException { lock.lock(); try { - resize(entrySize() * entries); + if (mmap != null) { + resize(entrySize() * entries); + } } finally { lock.unlock(); } 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 e07d8a2d6a..b404f8d079 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 @@ -763,10 +763,7 @@ public Optional findOffsetByTimestamp(long times public void close() throws IOException { if (maxTimestampAndOffsetSoFar != TimestampOffset.UNKNOWN) Utils.swallow(LOGGER, Level.WARN, "maybeAppend", () -> timeIndex().maybeAppend(maxTimestampSoFar(), shallowOffsetOfMaxTimestampSoFar(), true)); - Utils.closeQuietly(lazyOffsetIndex, "offsetIndex", LOGGER); - Utils.closeQuietly(lazyTimeIndex, "timeIndex", LOGGER); - Utils.closeQuietly(log, "log", LOGGER); - Utils.closeQuietly(txnIndex, "txnIndex", LOGGER); + Utils.closeAll(lazyOffsetIndex, lazyTimeIndex, log, txnIndex); } /** diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegments.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegments.java index 586d40e90b..1987e99e98 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegments.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegments.java @@ -17,6 +17,7 @@ package org.apache.kafka.storage.internals.log; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Utils; import java.io.Closeable; import java.io.File; @@ -105,8 +106,7 @@ public void clear() { */ @Override public void close() throws IOException { - for (LogSegment s : values()) - s.close(); + Utils.closeAll(values().toArray(new LogSegment[0])); } /** diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/TransactionIndex.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/TransactionIndex.java index 8e089dc3cf..1640312681 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/TransactionIndex.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/TransactionIndex.java @@ -110,7 +110,7 @@ public void reset() throws IOException { public void close() throws IOException { FileChannel channel = channelOrNull(); - if (channel != null) + if (channel != null && channel.isOpen()) channel.close(); maybeChannel = Optional.empty(); } diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentsTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentsTest.java index 43da918b29..97ffb7072b 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentsTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentsTest.java @@ -40,7 +40,10 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; public class LogSegmentsTest { @@ -49,7 +52,7 @@ public class LogSegmentsTest { /* create a segment with the given base offset */ private static LogSegment createSegment(Long offset) throws IOException { - return LogTestUtils.createSegment(offset, logDir, 10, Time.SYSTEM); + return spy(LogTestUtils.createSegment(offset, logDir, 10, Time.SYSTEM)); } @BeforeEach @@ -276,4 +279,22 @@ public void testUpdateDir() throws IOException { } } + @Test + public void testCloseClosesAllLogSegmentsOnExceptionWhileClosingOne() throws IOException { + LogSegment seg1 = createSegment(0L); + LogSegment seg2 = createSegment(100L); + LogSegment seg3 = createSegment(200L); + LogSegments segments = new LogSegments(topicPartition); + segments.add(seg1); + segments.add(seg2); + segments.add(seg3); + + doThrow(new IOException("Failure")).when(seg2).close(); + + assertThrows(IOException.class, segments::close, "Expected IOException to be thrown"); + verify(seg1).close(); + verify(seg2).close(); + verify(seg3).close(); + } + } diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/OffsetIndexTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/OffsetIndexTest.java index 918e9dd409..ad7fa59085 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/OffsetIndexTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/OffsetIndexTest.java @@ -225,7 +225,6 @@ public void forceUnmapTest() throws IOException { idx.forceUnmap(); // mmap should be null after unmap causing lookup to throw a NPE assertThrows(NullPointerException.class, () -> idx.lookup(1)); - assertThrows(NullPointerException.class, idx::close); } @Test From 80b9abebade920148b4295a1bccf84005c5989c4 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 7 Jul 2025 08:44:24 -0700 Subject: [PATCH 53/72] KAFKA-19444: Add back JoinGroup v0 & v1 (#20116) This fixes librdkafka older than the recently released 2.11.0 with Kerberos authentication and Apache Kafka 4.x. Even though this is a bug in librdkafka, a key goal of KIP-896 is not to break the popular client libraries listed in it. Adding back JoinGroup v0 & v1 is a very small change and worth it from that perspective. Reviewers: Chia-Ping Tsai --- .../common/message/JoinGroupRequest.json | 4 +--- .../common/message/JoinGroupResponse.json | 4 +--- .../common/requests/JoinGroupRequestTest.java | 19 +++++++++++++++++++ .../common/requests/RequestResponseTest.java | 8 ++++++++ 4 files changed, 29 insertions(+), 6 deletions(-) diff --git a/clients/src/main/resources/common/message/JoinGroupRequest.json b/clients/src/main/resources/common/message/JoinGroupRequest.json index 41d7c1acba..31afdb1a32 100644 --- a/clients/src/main/resources/common/message/JoinGroupRequest.json +++ b/clients/src/main/resources/common/message/JoinGroupRequest.json @@ -18,8 +18,6 @@ "type": "request", "listeners": ["broker"], "name": "JoinGroupRequest", - // Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline. - // // Version 1 adds RebalanceTimeoutMs. Version 2 and 3 are the same as version 1. // // Starting from version 4, the client needs to issue a second request to join group @@ -34,7 +32,7 @@ // Version 8 adds the Reason field (KIP-800). // // Version 9 is the same as version 8. - "validVersions": "2-9", + "validVersions": "0-9", "flexibleVersions": "6+", "fields": [ { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId", diff --git a/clients/src/main/resources/common/message/JoinGroupResponse.json b/clients/src/main/resources/common/message/JoinGroupResponse.json index 364309596e..d2f016f62f 100644 --- a/clients/src/main/resources/common/message/JoinGroupResponse.json +++ b/clients/src/main/resources/common/message/JoinGroupResponse.json @@ -17,8 +17,6 @@ "apiKey": 11, "type": "response", "name": "JoinGroupResponse", - // Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline. - // // Version 1 is the same as version 0. // // Version 2 adds throttle time. @@ -37,7 +35,7 @@ // Version 8 is the same as version 7. // // Version 9 adds the SkipAssignment field. - "validVersions": "2-9", + "validVersions": "0-9", "flexibleVersions": "6+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true, diff --git a/clients/src/test/java/org/apache/kafka/common/requests/JoinGroupRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/JoinGroupRequestTest.java index 60d10a6893..a330190895 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/JoinGroupRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/JoinGroupRequestTest.java @@ -19,12 +19,15 @@ import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.message.JoinGroupRequestData; +import org.apache.kafka.common.protocol.MessageUtil; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Test; +import java.nio.ByteBuffer; import java.util.Arrays; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.fail; @@ -65,4 +68,20 @@ public void testRequestVersionCompatibilityFailBuild() { .setProtocolType("consumer") ).build((short) 4)); } + + @Test + public void testRebalanceTimeoutDefaultsToSessionTimeoutV0() { + int sessionTimeoutMs = 30000; + short version = 0; + + ByteBuffer buffer = MessageUtil.toByteBuffer(new JoinGroupRequestData() + .setGroupId("groupId") + .setMemberId("consumerId") + .setProtocolType("consumer") + .setSessionTimeoutMs(sessionTimeoutMs), version); + + JoinGroupRequest request = JoinGroupRequest.parse(buffer, version); + assertEquals(sessionTimeoutMs, request.data().sessionTimeoutMs()); + assertEquals(sessionTimeoutMs, request.data().rebalanceTimeoutMs()); + } } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 6578302e81..ad52482a3b 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -665,6 +665,14 @@ public void testFetchRequestWithMetadata() { assertEquals(request.isolationLevel(), deserialized.isolationLevel()); } + @Test + public void testJoinGroupRequestV0RebalanceTimeout() { + final short version = 0; + JoinGroupRequest jgr = createJoinGroupRequest(version); + JoinGroupRequest jgr2 = JoinGroupRequest.parse(jgr.serialize(), version); + assertEquals(jgr2.data().rebalanceTimeoutMs(), jgr.data().rebalanceTimeoutMs()); + } + @Test public void testSerializeWithHeader() { CreatableTopicCollection topicsToCreate = new CreatableTopicCollection(1); From d95857a155943013fe6da5d613b505e6e9688c16 Mon Sep 17 00:00:00 2001 From: Bill Bejeck Date: Mon, 14 Jul 2025 20:19:16 -0400 Subject: [PATCH 54/72] KAFKA-19504: Remove unused metrics reporter initialization in KafkaAdminClient (#20166) The `AdminClient` adds a telemetry reporter to the metrics reporters list in the constructor. The problem is that the reporter was already added in the `createInternal` method. In the `createInternal` method call, the `clientTelemetryReporter` is added to a `List` which is passed to the `Metrics` object, will get closed when `Metrics.close()` is called. But adding a reporter to the reporters list in the constructor is not used by the `Metrics` object and hence doesn't get closed, causing a memory leak. All related tests pass after this change. Reviewers: Apoorv Mittal , Matthias J. Sax , Chia-Ping Tsai , Jhen-Yung Hsu --- .../org/apache/kafka/clients/admin/KafkaAdminClient.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index dc3164993b..12e122b123 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -577,10 +577,12 @@ static KafkaAdminClient createInternal(AdminClientConfig config, Time time) { Metrics metrics = null; String clientId = generateClientId(config); + List reporters = CommonClientConfigs.metricsReporters(clientId, config); Optional clientTelemetryReporter = CommonClientConfigs.telemetryReporter(clientId, config); + clientTelemetryReporter.ifPresent(reporters::add); try { - metrics = new Metrics(new MetricConfig(), new LinkedList<>(), time); + metrics = new Metrics(new MetricConfig(), reporters, time); LogContext logContext = createLogContext(clientId); return new KafkaAdminClient(config, clientId, time, metadataManager, metrics, client, null, logContext, clientTelemetryReporter); @@ -625,9 +627,7 @@ private KafkaAdminClient(AdminClientConfig config, CommonClientConfigs.RETRY_BACKOFF_EXP_BASE, retryBackoffMaxMs, CommonClientConfigs.RETRY_BACKOFF_JITTER); - List reporters = CommonClientConfigs.metricsReporters(this.clientId, config); this.clientTelemetryReporter = clientTelemetryReporter; - this.clientTelemetryReporter.ifPresent(reporters::add); this.metadataRecoveryStrategy = MetadataRecoveryStrategy.forName(config.getString(AdminClientConfig.METADATA_RECOVERY_STRATEGY_CONFIG)); this.partitionLeaderCache = new HashMap<>(); this.adminFetchMetricsManager = new AdminFetchMetricsManager(metrics); From eefee6d58d8bc426ab54d4bc05d49ddb5566c025 Mon Sep 17 00:00:00 2001 From: Ming-Yen Chung Date: Wed, 16 Jul 2025 22:06:33 +0800 Subject: [PATCH 55/72] KAFKA-19427 Allow the coordinator to grow its buffer dynamically (#20040) * Coordinator starts with a smaller buffer, which can grow as needed. * In freeCurrentBatch, release the appropriate buffer: * The Coordinator recycles the expanded buffer (`currentBatch.builder.buffer()`), not `currentBatch.buffer`, because `MemoryBuilder` may allocate a new `ByteBuffer` if the existing one isn't large enough. * There are two cases that buffer may exceeds `maxMessageSize` 1. If there's a single record whose size exceeds `maxMessageSize` (which, so far, is derived from `max.message.bytes`) and the write is in `non-atomic` mode, it's still possible for the buffer to grow beyond `maxMessageSize`. In this case, the Coordinator should revert to using a smaller buffer afterward. 2. Coordinator do not recycles the buffer that larger than `maxMessageSize`. If the user dynamically reduces `maxMessageSize` to a value even smaller than `INITIAL_BUFFER_SIZE`, the Coordinator should avoid recycling any buffer larger than `maxMessageSize` so that Coordinator can allocate the smaller buffer in the next round. * Add tests to verify the above scenarios. Reviewers: David Jacot , Sean Quah , Ken Huang , PoAn Yang , TaiJuWu , Jhen-Yung Hsu , Chia-Ping Tsai --- .../common/runtime/CoordinatorRuntime.java | 17 +- .../runtime/CoordinatorRuntimeTest.java | 215 +++++++++++++++++- 2 files changed, 222 insertions(+), 10 deletions(-) diff --git a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java index 1e9724a57a..e1e80476cf 100644 --- a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java +++ b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java @@ -70,6 +70,7 @@ import java.util.function.Consumer; import java.util.stream.Collectors; +import static java.lang.Math.min; import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime.CoordinatorWriteEvent.NOT_QUEUED; /** @@ -758,8 +759,14 @@ private void freeCurrentBatch() { // Cancel the linger timeout. currentBatch.lingerTimeoutTask.ifPresent(TimerTask::cancel); - // Release the buffer. - bufferSupplier.release(currentBatch.buffer); + // Release the buffer only if it is not larger than the maxBatchSize. + int maxBatchSize = partitionWriter.config(tp).maxMessageSize(); + + if (currentBatch.builder.buffer().capacity() <= maxBatchSize) { + bufferSupplier.release(currentBatch.builder.buffer()); + } else if (currentBatch.buffer.capacity() <= maxBatchSize) { + bufferSupplier.release(currentBatch.buffer); + } currentBatch = null; } @@ -859,7 +866,7 @@ private void maybeAllocateNewBatch( LogConfig logConfig = partitionWriter.config(tp); int maxBatchSize = logConfig.maxMessageSize(); long prevLastWrittenOffset = coordinator.lastWrittenOffset(); - ByteBuffer buffer = bufferSupplier.get(maxBatchSize); + ByteBuffer buffer = bufferSupplier.get(min(INITIAL_BUFFER_SIZE, maxBatchSize)); MemoryRecordsBuilder builder = new MemoryRecordsBuilder( buffer, @@ -1888,9 +1895,9 @@ public void onHighWatermarkUpdated( } /** - * 16KB. Used for initial buffer size for write operations. + * 512KB. Used for initial buffer size for write operations. */ - static final int MIN_BUFFER_SIZE = 16384; + static final int INITIAL_BUFFER_SIZE = 512 * 1024; /** * The log prefix. diff --git a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java index 9e4e6f7bb9..9198e207e4 100644 --- a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java +++ b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.KafkaException; 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.errors.NotCoordinatorException; import org.apache.kafka.common.errors.NotEnoughReplicasException; import org.apache.kafka.common.errors.RecordTooLargeException; @@ -65,6 +66,7 @@ import java.util.Deque; import java.util.LinkedList; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.OptionalInt; import java.util.Set; @@ -84,7 +86,7 @@ import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime.CoordinatorState.INITIAL; import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime.CoordinatorState.LOADING; import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime.HighWatermarkListener.NO_OFFSET; -import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime.MIN_BUFFER_SIZE; +import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime.INITIAL_BUFFER_SIZE; import static org.apache.kafka.test.TestUtils.assertFutureThrows; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -3486,11 +3488,11 @@ public void testAppendRecordBatchSize() { assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); int maxBatchSize = writer.config(TP).maxMessageSize(); - assertTrue(maxBatchSize > MIN_BUFFER_SIZE); + assertTrue(maxBatchSize > INITIAL_BUFFER_SIZE); - // Generate enough records to create a batch that has 16KB < batchSize < maxBatchSize + // Generate enough records to create a batch that has INITIAL_BUFFER_SIZE < batchSize < maxBatchSize List records = new ArrayList<>(); - for (int i = 0; i < 3000; i++) { + for (int i = 0; i < 50000; i++) { records.add("record-" + i); } @@ -3504,7 +3506,210 @@ public void testAppendRecordBatchSize() { assertFalse(write1.isCompletedExceptionally()); int batchSize = writer.entries(TP).get(0).sizeInBytes(); - assertTrue(batchSize > MIN_BUFFER_SIZE && batchSize < maxBatchSize); + assertTrue(batchSize > INITIAL_BUFFER_SIZE && batchSize < maxBatchSize); + } + + @Test + public void testCoordinatorDoNotRetainBufferLargeThanMaxMessageSize() { + MockTimer timer = new MockTimer(); + InMemoryPartitionWriter mockWriter = new InMemoryPartitionWriter(false) { + @Override + public LogConfig config(TopicPartition tp) { + return new LogConfig(Map.of( + TopicConfig.MAX_MESSAGE_BYTES_CONFIG, String.valueOf(1024 * 1024) // 1MB + )); + } + }; + StringSerializer serializer = new StringSerializer(); + + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(DEFAULT_WRITE_TIMEOUT) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(new DirectEventProcessor()) + .withPartitionWriter(mockWriter) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) + .withSerializer(serializer) + .withExecutorService(mock(ExecutorService.class)) + .build(); + + // Schedule the loading. + runtime.scheduleLoadOperation(TP, 10); + + // Verify the initial state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); + + // Generate a record larger than the maxBatchSize. + List largeRecords = List.of("A".repeat(100 * 1024 * 1024)); + + // Write #1. + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, DEFAULT_WRITE_TIMEOUT, + state -> new CoordinatorResult<>(largeRecords, "response1", null, true, false) + ); + + // Verify that the write has not completed exceptionally. + // This will catch any exceptions thrown including RecordTooLargeException. + assertFalse(write1.isCompletedExceptionally()); + + // Verify that the next buffer retrieved from the bufferSupplier is the initial small one, not the large buffer. + assertEquals(INITIAL_BUFFER_SIZE, ctx.bufferSupplier.get(1).capacity()); + } + + @Test + public void testCoordinatorRetainExpandedBufferLessOrEqualToMaxMessageSize() { + MockTimer timer = new MockTimer(); + InMemoryPartitionWriter mockWriter = new InMemoryPartitionWriter(false) { + @Override + public LogConfig config(TopicPartition tp) { + return new LogConfig(Map.of( + TopicConfig.MAX_MESSAGE_BYTES_CONFIG, String.valueOf(1024 * 1024 * 1024) // 1GB + )); + } + }; + StringSerializer serializer = new StringSerializer(); + + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(DEFAULT_WRITE_TIMEOUT) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(new DirectEventProcessor()) + .withPartitionWriter(mockWriter) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) + .withSerializer(serializer) + .withExecutorService(mock(ExecutorService.class)) + .build(); + + // Schedule the loading. + runtime.scheduleLoadOperation(TP, 10); + + // Verify the initial state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); + + // Generate enough records to create a batch that has INITIAL_BUFFER_SIZE < batchSize < maxBatchSize + List records = new ArrayList<>(); + for (int i = 0; i < 1000000; i++) { + records.add("record-" + i); + } + + // Write #1. + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, DEFAULT_WRITE_TIMEOUT, + state -> new CoordinatorResult<>(records, "response1") + ); + + // Verify that the write has not completed exceptionally. + // This will catch any exceptions thrown including RecordTooLargeException. + assertFalse(write1.isCompletedExceptionally()); + + int batchSize = mockWriter.entries(TP).get(0).sizeInBytes(); + int maxBatchSize = mockWriter.config(TP).maxMessageSize(); + assertTrue(INITIAL_BUFFER_SIZE < batchSize && batchSize <= maxBatchSize); + + // Verify that the next buffer retrieved from the bufferSupplier is the expanded buffer. + assertTrue(ctx.bufferSupplier.get(1).capacity() > INITIAL_BUFFER_SIZE); + } + + @Test + public void testBufferShrinkWhenMaxMessageSizeReducedBelowInitialBufferSize() { + MockTimer timer = new MockTimer(); + var mockWriter = new InMemoryPartitionWriter(false) { + private LogConfig config = new LogConfig(Map.of( + TopicConfig.MAX_MESSAGE_BYTES_CONFIG, String.valueOf(1024 * 1024) // 1MB + )); + + @Override + public LogConfig config(TopicPartition tp) { + return config; + } + + public void updateConfig(LogConfig newConfig) { + this.config = newConfig; + } + }; + StringSerializer serializer = new StringSerializer(); + + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(DEFAULT_WRITE_TIMEOUT) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(new DirectEventProcessor()) + .withPartitionWriter(mockWriter) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) + .withSerializer(serializer) + .withExecutorService(mock(ExecutorService.class)) + .build(); + + // Schedule the loading. + runtime.scheduleLoadOperation(TP, 10); + + // Verify the initial state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); + + List records = new ArrayList<>(); + for (int i = 0; i < 1000; i++) { + records.add("record-" + i); + } + + // Write #1. + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, DEFAULT_WRITE_TIMEOUT, + state -> new CoordinatorResult<>(records, "response1") + ); + + // Verify that the write has not completed exceptionally. + // This will catch any exceptions thrown including RecordTooLargeException. + assertFalse(write1.isCompletedExceptionally()); + + int batchSize = mockWriter.entries(TP).get(0).sizeInBytes(); + int maxBatchSize = mockWriter.config(TP).maxMessageSize(); + assertTrue(batchSize <= INITIAL_BUFFER_SIZE && INITIAL_BUFFER_SIZE <= maxBatchSize); + + ByteBuffer cachedBuffer = ctx.bufferSupplier.get(1); + assertEquals(INITIAL_BUFFER_SIZE, cachedBuffer.capacity()); + // ctx.bufferSupplier.get(1); will clear cachedBuffer in bufferSupplier. Use release to put it back to bufferSupplier + ctx.bufferSupplier.release(cachedBuffer); + + // Reduce max message size below initial buffer size. + mockWriter.updateConfig(new LogConfig( + Map.of(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, String.valueOf(INITIAL_BUFFER_SIZE - 66)))); + assertEquals(INITIAL_BUFFER_SIZE - 66, mockWriter.config(TP).maxMessageSize()); + + // Write #2. + CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, DEFAULT_WRITE_TIMEOUT, + state -> new CoordinatorResult<>(records, "response2") + ); + assertFalse(write2.isCompletedExceptionally()); + + // Verify that there is no cached buffer since the cached buffer size is greater than new maxMessageSize. + assertEquals(1, ctx.bufferSupplier.get(1).capacity()); + + // Write #3. + CompletableFuture write3 = runtime.scheduleWriteOperation("write#3", TP, DEFAULT_WRITE_TIMEOUT, + state -> new CoordinatorResult<>(records, "response3") + ); + assertFalse(write3.isCompletedExceptionally()); + + // Verify that the cached buffer size is equals to new maxMessageSize that less than INITIAL_BUFFER_SIZE. + assertEquals(mockWriter.config(TP).maxMessageSize(), ctx.bufferSupplier.get(1).capacity()); } @Test From 70c51641fbf54c3cea12b2ab2d730683813d00e8 Mon Sep 17 00:00:00 2001 From: Kaushik Raina <103954755+k-raina@users.noreply.github.com> Date: Wed, 16 Jul 2025 22:08:02 +0530 Subject: [PATCH 56/72] Cherrypick "MINOR : Handle error for client telemetry push (#19881)" (#20176) Update catch to handle compression errors Before : ![image](https://github.com/user-attachments/assets/c5ca121e-ba0c-4664-91f1-20b54abf67cc) After ``` Sent message: KR Message 376 [kafka-producer-network-thread | kr-kafka-producer] INFO org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter - KR: Failed to compress telemetry payload for compression: zstd, sending uncompressed data Sent message: KR Message 377 ``` Reviewers: Apoorv Mittal , Bill Bejeck , Chia-Ping Tsai --- .../common/telemetry/internals/ClientTelemetryReporter.java | 5 ++--- .../java/org/apache/kafka/server/ClientMetricsManager.java | 2 +- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java b/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java index 705aafaaa7..e0491943fe 100644 --- a/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java +++ b/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java @@ -41,7 +41,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.nio.ByteBuffer; import java.time.Duration; import java.util.Collections; @@ -718,8 +717,8 @@ private Optional> createPushRequest(ClientTelemetrySubscription local ByteBuffer compressedPayload; try { compressedPayload = ClientTelemetryUtils.compress(payload, compressionType); - } catch (IOException e) { - log.info("Failed to compress telemetry payload for compression: {}, sending uncompressed data", compressionType); + } catch (Throwable e) { + log.debug("Failed to compress telemetry payload for compression: {}, sending uncompressed data", compressionType); compressedPayload = ByteBuffer.wrap(payload.toByteArray()); compressionType = CompressionType.NONE; } diff --git a/server/src/main/java/org/apache/kafka/server/ClientMetricsManager.java b/server/src/main/java/org/apache/kafka/server/ClientMetricsManager.java index dcd17a3ecc..2487ccb6f3 100644 --- a/server/src/main/java/org/apache/kafka/server/ClientMetricsManager.java +++ b/server/src/main/java/org/apache/kafka/server/ClientMetricsManager.java @@ -217,7 +217,7 @@ public PushTelemetryResponse processPushTelemetryRequest(PushTelemetryRequest re long exportTimeStartMs = time.hiResClockMs(); receiverPlugin.exportMetrics(requestContext, request); clientMetricsStats.recordPluginExport(clientInstanceId, time.hiResClockMs() - exportTimeStartMs); - } catch (Exception exception) { + } catch (Throwable exception) { clientMetricsStats.recordPluginErrorCount(clientInstanceId); clientInstance.lastKnownError(Errors.INVALID_RECORD); log.error("Error exporting client metrics to the plugin for client instance id: {}", clientInstanceId, exception); From 12e695e29861a90ce14a541b50338e9db4ec978e Mon Sep 17 00:00:00 2001 From: Dmitry Werner Date: Sat, 19 Jul 2025 12:05:50 +0500 Subject: [PATCH 57/72] KAFKA-19520 Bump Commons-Lang for CVE-2025-48924 (#20196) Bump Commons-Lang for CVE-2025-48924. Reviewers: Luke Chen , Federico Valeri --- LICENSE-binary | 2 +- build.gradle | 3 ++- gradle/dependencies.gradle | 2 ++ 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index 8ceda84ebe..b07b5e0472 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -209,7 +209,7 @@ License Version 2.0: - commons-beanutils-1.11.0 - commons-collections-3.2.2 - commons-digester-2.1 -- commons-lang3-3.12.0 +- commons-lang3-3.18.0 - commons-logging-1.3.5 - commons-validator-1.9.0 - jackson-annotations-2.16.2 diff --git a/build.gradle b/build.gradle index 05d38aface..ed59bbe0dd 100644 --- a/build.gradle +++ b/build.gradle @@ -199,7 +199,8 @@ allprojects { libs.scalaReflect, // Workaround before `commons-validator` has new release. See KAFKA-19359. libs.commonsBeanutils, - libs.jacksonAnnotations + libs.jacksonAnnotations, + libs.commonsLang ) } } diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 5debdea5a0..dd4aaa2216 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -61,6 +61,7 @@ versions += [ bndlib: "7.0.0", checkstyle: project.hasProperty('checkstyleVersion') ? checkstyleVersion : "10.20.2", commonsBeanutils: "1.11.0", + commonsLang: "3.18.0", commonsValidator: "1.9.0", classgraph: "4.8.173", gradle: "8.10.2", @@ -150,6 +151,7 @@ libs += [ caffeine: "com.github.ben-manes.caffeine:caffeine:$versions.caffeine", classgraph: "io.github.classgraph:classgraph:$versions.classgraph", commonsBeanutils: "commons-beanutils:commons-beanutils:$versions.commonsBeanutils", + commonsLang: "org.apache.commons:commons-lang3:$versions.commonsLang", commonsValidator: "commons-validator:commons-validator:$versions.commonsValidator", jacksonAnnotations: "com.fasterxml.jackson.core:jackson-annotations:$versions.jackson", jacksonDatabind: "com.fasterxml.jackson.core:jackson-databind:$versions.jackson", From 74d93adadb74ff682882b31dde03fea07fb4af97 Mon Sep 17 00:00:00 2001 From: "Tsung-Han Ho (Miles Ho)" Date: Thu, 17 Jul 2025 03:27:07 +0800 Subject: [PATCH 58/72] KAFKA-19501 Update OpenJDK base image from buster to bullseye (#20165) The changes update the OpenJDK base image from 17-buster to 17-bullseye: - Updates tests/docker/Dockerfile to use openjdk:17-bullseye instead of openjdk:17-buster - Updates tests/docker/ducker-ak script to use the new default image - Updates documentation in tests/README.md with the new image name examples Reviewers: Federico Valeri , TengYao Chi , Ken Huang , Chia-Ping Tsai --- tests/README.md | 5 +++-- tests/docker/Dockerfile | 4 +--- tests/docker/ducker-ak | 11 ++++++++--- 3 files changed, 12 insertions(+), 8 deletions(-) diff --git a/tests/README.md b/tests/README.md index 7a1d4e7339..a64946c892 100644 --- a/tests/README.md +++ b/tests/README.md @@ -49,12 +49,13 @@ TC_PATHS="tests/kafkatest/tests/streams/streams_upgrade_test.py::StreamsUpgradeT ``` * Run tests with a specific image name ``` -image_name="ducker-ak-openjdk:17-buster" bash tests/docker/run_tests.sh +image_name="ducker-ak-openjdk:17" bash tests/docker/run_tests.sh ``` * Run tests with a different JVM ``` -bash tests/docker/ducker-ak up -j 'openjdk:17-buster'; tests/docker/run_tests.sh +bash tests/docker/ducker-ak up -j ''; tests/docker/run_tests.sh ``` + You can customize the OpenJDK base image using the `-j` or `--jdk` parameter, otherwise a default value will be used. * Remove ducker-ak containers ``` bash tests/docker/ducker-ak down -f diff --git a/tests/docker/Dockerfile b/tests/docker/Dockerfile index 7dce5ef86d..9fcca16b94 100644 --- a/tests/docker/Dockerfile +++ b/tests/docker/Dockerfile @@ -13,9 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -# The base image of openjdk:17 is typically oraclelinux:8-slim, which doesn't include apt-get. -# Therefore, use openjdk:17-buster instead. -ARG jdk_version=openjdk:17-buster +ARG jdk_version FROM $jdk_version AS build-native-image WORKDIR /build diff --git a/tests/docker/ducker-ak b/tests/docker/ducker-ak index 8283438476..368e7ebe08 100755 --- a/tests/docker/ducker-ak +++ b/tests/docker/ducker-ak @@ -45,7 +45,9 @@ docker_run_memory_limit="2000m" default_num_nodes=14 # The default OpenJDK base image. -default_jdk="openjdk:17-buster" +# The base image of openjdk:17 is typically oraclelinux:8-slim, which doesn't include apt-get. +# Therefore, use openjdk:17-bullseye instead. +default_jdk="openjdk:17-bullseye" # The default ducker-ak image name. default_image_name="ducker-ak" @@ -70,7 +72,7 @@ help|-h|--help Display this help message up [-n|--num-nodes NUM_NODES] [-f|--force] [docker-image] - [-C|--custom-ducktape DIR] [-e|--expose-ports ports] + [-C|--custom-ducktape DIR] [-e|--expose-ports ports] [-j|--jdk JDK_VERSION] Bring up a cluster with the specified amount of nodes (defaults to ${default_num_nodes}). The docker image name defaults to ${default_image_name}. If --force is specified, we will attempt to bring up an image even some parameters are not valid. @@ -83,7 +85,10 @@ up [-n|--num-nodes NUM_NODES] [-f|--force] [docker-image] on the host. The argument can be a single port (like 5005), a port range like (5005-5009) or a combination of port/port-range separated by comma (like 2181,9092 or 2181,5005-5008). By default no port is exposed. See README.md for more detail on this option. - + + If -j|--jdk is specified, you can customize the OpenJDK base image used for building + the ducker container. Defaults to ${default_jdk}. Example: -j openjdk:17-bullseye + Note that port 5678 will be automatically exposed for ducker01 node and will be mapped to 5678 on your local machine to enable debugging in VS Code. From b3aeb69cb4c84a4ade32083216be987829b21574 Mon Sep 17 00:00:00 2001 From: Christo Date: Thu, 24 Jul 2025 12:34:54 +0100 Subject: [PATCH 59/72] Bump version to 4.0.1 --- gradle.properties | 2 +- streams/quickstart/java/pom.xml | 2 +- .../java/src/main/resources/archetype-resources/pom.xml | 2 +- streams/quickstart/pom.xml | 2 +- tests/kafkatest/__init__.py | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/gradle.properties b/gradle.properties index 04ea428dd3..3bd3cee6c7 100644 --- a/gradle.properties +++ b/gradle.properties @@ -23,7 +23,7 @@ group=org.apache.kafka # - streams/quickstart/pom.xml # - streams/quickstart/java/src/main/resources/archetype-resources/pom.xml # - streams/quickstart/java/pom.xml -version=4.0.1-SNAPSHOT +version=4.0.1 scalaVersion=2.13.15 # Adding swaggerVersion in gradle.properties to have a single version in place for swagger swaggerVersion=2.2.25 diff --git a/streams/quickstart/java/pom.xml b/streams/quickstart/java/pom.xml index ffbf57e2fd..312b2029f3 100644 --- a/streams/quickstart/java/pom.xml +++ b/streams/quickstart/java/pom.xml @@ -26,7 +26,7 @@ org.apache.kafka streams-quickstart - 4.0.1-SNAPSHOT + 4.0.1 .. diff --git a/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml b/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml index 14bf05d7f9..d4ed1f7c18 100644 --- a/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml +++ b/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml @@ -29,7 +29,7 @@ UTF-8 - 4.0.1-SNAPSHOT + 4.0.1 2.0.16 diff --git a/streams/quickstart/pom.xml b/streams/quickstart/pom.xml index 7f267a1828..794363226c 100644 --- a/streams/quickstart/pom.xml +++ b/streams/quickstart/pom.xml @@ -22,7 +22,7 @@ org.apache.kafka streams-quickstart pom - 4.0.1-SNAPSHOT + 4.0.1 Kafka Streams :: Quickstart diff --git a/tests/kafkatest/__init__.py b/tests/kafkatest/__init__.py index 2f93446fdc..56da891c65 100644 --- a/tests/kafkatest/__init__.py +++ b/tests/kafkatest/__init__.py @@ -22,4 +22,4 @@ # Instead, in development branches, the version should have a suffix of the form ".devN" # # For example, when Kafka is at version 1.0.0-SNAPSHOT, this should be something like "1.0.0.dev0" -__version__ = '4.0.1.dev0' +__version__ = '4.0.1' From 01d6edc510d03c51ab5dac22251233c3c4e9de8f Mon Sep 17 00:00:00 2001 From: Christo Date: Thu, 24 Jul 2025 12:39:43 +0100 Subject: [PATCH 60/72] Bump version to 4.0.1 --- gradle.properties | 2 +- streams/quickstart/java/pom.xml | 2 +- .../java/src/main/resources/archetype-resources/pom.xml | 2 +- streams/quickstart/pom.xml | 2 +- tests/kafkatest/__init__.py | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/gradle.properties b/gradle.properties index 04ea428dd3..3bd3cee6c7 100644 --- a/gradle.properties +++ b/gradle.properties @@ -23,7 +23,7 @@ group=org.apache.kafka # - streams/quickstart/pom.xml # - streams/quickstart/java/src/main/resources/archetype-resources/pom.xml # - streams/quickstart/java/pom.xml -version=4.0.1-SNAPSHOT +version=4.0.1 scalaVersion=2.13.15 # Adding swaggerVersion in gradle.properties to have a single version in place for swagger swaggerVersion=2.2.25 diff --git a/streams/quickstart/java/pom.xml b/streams/quickstart/java/pom.xml index ffbf57e2fd..312b2029f3 100644 --- a/streams/quickstart/java/pom.xml +++ b/streams/quickstart/java/pom.xml @@ -26,7 +26,7 @@ org.apache.kafka streams-quickstart - 4.0.1-SNAPSHOT + 4.0.1 .. diff --git a/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml b/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml index 14bf05d7f9..d4ed1f7c18 100644 --- a/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml +++ b/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml @@ -29,7 +29,7 @@ UTF-8 - 4.0.1-SNAPSHOT + 4.0.1 2.0.16 diff --git a/streams/quickstart/pom.xml b/streams/quickstart/pom.xml index 7f267a1828..794363226c 100644 --- a/streams/quickstart/pom.xml +++ b/streams/quickstart/pom.xml @@ -22,7 +22,7 @@ org.apache.kafka streams-quickstart pom - 4.0.1-SNAPSHOT + 4.0.1 Kafka Streams :: Quickstart diff --git a/tests/kafkatest/__init__.py b/tests/kafkatest/__init__.py index 2f93446fdc..56da891c65 100644 --- a/tests/kafkatest/__init__.py +++ b/tests/kafkatest/__init__.py @@ -22,4 +22,4 @@ # Instead, in development branches, the version should have a suffix of the form ".devN" # # For example, when Kafka is at version 1.0.0-SNAPSHOT, this should be something like "1.0.0.dev0" -__version__ = '4.0.1.dev0' +__version__ = '4.0.1' From 4432d6a55800d0a7f943c9fe421186c32548de8b Mon Sep 17 00:00:00 2001 From: Christo Date: Thu, 24 Jul 2025 13:20:49 +0100 Subject: [PATCH 61/72] Bump version to 4.0.1 --- gradle.properties | 2 +- streams/quickstart/java/pom.xml | 2 +- .../java/src/main/resources/archetype-resources/pom.xml | 2 +- streams/quickstart/pom.xml | 2 +- tests/kafkatest/__init__.py | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/gradle.properties b/gradle.properties index 04ea428dd3..3bd3cee6c7 100644 --- a/gradle.properties +++ b/gradle.properties @@ -23,7 +23,7 @@ group=org.apache.kafka # - streams/quickstart/pom.xml # - streams/quickstart/java/src/main/resources/archetype-resources/pom.xml # - streams/quickstart/java/pom.xml -version=4.0.1-SNAPSHOT +version=4.0.1 scalaVersion=2.13.15 # Adding swaggerVersion in gradle.properties to have a single version in place for swagger swaggerVersion=2.2.25 diff --git a/streams/quickstart/java/pom.xml b/streams/quickstart/java/pom.xml index ffbf57e2fd..312b2029f3 100644 --- a/streams/quickstart/java/pom.xml +++ b/streams/quickstart/java/pom.xml @@ -26,7 +26,7 @@ org.apache.kafka streams-quickstart - 4.0.1-SNAPSHOT + 4.0.1 .. diff --git a/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml b/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml index 14bf05d7f9..d4ed1f7c18 100644 --- a/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml +++ b/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml @@ -29,7 +29,7 @@ UTF-8 - 4.0.1-SNAPSHOT + 4.0.1 2.0.16 diff --git a/streams/quickstart/pom.xml b/streams/quickstart/pom.xml index 7f267a1828..794363226c 100644 --- a/streams/quickstart/pom.xml +++ b/streams/quickstart/pom.xml @@ -22,7 +22,7 @@ org.apache.kafka streams-quickstart pom - 4.0.1-SNAPSHOT + 4.0.1 Kafka Streams :: Quickstart diff --git a/tests/kafkatest/__init__.py b/tests/kafkatest/__init__.py index 2f93446fdc..56da891c65 100644 --- a/tests/kafkatest/__init__.py +++ b/tests/kafkatest/__init__.py @@ -22,4 +22,4 @@ # Instead, in development branches, the version should have a suffix of the form ".devN" # # For example, when Kafka is at version 1.0.0-SNAPSHOT, this should be something like "1.0.0.dev0" -__version__ = '4.0.1.dev0' +__version__ = '4.0.1' From aaf1864d0cfee4880471e2ab42409285d600a199 Mon Sep 17 00:00:00 2001 From: Christo Date: Thu, 24 Jul 2025 13:57:57 +0100 Subject: [PATCH 62/72] Bump version to 4.0.1 --- gradle.properties | 2 +- streams/quickstart/java/pom.xml | 2 +- .../java/src/main/resources/archetype-resources/pom.xml | 2 +- streams/quickstart/pom.xml | 2 +- tests/kafkatest/__init__.py | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/gradle.properties b/gradle.properties index 04ea428dd3..3bd3cee6c7 100644 --- a/gradle.properties +++ b/gradle.properties @@ -23,7 +23,7 @@ group=org.apache.kafka # - streams/quickstart/pom.xml # - streams/quickstart/java/src/main/resources/archetype-resources/pom.xml # - streams/quickstart/java/pom.xml -version=4.0.1-SNAPSHOT +version=4.0.1 scalaVersion=2.13.15 # Adding swaggerVersion in gradle.properties to have a single version in place for swagger swaggerVersion=2.2.25 diff --git a/streams/quickstart/java/pom.xml b/streams/quickstart/java/pom.xml index ffbf57e2fd..312b2029f3 100644 --- a/streams/quickstart/java/pom.xml +++ b/streams/quickstart/java/pom.xml @@ -26,7 +26,7 @@ org.apache.kafka streams-quickstart - 4.0.1-SNAPSHOT + 4.0.1 .. diff --git a/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml b/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml index 14bf05d7f9..d4ed1f7c18 100644 --- a/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml +++ b/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml @@ -29,7 +29,7 @@ UTF-8 - 4.0.1-SNAPSHOT + 4.0.1 2.0.16 diff --git a/streams/quickstart/pom.xml b/streams/quickstart/pom.xml index 7f267a1828..794363226c 100644 --- a/streams/quickstart/pom.xml +++ b/streams/quickstart/pom.xml @@ -22,7 +22,7 @@ org.apache.kafka streams-quickstart pom - 4.0.1-SNAPSHOT + 4.0.1 Kafka Streams :: Quickstart diff --git a/tests/kafkatest/__init__.py b/tests/kafkatest/__init__.py index 2f93446fdc..56da891c65 100644 --- a/tests/kafkatest/__init__.py +++ b/tests/kafkatest/__init__.py @@ -22,4 +22,4 @@ # Instead, in development branches, the version should have a suffix of the form ".devN" # # For example, when Kafka is at version 1.0.0-SNAPSHOT, this should be something like "1.0.0.dev0" -__version__ = '4.0.1.dev0' +__version__ = '4.0.1' From b81230cfff1e8950775a7128b41f8b1709104a82 Mon Sep 17 00:00:00 2001 From: Christo Date: Thu, 24 Jul 2025 14:21:04 +0100 Subject: [PATCH 63/72] Bump version to 4.0.1 --- gradle.properties | 2 +- streams/quickstart/java/pom.xml | 2 +- .../java/src/main/resources/archetype-resources/pom.xml | 2 +- streams/quickstart/pom.xml | 2 +- tests/kafkatest/__init__.py | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/gradle.properties b/gradle.properties index 04ea428dd3..3bd3cee6c7 100644 --- a/gradle.properties +++ b/gradle.properties @@ -23,7 +23,7 @@ group=org.apache.kafka # - streams/quickstart/pom.xml # - streams/quickstart/java/src/main/resources/archetype-resources/pom.xml # - streams/quickstart/java/pom.xml -version=4.0.1-SNAPSHOT +version=4.0.1 scalaVersion=2.13.15 # Adding swaggerVersion in gradle.properties to have a single version in place for swagger swaggerVersion=2.2.25 diff --git a/streams/quickstart/java/pom.xml b/streams/quickstart/java/pom.xml index ffbf57e2fd..312b2029f3 100644 --- a/streams/quickstart/java/pom.xml +++ b/streams/quickstart/java/pom.xml @@ -26,7 +26,7 @@ org.apache.kafka streams-quickstart - 4.0.1-SNAPSHOT + 4.0.1 .. diff --git a/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml b/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml index 14bf05d7f9..d4ed1f7c18 100644 --- a/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml +++ b/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml @@ -29,7 +29,7 @@ UTF-8 - 4.0.1-SNAPSHOT + 4.0.1 2.0.16 diff --git a/streams/quickstart/pom.xml b/streams/quickstart/pom.xml index 7f267a1828..794363226c 100644 --- a/streams/quickstart/pom.xml +++ b/streams/quickstart/pom.xml @@ -22,7 +22,7 @@ org.apache.kafka streams-quickstart pom - 4.0.1-SNAPSHOT + 4.0.1 Kafka Streams :: Quickstart diff --git a/tests/kafkatest/__init__.py b/tests/kafkatest/__init__.py index 2f93446fdc..56da891c65 100644 --- a/tests/kafkatest/__init__.py +++ b/tests/kafkatest/__init__.py @@ -22,4 +22,4 @@ # Instead, in development branches, the version should have a suffix of the form ".devN" # # For example, when Kafka is at version 1.0.0-SNAPSHOT, this should be something like "1.0.0.dev0" -__version__ = '4.0.1.dev0' +__version__ = '4.0.1' From deb58910c8b573a8ed354cad3a76350fbafbe4d1 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Fri, 1 Aug 2025 14:59:15 +0200 Subject: [PATCH 64/72] KAFKA-19529: State updater sensor names should be unique (#20262) (#20273) All state updater threads use the same metrics instance, but do not use unique names for their sensors. This can have the following symptoms: 1) Data inserted into one sensor by one thread can affect the metrics of all state updater threads. 2) If one state updater thread is shutdown, the metrics associated to all state updater threads are removed. 3) If one state updater thread is started, while another one is removed, it can happen that a metric is registered with the `Metrics` instance, but not associated to any `Sensor` (because it is concurrently removed), which means that the metric will not be removed upon shutdown. If a thread with the same name later tries to register the same metric, we may run into a `java.lang.IllegalArgumentException: A metric named ... already exists`, as described in the ticket. This change fixes the bug giving unique names to the sensors. A test is added that there is no interference of the removal of sensors and metrics during shutdown. Reviewers: Matthias J. Sax --- .../internals/DefaultStateUpdater.java | 54 +++++++++--------- .../processor/internals/StreamThread.java | 2 +- .../internals/DefaultStateUpdaterTest.java | 56 ++++++++++++++++++- 3 files changed, 83 insertions(+), 29 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdater.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdater.java index bcb76966f6..48d42590c1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdater.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdater.java @@ -36,6 +36,7 @@ import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.Task.State; import org.apache.kafka.streams.processor.internals.TaskAndAction.Action; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.slf4j.Logger; @@ -89,7 +90,7 @@ private class StateUpdaterThread extends Thread { private volatile KafkaFutureImpl clientInstanceIdFuture = new KafkaFutureImpl<>(); public StateUpdaterThread(final String name, - final Metrics metrics, + final StreamsMetricsImpl metrics, final ChangelogReader changelogReader) { super(name); this.changelogReader = changelogReader; @@ -745,7 +746,7 @@ private void recordMetrics(final long now, final long totalLatency, final long t private final Time time; private final Logger log; private final String name; - private final Metrics metrics; + private final StreamsMetricsImpl metrics; private final Consumer restoreConsumer; private final ChangelogReader changelogReader; private final TopologyMetadata topologyMetadata; @@ -766,7 +767,7 @@ private void recordMetrics(final long now, final long totalLatency, final long t private StateUpdaterThread stateUpdaterThread = null; public DefaultStateUpdater(final String name, - final Metrics metrics, + final StreamsMetricsImpl metrics, final StreamsConfig config, final Consumer restoreConsumer, final ChangelogReader changelogReader, @@ -1059,74 +1060,75 @@ private class StateUpdaterMetrics { private final Sensor standbyRestoreRatioSensor; private final Sensor checkpointRatioSensor; - private final Deque allSensorNames = new LinkedList<>(); + private final Deque allSensors = new LinkedList<>(); private final Deque allMetricNames = new LinkedList<>(); - private StateUpdaterMetrics(final Metrics metrics, final String threadId) { + private StateUpdaterMetrics(final StreamsMetricsImpl metrics, final String threadId) { final Map threadLevelTags = new LinkedHashMap<>(); threadLevelTags.put(THREAD_ID_TAG, threadId); + final Metrics metricsRegistry = metrics.metricsRegistry(); - MetricName metricName = metrics.metricName("active-restoring-tasks", + MetricName metricName = metricsRegistry.metricName("active-restoring-tasks", STATE_LEVEL_GROUP, "The number of active tasks currently undergoing restoration", threadLevelTags); - metrics.addMetric(metricName, (config, now) -> stateUpdaterThread != null ? + metricsRegistry.addMetric(metricName, (config, now) -> stateUpdaterThread != null ? stateUpdaterThread.numRestoringActiveTasks() : 0); allMetricNames.push(metricName); - metricName = metrics.metricName("standby-updating-tasks", + metricName = metricsRegistry.metricName("standby-updating-tasks", STATE_LEVEL_GROUP, "The number of standby tasks currently undergoing state update", threadLevelTags); - metrics.addMetric(metricName, (config, now) -> stateUpdaterThread != null ? + metricsRegistry.addMetric(metricName, (config, now) -> stateUpdaterThread != null ? stateUpdaterThread.numUpdatingStandbyTasks() : 0); allMetricNames.push(metricName); - metricName = metrics.metricName("active-paused-tasks", + metricName = metricsRegistry.metricName("active-paused-tasks", STATE_LEVEL_GROUP, "The number of active tasks paused restoring", threadLevelTags); - metrics.addMetric(metricName, (config, now) -> stateUpdaterThread != null ? + metricsRegistry.addMetric(metricName, (config, now) -> stateUpdaterThread != null ? stateUpdaterThread.numPausedActiveTasks() : 0); allMetricNames.push(metricName); - metricName = metrics.metricName("standby-paused-tasks", + metricName = metricsRegistry.metricName("standby-paused-tasks", STATE_LEVEL_GROUP, "The number of standby tasks paused state update", threadLevelTags); - metrics.addMetric(metricName, (config, now) -> stateUpdaterThread != null ? + metricsRegistry.addMetric(metricName, (config, now) -> stateUpdaterThread != null ? stateUpdaterThread.numPausedStandbyTasks() : 0); allMetricNames.push(metricName); - this.idleRatioSensor = metrics.sensor("idle-ratio", RecordingLevel.INFO); + this.idleRatioSensor = metrics.threadLevelSensor(threadId, "idle-ratio", RecordingLevel.INFO); this.idleRatioSensor.add(new MetricName("idle-ratio", STATE_LEVEL_GROUP, IDLE_RATIO_DESCRIPTION, threadLevelTags), new Avg()); - allSensorNames.add("idle-ratio"); + allSensors.add(this.idleRatioSensor); - this.activeRestoreRatioSensor = metrics.sensor("active-restore-ratio", RecordingLevel.INFO); + this.activeRestoreRatioSensor = metrics.threadLevelSensor(threadId, "active-restore-ratio", RecordingLevel.INFO); this.activeRestoreRatioSensor.add(new MetricName("active-restore-ratio", STATE_LEVEL_GROUP, RESTORE_RATIO_DESCRIPTION, threadLevelTags), new Avg()); - allSensorNames.add("active-restore-ratio"); + allSensors.add(this.activeRestoreRatioSensor); - this.standbyRestoreRatioSensor = metrics.sensor("standby-update-ratio", RecordingLevel.INFO); + this.standbyRestoreRatioSensor = metrics.threadLevelSensor(threadId, "standby-update-ratio", RecordingLevel.INFO); this.standbyRestoreRatioSensor.add(new MetricName("standby-update-ratio", STATE_LEVEL_GROUP, UPDATE_RATIO_DESCRIPTION, threadLevelTags), new Avg()); - allSensorNames.add("standby-update-ratio"); + allSensors.add(this.standbyRestoreRatioSensor); - this.checkpointRatioSensor = metrics.sensor("checkpoint-ratio", RecordingLevel.INFO); + this.checkpointRatioSensor = metrics.threadLevelSensor(threadId, "checkpoint-ratio", RecordingLevel.INFO); this.checkpointRatioSensor.add(new MetricName("checkpoint-ratio", STATE_LEVEL_GROUP, CHECKPOINT_RATIO_DESCRIPTION, threadLevelTags), new Avg()); - allSensorNames.add("checkpoint-ratio"); + allSensors.add(this.checkpointRatioSensor); - this.restoreSensor = metrics.sensor("restore-records", RecordingLevel.INFO); + this.restoreSensor = metrics.threadLevelSensor(threadId, "restore-records", RecordingLevel.INFO); this.restoreSensor.add(new MetricName("restore-records-rate", STATE_LEVEL_GROUP, RESTORE_RECORDS_RATE_DESCRIPTION, threadLevelTags), new Rate()); this.restoreSensor.add(new MetricName("restore-call-rate", STATE_LEVEL_GROUP, RESTORE_RATE_DESCRIPTION, threadLevelTags), new Rate(new WindowedCount())); - allSensorNames.add("restore-records"); + allSensors.add(this.restoreSensor); } void clear() { - while (!allSensorNames.isEmpty()) { - metrics.removeSensor(allSensorNames.pop()); + while (!allSensors.isEmpty()) { + metrics.removeSensor(allSensors.pop()); } while (!allMetricNames.isEmpty()) { - metrics.removeMetric(allMetricNames.pop()); + metrics.metricsRegistry().removeMetric(allMetricNames.pop()); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 20d49b44db..782775657e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -550,7 +550,7 @@ private static StateUpdater maybeCreateAndStartStateUpdater(final boolean stateU final String name = clientId + STATE_UPDATER_ID_SUBSTRING + threadIdx; final StateUpdater stateUpdater = new DefaultStateUpdater( name, - streamsMetrics.metricsRegistry(), + streamsMetrics, streamsConfig, restoreConsumer, changelogReader, diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdaterTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdaterTest.java index 25f3f0e587..9f2a4c9356 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdaterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdaterTest.java @@ -28,6 +28,7 @@ import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.StateUpdater.ExceptionAndTask; import org.apache.kafka.streams.processor.internals.Task.State; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.hamcrest.Matcher; import org.junit.jupiter.api.AfterEach; @@ -105,7 +106,7 @@ class DefaultStateUpdaterTest { // need an auto-tick timer to work for draining with timeout private final Time time = new MockTime(1L); - private final Metrics metrics = new Metrics(time); + private final StreamsMetricsImpl metrics = new StreamsMetricsImpl(new Metrics(time), "", "", time); private final StreamsConfig config = new StreamsConfig(configProps(COMMIT_INTERVAL)); private final ChangelogReader changelogReader = mock(ChangelogReader.class); private final TopologyMetadata topologyMetadata = unnamedTopology().build(); @@ -1672,8 +1673,59 @@ public void shouldRecordMetrics() throws Exception { assertThat(metrics.metrics().size(), is(1)); } + @Test + public void shouldRemoveMetricsWithoutInterference() { + final DefaultStateUpdater stateUpdater2 = + new DefaultStateUpdater("test-state-updater2", metrics, config, null, changelogReader, topologyMetadata, time); + final List threadMetrics = getMetricNames("test-state-updater"); + final List threadMetrics2 = getMetricNames("test-state-updater2"); + + stateUpdater.start(); + stateUpdater2.start(); + + for (final MetricName metricName : threadMetrics) { + assertTrue(metrics.metrics().containsKey(metricName)); + } + for (final MetricName metricName : threadMetrics2) { + assertTrue(metrics.metrics().containsKey(metricName)); + } + + stateUpdater2.shutdown(Duration.ofMinutes(1)); + + for (final MetricName metricName : threadMetrics) { + assertTrue(metrics.metrics().containsKey(metricName)); + } + for (final MetricName metricName : threadMetrics2) { + assertFalse(metrics.metrics().containsKey(metricName)); + } + + stateUpdater.shutdown(Duration.ofMinutes(1)); + + for (final MetricName metricName : threadMetrics) { + assertFalse(metrics.metrics().containsKey(metricName)); + } + for (final MetricName metricName : threadMetrics2) { + assertFalse(metrics.metrics().containsKey(metricName)); + } + } + + private static List getMetricNames(final String threadId) { + final Map tagMap = Map.of("thread-id", threadId); + return List.of( + new MetricName("active-restoring-tasks", "stream-state-updater-metrics", "", tagMap), + new MetricName("standby-updating-tasks", "stream-state-updater-metrics", "", tagMap), + new MetricName("active-paused-tasks", "stream-state-updater-metrics", "", tagMap), + new MetricName("standby-paused-tasks", "stream-state-updater-metrics", "", tagMap), + new MetricName("idle-ratio", "stream-state-updater-metrics", "", tagMap), + new MetricName("standby-update-ratio", "stream-state-updater-metrics", "", tagMap), + new MetricName("checkpoint-ratio", "stream-state-updater-metrics", "", tagMap), + new MetricName("restore-records-rate", "stream-state-updater-metrics", "", tagMap), + new MetricName("restore-call-rate", "stream-state-updater-metrics", "", tagMap) + ); + } + @SuppressWarnings("unchecked") - private static void verifyMetric(final Metrics metrics, + private static void verifyMetric(final StreamsMetricsImpl metrics, final MetricName metricName, final Matcher matcher) { assertThat(metrics.metrics().get(metricName).metricName().description(), is(metricName.description())); From 0f9b3127030f596c8c187e22b1badd88e4bf9662 Mon Sep 17 00:00:00 2001 From: Jared Harley Date: Mon, 4 Aug 2025 22:22:54 -0600 Subject: [PATCH 65/72] KAFKA-19576 Fix typo in state-change log filename after rotate (#20269) The `state-change.log` file is being incorrectly rotated to `stage-change.log.[date]`. This change fixes the typo to have the log file correctly rotated to `state-change.log.[date]` _No functional changes._ Reviewers: Mickael Maison , Christo Lolov , Luke Chen , Ken Huang , TengYao Chi , Chia-Ping Tsai --- config/log4j2.yaml | 2 +- docs/upgrade.html | 20 +++++++++++++++----- 2 files changed, 16 insertions(+), 6 deletions(-) diff --git a/config/log4j2.yaml b/config/log4j2.yaml index 7ee6f001e1..891a5c71cd 100644 --- a/config/log4j2.yaml +++ b/config/log4j2.yaml @@ -44,7 +44,7 @@ Configuration: # State Change appender - name: StateChangeAppender fileName: "${sys:kafka.logs.dir}/state-change.log" - filePattern: "${sys:kafka.logs.dir}/stage-change.log.%d{yyyy-MM-dd-HH}" + filePattern: "${sys:kafka.logs.dir}/state-change.log.%d{yyyy-MM-dd-HH}" PatternLayout: pattern: "${logPattern}" TimeBasedTriggeringPolicy: diff --git a/docs/upgrade.html b/docs/upgrade.html index 4e24c14932..cb6f4cdc66 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -19,9 +19,9 @@