From 1debe64e8c8234de56c465bbaffdf8229d5322c6 Mon Sep 17 00:00:00 2001 From: Jonah Hooper Date: Tue, 9 Sep 2025 01:11:45 -0400 Subject: [PATCH 001/100] MINOR: Rename RecordSnapshotWriter.Builder.setMaxBatchSize to setMaxBatchSizeBytes (#20446) The original name is confusing which could cause engineers to make a mistake and confuse the `batchSize` with some other unit like number of records. Reviewers: Chia-Ping Tsai --- .../kafka/metadata/storage/Formatter.java | 2 +- .../apache/kafka/raft/KafkaRaftClient.java | 2 +- .../raft/internals/BatchAccumulator.java | 16 +++++++------- .../kafka/snapshot/RecordsSnapshotWriter.java | 12 +++++----- .../snapshot/RecordsSnapshotWriterTest.java | 22 +++++++++---------- 5 files changed, 27 insertions(+), 27 deletions(-) diff --git a/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java b/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java index 3f7cba37724..daac6630550 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java @@ -506,7 +506,7 @@ public class Formatter { VoterSet voterSet = initialControllers.toVoterSet(controllerListenerName); RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder(). setLastContainedLogTimestamp(Time.SYSTEM.milliseconds()). - setMaxBatchSize(KafkaRaftClient.MAX_BATCH_SIZE_BYTES). + setMaxBatchSizeBytes(KafkaRaftClient.MAX_BATCH_SIZE_BYTES). setRawSnapshotWriter(FileRawSnapshotWriter.create( clusterMetadataDirectory.toPath(), Snapshots.BOOTSTRAP_SNAPSHOT_ID)). 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 384a47b5ff2..343f4db6b02 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -3721,7 +3721,7 @@ public final class KafkaRaftClient implements RaftClient { return new RecordsSnapshotWriter.Builder() .setLastContainedLogTimestamp(lastContainedLogTimestamp) .setTime(time) - .setMaxBatchSize(MAX_BATCH_SIZE_BYTES) + .setMaxBatchSizeBytes(MAX_BATCH_SIZE_BYTES) .setMemoryPool(memoryPool) .setRawSnapshotWriter(wrappedWriter) .setKraftVersion(partitionState.kraftVersionAtOffset(lastContainedLogOffset)) diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java b/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java index bb3b2a39b46..e202f8a1ab4 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java @@ -58,7 +58,7 @@ public class BatchAccumulator implements Closeable { private final int epoch; private final Time time; private final int lingerMs; - private final int maxBatchSize; + private final int maxBatchSizeBytes; private final int maxNumberOfBatches; private final Compression compression; private final MemoryPool memoryPool; @@ -82,7 +82,7 @@ public class BatchAccumulator implements Closeable { int epoch, long baseOffset, int lingerMs, - int maxBatchSize, + int maxBatchSizeBytes, int maxNumberOfBatches, MemoryPool memoryPool, Time time, @@ -91,7 +91,7 @@ public class BatchAccumulator implements Closeable { ) { this.epoch = epoch; this.lingerMs = lingerMs; - this.maxBatchSize = maxBatchSize; + this.maxBatchSizeBytes = maxBatchSizeBytes; this.maxNumberOfBatches = maxNumberOfBatches; this.memoryPool = memoryPool; this.time = time; @@ -182,12 +182,12 @@ public class BatchAccumulator implements Closeable { if (currentBatch != null) { OptionalInt bytesNeeded = currentBatch.bytesNeeded(records, serializationCache); - if (bytesNeeded.isPresent() && bytesNeeded.getAsInt() > maxBatchSize) { + if (bytesNeeded.isPresent() && bytesNeeded.getAsInt() > maxBatchSizeBytes) { throw new RecordBatchTooLargeException( String.format( "The total record(s) size of %d exceeds the maximum allowed batch size of %d", bytesNeeded.getAsInt(), - maxBatchSize + maxBatchSizeBytes ) ); } else if (bytesNeeded.isPresent()) { @@ -231,7 +231,7 @@ public class BatchAccumulator implements Closeable { public long appendControlMessages(MemoryRecordsCreator valueCreator) { appendLock.lock(); try { - ByteBuffer buffer = memoryPool.tryAllocate(maxBatchSize); + ByteBuffer buffer = memoryPool.tryAllocate(maxBatchSizeBytes); if (buffer != null) { try { forceDrain(); @@ -421,7 +421,7 @@ public class BatchAccumulator implements Closeable { } private void startNewBatch() { - ByteBuffer buffer = memoryPool.tryAllocate(maxBatchSize); + ByteBuffer buffer = memoryPool.tryAllocate(maxBatchSizeBytes); if (buffer != null) { currentBatch = new BatchBuilder<>( buffer, @@ -430,7 +430,7 @@ public class BatchAccumulator implements Closeable { nextOffset, time.milliseconds(), epoch, - maxBatchSize + maxBatchSizeBytes ); } } diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java index 09f88306638..7e008edcb34 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java @@ -44,7 +44,7 @@ public final class RecordsSnapshotWriter implements SnapshotWriter { private RecordsSnapshotWriter( RawSnapshotWriter snapshot, - int maxBatchSize, + int maxBatchSizeBytes, MemoryPool memoryPool, Time time, Compression compression, @@ -57,7 +57,7 @@ public final class RecordsSnapshotWriter implements SnapshotWriter { snapshot.snapshotId().epoch(), 0, Integer.MAX_VALUE, - maxBatchSize, + maxBatchSizeBytes, 10, // maxNumberOfBatches memoryPool, time, @@ -145,7 +145,7 @@ public final class RecordsSnapshotWriter implements SnapshotWriter { private long lastContainedLogTimestamp = 0; private Compression compression = Compression.NONE; private Time time = Time.SYSTEM; - private int maxBatchSize = 1024; + private int maxBatchSizeBytes = 1024; private MemoryPool memoryPool = MemoryPool.NONE; private KRaftVersion kraftVersion = KRaftVersion.KRAFT_VERSION_1; private Optional voterSet = Optional.empty(); @@ -166,8 +166,8 @@ public final class RecordsSnapshotWriter implements SnapshotWriter { return this; } - public Builder setMaxBatchSize(int maxBatchSize) { - this.maxBatchSize = maxBatchSize; + public Builder setMaxBatchSizeBytes(int maxBatchSizeBytes) { + this.maxBatchSizeBytes = maxBatchSizeBytes; return this; } @@ -206,7 +206,7 @@ public final class RecordsSnapshotWriter implements SnapshotWriter { RecordsSnapshotWriter writer = new RecordsSnapshotWriter<>( rawSnapshotWriter.get(), - maxBatchSize, + maxBatchSizeBytes, memoryPool, time, compression, diff --git a/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java b/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java index 35d126dba97..e1b09e06e6a 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java @@ -50,13 +50,13 @@ final class RecordsSnapshotWriterTest { @Test void testBuilderKRaftVersion0() { OffsetAndEpoch snapshotId = new OffsetAndEpoch(100, 10); - int maxBatchSize = 1024; + int maxBatchSizeBytes = 1024; AtomicReference buffer = new AtomicReference<>(null); RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() .setKraftVersion(KRaftVersion.KRAFT_VERSION_0) .setVoterSet(Optional.empty()) .setTime(new MockTime()) - .setMaxBatchSize(maxBatchSize) + .setMaxBatchSizeBytes(maxBatchSizeBytes) .setRawSnapshotWriter( new MockRawSnapshotWriter(snapshotId, buffer::set) ); @@ -68,7 +68,7 @@ final class RecordsSnapshotWriterTest { new MockRawSnapshotReader(snapshotId, buffer.get()), STRING_SERDE, BufferSupplier.NO_CACHING, - maxBatchSize, + maxBatchSizeBytes, true, new LogContext() ) @@ -100,7 +100,7 @@ final class RecordsSnapshotWriterTest { @Test void testBuilderKRaftVersion0WithVoterSet() { OffsetAndEpoch snapshotId = new OffsetAndEpoch(100, 10); - int maxBatchSize = 1024; + int maxBatchSizeBytes = 1024; VoterSet voterSet = VoterSetTest.voterSet( new HashMap<>(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true)) ); @@ -109,7 +109,7 @@ final class RecordsSnapshotWriterTest { .setKraftVersion(KRaftVersion.KRAFT_VERSION_0) .setVoterSet(Optional.of(voterSet)) .setTime(new MockTime()) - .setMaxBatchSize(maxBatchSize) + .setMaxBatchSizeBytes(maxBatchSizeBytes) .setRawSnapshotWriter( new MockRawSnapshotWriter(snapshotId, buffer::set) ); @@ -120,7 +120,7 @@ final class RecordsSnapshotWriterTest { @Test void testKBuilderRaftVersion1WithVoterSet() { OffsetAndEpoch snapshotId = new OffsetAndEpoch(100, 10); - int maxBatchSize = 1024; + int maxBatchSizeBytes = 1024; VoterSet voterSet = VoterSetTest.voterSet( new HashMap<>(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true)) ); @@ -129,7 +129,7 @@ final class RecordsSnapshotWriterTest { .setKraftVersion(KRaftVersion.KRAFT_VERSION_1) .setVoterSet(Optional.of(voterSet)) .setTime(new MockTime()) - .setMaxBatchSize(maxBatchSize) + .setMaxBatchSizeBytes(maxBatchSizeBytes) .setRawSnapshotWriter( new MockRawSnapshotWriter(snapshotId, buffer::set) ); @@ -141,7 +141,7 @@ final class RecordsSnapshotWriterTest { new MockRawSnapshotReader(snapshotId, buffer.get()), STRING_SERDE, BufferSupplier.NO_CACHING, - maxBatchSize, + maxBatchSizeBytes, true, new LogContext() ) @@ -181,13 +181,13 @@ final class RecordsSnapshotWriterTest { @Test void testBuilderKRaftVersion1WithoutVoterSet() { OffsetAndEpoch snapshotId = new OffsetAndEpoch(100, 10); - int maxBatchSize = 1024; + int maxBatchSizeBytes = 1024; AtomicReference buffer = new AtomicReference<>(null); RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() .setKraftVersion(KRaftVersion.KRAFT_VERSION_1) .setVoterSet(Optional.empty()) .setTime(new MockTime()) - .setMaxBatchSize(maxBatchSize) + .setMaxBatchSizeBytes(maxBatchSizeBytes) .setRawSnapshotWriter( new MockRawSnapshotWriter(snapshotId, buffer::set) ); @@ -199,7 +199,7 @@ final class RecordsSnapshotWriterTest { new MockRawSnapshotReader(snapshotId, buffer.get()), STRING_SERDE, BufferSupplier.NO_CACHING, - maxBatchSize, + maxBatchSizeBytes, true, new LogContext() ) From 9c9f1446a16124c177bf7412869e14ded9032f88 Mon Sep 17 00:00:00 2001 From: Linsiyuan9 <154567180+Linsiyuan9@users.noreply.github.com> Date: Tue, 9 Sep 2025 13:58:10 +0800 Subject: [PATCH 002/100] KAFKA-18061 AddRaftVoter responds with error message "NONE" instead of null (#17930) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit In `RaftUtil.addVoterResponse` and `RaftUtil.removeVoterResponse` methods, when the input `errorMessage` is `null`, the returned string is not actually null but `NONE`. This introduces an inconsistency: semantically, `null` should represent “no error message,” while `NONE` looks like a real string value and may confuse clients. Reviewers: Alyssa Huang , José Armando García Sancio , Anton Agestam , Chia-Ping Tsai --- .../java/org/apache/kafka/raft/RaftUtil.java | 12 ++++++--- .../kafka/raft/RaftClientTestContext.java | 17 ++++++++++--- .../org/apache/kafka/raft/RaftUtilTest.java | 25 +++++++++++++++++++ 3 files changed, 46 insertions(+), 8 deletions(-) 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 f3f411885a7..d3a4f760968 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java @@ -542,8 +542,10 @@ public class RaftUtil { Errors error, String errorMessage ) { - errorMessage = errorMessage == null ? error.message() : errorMessage; - + // return the provided errorMessage if it exists, Errors.NONE should have a null message + if (errorMessage == null && error != Errors.NONE) { + errorMessage = error.message(); + } return new AddRaftVoterResponseData() .setErrorCode(error.code()) .setErrorMessage(errorMessage); @@ -563,8 +565,10 @@ public class RaftUtil { Errors error, String errorMessage ) { - errorMessage = errorMessage == null ? error.message() : errorMessage; - + // return the provided errorMessage if it exists, Errors.NONE should have a null message + if (errorMessage == null && error != Errors.NONE) { + errorMessage = error.message(); + } return new RemoveRaftVoterResponseData() .setErrorCode(error.code()) .setErrorMessage(errorMessage); diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index a98fb79d09a..42fdde0fa0e 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -108,6 +108,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; public final class RaftClientTestContext { @@ -1341,8 +1342,12 @@ public final class RaftClientTestContext { assertInstanceOf(AddRaftVoterResponseData.class, response.data()); AddRaftVoterResponseData addVoterResponse = (AddRaftVoterResponseData) response.data(); - assertEquals(error, Errors.forCode(addVoterResponse.errorCode())); - + if (Errors.NONE.equals(error)) { + assertEquals(error, Errors.forCode(addVoterResponse.errorCode())); + assertNull(addVoterResponse.errorMessage()); + } else { + assertEquals(error, Errors.forCode(addVoterResponse.errorCode())); + } return addVoterResponse; } @@ -1371,8 +1376,12 @@ public final class RaftClientTestContext { assertInstanceOf(RemoveRaftVoterResponseData.class, response.data()); RemoveRaftVoterResponseData removeVoterResponse = (RemoveRaftVoterResponseData) response.data(); - assertEquals(error, Errors.forCode(removeVoterResponse.errorCode())); - + if (Errors.NONE.equals(error)) { + assertEquals(error, Errors.forCode(removeVoterResponse.errorCode())); + assertNull(removeVoterResponse.errorMessage()); + } else { + assertEquals(error, Errors.forCode(removeVoterResponse.errorCode())); + } return removeVoterResponse; } 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 cdbd728cb1e..5aacab01df6 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftUtilTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftUtilTest.java @@ -72,6 +72,7 @@ import java.util.Map; import java.util.stream.Stream; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; public class RaftUtilTest { @@ -646,6 +647,30 @@ public class RaftUtilTest { ); } + @Test + public void testAddVoterResponse() { + for (Errors error : Errors.values()) { + AddRaftVoterResponseData addRaftVoterResponseData = RaftUtil.addVoterResponse(error, null); + assertEquals(error.code(), addRaftVoterResponseData.errorCode()); + if (Errors.NONE.equals(error)) + assertNull(addRaftVoterResponseData.errorMessage()); + else + assertEquals(error.message(), addRaftVoterResponseData.errorMessage()); + } + } + + @Test + public void testRemoveVoterResponse() { + for (Errors error : Errors.values()) { + RemoveRaftVoterResponseData removeRaftVoterResponseData = RaftUtil.removeVoterResponse(error, null); + assertEquals(error.code(), removeRaftVoterResponseData.errorCode()); + if (Errors.NONE.equals(error)) + assertNull(removeRaftVoterResponseData.errorMessage()); + else + assertEquals(error.message(), removeRaftVoterResponseData.errorMessage()); + } + } + private Records createRecords() { ByteBuffer allocate = ByteBuffer.allocate(1024); From f6f5b4cb279e12d67cefc732ff71f0cff6386000 Mon Sep 17 00:00:00 2001 From: lucliu1108 Date: Tue, 9 Sep 2025 03:30:39 -0500 Subject: [PATCH 003/100] KAFKA-19565: Integration test for Streams-related Admin APIs [2/N] (#20266) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Integration tests for Stream Admin related API Previous PR: https://github.com/apache/kafka/pull/20244 This one adds: - Integration test for Admin#listStreamsGroupOffsets API - Integration test for Admin#deleteStreamsGroupOffsets API - Integration test for Admin#alterStreamsGroupOffsets API Reviewers: Alieh Saeedi , Lucas Brutschy --- .../api/PlaintextAdminIntegrationTest.scala | 213 ++++++++++++++++++ 1 file changed, 213 insertions(+) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 286ac2b098c..3d172a678bc 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -4521,6 +4521,219 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { Utils.closeQuietly(client, "adminClient") } } + + @Test + def testListStreamsGroupOffsets(): Unit = { + val streamsGroupId = "stream_group_id" + val testTopicName = "test_topic" + val testNumPartitions = 3 + + val config = createConfig + client = Admin.create(config) + val producer = createProducer(configOverrides = new Properties()) + + prepareTopics(List(testTopicName), testNumPartitions) + prepareRecords(testTopicName) + + // Producer sends messages + for (i <- 1 to 20) { + TestUtils.waitUntilTrue(() => { + val producerRecord = producer.send( + new ProducerRecord[Array[Byte], Array[Byte]](testTopicName, s"key-$i".getBytes(), s"value-$i".getBytes())) + .get() + producerRecord != null && producerRecord.topic() == testTopicName + }, "Fail to produce record to topic") + } + + val streams = createStreamsGroup( + inputTopic = testTopicName, + streamsGroupId = streamsGroupId, + ) + + try { + TestUtils.waitUntilTrue(() => { + streams.poll(JDuration.ofMillis(100L)) + !streams.assignment().isEmpty + }, "Consumer not assigned to partitions") + + streams.poll(JDuration.ofMillis(1000L)) + streams.commitSync() + + TestUtils.waitUntilTrue(() => { + val firstGroup = client.listGroups().all().get().stream().findFirst().orElse(null) + firstGroup.groupState().orElse(null) == GroupState.STABLE && firstGroup.groupId() == streamsGroupId + }, "Stream group not stable yet") + + val allTopicPartitions = client.listStreamsGroupOffsets( + util.Map.of(streamsGroupId, new ListStreamsGroupOffsetsSpec()) + ).partitionsToOffsetAndMetadata(streamsGroupId).get() + assertNotNull(allTopicPartitions) + assertEquals(allTopicPartitions.size(), 3) + allTopicPartitions.forEach((topicPartition, offsetAndMetadata) => { + assertNotNull(topicPartition) + assertNotNull(offsetAndMetadata) + assertTrue(topicPartition.topic().startsWith(testTopicName)) + assertTrue(offsetAndMetadata.offset() >= 0) + }) + + } finally { + Utils.closeQuietly(streams, "streams") + Utils.closeQuietly(client, "adminClient") + Utils.closeQuietly(producer, "producer") + } + } + + @Test + def testDeleteStreamsGroupOffsets(): Unit = { + val streamsGroupId = "stream_group_id" + val testTopicName = "test_topic" + val testNumPartitions = 3 + + val config = createConfig + client = Admin.create(config) + val producer = createProducer(configOverrides = new Properties()) + + prepareTopics(List(testTopicName), testNumPartitions) + prepareRecords(testTopicName) + // Producer sends messages + for (i <- 1 to 20) { + TestUtils.waitUntilTrue(() => { + val producerRecord = producer.send( + new ProducerRecord[Array[Byte], Array[Byte]](testTopicName, s"key-$i".getBytes(), s"value-$i".getBytes())) + .get() + producerRecord != null && producerRecord.topic() == testTopicName + }, "Fail to produce record to topic") + } + + val streams = createStreamsGroup( + inputTopic = testTopicName, + streamsGroupId = streamsGroupId, + ) + + try { + TestUtils.waitUntilTrue(() => { + streams.poll(JDuration.ofMillis(100L)) + !streams.assignment().isEmpty + }, "Consumer not assigned to partitions") + + streams.poll(JDuration.ofMillis(1000L)) + streams.commitSync() + + // List streams group offsets + TestUtils.waitUntilTrue(() => { + val allTopicPartitions = client.listStreamsGroupOffsets( + util.Map.of(streamsGroupId, new ListStreamsGroupOffsetsSpec()) + ).partitionsToOffsetAndMetadata(streamsGroupId).get() + allTopicPartitions!=null && allTopicPartitions.size() == testNumPartitions + },"Streams group offsets not ready to list yet") + + // Verify running Kstreams group cannot delete its own offsets + var deleteStreamsGroupOffsetsResult = client.deleteStreamsGroupOffsets(streamsGroupId, util.Set.of(new TopicPartition(testTopicName, 0))) + assertFutureThrows(classOf[GroupSubscribedToTopicException], deleteStreamsGroupOffsetsResult.all()) + + // Verity stopped Kstreams group can delete its own offsets + streams.close() + TestUtils.waitUntilTrue(() => { + val groupDescription = client.describeStreamsGroups(util.List.of(streamsGroupId)).all().get() + groupDescription.get(streamsGroupId).groupState() == GroupState.EMPTY + }, "Streams group not closed yet") + deleteStreamsGroupOffsetsResult = client.deleteStreamsGroupOffsets(streamsGroupId, util.Set.of(new TopicPartition(testTopicName, 0))) + val res = deleteStreamsGroupOffsetsResult.partitionResult(new TopicPartition(testTopicName, 0)).get() + assertNull(res) + + // Verify the group offsets after deletion + val allTopicPartitions = client.listStreamsGroupOffsets( + util.Map.of(streamsGroupId, new ListStreamsGroupOffsetsSpec()) + ).partitionsToOffsetAndMetadata(streamsGroupId).get() + assertEquals(testNumPartitions-1, allTopicPartitions.size()) + + // Verify non-existing topic partition couldn't be deleted + val deleteStreamsGroupOffsetsResultWithFakeTopic = client.deleteStreamsGroupOffsets(streamsGroupId, util.Set.of(new TopicPartition("mock-topic", 1))) + assertFutureThrows(classOf[UnknownTopicOrPartitionException], deleteStreamsGroupOffsetsResultWithFakeTopic.all()) + val deleteStreamsGroupOffsetsResultWithFakePartition = client.deleteStreamsGroupOffsets(streamsGroupId, util.Set.of(new TopicPartition(testTopicName, testNumPartitions))) + assertFutureThrows(classOf[UnknownTopicOrPartitionException], deleteStreamsGroupOffsetsResultWithFakePartition.all()) + } finally { + Utils.closeQuietly(streams, "streams") + Utils.closeQuietly(client, "adminClient") + Utils.closeQuietly(producer, "producer") + } + } + + @Test + def testAlterStreamsGroupOffsets(): Unit = { + val streamsGroupId = "stream_group_id" + val testTopicName = "test_topic" + val testNumPartitions = 3 + + val config = createConfig + client = Admin.create(config) + val producer = createProducer(configOverrides = new Properties()) + + prepareTopics(List(testTopicName), testNumPartitions) + prepareRecords(testTopicName) + + // Producer sends messages + for (i <- 1 to 20) { + TestUtils.waitUntilTrue(() => { + val producerRecord = producer.send( + new ProducerRecord[Array[Byte], Array[Byte]](testTopicName, s"key-$i".getBytes(), s"value-$i".getBytes())) + .get() + producerRecord != null && producerRecord.topic() == testTopicName + }, "Fail to produce record to topic") + } + + val streams = createStreamsGroup( + inputTopic = testTopicName, + streamsGroupId = streamsGroupId, + ) + + try { + TestUtils.waitUntilTrue(() => { + streams.poll(JDuration.ofMillis(100L)) + !streams.assignment().isEmpty + }, "Consumer not assigned to partitions") + + streams.poll(JDuration.ofMillis(1000L)) + streams.commitSync() + + // List streams group offsets + TestUtils.waitUntilTrue(() => { + val allTopicPartitions = client.listStreamsGroupOffsets( + util.Map.of(streamsGroupId, new ListStreamsGroupOffsetsSpec()) + ).partitionsToOffsetAndMetadata(streamsGroupId).get() + allTopicPartitions!=null && allTopicPartitions.size() == testNumPartitions + },"Streams group offsets not ready to list yet") + + // Verity stopped Kstreams group can delete its own offsets + streams.close() + TestUtils.waitUntilTrue(() => { + val groupDescription = client.describeStreamsGroups(util.List.of(streamsGroupId)).all().get() + groupDescription.get(streamsGroupId).groupState() == GroupState.EMPTY + }, "Streams group not closed yet") + + val offsets = util.Map.of( + new TopicPartition(testTopicName, 0), new OffsetAndMetadata(1L), + new TopicPartition(testTopicName, 1), new OffsetAndMetadata(10L) + ) + val alterStreamsGroupOffsetsResult = client.alterStreamsGroupOffsets(streamsGroupId, offsets) + val res0 = alterStreamsGroupOffsetsResult.partitionResult(new TopicPartition(testTopicName, 0)).get() + val res1 = alterStreamsGroupOffsetsResult.partitionResult(new TopicPartition(testTopicName, 1)).get() + assertTrue(res0 == null && res1 == null, "Alter streams group offsets should return null for each partition result") + + val allTopicPartitions = client.listStreamsGroupOffsets( + util.Map.of(streamsGroupId, new ListStreamsGroupOffsetsSpec()) + ).partitionsToOffsetAndMetadata(streamsGroupId).get() + assertNotNull(allTopicPartitions) + assertEquals(testNumPartitions, allTopicPartitions.size()) + assertEquals(1L, allTopicPartitions.get(new TopicPartition(testTopicName, 0)).offset()) + assertEquals(10L, allTopicPartitions.get(new TopicPartition(testTopicName, 1)).offset()) + + } finally { + Utils.closeQuietly(streams, "streams") + Utils.closeQuietly(client, "adminClient") + Utils.closeQuietly(producer, "producer") + } + } } object PlaintextAdminIntegrationTest { From 620a01b74bd8bbaf42eff002dd0a0692123ec2d5 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Tue, 9 Sep 2025 10:44:37 +0200 Subject: [PATCH 004/100] KAFKA-19661 [4/N]: Prefer range-style assignment (#20486) This is actually fixing a difference between the old and the new assignor. Given the assignment ordering, the legacy assignor has a preference for range-style assignments built in, that is, assigning C1: 0_0, 1_0 C2: 0_1, 1_1 instead of C1: 0_0, 0_1 C2: 1_0, 1_1 We add tests to both assignors to check for this behavior, and improve the new assingor by enforcing corresponding orderings. Reviewers: Bill Bejeck --- .../streams/assignor/StickyTaskAssignor.java | 30 +++- .../assignor/StickyTaskAssignorTest.java | 142 ++++++++++++++++++ .../LegacyStickyTaskAssignorTest.java | 113 ++++++++++++++ .../streams/DescribeStreamsGroupTest.java | 12 +- 4 files changed, 283 insertions(+), 14 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignor.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignor.java index 7ef5a382584..fc29f93b883 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignor.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignor.java @@ -25,6 +25,7 @@ import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedList; import java.util.Map; import java.util.Optional; import java.util.PriorityQueue; @@ -53,8 +54,7 @@ public class StickyTaskAssignor implements TaskAssignor { } private GroupAssignment doAssign(final GroupSpec groupSpec, final TopologyDescriber topologyDescriber) { - //active - final Set activeTasks = taskIds(topologyDescriber, true); + final LinkedList activeTasks = taskIds(topologyDescriber, true); assignActive(activeTasks); //standby @@ -62,15 +62,15 @@ public class StickyTaskAssignor implements TaskAssignor { groupSpec.assignmentConfigs().isEmpty() ? 0 : Integer.parseInt(groupSpec.assignmentConfigs().get("num.standby.replicas")); if (numStandbyReplicas > 0) { - final Set statefulTasks = taskIds(topologyDescriber, false); + final LinkedList statefulTasks = taskIds(topologyDescriber, false); assignStandby(statefulTasks, numStandbyReplicas); } return buildGroupAssignment(groupSpec.members().keySet()); } - private Set taskIds(final TopologyDescriber topologyDescriber, final boolean isActive) { - final Set ret = new HashSet<>(); + private LinkedList taskIds(final TopologyDescriber topologyDescriber, final boolean isActive) { + final LinkedList ret = new LinkedList<>(); for (final String subtopology : topologyDescriber.subtopologies()) { if (isActive || topologyDescriber.isStateful(subtopology)) { final int numberOfPartitions = topologyDescriber.maxNumInputPartitions(subtopology); @@ -166,7 +166,10 @@ public class StickyTaskAssignor implements TaskAssignor { return ret; } - private void assignActive(final Set activeTasks) { + private void assignActive(final LinkedList activeTasks) { + + // Assuming our current assignment pairs same partitions (range-based), we want to sort by partition first + activeTasks.sort(Comparator.comparing(TaskId::partition).thenComparing(TaskId::subtopologyId)); // 1. re-assigning existing active tasks to clients that previously had the same active tasks for (final Iterator it = activeTasks.iterator(); it.hasNext();) { @@ -193,6 +196,9 @@ public class StickyTaskAssignor implements TaskAssignor { } } + // To achieve an initially range-based assignment, sort by subtopology + activeTasks.sort(Comparator.comparing(TaskId::subtopologyId).thenComparing(TaskId::partition)); + // 3. assign any remaining unassigned tasks final PriorityQueue processByLoad = new PriorityQueue<>(Comparator.comparingDouble(ProcessState::load)); processByLoad.addAll(localState.processIdToState.values()); @@ -296,9 +302,13 @@ public class StickyTaskAssignor implements TaskAssignor { return localState.processIdToState.get(member.processId).memberToTaskCounts().get(member.memberId) < localState.tasksPerMember; } - private void assignStandby(final Set standbyTasks, final int numStandbyReplicas) { + private void assignStandby(final LinkedList standbyTasks, int numStandbyReplicas) { final ArrayList toLeastLoaded = new ArrayList<>(standbyTasks.size() * numStandbyReplicas); - for (final TaskId task : standbyTasks) { + + // Assuming our current assignment is range-based, we want to sort by partition first. + standbyTasks.sort(Comparator.comparing(TaskId::partition).thenComparing(TaskId::subtopologyId).reversed()); + + for (TaskId task : standbyTasks) { for (int i = 0; i < numStandbyReplicas; i++) { // prev active task @@ -329,6 +339,10 @@ public class StickyTaskAssignor implements TaskAssignor { } } + // To achieve a range-based assignment, sort by subtopology + toLeastLoaded.sort(Comparator.comparing(x -> x.taskId.subtopologyId()) + .thenComparing(x -> x.taskId.partition()).reversed()); + final PriorityQueue processByLoad = new PriorityQueue<>(Comparator.comparingDouble(ProcessState::load)); processByLoad.addAll(localState.processIdToState.values()); for (final StandbyToAssign toAssign : toLeastLoaded) { diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignorTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignorTest.java index b4fa9c4db99..1e9d4115cb2 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignorTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignorTest.java @@ -1091,6 +1091,148 @@ public class StickyTaskAssignorTest { assertEquals(numTasks, allStandbyTasks.size()); } + @Test + public void shouldReassignTasksWhenNewNodeJoinsWithExistingActiveAndStandbyAssignments() { + // Initial setup: Node 1 has active tasks 0,1 and standby tasks 2,3 + // Node 2 has active tasks 2,3 and standby tasks 0,1 + final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", + mkMap(mkEntry("test-subtopology", Sets.newSet(0, 1))), + mkMap(mkEntry("test-subtopology", Sets.newSet(2, 3)))); + + final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", + mkMap(mkEntry("test-subtopology", Sets.newSet(2, 3))), + mkMap(mkEntry("test-subtopology", Sets.newSet(0, 1)))); + + // Node 3 joins as new client + final AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process3"); + + final Map members = mkMap( + mkEntry("member1", memberSpec1), mkEntry("member2", memberSpec2), mkEntry("member3", memberSpec3)); + + final GroupAssignment result = assignor.assign( + new GroupSpecImpl(members, mkMap(mkEntry(NUM_STANDBY_REPLICAS_CONFIG, "1"))), + new TopologyDescriberImpl(4, true, List.of("test-subtopology")) + ); + + // Verify all active tasks are assigned + final Set allAssignedActiveTasks = new HashSet<>(); + allAssignedActiveTasks.addAll(getAllActiveTaskIds(result, "member1")); + allAssignedActiveTasks.addAll(getAllActiveTaskIds(result, "member2")); + allAssignedActiveTasks.addAll(getAllActiveTaskIds(result, "member3")); + assertEquals(Sets.newSet(0, 1, 2, 3), allAssignedActiveTasks); + + // Verify all standby tasks are assigned + final Set allAssignedStandbyTasks = new HashSet<>(); + allAssignedStandbyTasks.addAll(getAllStandbyTaskIds(result, "member1")); + allAssignedStandbyTasks.addAll(getAllStandbyTaskIds(result, "member2")); + allAssignedStandbyTasks.addAll(getAllStandbyTaskIds(result, "member3")); + assertEquals(Sets.newSet(0, 1, 2, 3), allAssignedStandbyTasks); + + // Verify each member has 1-2 active tasks and at most 3 tasks total + assertTrue(getAllActiveTaskIds(result, "member1").size() >= 1 && getAllActiveTaskIds(result, "member1").size() <= 2); + assertTrue(getAllActiveTaskIds(result, "member1").size() + getAllStandbyTaskIds(result, "member1").size() <= 3); + + assertTrue(getAllActiveTaskIds(result, "member2").size() >= 1 && getAllActiveTaskIds(result, "member2").size() <= 2); + assertTrue(getAllActiveTaskIds(result, "member2").size() + getAllStandbyTaskIds(result, "member2").size() <= 3); + + assertTrue(getAllActiveTaskIds(result, "member3").size() >= 1 && getAllActiveTaskIds(result, "member3").size() <= 2); + assertTrue(getAllActiveTaskIds(result, "member3").size() + getAllStandbyTaskIds(result, "member3").size() <= 3); + } + + @Test + public void shouldRangeAssignTasksWhenScalingUp() { + // Two clients, the second one is new + final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", + Map.of("test-subtopology1", Set.of(0, 1), "test-subtopology2", Set.of(0, 1)), + Map.of()); + final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2"); + final Map members = mkMap( + mkEntry("member1", memberSpec1), mkEntry("member2", memberSpec2)); + + // Two subtopologies with 2 tasks each (4 tasks total) with standby replicas enabled + final GroupAssignment result = assignor.assign( + new GroupSpecImpl(members, mkMap(mkEntry(NUM_STANDBY_REPLICAS_CONFIG, String.valueOf(1)))), + new TopologyDescriberImpl(2, true, Arrays.asList("test-subtopology1", "test-subtopology2")) + ); + + // Each client should get one task from each subtopology + final MemberAssignment testMember1 = result.members().get("member1"); + assertNotNull(testMember1); + assertEquals(1, testMember1.activeTasks().get("test-subtopology1").size()); + assertEquals(1, testMember1.activeTasks().get("test-subtopology2").size()); + + final MemberAssignment testMember2 = result.members().get("member2"); + assertNotNull(testMember2); + assertEquals(1, testMember2.activeTasks().get("test-subtopology1").size()); + assertEquals(1, testMember2.activeTasks().get("test-subtopology2").size()); + + // Verify all tasks are assigned exactly once + final Set allSubtopology1Tasks = new HashSet<>(); + allSubtopology1Tasks.addAll(testMember1.activeTasks().get("test-subtopology1")); + allSubtopology1Tasks.addAll(testMember2.activeTasks().get("test-subtopology1")); + assertEquals(Sets.newSet(0, 1), allSubtopology1Tasks); + + final Set allSubtopology2Tasks = new HashSet<>(); + allSubtopology2Tasks.addAll(testMember1.activeTasks().get("test-subtopology2")); + allSubtopology2Tasks.addAll(testMember2.activeTasks().get("test-subtopology2")); + assertEquals(Sets.newSet(0, 1), allSubtopology2Tasks); + + // Each client should get one task from each subtopology + assertNotNull(testMember1); + assertEquals(1, testMember1.standbyTasks().get("test-subtopology1").size()); + assertEquals(1, testMember1.standbyTasks().get("test-subtopology2").size()); + + assertNotNull(testMember2); + assertEquals(1, testMember2.standbyTasks().get("test-subtopology1").size()); + assertEquals(1, testMember2.standbyTasks().get("test-subtopology2").size()); + } + + @Test + public void shouldRangeAssignTasksWhenStartingEmpty() { + // Two clients starting empty (no previous tasks) + final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1"); + final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2"); + final Map members = mkMap( + mkEntry("member1", memberSpec1), mkEntry("member2", memberSpec2)); + + // Two subtopologies with 2 tasks each (4 tasks total) with standby replicas enabled + final GroupAssignment result = assignor.assign( + new GroupSpecImpl(members, mkMap(mkEntry(NUM_STANDBY_REPLICAS_CONFIG, String.valueOf(1)))), + new TopologyDescriberImpl(2, true, Arrays.asList("test-subtopology1", "test-subtopology2")) + ); + + // Each client should get one task from each subtopology + final MemberAssignment testMember1 = result.members().get("member1"); + assertNotNull(testMember1); + assertEquals(1, testMember1.activeTasks().get("test-subtopology1").size()); + assertEquals(1, testMember1.activeTasks().get("test-subtopology2").size()); + + final MemberAssignment testMember2 = result.members().get("member2"); + assertNotNull(testMember2); + assertEquals(1, testMember2.activeTasks().get("test-subtopology1").size()); + assertEquals(1, testMember2.activeTasks().get("test-subtopology2").size()); + + // Verify all tasks are assigned exactly once + final Set allSubtopology1Tasks = new HashSet<>(); + allSubtopology1Tasks.addAll(testMember1.activeTasks().get("test-subtopology1")); + allSubtopology1Tasks.addAll(testMember2.activeTasks().get("test-subtopology1")); + assertEquals(Sets.newSet(0, 1), allSubtopology1Tasks); + + final Set allSubtopology2Tasks = new HashSet<>(); + allSubtopology2Tasks.addAll(testMember1.activeTasks().get("test-subtopology2")); + allSubtopology2Tasks.addAll(testMember2.activeTasks().get("test-subtopology2")); + assertEquals(Sets.newSet(0, 1), allSubtopology2Tasks); + + // Each client should get one task from each subtopology + assertNotNull(testMember1); + assertEquals(1, testMember1.standbyTasks().get("test-subtopology1").size()); + assertEquals(1, testMember1.standbyTasks().get("test-subtopology2").size()); + + assertNotNull(testMember2); + assertEquals(1, testMember2.standbyTasks().get("test-subtopology1").size()); + assertEquals(1, testMember2.standbyTasks().get("test-subtopology2").size()); + } + private int getAllActiveTaskCount(GroupAssignment result, String... memberIds) { int size = 0; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/LegacyStickyTaskAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/LegacyStickyTaskAssignorTest.java index 3103e72cd52..64830546f0f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/LegacyStickyTaskAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/LegacyStickyTaskAssignorTest.java @@ -1263,6 +1263,119 @@ public class LegacyStickyTaskAssignorTest { } } + @ParameterizedTest + @ValueSource(strings = { + StreamsConfig.RACK_AWARE_ASSIGNMENT_STRATEGY_NONE, + StreamsConfig.RACK_AWARE_ASSIGNMENT_STRATEGY_MIN_TRAFFIC, + StreamsConfig.RACK_AWARE_ASSIGNMENT_STRATEGY_BALANCE_SUBTOPOLOGY + }) + public void shouldReassignTasksWhenNewNodeJoinsWithExistingActiveAndStandbyAssignments(final String rackAwareStrategy) { + setUp(rackAwareStrategy); + + // Initial setup: Node 1 has active tasks 0,1 and standby tasks 2,3 + // Node 2 has active tasks 2,3 and standby tasks 0,1 + final ClientState node1 = createClientWithPreviousActiveTasks(PID_1, 1, TASK_0_0, TASK_0_1); + node1.addPreviousStandbyTasks(Set.of(TASK_0_2, TASK_0_3)); + + final ClientState node2 = createClientWithPreviousActiveTasks(PID_2, 1, TASK_0_2, TASK_0_3); + node2.addPreviousStandbyTasks(Set.of(TASK_0_0, TASK_0_1)); + + // Node 3 joins as new client + final ClientState node3 = createClient(PID_3, 1); + + final boolean probingRebalanceNeeded = assign(1, rackAwareStrategy, TASK_0_0, TASK_0_1, TASK_0_2, TASK_0_3); + assertThat(probingRebalanceNeeded, is(false)); + + // Verify all active tasks are assigned + final Set allAssignedActiveTasks = new HashSet<>(); + allAssignedActiveTasks.addAll(node1.activeTasks()); + allAssignedActiveTasks.addAll(node2.activeTasks()); + allAssignedActiveTasks.addAll(node3.activeTasks()); + assertThat(allAssignedActiveTasks, equalTo(Set.of(TASK_0_0, TASK_0_1, TASK_0_2, TASK_0_3))); + + // Verify all standby tasks are assigned + final Set allAssignedStandbyTasks = new HashSet<>(); + allAssignedStandbyTasks.addAll(node1.standbyTasks()); + allAssignedStandbyTasks.addAll(node2.standbyTasks()); + allAssignedStandbyTasks.addAll(node3.standbyTasks()); + assertThat(allAssignedStandbyTasks, equalTo(Set.of(TASK_0_0, TASK_0_1, TASK_0_2, TASK_0_3))); + + // Verify each client has 1-2 active tasks and at most 3 tasks total + assertThat(node1.activeTasks().size(), greaterThanOrEqualTo(1)); + assertThat(node1.activeTasks().size(), lessThanOrEqualTo(2)); + assertThat(node1.activeTasks().size() + node1.standbyTasks().size(), lessThanOrEqualTo(3)); + + assertThat(node2.activeTasks().size(), greaterThanOrEqualTo(1)); + assertThat(node2.activeTasks().size(), lessThanOrEqualTo(2)); + assertThat(node2.activeTasks().size() + node2.standbyTasks().size(), lessThanOrEqualTo(3)); + + assertThat(node3.activeTasks().size(), greaterThanOrEqualTo(1)); + assertThat(node3.activeTasks().size(), lessThanOrEqualTo(2)); + assertThat(node3.activeTasks().size() + node3.standbyTasks().size(), lessThanOrEqualTo(3)); + } + + @ParameterizedTest + @ValueSource(strings = { + StreamsConfig.RACK_AWARE_ASSIGNMENT_STRATEGY_NONE, + StreamsConfig.RACK_AWARE_ASSIGNMENT_STRATEGY_MIN_TRAFFIC, + StreamsConfig.RACK_AWARE_ASSIGNMENT_STRATEGY_BALANCE_SUBTOPOLOGY + }) + public void shouldRangeAssignTasksWhenStartingEmpty(final String rackAwareStrategy) { + setUp(rackAwareStrategy); + + // Two clients with capacity 1 each, starting empty (no previous tasks) + createClient(PID_1, 1); + createClient(PID_2, 1); + + // Two subtopologies with 2 tasks each (4 tasks total) + final boolean probingRebalanceNeeded = assign(1, rackAwareStrategy, TASK_0_0, TASK_0_1, TASK_1_0, TASK_1_1); + assertThat(probingRebalanceNeeded, is(false)); + + // Each client should get one active task from each subtopology + final ClientState client1 = clients.get(PID_1); + final ClientState client2 = clients.get(PID_2); + + // Check that each client has one active task from subtopology 0 + final long client1Subtopology0ActiveCount = client1.activeTasks().stream() + .filter(task -> task.subtopology() == 0) + .count(); + final long client2Subtopology0ActiveCount = client2.activeTasks().stream() + .filter(task -> task.subtopology() == 0) + .count(); + assertThat(client1Subtopology0ActiveCount, equalTo(1L)); + assertThat(client2Subtopology0ActiveCount, equalTo(1L)); + + // Check that each client has one active task from subtopology 1 + final long client1Subtopology1ActiveCount = client1.activeTasks().stream() + .filter(task -> task.subtopology() == 1) + .count(); + final long client2Subtopology1ActiveCount = client2.activeTasks().stream() + .filter(task -> task.subtopology() == 1) + .count(); + assertThat(client1Subtopology1ActiveCount, equalTo(1L)); + assertThat(client2Subtopology1ActiveCount, equalTo(1L)); + + // Check that each client has one standby task from subtopology 0 + final long client1Subtopology0StandbyCount = client1.standbyTasks().stream() + .filter(task -> task.subtopology() == 0) + .count(); + final long client2Subtopology0StandbyCount = client2.standbyTasks().stream() + .filter(task -> task.subtopology() == 0) + .count(); + assertThat(client1Subtopology0StandbyCount, equalTo(1L)); + assertThat(client2Subtopology0StandbyCount, equalTo(1L)); + + // Check that each client has one standby task from subtopology 1 + final long client1Subtopology1StandbyCount = client1.standbyTasks().stream() + .filter(task -> task.subtopology() == 1) + .count(); + final long client2Subtopology1StandbyCount = client2.standbyTasks().stream() + .filter(task -> task.subtopology() == 1) + .count(); + assertThat(client1Subtopology1StandbyCount, equalTo(1L)); + assertThat(client2Subtopology1StandbyCount, equalTo(1L)); + } + private boolean assign(final String rackAwareStrategy, final TaskId... tasks) { return assign(0, rackAwareStrategy, tasks); } diff --git a/tools/src/test/java/org/apache/kafka/tools/streams/DescribeStreamsGroupTest.java b/tools/src/test/java/org/apache/kafka/tools/streams/DescribeStreamsGroupTest.java index 9c4d4016748..6d7ea57b8ac 100644 --- a/tools/src/test/java/org/apache/kafka/tools/streams/DescribeStreamsGroupTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/streams/DescribeStreamsGroupTest.java @@ -180,8 +180,8 @@ public class DescribeStreamsGroupTest { public void testDescribeStreamsGroupWithMembersOption() throws Exception { final List expectedHeader = List.of("GROUP", "MEMBER", "PROCESS", "CLIENT-ID", "ASSIGNMENTS"); final Set> expectedRows = Set.of( - List.of(APP_ID, "", "", "", "ACTIVE:", "0:[0,1];"), - List.of(APP_ID, "", "", "", "ACTIVE:", "1:[0,1];")); + List.of(APP_ID, "", "", "", "ACTIVE:", "0:[1];", "1:[1];"), + List.of(APP_ID, "", "", "", "ACTIVE:", "0:[0];", "1:[0];")); // The member and process names as well as client-id are not deterministic, so we don't care about them. final List dontCares = List.of(1, 2, 3); @@ -193,8 +193,8 @@ public class DescribeStreamsGroupTest { public void testDescribeStreamsGroupWithMembersAndVerboseOptions() throws Exception { final List expectedHeader = List.of("GROUP", "TARGET-ASSIGNMENT-EPOCH", "TOPOLOGY-EPOCH", "MEMBER", "MEMBER-PROTOCOL", "MEMBER-EPOCH", "PROCESS", "CLIENT-ID", "ASSIGNMENTS"); final Set> expectedRows = Set.of( - List.of(APP_ID, "3", "0", "", "streams", "3", "", "", "ACTIVE:", "0:[0,1];", "TARGET-ACTIVE:", "0:[0,1];"), - List.of(APP_ID, "3", "0", "", "streams", "3", "", "", "ACTIVE:", "1:[0,1];", "TARGET-ACTIVE:", "1:[0,1];")); + List.of(APP_ID, "3", "0", "", "streams", "3", "", "", "ACTIVE:", "0:[1];", "1:[1];", "TARGET-ACTIVE:", "0:[1];", "1:[1];"), + List.of(APP_ID, "3", "0", "", "streams", "3", "", "", "ACTIVE:", "0:[0];", "1:[0];", "TARGET-ACTIVE:", "0:[0];", "1:[0];")); // The member and process names as well as client-id are not deterministic, so we don't care about them. final List dontCares = List.of(3, 6, 7); @@ -212,8 +212,8 @@ public class DescribeStreamsGroupTest { final List expectedHeader = List.of("GROUP", "TARGET-ASSIGNMENT-EPOCH", "TOPOLOGY-EPOCH", "MEMBER", "MEMBER-PROTOCOL", "MEMBER-EPOCH", "PROCESS", "CLIENT-ID", "ASSIGNMENTS"); final Set> expectedRows1 = Set.of( - List.of(APP_ID, "3", "0", "", "streams", "3", "", "", "ACTIVE:", "0:[0,1];", "TARGET-ACTIVE:", "0:[0,1];"), - List.of(APP_ID, "3", "0", "", "streams", "3", "", "", "ACTIVE:", "1:[0,1];", "TARGET-ACTIVE:", "1:[0,1];")); + List.of(APP_ID, "3", "0", "", "streams", "3", "", "", "ACTIVE:", "0:[1];", "1:[1];", "TARGET-ACTIVE:", "0:[1];", "1:[1];"), + List.of(APP_ID, "3", "0", "", "streams", "3", "", "", "ACTIVE:", "0:[0];", "1:[0];", "TARGET-ACTIVE:", "0:[0];", "1:[0];")); final Set> expectedRows2 = Set.of( List.of(APP_ID_2, "3", "0", "", "streams", "3", "", "", "ACTIVE:", "1:[0];", "TARGET-ACTIVE:", "1:[0];"), List.of(APP_ID_2, "3", "0", "", "streams", "3", "", "", "ACTIVE:", "0:[0];", "TARGET-ACTIVE:", "0:[0];")); From d5e624e9183d8a8dde02b73d4d2c90b6e7d40113 Mon Sep 17 00:00:00 2001 From: Chirag Wadhwa Date: Tue, 9 Sep 2025 15:51:42 +0530 Subject: [PATCH 005/100] KAFKA-19693: Added PersisterBatch record in Share Partition which includes updatedState and stateBatch (#20507) The method rollbackOrProcessStateUpdates in SharePartition received 2 separate lists of updatedStates (InFlightState) and stateBatches (PersisterStateBatch). This PR introduces a new subclass called `PersisterBatch` which encompasses both these objects. Reviewers: Apoorv Mittal --- .../kafka/server/share/SharePartition.java | 160 +++++++++--------- 1 file changed, 78 insertions(+), 82 deletions(-) diff --git a/core/src/main/java/kafka/server/share/SharePartition.java b/core/src/main/java/kafka/server/share/SharePartition.java index 3ca11cbab1f..2c330076733 100644 --- a/core/src/main/java/kafka/server/share/SharePartition.java +++ b/core/src/main/java/kafka/server/share/SharePartition.java @@ -884,8 +884,7 @@ public class SharePartition { CompletableFuture future = new CompletableFuture<>(); Throwable throwable = null; - List updatedStates = new ArrayList<>(); - List stateBatches = new ArrayList<>(); + List persisterBatches = new ArrayList<>(); lock.writeLock().lock(); try { // Avoided using enhanced for loop as need to check if the last batch have offsets @@ -925,8 +924,7 @@ public class SharePartition { batch, recordStateMap, subMap, - updatedStates, - stateBatches + persisterBatches ); if (ackThrowable.isPresent()) { @@ -939,7 +937,7 @@ public class SharePartition { } // If the acknowledgement is successful then persist state, complete the state transition // and update the cached state for start offset. Else rollback the state transition. - rollbackOrProcessStateUpdates(future, throwable, updatedStates, stateBatches); + rollbackOrProcessStateUpdates(future, throwable, persisterBatches); return future; } @@ -955,8 +953,7 @@ public class SharePartition { CompletableFuture future = new CompletableFuture<>(); Throwable throwable = null; - List updatedStates = new ArrayList<>(); - List stateBatches = new ArrayList<>(); + List persisterBatches = new ArrayList<>(); lock.writeLock().lock(); try { @@ -975,14 +972,14 @@ public class SharePartition { } if (inFlightBatch.offsetState() != null) { - Optional releaseAcquiredRecordsThrowable = releaseAcquiredRecordsForPerOffsetBatch(memberId, inFlightBatch, recordState, updatedStates, stateBatches); + Optional releaseAcquiredRecordsThrowable = releaseAcquiredRecordsForPerOffsetBatch(memberId, inFlightBatch, recordState, persisterBatches); if (releaseAcquiredRecordsThrowable.isPresent()) { throwable = releaseAcquiredRecordsThrowable.get(); break; } continue; } - Optional releaseAcquiredRecordsThrowable = releaseAcquiredRecordsForCompleteBatch(memberId, inFlightBatch, recordState, updatedStates, stateBatches); + Optional releaseAcquiredRecordsThrowable = releaseAcquiredRecordsForCompleteBatch(memberId, inFlightBatch, recordState, persisterBatches); if (releaseAcquiredRecordsThrowable.isPresent()) { throwable = releaseAcquiredRecordsThrowable.get(); break; @@ -993,7 +990,7 @@ public class SharePartition { } // If the release acquired records is successful then persist state, complete the state transition // and update the cached state for start offset. Else rollback the state transition. - rollbackOrProcessStateUpdates(future, throwable, updatedStates, stateBatches); + rollbackOrProcessStateUpdates(future, throwable, persisterBatches); return future; } @@ -1004,8 +1001,7 @@ public class SharePartition { private Optional releaseAcquiredRecordsForPerOffsetBatch(String memberId, InFlightBatch inFlightBatch, RecordState recordState, - List updatedStates, - List stateBatches) { + List persisterBatches) { log.trace("Offset tracked batch record found, batch: {} for the share partition: {}-{}", inFlightBatch, groupId, topicIdPartition); @@ -1032,10 +1028,9 @@ public class SharePartition { return Optional.of(new InvalidRecordStateException("Unable to release acquired records for the offset")); } - // Successfully updated the state of the offset. - updatedStates.add(updateResult); - stateBatches.add(new PersisterStateBatch(offsetState.getKey(), offsetState.getKey(), - updateResult.state().id(), (short) updateResult.deliveryCount())); + // Successfully updated the state of the offset and created a persister state batch for write to persister. + persisterBatches.add(new PersisterBatch(updateResult, new PersisterStateBatch(offsetState.getKey(), + offsetState.getKey(), updateResult.state().id(), (short) updateResult.deliveryCount()))); // Do not update the next fetch offset as the offset has not completed the transition yet. } } @@ -1045,8 +1040,7 @@ public class SharePartition { private Optional releaseAcquiredRecordsForCompleteBatch(String memberId, InFlightBatch inFlightBatch, RecordState recordState, - List updatedStates, - List stateBatches) { + List persisterBatches) { // Check if member id is the owner of the batch. if (!inFlightBatch.batchMemberId().equals(memberId) && !inFlightBatch.batchMemberId().equals(EMPTY_MEMBER_ID)) { @@ -1072,10 +1066,9 @@ public class SharePartition { return Optional.of(new InvalidRecordStateException("Unable to release acquired records for the batch")); } - // Successfully updated the state of the batch. - updatedStates.add(updateResult); - stateBatches.add(new PersisterStateBatch(inFlightBatch.firstOffset(), inFlightBatch.lastOffset(), - updateResult.state().id(), (short) updateResult.deliveryCount())); + // Successfully updated the state of the batch and created a persister state batch for write to persister. + persisterBatches.add(new PersisterBatch(updateResult, new PersisterStateBatch(inFlightBatch.firstOffset(), + inFlightBatch.lastOffset(), updateResult.state().id(), (short) updateResult.deliveryCount()))); // Do not update the next fetch offset as the batch has not completed the transition yet. } return Optional.empty(); @@ -1826,8 +1819,7 @@ public class SharePartition { ShareAcknowledgementBatch batch, Map recordStateMap, NavigableMap subMap, - final List updatedStates, - List stateBatches + List persisterBatches ) { Optional throwable; lock.writeLock().lock(); @@ -1889,11 +1881,11 @@ public class SharePartition { } throwable = acknowledgePerOffsetBatchRecords(memberId, batch, inFlightBatch, - recordStateMap, updatedStates, stateBatches); + recordStateMap, persisterBatches); } else { // The in-flight batch is a full match hence change the state of the complete batch. throwable = acknowledgeCompleteBatch(batch, inFlightBatch, - recordStateMap.get(batch.firstOffset()), updatedStates, stateBatches); + recordStateMap.get(batch.firstOffset()), persisterBatches); } if (throwable.isPresent()) { @@ -1930,8 +1922,7 @@ public class SharePartition { ShareAcknowledgementBatch batch, InFlightBatch inFlightBatch, Map recordStateMap, - List updatedStates, - List stateBatches + List persisterBatches ) { lock.writeLock().lock(); try { @@ -1995,10 +1986,9 @@ public class SharePartition { return Optional.of(new InvalidRecordStateException( "Unable to acknowledge records for the batch")); } - // Successfully updated the state of the offset. - updatedStates.add(updateResult); - stateBatches.add(new PersisterStateBatch(offsetState.getKey(), offsetState.getKey(), - updateResult.state().id(), (short) updateResult.deliveryCount())); + // Successfully updated the state of the offset and created a persister state batch for write to persister. + persisterBatches.add(new PersisterBatch(updateResult, new PersisterStateBatch(offsetState.getKey(), + offsetState.getKey(), updateResult.state().id(), (short) updateResult.deliveryCount()))); // Do not update the nextFetchOffset as the offset has not completed the transition yet. } } finally { @@ -2011,8 +2001,7 @@ public class SharePartition { ShareAcknowledgementBatch batch, InFlightBatch inFlightBatch, RecordState recordState, - List updatedStates, - List stateBatches + List persisterBatches ) { lock.writeLock().lock(); try { @@ -2044,11 +2033,9 @@ public class SharePartition { new InvalidRecordStateException("Unable to acknowledge records for the batch")); } - // Successfully updated the state of the batch. - updatedStates.add(updateResult); - stateBatches.add( - new PersisterStateBatch(inFlightBatch.firstOffset(), inFlightBatch.lastOffset(), - updateResult.state().id(), (short) updateResult.deliveryCount())); + // Successfully updated the state of the batch and created a persister state batch for write to persister. + persisterBatches.add(new PersisterBatch(updateResult, new PersisterStateBatch(inFlightBatch.firstOffset(), + inFlightBatch.lastOffset(), updateResult.state().id(), (short) updateResult.deliveryCount()))); // Do not update the next fetch offset as the batch has not completed the transition yet. } finally { lock.writeLock().unlock(); @@ -2090,8 +2077,7 @@ public class SharePartition { void rollbackOrProcessStateUpdates( CompletableFuture future, Throwable throwable, - List updatedStates, - List stateBatches + List persisterBatches ) { lock.writeLock().lock(); try { @@ -2099,9 +2085,9 @@ public class SharePartition { // Log in DEBUG to avoid flooding of logs for a faulty client. log.debug("Request failed for updating state, rollback any changed state" + " for the share partition: {}-{}", groupId, topicIdPartition); - updatedStates.forEach(state -> { - state.completeStateTransition(false); - if (state.state() == RecordState.AVAILABLE) { + persisterBatches.forEach(persisterBatch -> { + persisterBatch.updatedState.completeStateTransition(false); + if (persisterBatch.updatedState.state() == RecordState.AVAILABLE) { updateFindNextFetchOffset(true); } }); @@ -2109,7 +2095,7 @@ public class SharePartition { return; } - if (stateBatches.isEmpty() && updatedStates.isEmpty()) { + if (persisterBatches.isEmpty()) { future.complete(null); return; } @@ -2117,47 +2103,48 @@ public class SharePartition { lock.writeLock().unlock(); } - writeShareGroupState(stateBatches).whenComplete((result, exception) -> { - // There can be a pending delayed share fetch requests for the share partition which are waiting - // on the startOffset to move ahead, hence track if the state is updated in the cache. If - // yes, then notify the delayed share fetch purgatory to complete the pending requests. - boolean cacheStateUpdated = false; - lock.writeLock().lock(); - try { - if (exception != null) { - log.debug("Failed to write state to persister for the share partition: {}-{}", - groupId, topicIdPartition, exception); - // In case of failure when transition state is rolled back then it should be rolled - // back to ACQUIRED state, unless acquisition lock for the state has expired. - updatedStates.forEach(state -> { - state.completeStateTransition(false); - if (state.state() == RecordState.AVAILABLE) { + writeShareGroupState(persisterBatches.stream().map(PersisterBatch::stateBatch).toList()) + .whenComplete((result, exception) -> { + // There can be a pending delayed share fetch requests for the share partition which are waiting + // on the startOffset to move ahead, hence track if the state is updated in the cache. If + // yes, then notify the delayed share fetch purgatory to complete the pending requests. + boolean cacheStateUpdated = false; + lock.writeLock().lock(); + try { + if (exception != null) { + log.debug("Failed to write state to persister for the share partition: {}-{}", + groupId, topicIdPartition, exception); + // In case of failure when transition state is rolled back then it should be rolled + // back to ACQUIRED state, unless acquisition lock for the state has expired. + persisterBatches.forEach(persisterBatch -> { + persisterBatch.updatedState.completeStateTransition(false); + if (persisterBatch.updatedState.state() == RecordState.AVAILABLE) { + updateFindNextFetchOffset(true); + } + }); + future.completeExceptionally(exception); + return; + } + + log.trace("State change request successful for share partition: {}-{}", + groupId, topicIdPartition); + persisterBatches.forEach(persisterBatch -> { + persisterBatch.updatedState.completeStateTransition(true); + if (persisterBatch.updatedState.state() == RecordState.AVAILABLE) { updateFindNextFetchOffset(true); } }); - future.completeExceptionally(exception); - return; + // Update the cached state and start and end offsets after acknowledging/releasing the acquired records. + cacheStateUpdated = maybeUpdateCachedStateAndOffsets(); + future.complete(null); + } finally { + lock.writeLock().unlock(); + // Maybe complete the delayed share fetch request if the state has been changed in cache + // which might have moved start offset ahead. Hence, the pending delayed share fetch + // request can be completed. The call should be made outside the lock to avoid deadlock. + maybeCompleteDelayedShareFetchRequest(cacheStateUpdated); } - - log.trace("State change request successful for share partition: {}-{}", - groupId, topicIdPartition); - updatedStates.forEach(state -> { - state.completeStateTransition(true); - if (state.state() == RecordState.AVAILABLE) { - updateFindNextFetchOffset(true); - } - }); - // Update the cached state and start and end offsets after acknowledging/releasing the acquired records. - cacheStateUpdated = maybeUpdateCachedStateAndOffsets(); - future.complete(null); - } finally { - lock.writeLock().unlock(); - // Maybe complete the delayed share fetch request if the state has been changed in cache - // which might have moved start offset ahead. Hence, the pending delayed share fetch - // request can be completed. The call should be made outside the lock to avoid deadlock. - maybeCompleteDelayedShareFetchRequest(cacheStateUpdated); - } - }); + }); } private boolean maybeUpdateCachedStateAndOffsets() { @@ -2929,6 +2916,15 @@ public class SharePartition { } } + /** + * PersisterBatch class is used to record the state updates for a batch or an offset. + * It contains the updated in-flight state and the persister state batch to be sent to persister. + */ + private record PersisterBatch( + InFlightState updatedState, + PersisterStateBatch stateBatch + ) { } + /** * LastOffsetAndMaxRecords class is used to track the last offset to acquire and the maximum number * of records that can be acquired in a fetch request. From 93512704dad1865da5ee0cb8a165ff08d7b72743 Mon Sep 17 00:00:00 2001 From: Mickael Maison Date: Tue, 9 Sep 2025 18:14:57 +0200 Subject: [PATCH 006/100] MINOR: Add 4.1.0 to system tests (#20476) Adding 4.1.0 to system tests as per https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=34840886#ReleaseProcess-Afterrelease Reviewers: Chia-Ping Tsai --- gradle/dependencies.gradle | 2 ++ tests/docker/Dockerfile | 2 ++ vagrant/base.sh | 2 ++ 3 files changed, 6 insertions(+) diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 59e53409145..390c7e830c8 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -105,6 +105,7 @@ versions += [ kafka_38: "3.8.1", kafka_39: "3.9.1", kafka_40: "4.0.0", + kafka_41: "4.1.0", log4j2: "2.25.1", // When updating lz4 make sure the compression levels in org.apache.kafka.common.record.CompressionType are still valid lz4: "1.8.0", @@ -207,6 +208,7 @@ libs += [ kafkaStreams_38: "org.apache.kafka:kafka-streams:$versions.kafka_38", kafkaStreams_39: "org.apache.kafka:kafka-streams:$versions.kafka_39", kafkaStreams_40: "org.apache.kafka:kafka-streams:$versions.kafka_40", + kafkaStreams_41: "org.apache.kafka:kafka-streams:$versions.kafka_41", log4j1Bridge2Api: "org.apache.logging.log4j:log4j-1.2-api:$versions.log4j2", log4j2Api: "org.apache.logging.log4j:log4j-api:$versions.log4j2", log4j2Core: "org.apache.logging.log4j:log4j-core:$versions.log4j2", diff --git a/tests/docker/Dockerfile b/tests/docker/Dockerfile index 01c1a0533d3..bf69bc0e371 100644 --- a/tests/docker/Dockerfile +++ b/tests/docker/Dockerfile @@ -95,6 +95,7 @@ RUN mkdir -p "/opt/kafka-3.7.2" && chmod a+rw /opt/kafka-3.7.2 && curl -s "$KAFK RUN mkdir -p "/opt/kafka-3.8.1" && chmod a+rw /opt/kafka-3.8.1 && curl -s "$KAFKA_MIRROR/kafka_2.12-3.8.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-3.8.1" RUN mkdir -p "/opt/kafka-3.9.1" && chmod a+rw /opt/kafka-3.9.1 && curl -s "$KAFKA_MIRROR/kafka_2.12-3.9.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-3.9.1" RUN mkdir -p "/opt/kafka-4.0.0" && chmod a+rw /opt/kafka-4.0.0 && curl -s "$KAFKA_MIRROR/kafka_2.13-4.0.0.tgz" | tar xz --strip-components=1 -C "/opt/kafka-4.0.0" +RUN mkdir -p "/opt/kafka-4.1.0" && chmod a+rw /opt/kafka-4.1.0 && curl -s "$KAFKA_MIRROR/kafka_2.13-4.1.0.tgz" | tar xz --strip-components=1 -C "/opt/kafka-4.1.0" # Streams test dependencies @@ -117,6 +118,7 @@ RUN curl -s "$KAFKA_MIRROR/kafka-streams-3.7.2-test.jar" -o /opt/kafka-3.7.2/lib RUN curl -s "$KAFKA_MIRROR/kafka-streams-3.8.1-test.jar" -o /opt/kafka-3.8.1/libs/kafka-streams-3.8.1-test.jar RUN curl -s "$KAFKA_MIRROR/kafka-streams-3.9.1-test.jar" -o /opt/kafka-3.9.1/libs/kafka-streams-3.9.1-test.jar RUN curl -s "$KAFKA_MIRROR/kafka-streams-4.0.0-test.jar" -o /opt/kafka-4.0.0/libs/kafka-streams-4.0.0-test.jar +RUN curl -s "$KAFKA_MIRROR/kafka-streams-4.1.0-test.jar" -o /opt/kafka-4.1.0/libs/kafka-streams-4.1.0-test.jar # To ensure the Kafka cluster starts successfully under JDK 17, we need to update the Zookeeper # client from version 3.4.x to 3.5.7 in Kafka versions 2.1.1, 2.2.2, and 2.3.1, as the older Zookeeper diff --git a/vagrant/base.sh b/vagrant/base.sh index 87597de5022..4f5034042c7 100755 --- a/vagrant/base.sh +++ b/vagrant/base.sh @@ -152,6 +152,8 @@ get_kafka 3.9.1 2.12 chmod a+rw /opt/kafka-3.9.1 get_kafka 4.0.0 2.13 chmod a+rw /opt/kafka-4.0.0 +get_kafka 4.1.0 2.13 +chmod a+rw /opt/kafka-4.1.0 # To ensure the Kafka cluster starts successfully under JDK 17, we need to update the Zookeeper # client from version 3.4.x to 3.5.7 in Kafka versions 2.1.1, 2.2.2, and 2.3.1, as the older Zookeeper From 0bc2c6e69994c59252d3dc0992f09993445b6ad4 Mon Sep 17 00:00:00 2001 From: lucliu1108 Date: Tue, 9 Sep 2025 17:05:45 -0500 Subject: [PATCH 007/100] MINOR: Move topic creation before consumer creation in testListGroups integration test (#20496) This PR moves the topic creation before consumer creations in `PlaintextAdminIntegrationTest.testListGroups`, to avoid potential errors if consumer creates topic due to metadata update. See discussion https://github.com/apache/kafka/pull/20244#discussion_r2325557949 Reviewers: @chia7712, bbejeck@apache.org --- .../api/PlaintextAdminIntegrationTest.scala | 40 +++++++++---------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 3d172a678bc..6b1e3dda727 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -2580,6 +2580,12 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val config = createConfig client = Admin.create(config) + client.createTopics(util.Set.of( + new NewTopic(testTopicName, 1, 1.toShort) + )).all().get() + waitForTopics(client, List(testTopicName), List()) + val topicPartition = new TopicPartition(testTopicName, 0) + consumerConfig.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name) val classicGroupConfig = new Properties(consumerConfig) classicGroupConfig.put(ConsumerConfig.GROUP_ID_CONFIG, classicGroupId) @@ -2600,12 +2606,6 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { ) try { - client.createTopics(util.Set.of( - new NewTopic(testTopicName, 1, 1.toShort) - )).all().get() - waitForTopics(client, List(testTopicName), List()) - val topicPartition = new TopicPartition(testTopicName, 0) - classicGroup.subscribe(util.Set.of(testTopicName)) classicGroup.poll(JDuration.ofMillis(1000)) consumerGroup.subscribe(util.Set.of(testTopicName)) @@ -2628,20 +2628,22 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val consumerGroupListing = new GroupListing(consumerGroupId, Optional.of(GroupType.CONSUMER), "consumer", Optional.of(GroupState.STABLE)) val shareGroupListing = new GroupListing(shareGroupId, Optional.of(GroupType.SHARE), "share", Optional.of(GroupState.STABLE)) val simpleGroupListing = new GroupListing(simpleGroupId, Optional.of(GroupType.CLASSIC), "", Optional.of(GroupState.EMPTY)) - // Streams group could either be in STABLE or NOT_READY state - val streamsGroupListingStable = new GroupListing(streamsGroupId, Optional.of(GroupType.STREAMS), "streams", Optional.of(GroupState.STABLE)) - val streamsGroupListingNotReady = new GroupListing(streamsGroupId, Optional.of(GroupType.STREAMS), "streams", Optional.of(GroupState.NOT_READY)) + val streamsGroupListing = new GroupListing(streamsGroupId, Optional.of(GroupType.STREAMS), "streams", Optional.of(GroupState.STABLE)) var listGroupsResult = client.listGroups() assertTrue(listGroupsResult.errors().get().isEmpty) - val expectedStreamListings = Set(streamsGroupListingStable, streamsGroupListingNotReady) - val expectedListings = Set(classicGroupListing, simpleGroupListing, consumerGroupListing, shareGroupListing) - val actualListings = listGroupsResult.all().get().asScala.toSet - - // Check that actualListings contains all expectedListings and one of the streams listings - assertTrue(expectedListings.subsetOf(actualListings)) - assertTrue(actualListings.exists(expectedStreamListings.contains)) + TestUtils.waitUntilTrue(() => { + val listGroupResultScala = client.listGroups().all().get().asScala + val filteredStreamsGroups = listGroupResultScala.filter(_.groupId() == streamsGroupId) + val filteredClassicGroups = listGroupResultScala.filter(_.groupId() == classicGroupId) + val filteredConsumerGroups = listGroupResultScala.filter(_.groupId() == consumerGroupId) + val filteredShareGroups = listGroupResultScala.filter(_.groupId() == shareGroupId) + filteredClassicGroups.forall(_.groupState().orElse(null) == GroupState.STABLE) && + filteredConsumerGroups.forall(_.groupState().orElse(null) == GroupState.STABLE) && + filteredShareGroups.forall(_.groupState().orElse(null) == GroupState.STABLE) && + filteredStreamsGroups.forall(_.groupState().orElse(null) == GroupState.STABLE) + }, "Groups not stable yet") listGroupsResult = client.listGroups(new ListGroupsOptions().withTypes(util.Set.of(GroupType.CLASSIC))) assertTrue(listGroupsResult.errors().get().isEmpty) @@ -2660,10 +2662,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { listGroupsResult = client.listGroups(new ListGroupsOptions().withTypes(util.Set.of(GroupType.STREAMS))) assertTrue(listGroupsResult.errors().get().isEmpty) - assertTrue(listGroupsResult.all().get().asScala.toSet.equals(Set(streamsGroupListingStable)) || - listGroupsResult.all().get().asScala.toSet.equals(Set(streamsGroupListingNotReady))) - assertTrue(listGroupsResult.valid().get().asScala.toSet.equals(Set(streamsGroupListingStable)) || - listGroupsResult.valid().get().asScala.toSet.equals(Set(streamsGroupListingNotReady))) + assertEquals(Set(streamsGroupListing), listGroupsResult.all().get().asScala.toSet) + assertEquals(Set(streamsGroupListing), listGroupsResult.valid().get().asScala.toSet) } finally { Utils.closeQuietly(classicGroup, "classicGroup") From 45b96cb3a7493816860f837e378f2d91ce6f8bfe Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Wed, 10 Sep 2025 06:59:32 +0800 Subject: [PATCH 008/100] MINOR: add the explanation of `null` for DeleteAclsRequest#ResourceNameFilter (#20502) Add the explanation of `null` for DeleteAclsRequest#ResourceNameFilter Reviewers: Ken Huang , TengYao Chi , Chia-Ping Tsai --- .../src/main/resources/common/message/DeleteAclsRequest.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/resources/common/message/DeleteAclsRequest.json b/clients/src/main/resources/common/message/DeleteAclsRequest.json index db605305ae2..d794295af56 100644 --- a/clients/src/main/resources/common/message/DeleteAclsRequest.json +++ b/clients/src/main/resources/common/message/DeleteAclsRequest.json @@ -30,7 +30,7 @@ { "name": "ResourceTypeFilter", "type": "int8", "versions": "0+", "about": "The resource type." }, { "name": "ResourceNameFilter", "type": "string", "versions": "0+", "nullableVersions": "0+", - "about": "The resource name." }, + "about": "The resource name, or null to match any resource name." }, { "name": "PatternTypeFilter", "type": "int8", "versions": "1+", "default": "3", "ignorable": false, "about": "The pattern type." }, { "name": "PrincipalFilter", "type": "string", "versions": "0+", "nullableVersions": "0+", From 8065d5cb1d13e183403be9df7f8a072e44d9f49e Mon Sep 17 00:00:00 2001 From: Genseric Ghiro Date: Tue, 9 Sep 2025 19:03:19 -0400 Subject: [PATCH 009/100] MINOR: Making sure log appender is closed in ShareConsumerImplTest.java::testFailConstructor (#20514) Similarly to what was done for AsyncKafkaConsumerTest::testFailConstructor, [here](https://github.com/apache/kafka/pull/20491) Reviewers: Lianet Magrans , Chia-Ping Tsai --- .../consumer/internals/ShareConsumerImplTest.java | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImplTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImplTest.java index 0fa3def7c15..5dddd0772df 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImplTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImplTest.java @@ -213,18 +213,19 @@ public class ShareConsumerImplTest { props.put(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG, "an.invalid.class"); final ConsumerConfig config = new ConsumerConfig(props); - LogCaptureAppender appender = LogCaptureAppender.createAndRegister(); - KafkaException ce = assertThrows( + try (LogCaptureAppender appender = LogCaptureAppender.createAndRegister()) { + KafkaException ce = assertThrows( KafkaException.class, () -> newConsumer(config)); - assertTrue(ce.getMessage().contains("Failed to construct Kafka share consumer"), "Unexpected exception message: " + ce.getMessage()); - assertTrue(ce.getCause().getMessage().contains("Class an.invalid.class cannot be found"), "Unexpected cause: " + ce.getCause()); + assertTrue(ce.getMessage().contains("Failed to construct Kafka share consumer"), "Unexpected exception message: " + ce.getMessage()); + assertTrue(ce.getCause().getMessage().contains("Class an.invalid.class cannot be found"), "Unexpected cause: " + ce.getCause()); - boolean npeLogged = appender.getEvents().stream() + boolean npeLogged = appender.getEvents().stream() .flatMap(event -> event.getThrowableInfo().stream()) .anyMatch(str -> str.contains("NullPointerException")); - assertFalse(npeLogged, "Unexpected NullPointerException during consumer construction"); + assertFalse(npeLogged, "Unexpected NullPointerException during consumer construction"); + } } @Test From 8015c87390a4db7702468e9cea0ccd45ab61abf0 Mon Sep 17 00:00:00 2001 From: Nikita Shupletsov Date: Tue, 9 Sep 2025 21:18:00 -0700 Subject: [PATCH 010/100] KAFKA-19679: Fix NoSuchElementException in oldest open iterator metric (#20512) Querying the oldest-open-iterator metric can result in a NoSuchElementException when the last open iterator gets removed, due to a race condition between the query and the metric update. To avoid this race condition, this PR caches the metric result, to avoid accessing the list of open iterator directly. We don't need to clear this cache, because the entire metric is removed when the last iterator gets removed. Reviewers: Matthias J. Sax --- .../internals/metrics/OpenIterators.java | 14 +++- .../internals/metrics/OpenIteratorsTest.java | 73 +++++++++++++++++++ 2 files changed, 86 insertions(+), 1 deletion(-) create mode 100644 streams/src/test/java/org/apache/kafka/streams/internals/metrics/OpenIteratorsTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/OpenIterators.java b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/OpenIterators.java index d66e76f73cd..736af467bd2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/OpenIterators.java +++ b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/OpenIterators.java @@ -23,8 +23,10 @@ import org.apache.kafka.streams.state.internals.MeteredIterator; import org.apache.kafka.streams.state.internals.metrics.StateStoreMetrics; import java.util.Comparator; +import java.util.Iterator; import java.util.NavigableSet; import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.atomic.AtomicLong; public class OpenIterators { private final TaskId taskId; @@ -33,6 +35,7 @@ public class OpenIterators { private final StreamsMetricsImpl streamsMetrics; private final NavigableSet openIterators = new ConcurrentSkipListSet<>(Comparator.comparingLong(MeteredIterator::startTimestamp)); + private final AtomicLong oldestStartTimestamp = new AtomicLong(); private MetricName metricName; @@ -48,10 +51,11 @@ public class OpenIterators { public void add(final MeteredIterator iterator) { openIterators.add(iterator); + updateOldestStartTimestamp(); if (openIterators.size() == 1) { metricName = StateStoreMetrics.addOldestOpenIteratorGauge(taskId.toString(), metricsScope, name, streamsMetrics, - (config, now) -> openIterators.first().startTimestamp() + (config, now) -> oldestStartTimestamp.get() ); } } @@ -61,9 +65,17 @@ public class OpenIterators { streamsMetrics.removeMetric(metricName); } openIterators.remove(iterator); + updateOldestStartTimestamp(); } public long sum() { return openIterators.size(); } + + private void updateOldestStartTimestamp() { + final Iterator openIteratorsIterator = openIterators.iterator(); + if (openIteratorsIterator.hasNext()) { + oldestStartTimestamp.set(openIteratorsIterator.next().startTimestamp()); + } + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/internals/metrics/OpenIteratorsTest.java b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/OpenIteratorsTest.java new file mode 100644 index 00000000000..3464ecbdaee --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/OpenIteratorsTest.java @@ -0,0 +1,73 @@ +/* + * 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.streams.internals.metrics; + +import org.apache.kafka.common.metrics.Gauge; +import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.state.internals.MeteredIterator; + +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.verify; + +public class OpenIteratorsTest { + + private final StreamsMetricsImpl streamsMetrics = mock(StreamsMetricsImpl.class); + + @SuppressWarnings("unchecked") + @Test + public void shouldCalculateOldestStartTimestampCorrectly() { + final OpenIterators openIterators = new OpenIterators(new TaskId(0, 0), "scope", "name", streamsMetrics); + + final MeteredIterator meteredIterator1 = () -> 5; + final MeteredIterator meteredIterator2 = () -> 2; + final MeteredIterator meteredIterator3 = () -> 6; + + openIterators.add(meteredIterator1); + final ArgumentCaptor> gaugeCaptor = ArgumentCaptor.forClass(Gauge.class); + verify(streamsMetrics).addStoreLevelMutableMetric(any(), any(), any(), any(), any(), any(), gaugeCaptor.capture()); + final Gauge gauge = gaugeCaptor.getValue(); + assertThat(gauge.value(null, 0), is(5L)); + reset(streamsMetrics); + + openIterators.add(meteredIterator2); + verify(streamsMetrics, never()).addStoreLevelMutableMetric(any(), any(), any(), any(), any(), any(), gaugeCaptor.capture()); + assertThat(gauge.value(null, 0), is(2L)); + + openIterators.remove(meteredIterator2); + verify(streamsMetrics, never()).removeMetric(any()); + assertThat(gauge.value(null, 0), is(5L)); + + openIterators.remove(meteredIterator1); + verify(streamsMetrics).removeMetric(any()); + assertThat(gauge.value(null, 0), is(5L)); + + openIterators.add(meteredIterator3); + verify(streamsMetrics).addStoreLevelMutableMetric(any(), any(), any(), any(), any(), any(), gaugeCaptor.capture()); + assertThat(gaugeCaptor.getValue(), not(gauge)); + assertThat(gaugeCaptor.getValue().value(null, 0), is(6L)); + } +} From 675552a724fc8da5c3b09083f00bf2500930b50d Mon Sep 17 00:00:00 2001 From: PoAn Yang Date: Wed, 10 Sep 2025 13:02:03 +0800 Subject: [PATCH 011/100] KAFKA-19490: Remove usages of distutils in docker scripts (#20178) The [distutils](https://docs.python.org/3.13/whatsnew/3.12.html#distutils) package is removed from Python 3.12. Change `distutils` usage to `shutil`. Reviewers: Mickael Maison --------- Signed-off-by: PoAn Yang --- docker/common.py | 11 ++++------- docker/docker_build_test.py | 3 +-- docker/docker_official_image_build_test.py | 10 +++++----- docker/prepare_docker_official_image_source.py | 11 ++++------- 4 files changed, 14 insertions(+), 21 deletions(-) diff --git a/docker/common.py b/docker/common.py index f04a484a187..5099a789da1 100644 --- a/docker/common.py +++ b/docker/common.py @@ -18,9 +18,6 @@ import subprocess import tempfile import os -from distutils.file_util import copy_file - -from distutils.dir_util import copy_tree import shutil def execute(command): @@ -36,11 +33,11 @@ def get_input(message): def build_docker_image_runner(command, image_type, kafka_archive=None): temp_dir_path = tempfile.mkdtemp() current_dir = os.path.dirname(os.path.realpath(__file__)) - copy_tree(f"{current_dir}/{image_type}", f"{temp_dir_path}/{image_type}") - copy_tree(f"{current_dir}/resources", f"{temp_dir_path}/{image_type}/resources") - copy_file(f"{current_dir}/server.properties", f"{temp_dir_path}/{image_type}") + shutil.copytree(f"{current_dir}/{image_type}", f"{temp_dir_path}/{image_type}", dirs_exist_ok=True) + shutil.copytree(f"{current_dir}/resources", f"{temp_dir_path}/{image_type}/resources", dirs_exist_ok=True) + shutil.copy(f"{current_dir}/server.properties", f"{temp_dir_path}/{image_type}") if kafka_archive: - copy_file(kafka_archive, f"{temp_dir_path}/{image_type}/kafka.tgz") + shutil.copy(kafka_archive, f"{temp_dir_path}/{image_type}/kafka.tgz") command = command.replace("$DOCKER_FILE", f"{temp_dir_path}/{image_type}/Dockerfile") command = command.replace("$DOCKER_DIR", f"{temp_dir_path}/{image_type}") try: diff --git a/docker/docker_build_test.py b/docker/docker_build_test.py index fab6e65263d..9a986875fe3 100755 --- a/docker/docker_build_test.py +++ b/docker/docker_build_test.py @@ -36,7 +36,6 @@ Usage: from datetime import date import argparse -from distutils.dir_util import copy_tree import shutil from test.docker_sanity_test import run_tests from common import execute, build_docker_image_runner @@ -47,7 +46,7 @@ def run_docker_tests(image, tag, kafka_url, image_type): temp_dir_path = tempfile.mkdtemp() try: current_dir = os.path.dirname(os.path.realpath(__file__)) - copy_tree(f"{current_dir}/test/fixtures", f"{temp_dir_path}/fixtures") + shutil.copytree(f"{current_dir}/test/fixtures", f"{temp_dir_path}/fixtures", dirs_exist_ok=True) execute(["wget", "-nv", "-O", f"{temp_dir_path}/kafka.tgz", kafka_url]) execute(["mkdir", f"{temp_dir_path}/fixtures/kafka"]) execute(["tar", "xfz", f"{temp_dir_path}/kafka.tgz", "-C", f"{temp_dir_path}/fixtures/kafka", "--strip-components", "1"]) diff --git a/docker/docker_official_image_build_test.py b/docker/docker_official_image_build_test.py index 3da68854c23..32869a1f4b2 100644 --- a/docker/docker_official_image_build_test.py +++ b/docker/docker_official_image_build_test.py @@ -34,7 +34,6 @@ Usage: """ import argparse -from distutils.dir_util import copy_tree import shutil from common import execute from docker_build_test import run_docker_tests @@ -46,10 +45,11 @@ def build_docker_official_image(image, tag, kafka_version, image_type): image = f'{image}:{tag}' current_dir = os.path.dirname(os.path.realpath(__file__)) temp_dir_path = tempfile.mkdtemp() - copy_tree(f"{current_dir}/docker_official_images/{kafka_version}/{image_type}", - f"{temp_dir_path}/{image_type}") - copy_tree(f"{current_dir}/docker_official_images/{kafka_version}/jvm/resources", - f"{temp_dir_path}/{image_type}/resources") + shutil.copytree(f"{current_dir}/docker_official_images/{kafka_version}/{image_type}", + f"{temp_dir_path}/{image_type}", dirs_exist_ok=True) + shutil.copytree(f"{current_dir}/docker_official_images/{kafka_version}/jvm/resources", + f"{temp_dir_path}/{image_type}/resources", dirs_exist_ok=True) + shutil.copy(f"{current_dir}/server.properties", f"{temp_dir_path}/{image_type}") command = f"docker build -f $DOCKER_FILE -t {image} $DOCKER_DIR" command = command.replace("$DOCKER_FILE", f"{temp_dir_path}/{image_type}/Dockerfile") command = command.replace("$DOCKER_DIR", f"{temp_dir_path}/{image_type}") diff --git a/docker/prepare_docker_official_image_source.py b/docker/prepare_docker_official_image_source.py index 25d57c53e0f..bbc539b5c4c 100644 --- a/docker/prepare_docker_official_image_source.py +++ b/docker/prepare_docker_official_image_source.py @@ -33,7 +33,6 @@ Usage: from datetime import date import argparse -from distutils.dir_util import copy_tree import os import shutil import re @@ -61,12 +60,10 @@ if __name__ == '__main__': args = parser.parse_args() kafka_url = f"https://archive.apache.org/dist/kafka/{args.kafka_version}/kafka_2.13-{args.kafka_version}.tgz" current_dir = os.path.dirname(os.path.realpath(__file__)) - new_dir = os.path.join( - current_dir, f'docker_official_images', args.kafka_version) + new_dir = os.path.join(current_dir, 'docker_official_images', args.kafka_version) if os.path.exists(new_dir): shutil.rmtree(new_dir) os.makedirs(new_dir) - copy_tree(os.path.join(current_dir, args.image_type), os.path.join(new_dir, args.kafka_version, args.image_type)) - copy_tree(os.path.join(current_dir, 'resources'), os.path.join(new_dir, args.kafka_version, args.image_type, 'resources')) - remove_args_and_hardcode_values( - os.path.join(new_dir, args.kafka_version, args.image_type, 'Dockerfile'), args.kafka_version, kafka_url) + shutil.copytree(os.path.join(current_dir, args.image_type), os.path.join(new_dir, args.image_type), dirs_exist_ok=True) + shutil.copytree(os.path.join(current_dir, 'resources'), os.path.join(new_dir, args.image_type, 'resources'), dirs_exist_ok=True) + remove_args_and_hardcode_values(os.path.join(new_dir, args.image_type, 'Dockerfile'), args.kafka_version, kafka_url) From 1ea221c5e9ac0b85ecf2309b06fac2d0acc866db Mon Sep 17 00:00:00 2001 From: Mickael Maison Date: Wed, 10 Sep 2025 10:15:33 +0200 Subject: [PATCH 012/100] MINOR: Add 4.1.0 to core system tests (#20477) This PR updates all the core system tests to include 4.1.0. Reviewers: Chia-Ping Tsai --- .../client/client_compatibility_features_test.py | 3 ++- .../client_compatibility_produce_consume_test.py | 3 ++- .../tests/client/consumer_protocol_migration_test.py | 4 +++- .../tests/core/compatibility_test_new_broker_test.py | 3 ++- .../tests/core/transactions_mixed_versions_test.py | 4 ++-- .../kafkatest/tests/core/transactions_upgrade_test.py | 4 ++-- tests/kafkatest/tests/core/upgrade_test.py | 11 ++++++----- 7 files changed, 19 insertions(+), 13 deletions(-) diff --git a/tests/kafkatest/tests/client/client_compatibility_features_test.py b/tests/kafkatest/tests/client/client_compatibility_features_test.py index d6c70e7be27..d2026bb6788 100644 --- a/tests/kafkatest/tests/client/client_compatibility_features_test.py +++ b/tests/kafkatest/tests/client/client_compatibility_features_test.py @@ -29,7 +29,7 @@ from ducktape.tests.test import Test from kafkatest.version import DEV_BRANCH, \ LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, LATEST_2_8, \ LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, \ - LATEST_3_8, LATEST_3_9, LATEST_4_0, KafkaVersion + LATEST_3_8, LATEST_3_9, LATEST_4_0, LATEST_4_1, KafkaVersion def get_broker_features(broker_version): features = {} @@ -126,6 +126,7 @@ class ClientCompatibilityFeaturesTest(Test): @parametrize(broker_version=str(LATEST_3_8), metadata_quorum=quorum.isolated_kraft) @parametrize(broker_version=str(LATEST_3_9), metadata_quorum=quorum.isolated_kraft) @parametrize(broker_version=str(LATEST_4_0), metadata_quorum=quorum.isolated_kraft) + @parametrize(broker_version=str(LATEST_4_1), metadata_quorum=quorum.isolated_kraft) def run_compatibility_test(self, broker_version, metadata_quorum=quorum.zk): if self.zk: self.zk.start() diff --git a/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py b/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py index 449fb2ffdfd..2615d35ce47 100644 --- a/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py +++ b/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py @@ -26,7 +26,7 @@ from kafkatest.utils import is_int_with_prefix from kafkatest.version import DEV_BRANCH, \ LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, LATEST_2_8, \ LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, \ - LATEST_3_8, LATEST_3_9, LATEST_4_0, KafkaVersion + LATEST_3_8, LATEST_3_9, LATEST_4_0, LATEST_4_1, KafkaVersion class ClientCompatibilityProduceConsumeTest(ProduceConsumeValidateTest): """ @@ -86,6 +86,7 @@ class ClientCompatibilityProduceConsumeTest(ProduceConsumeValidateTest): @parametrize(broker_version=str(LATEST_3_8), metadata_quorum=quorum.isolated_kraft) @parametrize(broker_version=str(LATEST_3_9), metadata_quorum=quorum.isolated_kraft) @parametrize(broker_version=str(LATEST_4_0), metadata_quorum=quorum.isolated_kraft) + @parametrize(broker_version=str(LATEST_4_1), metadata_quorum=quorum.isolated_kraft) def test_produce_consume(self, broker_version, metadata_quorum=quorum.zk): print("running producer_consumer_compat with broker_version = %s" % broker_version, flush=True) self.kafka.set_version(KafkaVersion(broker_version)) diff --git a/tests/kafkatest/tests/client/consumer_protocol_migration_test.py b/tests/kafkatest/tests/client/consumer_protocol_migration_test.py index 596502bf74a..ce4f0431bb2 100644 --- a/tests/kafkatest/tests/client/consumer_protocol_migration_test.py +++ b/tests/kafkatest/tests/client/consumer_protocol_migration_test.py @@ -20,7 +20,8 @@ from ducktape.mark.resource import cluster from kafkatest.tests.verifiable_consumer_test import VerifiableConsumerTest from kafkatest.services.kafka import TopicPartition, quorum, consumer_group from kafkatest.version import LATEST_2_1, LATEST_2_3, LATEST_2_4, LATEST_2_5, \ - LATEST_3_2, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, LATEST_3_8, LATEST_3_9, LATEST_4_0, DEV_BRANCH, KafkaVersion + LATEST_3_2, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, LATEST_3_8, LATEST_3_9, \ + LATEST_4_0, LATEST_4_1, DEV_BRANCH, KafkaVersion class ConsumerProtocolMigrationTest(VerifiableConsumerTest): """ @@ -44,6 +45,7 @@ class ConsumerProtocolMigrationTest(VerifiableConsumerTest): all_consumer_versions = [LATEST_2_1, LATEST_2_3, LATEST_2_4, LATEST_2_5, \ LATEST_3_2, LATEST_3_4, LATEST_3_5, LATEST_3_6, \ LATEST_3_7, LATEST_3_8, LATEST_3_9, LATEST_4_0, \ + LATEST_4_1, \ DEV_BRANCH] consumer_versions_supporting_range_assignnor = [str(v) for v in all_consumer_versions] consumer_versions_supporting_static_membership = [str(v) for v in all_consumer_versions if v >= LATEST_2_3] diff --git a/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py b/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py index a14604145ea..cabf2922f40 100644 --- a/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py +++ b/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py @@ -22,7 +22,7 @@ from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest from kafkatest.utils import is_int from kafkatest.version import LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, \ LATEST_2_7, LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, \ - LATEST_3_7, LATEST_3_8, LATEST_3_9, LATEST_4_0, DEV_BRANCH, KafkaVersion + LATEST_3_7, LATEST_3_8, LATEST_3_9, LATEST_4_0, LATEST_4_1, DEV_BRANCH, KafkaVersion # Compatibility tests for moving to a new broker (e.g., 0.10.x) and using a mix of old and new clients (e.g., 0.9.x) class ClientCompatibilityTestNewBroker(ProduceConsumeValidateTest): @@ -61,6 +61,7 @@ class ClientCompatibilityTestNewBroker(ProduceConsumeValidateTest): @matrix(producer_version=[str(LATEST_3_8)], consumer_version=[str(LATEST_3_8)], compression_types=[["none"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade) @matrix(producer_version=[str(LATEST_3_9)], consumer_version=[str(LATEST_3_9)], compression_types=[["none"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade) @matrix(producer_version=[str(LATEST_4_0)], consumer_version=[str(LATEST_4_0)], compression_types=[["none"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade) + @matrix(producer_version=[str(LATEST_4_1)], consumer_version=[str(LATEST_4_1)], compression_types=[["none"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade) @matrix(producer_version=[str(LATEST_2_1)], consumer_version=[str(LATEST_2_1)], compression_types=[["zstd"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade) def test_compatibility(self, producer_version, consumer_version, compression_types, timestamp_type=None, metadata_quorum=quorum.zk): self.kafka = KafkaService(self.test_context, num_nodes=3, zk=None, version=DEV_BRANCH, topics={self.topic: { diff --git a/tests/kafkatest/tests/core/transactions_mixed_versions_test.py b/tests/kafkatest/tests/core/transactions_mixed_versions_test.py index 65747036802..15144682b33 100644 --- a/tests/kafkatest/tests/core/transactions_mixed_versions_test.py +++ b/tests/kafkatest/tests/core/transactions_mixed_versions_test.py @@ -22,7 +22,7 @@ from kafkatest.utils import is_int from kafkatest.utils.transactions_utils import create_and_start_copiers from kafkatest.version import LATEST_3_3, LATEST_3_4, LATEST_3_5, \ LATEST_3_6, LATEST_3_7, LATEST_3_8, LATEST_3_9, LATEST_4_0, \ - DEV_BRANCH, KafkaVersion, LATEST_STABLE_METADATA_VERSION + LATEST_4_1, DEV_BRANCH, KafkaVersion, LATEST_STABLE_METADATA_VERSION from ducktape.tests.test import Test from ducktape.mark import matrix @@ -179,7 +179,7 @@ class TransactionsMixedVersionsTest(Test): @cluster(num_nodes=8) @matrix( - old_kafka_version=[str(LATEST_4_0), str(LATEST_3_9), str(LATEST_3_8), str(LATEST_3_7), str(LATEST_3_6), str(LATEST_3_5), str(LATEST_3_4), str(LATEST_3_3)], + old_kafka_version=[str(LATEST_4_1), str(LATEST_4_0), str(LATEST_3_9), str(LATEST_3_8), str(LATEST_3_7), str(LATEST_3_6), str(LATEST_3_5), str(LATEST_3_4), str(LATEST_3_3)], metadata_quorum=[isolated_kraft], group_protocol=[None] ) diff --git a/tests/kafkatest/tests/core/transactions_upgrade_test.py b/tests/kafkatest/tests/core/transactions_upgrade_test.py index 495e91e430b..724605c6b24 100644 --- a/tests/kafkatest/tests/core/transactions_upgrade_test.py +++ b/tests/kafkatest/tests/core/transactions_upgrade_test.py @@ -22,7 +22,7 @@ from kafkatest.utils import is_int from kafkatest.utils.transactions_utils import create_and_start_copiers from kafkatest.version import LATEST_3_3, LATEST_3_4, LATEST_3_5, \ LATEST_3_6, LATEST_3_7, LATEST_3_8, LATEST_3_9, LATEST_4_0, \ - DEV_BRANCH, KafkaVersion, LATEST_STABLE_METADATA_VERSION, LATEST_STABLE_TRANSACTION_VERSION + LATEST_4_1, DEV_BRANCH, KafkaVersion, LATEST_STABLE_METADATA_VERSION, LATEST_STABLE_TRANSACTION_VERSION from ducktape.tests.test import Test from ducktape.mark import matrix @@ -209,7 +209,7 @@ class TransactionsUpgradeTest(Test): @cluster(num_nodes=8) @matrix( - from_kafka_version=[str(LATEST_4_0), str(LATEST_3_9), str(LATEST_3_8), str(LATEST_3_7), str(LATEST_3_6), str(LATEST_3_5), str(LATEST_3_4), str(LATEST_3_3)], + from_kafka_version=[str(LATEST_4_1), str(LATEST_4_0), str(LATEST_3_9), str(LATEST_3_8), str(LATEST_3_7), str(LATEST_3_6), str(LATEST_3_5), str(LATEST_3_4), str(LATEST_3_3)], metadata_quorum=[isolated_kraft], group_protocol=[None] ) diff --git a/tests/kafkatest/tests/core/upgrade_test.py b/tests/kafkatest/tests/core/upgrade_test.py index 4bd188d2c17..50cb345a4be 100644 --- a/tests/kafkatest/tests/core/upgrade_test.py +++ b/tests/kafkatest/tests/core/upgrade_test.py @@ -23,7 +23,8 @@ from kafkatest.services.verifiable_producer import VerifiableProducer from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest from kafkatest.utils import is_int from kafkatest.version import LATEST_3_4, LATEST_3_5, \ - LATEST_3_6, LATEST_3_7, LATEST_3_8, LATEST_3_9, LATEST_4_0, DEV_BRANCH, KafkaVersion, LATEST_STABLE_METADATA_VERSION + LATEST_3_6, LATEST_3_7, LATEST_3_8, LATEST_3_9, LATEST_4_0, \ + LATEST_4_1, DEV_BRANCH, KafkaVersion, LATEST_STABLE_METADATA_VERSION # # Test upgrading between different KRaft versions. @@ -163,25 +164,25 @@ class TestUpgrade(ProduceConsumeValidateTest): assert self.kafka.check_protocol_errors(self) @cluster(num_nodes=5) - @matrix(from_kafka_version=[str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7), str(LATEST_3_8), str(LATEST_3_9), str(LATEST_4_0), str(DEV_BRANCH)], + @matrix(from_kafka_version=[str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7), str(LATEST_3_8), str(LATEST_3_9), str(LATEST_4_0), str(LATEST_4_1), str(DEV_BRANCH)], metadata_quorum=[combined_kraft]) def test_combined_mode_upgrade(self, from_kafka_version, metadata_quorum): self.run_upgrade(from_kafka_version) @cluster(num_nodes=8) - @matrix(from_kafka_version=[str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7), str(LATEST_3_8), str(LATEST_3_9), str(LATEST_4_0), str(DEV_BRANCH)], + @matrix(from_kafka_version=[str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7), str(LATEST_3_8), str(LATEST_3_9), str(LATEST_4_0), str(LATEST_4_1), str(DEV_BRANCH)], metadata_quorum=[isolated_kraft]) def test_isolated_mode_upgrade(self, from_kafka_version, metadata_quorum): self.run_upgrade(from_kafka_version) @cluster(num_nodes=5) - @matrix(from_kafka_version=[str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7), str(LATEST_3_8), str(LATEST_3_9), str(LATEST_4_0), str(DEV_BRANCH)], + @matrix(from_kafka_version=[str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7), str(LATEST_3_8), str(LATEST_3_9), str(LATEST_4_0), str(LATEST_4_1), str(DEV_BRANCH)], metadata_quorum=[combined_kraft]) def test_combined_mode_upgrade_downgrade(self, from_kafka_version, metadata_quorum): self.run_upgrade_downgrade(from_kafka_version) @cluster(num_nodes=8) - @matrix(from_kafka_version=[str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7), str(LATEST_3_8), str(LATEST_3_9), str(LATEST_4_0), str(DEV_BRANCH)], + @matrix(from_kafka_version=[str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7), str(LATEST_3_8), str(LATEST_3_9), str(LATEST_4_0), str(LATEST_4_1), str(DEV_BRANCH)], metadata_quorum=[isolated_kraft]) def test_isolated_mode_upgrade_downgrade(self, from_kafka_version, metadata_quorum): self.run_upgrade_downgrade(from_kafka_version) From 32b8e326da4668f8533091c41e0790abe9d68aca Mon Sep 17 00:00:00 2001 From: Mickael Maison Date: Wed, 10 Sep 2025 16:23:55 +0200 Subject: [PATCH 013/100] MINOR: Add 4.1.0 to streams system tests (#20480) This PR updates all the streams system tests to include 4.1.0. Reviewers: Lucas Brutschy --- build.gradle | 16 + settings.gradle | 1 + .../apache/kafka/streams/StreamsConfig.java | 6 + .../streams/internals/UpgradeFromValues.java | 3 +- .../kafka/streams/tests/SmokeTestClient.java | 299 ++++++++ .../kafka/streams/tests/SmokeTestDriver.java | 670 ++++++++++++++++++ .../kafka/streams/tests/SmokeTestUtil.java | 131 ++++ .../kafka/streams/tests/StreamsSmokeTest.java | 100 +++ .../streams/tests/StreamsUpgradeTest.java | 120 ++++ .../streams_application_upgrade_test.py | 4 +- .../streams_broker_compatibility_test.py | 6 +- .../tests/streams/streams_upgrade_test.py | 6 +- 12 files changed, 1353 insertions(+), 9 deletions(-) create mode 100644 streams/upgrade-system-tests-41/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java create mode 100644 streams/upgrade-system-tests-41/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java create mode 100644 streams/upgrade-system-tests-41/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java create mode 100644 streams/upgrade-system-tests-41/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java create mode 100644 streams/upgrade-system-tests-41/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java diff --git a/build.gradle b/build.gradle index 2b4f1294e9c..26ca93d0738 100644 --- a/build.gradle +++ b/build.gradle @@ -3316,6 +3316,22 @@ project(':streams:upgrade-system-tests-40') { } } +project(':streams:upgrade-system-tests-41') { + base { + archivesName = "kafka-streams-upgrade-system-tests-41" + } + + dependencies { + testImplementation libs.kafkaStreams_41 + testRuntimeOnly libs.junitJupiter + testRuntimeOnly runtimeTestLibs + } + + systemTestLibs { + dependsOn testJar + } +} + project(':jmh-benchmarks') { apply plugin: 'com.gradleup.shadow' diff --git a/settings.gradle b/settings.gradle index 906ea1f8f7d..7c37a046838 100644 --- a/settings.gradle +++ b/settings.gradle @@ -117,6 +117,7 @@ include 'clients', 'streams:upgrade-system-tests-38', 'streams:upgrade-system-tests-39', 'streams:upgrade-system-tests-40', + 'streams:upgrade-system-tests-41', 'tools', 'tools:tools-api', 'transaction-coordinator', diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 1bd533fbd14..7f498e32bdc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -443,6 +443,12 @@ public class StreamsConfig extends AbstractConfig { @SuppressWarnings("WeakerAccess") public static final String UPGRADE_FROM_40 = UpgradeFromValues.UPGRADE_FROM_40.toString(); + /** + * Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 4.1.x}. + */ + @SuppressWarnings("WeakerAccess") + public static final String UPGRADE_FROM_41 = UpgradeFromValues.UPGRADE_FROM_41.toString(); + /** * Config value for parameter {@link #PROCESSING_GUARANTEE_CONFIG "processing.guarantee"} for at-least-once processing guarantees. diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/UpgradeFromValues.java b/streams/src/main/java/org/apache/kafka/streams/internals/UpgradeFromValues.java index 12cf3ead085..798383980b5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/internals/UpgradeFromValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/internals/UpgradeFromValues.java @@ -42,7 +42,8 @@ public enum UpgradeFromValues { UPGRADE_FROM_37("3.7"), UPGRADE_FROM_38("3.8"), UPGRADE_FROM_39("3.9"), - UPGRADE_FROM_40("4.0"); + UPGRADE_FROM_40("4.0"), + UPGRADE_FROM_41("4.1"); private final String value; diff --git a/streams/upgrade-system-tests-41/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/upgrade-system-tests-41/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java new file mode 100644 index 00000000000..dc0ad4d5601 --- /dev/null +++ b/streams/upgrade-system-tests-41/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -0,0 +1,299 @@ +/* + * 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.streams.tests; + +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.KafkaThread; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.Grouped; +import org.apache.kafka.streams.kstream.KGroupedStream; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.kstream.Suppressed.BufferConfig; +import org.apache.kafka.streams.kstream.TimeWindows; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.WindowStore; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.time.Duration; +import java.time.Instant; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; + +public class SmokeTestClient extends SmokeTestUtil { + + private final String name; + + private KafkaStreams streams; + private boolean uncaughtException = false; + private boolean started; + private volatile boolean closed; + + private static void addShutdownHook(final String name, final Runnable runnable) { + if (name != null) { + Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); + } else { + Runtime.getRuntime().addShutdownHook(new Thread(runnable)); + } + } + + private static File tempDirectory() { + final String prefix = "kafka-"; + final File file; + try { + file = Files.createTempDirectory(prefix).toFile(); + } catch (final IOException ex) { + throw new RuntimeException("Failed to create a temp dir", ex); + } + file.deleteOnExit(); + + addShutdownHook("delete-temp-file-shutdown-hook", () -> { + try { + Utils.delete(file); + } catch (final IOException e) { + System.out.println("Error deleting " + file.getAbsolutePath()); + e.printStackTrace(System.out); + } + }); + + return file; + } + + public SmokeTestClient(final String name) { + this.name = name; + } + + public boolean started() { + return started; + } + + public boolean closed() { + return closed; + } + + public void start(final Properties streamsProperties) { + final Topology build = getTopology(); + streams = new KafkaStreams(build, getStreamsConfig(streamsProperties)); + + final CountDownLatch countDownLatch = new CountDownLatch(1); + streams.setStateListener((newState, oldState) -> { + System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); + if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { + started = true; + countDownLatch.countDown(); + } + + if (newState == KafkaStreams.State.NOT_RUNNING) { + closed = true; + } + }); + + streams.setUncaughtExceptionHandler(e -> { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); + System.out.println(name + ": FATAL: An unexpected exception is encountered: " + e); + e.printStackTrace(System.out); + uncaughtException = true; + return StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT; + }); + + addShutdownHook("streams-shutdown-hook", this::close); + + streams.start(); + try { + if (!countDownLatch.await(1, TimeUnit.MINUTES)) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't start in one minute"); + } + } catch (final InterruptedException e) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: " + e); + e.printStackTrace(System.out); + } + System.out.println(name + ": SMOKE-TEST-CLIENT-STARTED"); + System.out.println(name + " started at " + Instant.now()); + } + + public void closeAsync() { + streams.close(Duration.ZERO); + } + + public void close() { + final boolean closed = streams.close(Duration.ofMinutes(1)); + + if (closed && !uncaughtException) { + System.out.println(name + ": SMOKE-TEST-CLIENT-CLOSED"); + } else if (closed) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); + } else { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't close"); + } + } + + private Properties getStreamsConfig(final Properties props) { + final Properties fullProps = new Properties(props); + fullProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest"); + fullProps.put(StreamsConfig.CLIENT_ID_CONFIG, "SmokeTest-" + name); + fullProps.put(StreamsConfig.STATE_DIR_CONFIG, tempDirectory().getAbsolutePath()); + fullProps.putAll(props); + return fullProps; + } + + public Topology getTopology() { + final StreamsBuilder builder = new StreamsBuilder(); + final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde); + final KStream source = builder.stream("data", stringIntConsumed); + source.filterNot((k, v) -> k.equals("flush")) + .to("echo", Produced.with(stringSerde, intSerde)); + final KStream data = source.filter((key, value) -> value == null || value != END); + data.process(SmokeTestUtil.printProcessorSupplier("data", name)); + + // min + final KGroupedStream groupedData = data.groupByKey(Grouped.with(stringSerde, intSerde)); + + final KTable, Integer> minAggregation = groupedData + .windowedBy(TimeWindows.ofSizeAndGrace(Duration.ofDays(1), Duration.ofMinutes(1))) + .aggregate( + () -> Integer.MAX_VALUE, + (aggKey, value, aggregate) -> (value < aggregate) ? value : aggregate, + Materialized + .>as("uwin-min") + .withValueSerde(intSerde) + .withRetention(Duration.ofHours(25)) + ); + + streamify(minAggregation, "min-raw"); + + streamify(minAggregation.suppress(untilWindowCloses(BufferConfig.unbounded())), "min-suppressed"); + + minAggregation + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("min", Produced.with(stringSerde, intSerde)); + + final KTable, Integer> smallWindowSum = groupedData + .windowedBy(TimeWindows.ofSizeAndGrace(Duration.ofSeconds(2), Duration.ofSeconds(30)).advanceBy(Duration.ofSeconds(1))) + .reduce(Integer::sum); + + streamify(smallWindowSum, "sws-raw"); + streamify(smallWindowSum.suppress(untilWindowCloses(BufferConfig.unbounded())), "sws-suppressed"); + + final KTable minTable = builder.table( + "min", + Consumed.with(stringSerde, intSerde), + Materialized.as("minStoreName")); + + minTable.toStream().process(SmokeTestUtil.printProcessorSupplier("min", name)); + + // max + groupedData + .windowedBy(TimeWindows.ofSizeWithNoGrace(Duration.ofDays(2))) + .aggregate( + () -> Integer.MIN_VALUE, + (aggKey, value, aggregate) -> (value > aggregate) ? value : aggregate, + Materialized.>as("uwin-max").withValueSerde(intSerde)) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("max", Produced.with(stringSerde, intSerde)); + + final KTable maxTable = builder.table( + "max", + Consumed.with(stringSerde, intSerde), + Materialized.as("maxStoreName")); + maxTable.toStream().process(SmokeTestUtil.printProcessorSupplier("max", name)); + + // sum + groupedData + .windowedBy(TimeWindows.ofSizeWithNoGrace(Duration.ofDays(2))) + .aggregate( + () -> 0L, + (aggKey, value, aggregate) -> (long) value + aggregate, + Materialized.>as("win-sum").withValueSerde(longSerde)) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("sum", Produced.with(stringSerde, longSerde)); + + final Consumed stringLongConsumed = Consumed.with(stringSerde, longSerde); + final KTable sumTable = builder.table("sum", stringLongConsumed); + sumTable.toStream().process(SmokeTestUtil.printProcessorSupplier("sum", name)); + + // cnt + groupedData + .windowedBy(TimeWindows.ofSizeWithNoGrace(Duration.ofDays(2))) + .count(Materialized.as("uwin-cnt")) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("cnt", Produced.with(stringSerde, longSerde)); + + final KTable cntTable = builder.table( + "cnt", + Consumed.with(stringSerde, longSerde), + Materialized.as("cntStoreName")); + cntTable.toStream().process(SmokeTestUtil.printProcessorSupplier("cnt", name)); + + // dif + maxTable + .join( + minTable, + (value1, value2) -> value1 - value2) + .toStream() + .filterNot((k, v) -> k.equals("flush")) + .to("dif", Produced.with(stringSerde, intSerde)); + + // avg + sumTable + .join( + cntTable, + (value1, value2) -> (double) value1 / (double) value2) + .toStream() + .filterNot((k, v) -> k.equals("flush")) + .to("avg", Produced.with(stringSerde, doubleSerde)); + + // test repartition + final Agg agg = new Agg(); + cntTable.groupBy(agg.selector(), Grouped.with(stringSerde, longSerde)) + .aggregate(agg.init(), agg.adder(), agg.remover(), + Materialized.as(Stores.inMemoryKeyValueStore("cntByCnt")) + .withKeySerde(Serdes.String()) + .withValueSerde(Serdes.Long())) + .toStream() + .to("tagg", Produced.with(stringSerde, longSerde)); + + return builder.build(); + } + + private static void streamify(final KTable, Integer> windowedTable, final String topic) { + windowedTable + .toStream() + .filterNot((k, v) -> k.key().equals("flush")) + .map((key, value) -> new KeyValue<>(key.toString(), value)) + .to(topic, Produced.with(stringSerde, intSerde)); + } +} diff --git a/streams/upgrade-system-tests-41/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/upgrade-system-tests-41/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java new file mode 100644 index 00000000000..8ab48f7cf5f --- /dev/null +++ b/streams/upgrade-system-tests-41/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java @@ -0,0 +1,670 @@ +/* + * 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.streams.tests; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Utils; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static java.util.Collections.emptyMap; +import static org.apache.kafka.common.utils.Utils.mkEntry; + +public class SmokeTestDriver extends SmokeTestUtil { + private static final String[] NUMERIC_VALUE_TOPICS = { + "data", + "echo", + "max", + "min", "min-suppressed", "min-raw", + "dif", + "sum", + "sws-raw", "sws-suppressed", + "cnt", + "avg", + "tagg" + }; + private static final String[] STRING_VALUE_TOPICS = { + "fk" + }; + + private static final String[] TOPICS = new String[NUMERIC_VALUE_TOPICS.length + STRING_VALUE_TOPICS.length]; + static { + System.arraycopy(NUMERIC_VALUE_TOPICS, 0, TOPICS, 0, NUMERIC_VALUE_TOPICS.length); + System.arraycopy(STRING_VALUE_TOPICS, 0, TOPICS, NUMERIC_VALUE_TOPICS.length, STRING_VALUE_TOPICS.length); + } + + private static final int MAX_RECORD_EMPTY_RETRIES = 30; + + private static class ValueList { + public final String key; + private final int[] values; + private int index; + + ValueList(final int min, final int max) { + key = min + "-" + max; + + values = new int[max - min + 1]; + for (int i = 0; i < values.length; i++) { + values[i] = min + i; + } + // We want to randomize the order of data to test not completely predictable processing order + // However, values are also use as a timestamp of the record. (TODO: separate data and timestamp) + // We keep some correlation of time and order. Thus, the shuffling is done with a sliding window + shuffle(values, 10); + + index = 0; + } + + int next() { + return (index < values.length) ? values[index++] : -1; + } + } + + public static String[] topics() { + return Arrays.copyOf(TOPICS, TOPICS.length); + } + + static void generatePerpetually(final String kafka, + final int numKeys, + final int maxRecordsPerKey) { + final Properties producerProps = generatorProperties(kafka); + + int numRecordsProduced = 0; + + final ValueList[] data = new ValueList[numKeys]; + for (int i = 0; i < numKeys; i++) { + data[i] = new ValueList(i, i + maxRecordsPerKey - 1); + } + + final Random rand = new Random(); + + try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { + while (true) { + final int index = rand.nextInt(numKeys); + final String key = data[index].key; + final int value = data[index].next(); + + final ProducerRecord record = + new ProducerRecord<>( + "data", + stringSerde.serializer().serialize("", key), + intSerde.serializer().serialize("", value) + ); + producer.send(record); + + final ProducerRecord fkRecord = + new ProducerRecord<>( + "fk", + intSerde.serializer().serialize("", value), + stringSerde.serializer().serialize("", key) + ); + producer.send(fkRecord); + + numRecordsProduced++; + if (numRecordsProduced % 100 == 0) { + System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); + } + Utils.sleep(2); + } + } + } + + public static Map> generate(final String kafka, + final int numKeys, + final int maxRecordsPerKey, + final Duration timeToSpend) { + final Properties producerProps = generatorProperties(kafka); + + int numRecordsProduced = 0; + + final Map> allData = new HashMap<>(); + final ValueList[] data = new ValueList[numKeys]; + for (int i = 0; i < numKeys; i++) { + data[i] = new ValueList(i, i + maxRecordsPerKey - 1); + allData.put(data[i].key, new HashSet<>()); + } + final Random rand = new Random(); + + int remaining = data.length; + + final long recordPauseTime = timeToSpend.toMillis() / numKeys / maxRecordsPerKey; + + final List> dataNeedRetry = new ArrayList<>(); + final List> fkNeedRetry = new ArrayList<>(); + + try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { + while (remaining > 0) { + final int index = rand.nextInt(remaining); + final String key = data[index].key; + final int value = data[index].next(); + + if (value < 0) { + remaining--; + data[index] = data[remaining]; + } else { + final ProducerRecord record = + new ProducerRecord<>( + "data", + stringSerde.serializer().serialize("", key), + intSerde.serializer().serialize("", value) + ); + + producer.send(record, new TestCallback(record, dataNeedRetry)); + + final ProducerRecord fkRecord = + new ProducerRecord<>( + "fk", + intSerde.serializer().serialize("", value), + stringSerde.serializer().serialize("", key) + ); + + producer.send(fkRecord, new TestCallback(fkRecord, fkNeedRetry)); + + numRecordsProduced++; + allData.get(key).add(value); + if (numRecordsProduced % 100 == 0) { + System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); + } + Utils.sleep(Math.max(recordPauseTime, 2)); + } + } + producer.flush(); + + retry(producer, dataNeedRetry, stringSerde); + retry(producer, fkNeedRetry, intSerde); + + flush(producer, + "data", + stringSerde.serializer().serialize("", "flush"), + intSerde.serializer().serialize("", 0) + ); + flush(producer, + "fk", + intSerde.serializer().serialize("", 0), + stringSerde.serializer().serialize("", "flush") + ); + } + return Collections.unmodifiableMap(allData); + } + + private static void retry(final KafkaProducer producer, + List> needRetry, + final Serde keySerde) { + int remainingRetries = 5; + while (!needRetry.isEmpty()) { + final List> needRetry2 = new ArrayList<>(); + for (final ProducerRecord record : needRetry) { + System.out.println( + "retry producing " + keySerde.deserializer().deserialize("", record.key())); + producer.send(record, new TestCallback(record, needRetry2)); + } + producer.flush(); + needRetry = needRetry2; + if (--remainingRetries == 0 && !needRetry.isEmpty()) { + System.err.println("Failed to produce all records after multiple retries"); + Exit.exit(1); + } + } + } + + private static void flush(final KafkaProducer producer, + final String topic, + final byte[] keyBytes, + final byte[] valBytes) { + // now that we've sent everything, we'll send some final records with a timestamp high enough to flush out + // all suppressed records. + final List partitions = producer.partitionsFor(topic); + for (final PartitionInfo partition : partitions) { + producer.send(new ProducerRecord<>( + partition.topic(), + partition.partition(), + System.currentTimeMillis() + Duration.ofDays(2).toMillis(), + keyBytes, + valBytes + )); + } + } + + private static Properties generatorProperties(final String kafka) { + final Properties producerProps = new Properties(); + producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "SmokeTest"); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + producerProps.put(ProducerConfig.ACKS_CONFIG, "all"); + return producerProps; + } + + private static class TestCallback implements Callback { + private final ProducerRecord originalRecord; + private final List> needRetry; + + TestCallback(final ProducerRecord originalRecord, + final List> needRetry) { + this.originalRecord = originalRecord; + this.needRetry = needRetry; + } + + @Override + public void onCompletion(final RecordMetadata metadata, final Exception exception) { + if (exception != null) { + if (exception instanceof TimeoutException) { + needRetry.add(originalRecord); + } else { + exception.printStackTrace(); + Exit.exit(1); + } + } + } + } + + private static void shuffle(final int[] data, @SuppressWarnings("SameParameterValue") final int windowSize) { + final Random rand = new Random(); + for (int i = 0; i < data.length; i++) { + // we shuffle data within windowSize + final int j = rand.nextInt(Math.min(data.length - i, windowSize)) + i; + + // swap + final int tmp = data[i]; + data[i] = data[j]; + data[j] = tmp; + } + } + + public static class NumberDeserializer implements Deserializer { + @Override + public Number deserialize(final String topic, final byte[] data) { + final Number value; + switch (topic) { + case "data": + case "echo": + case "min": + case "min-raw": + case "min-suppressed": + case "sws-raw": + case "sws-suppressed": + case "max": + case "dif": + value = intSerde.deserializer().deserialize(topic, data); + break; + case "sum": + case "cnt": + case "tagg": + value = longSerde.deserializer().deserialize(topic, data); + break; + case "avg": + value = doubleSerde.deserializer().deserialize(topic, data); + break; + default: + throw new RuntimeException("unknown topic: " + topic); + } + return value; + } + } + + public static VerificationResult verify(final String kafka, + final Map> inputs, + final int maxRecordsPerKey) { + final Properties props = new Properties(); + props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier"); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, NumberDeserializer.class); + props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); + + final KafkaConsumer consumer = new KafkaConsumer<>(props); + final List partitions = getAllPartitions(consumer, NUMERIC_VALUE_TOPICS); + consumer.assign(partitions); + consumer.seekToBeginning(partitions); + + final int recordsGenerated = inputs.size() * maxRecordsPerKey; + int recordsProcessed = 0; + final Map processed = + Stream.of(NUMERIC_VALUE_TOPICS) + .collect(Collectors.toMap(t -> t, t -> new AtomicInteger(0))); + + final Map>>> events = new HashMap<>(); + + VerificationResult verificationResult = new VerificationResult(false, "no results yet"); + int retry = 0; + final long start = System.currentTimeMillis(); + while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) { + final ConsumerRecords records = consumer.poll(Duration.ofSeconds(5)); + if (records.isEmpty() && recordsProcessed >= recordsGenerated) { + verificationResult = verifyAll(inputs, events, false); + if (verificationResult.passed()) { + break; + } else if (retry++ > MAX_RECORD_EMPTY_RETRIES) { + System.out.println(Instant.now() + " Didn't get any more results, verification hasn't passed, and out of retries."); + break; + } else { + System.out.println(Instant.now() + " Didn't get any more results, but verification hasn't passed (yet). Retrying..." + retry); + } + } else { + System.out.println(Instant.now() + " Get some more results from " + records.partitions() + ", resetting retry."); + + retry = 0; + for (final ConsumerRecord record : records) { + final String key = record.key(); + + final String topic = record.topic(); + processed.get(topic).incrementAndGet(); + + if (topic.equals("echo")) { + recordsProcessed++; + if (recordsProcessed % 100 == 0) { + System.out.println("Echo records processed = " + recordsProcessed); + } + } + + events.computeIfAbsent(topic, t -> new HashMap<>()) + .computeIfAbsent(key, k -> new LinkedList<>()) + .add(record); + } + + System.out.println(processed); + } + } + consumer.close(); + final long finished = System.currentTimeMillis() - start; + System.out.println("Verification time=" + finished); + System.out.println("-------------------"); + System.out.println("Result Verification"); + System.out.println("-------------------"); + System.out.println("recordGenerated=" + recordsGenerated); + System.out.println("recordProcessed=" + recordsProcessed); + + if (recordsProcessed > recordsGenerated) { + System.out.println("PROCESSED-MORE-THAN-GENERATED"); + } else if (recordsProcessed < recordsGenerated) { + System.out.println("PROCESSED-LESS-THAN-GENERATED"); + } + + boolean success; + + final Map> received = + events.get("echo") + .entrySet() + .stream() + .map(entry -> mkEntry( + entry.getKey(), + entry.getValue().stream().map(ConsumerRecord::value).collect(Collectors.toSet())) + ) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + success = inputs.equals(received); + + if (success) { + System.out.println("ALL-RECORDS-DELIVERED"); + } else { + int missedCount = 0; + for (final Map.Entry> entry : inputs.entrySet()) { + missedCount += received.get(entry.getKey()).size(); + } + System.out.println("missedRecords=" + missedCount); + } + + // give it one more try if it's not already passing. + if (!verificationResult.passed()) { + verificationResult = verifyAll(inputs, events, true); + } + success &= verificationResult.passed(); + + System.out.println(verificationResult.result()); + + System.out.println(success ? "SUCCESS" : "FAILURE"); + return verificationResult; + } + + public static class VerificationResult { + private final boolean passed; + private final String result; + + VerificationResult(final boolean passed, final String result) { + this.passed = passed; + this.result = result; + } + + public boolean passed() { + return passed; + } + + public String result() { + return result; + } + } + + private static VerificationResult verifyAll(final Map> inputs, + final Map>>> events, + final boolean printResults) { + final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + boolean pass; + try (final PrintStream resultStream = new PrintStream(byteArrayOutputStream)) { + pass = verifyTAgg(resultStream, inputs, events.get("tagg"), printResults); + pass &= verifySuppressed(resultStream, "min-suppressed", events, printResults); + pass &= verify(resultStream, "min-suppressed", inputs, events, windowedKey -> { + final String unwindowedKey = windowedKey.substring(1, windowedKey.length() - 1).replaceAll("@.*", ""); + return getMin(unwindowedKey); + }, printResults); + pass &= verifySuppressed(resultStream, "sws-suppressed", events, printResults); + pass &= verify(resultStream, "min", inputs, events, SmokeTestDriver::getMin, printResults); + pass &= verify(resultStream, "max", inputs, events, SmokeTestDriver::getMax, printResults); + pass &= verify(resultStream, "dif", inputs, events, key -> getMax(key).intValue() - getMin(key).intValue(), printResults); + pass &= verify(resultStream, "sum", inputs, events, SmokeTestDriver::getSum, printResults); + pass &= verify(resultStream, "cnt", inputs, events, key1 -> getMax(key1).intValue() - getMin(key1).intValue() + 1L, printResults); + pass &= verify(resultStream, "avg", inputs, events, SmokeTestDriver::getAvg, printResults); + } + return new VerificationResult(pass, new String(byteArrayOutputStream.toByteArray(), StandardCharsets.UTF_8)); + } + + private static boolean verify(final PrintStream resultStream, + final String topic, + final Map> inputData, + final Map>>> events, + final Function keyToExpectation, + final boolean printResults) { + final Map>> observedInputEvents = events.get("data"); + final Map>> outputEvents = events.getOrDefault(topic, emptyMap()); + if (outputEvents.isEmpty()) { + resultStream.println(topic + " is empty"); + return false; + } else { + resultStream.printf("verifying %s with %d keys%n", topic, outputEvents.size()); + + if (outputEvents.size() != inputData.size()) { + resultStream.printf("fail: resultCount=%d expectedCount=%s%n\tresult=%s%n\texpected=%s%n", + outputEvents.size(), inputData.size(), outputEvents.keySet(), inputData.keySet()); + return false; + } + for (final Map.Entry>> entry : outputEvents.entrySet()) { + final String key = entry.getKey(); + final Number expected = keyToExpectation.apply(key); + final Number actual = entry.getValue().getLast().value(); + if (!expected.equals(actual)) { + resultStream.printf("%s fail: key=%s actual=%s expected=%s%n", topic, key, actual, expected); + + if (printResults) { + resultStream.printf("\t inputEvents=%n%s%n\t" + + "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n", + indent("\t\t", observedInputEvents.get(key)), + indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("dif", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("cnt", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("tagg", emptyMap()).getOrDefault(key, new LinkedList<>()))); + + if (!Set.of("echo", "max", "min", "dif", "cnt", "tagg").contains(topic)) + resultStream.printf("%sEvents=%n%s%n", topic, indent("\t\t", entry.getValue())); + } + + return false; + } + } + return true; + } + } + + + private static boolean verifySuppressed(final PrintStream resultStream, + @SuppressWarnings("SameParameterValue") final String topic, + final Map>>> events, + final boolean printResults) { + resultStream.println("verifying suppressed " + topic); + final Map>> topicEvents = events.getOrDefault(topic, emptyMap()); + for (final Map.Entry>> entry : topicEvents.entrySet()) { + if (entry.getValue().size() != 1) { + final String unsuppressedTopic = topic.replace("-suppressed", "-raw"); + final String key = entry.getKey(); + final String unwindowedKey = key.substring(1, key.length() - 1).replaceAll("@.*", ""); + resultStream.printf("fail: key=%s%n\tnon-unique result:%n%s%n", + key, + indent("\t\t", entry.getValue())); + + if (printResults) + resultStream.printf("\tresultEvents:%n%s%n\tinputEvents:%n%s%n", + indent("\t\t", events.get(unsuppressedTopic).get(key)), + indent("\t\t", events.get("data").get(unwindowedKey))); + + return false; + } + } + return true; + } + + private static String indent(@SuppressWarnings("SameParameterValue") final String prefix, + final Iterable> list) { + final StringBuilder stringBuilder = new StringBuilder(); + for (final ConsumerRecord record : list) { + stringBuilder.append(prefix).append(record).append('\n'); + } + return stringBuilder.toString(); + } + + private static Long getSum(final String key) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + return ((long) min + max) * (max - min + 1L) / 2L; + } + + private static Double getAvg(final String key) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + return ((long) min + max) / 2.0; + } + + + private static boolean verifyTAgg(final PrintStream resultStream, + final Map> allData, + final Map>> taggEvents, + final boolean printResults) { + if (taggEvents == null) { + resultStream.println("tagg is missing"); + return false; + } else if (taggEvents.isEmpty()) { + resultStream.println("tagg is empty"); + return false; + } else { + resultStream.println("verifying tagg"); + + // generate expected answer + final Map expected = new HashMap<>(); + for (final String key : allData.keySet()) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + final String cnt = Long.toString(max - min + 1L); + + expected.put(cnt, expected.getOrDefault(cnt, 0L) + 1); + } + + // check the result + for (final Map.Entry>> entry : taggEvents.entrySet()) { + final String key = entry.getKey(); + Long expectedCount = expected.remove(key); + if (expectedCount == null) { + expectedCount = 0L; + } + + if (entry.getValue().getLast().value().longValue() != expectedCount) { + resultStream.println("fail: key=" + key + " tagg=" + entry.getValue() + " expected=" + expectedCount); + + if (printResults) + resultStream.println("\t taggEvents: " + entry.getValue()); + return false; + } + } + + } + return true; + } + + private static Number getMin(final String key) { + return Integer.parseInt(key.split("-")[0]); + } + + private static Number getMax(final String key) { + return Integer.parseInt(key.split("-")[1]); + } + + private static List getAllPartitions(final KafkaConsumer consumer, final String... topics) { + final List partitions = new ArrayList<>(); + + for (final String topic : topics) { + for (final PartitionInfo info : consumer.partitionsFor(topic)) { + partitions.add(new TopicPartition(info.topic(), info.partition())); + } + } + return partitions; + } + +} diff --git a/streams/upgrade-system-tests-41/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/upgrade-system-tests-41/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java new file mode 100644 index 00000000000..2e4938edfe7 --- /dev/null +++ b/streams/upgrade-system-tests-41/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java @@ -0,0 +1,131 @@ +/* + * 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.streams.tests; + +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.Aggregator; +import org.apache.kafka.streams.kstream.Initializer; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; + +import java.time.Instant; + +public class SmokeTestUtil { + + static final int END = Integer.MAX_VALUE; + + static ProcessorSupplier printProcessorSupplier(final String topic) { + return printProcessorSupplier(topic, ""); + } + + static ProcessorSupplier printProcessorSupplier(final String topic, final String name) { + return () -> new ContextualProcessor() { + private int numRecordsProcessed = 0; + private long smallestOffset = Long.MAX_VALUE; + private long largestOffset = Long.MIN_VALUE; + + @Override + public void init(final ProcessorContext context) { + super.init(context); + System.out.println("[4.1] initializing processor: topic=" + topic + " taskId=" + context.taskId()); + System.out.flush(); + numRecordsProcessed = 0; + smallestOffset = Long.MAX_VALUE; + largestOffset = Long.MIN_VALUE; + } + + @Override + public void process(final Record record) { + numRecordsProcessed++; + if (numRecordsProcessed % 100 == 0) { + System.out.printf("%s: %s%n", name, Instant.now()); + System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); + } + + context().recordMetadata().ifPresent(recordMetadata -> { + if (smallestOffset > recordMetadata.offset()) { + smallestOffset = recordMetadata.offset(); + } + if (largestOffset < recordMetadata.offset()) { + largestOffset = recordMetadata.offset(); + } + }); + } + + @Override + public void close() { + System.out.printf("Close processor for task %s%n", context().taskId()); + System.out.println("processed " + numRecordsProcessed + " records"); + final long processed; + if (largestOffset >= smallestOffset) { + processed = 1L + largestOffset - smallestOffset; + } else { + processed = 0L; + } + System.out.println("offset " + smallestOffset + " to " + largestOffset + " -> processed " + processed); + System.out.flush(); + } + }; + } + + public static final class Unwindow implements KeyValueMapper, V, K> { + @Override + public K apply(final Windowed winKey, final V value) { + return winKey.key(); + } + } + + public static class Agg { + + KeyValueMapper> selector() { + return (key, value) -> new KeyValue<>(value == null ? null : Long.toString(value), 1L); + } + + public Initializer init() { + return () -> 0L; + } + + Aggregator adder() { + return (aggKey, value, aggregate) -> aggregate + value; + } + + Aggregator remover() { + return (aggKey, value, aggregate) -> aggregate - value; + } + } + + public static Serde stringSerde = Serdes.String(); + + public static Serde intSerde = Serdes.Integer(); + + static Serde longSerde = Serdes.Long(); + + static Serde doubleSerde = Serdes.Double(); + + public static void sleep(final long duration) { + try { + Thread.sleep(duration); + } catch (final Exception ignore) { } + } + +} diff --git a/streams/upgrade-system-tests-41/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java b/streams/upgrade-system-tests-41/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java new file mode 100644 index 00000000000..5803b2fbd02 --- /dev/null +++ b/streams/upgrade-system-tests-41/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java @@ -0,0 +1,100 @@ +/* + * 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.streams.tests; + +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.StreamsConfig; + +import java.io.IOException; +import java.time.Duration; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.UUID; + +import static org.apache.kafka.streams.tests.SmokeTestDriver.generate; +import static org.apache.kafka.streams.tests.SmokeTestDriver.generatePerpetually; + +public class StreamsSmokeTest { + + /** + * args ::= kafka propFileName command disableAutoTerminate + * command := "run" | "process" + * + * @param args + */ + public static void main(final String[] args) throws IOException { + if (args.length < 2) { + System.err.println("StreamsSmokeTest are expecting two parameters: propFile, command; but only see " + args.length + " parameter"); + Exit.exit(1); + } + + final String propFileName = args[0]; + final String command = args[1]; + final boolean disableAutoTerminate = args.length > 2; + + final Properties streamsProperties = Utils.loadProps(propFileName); + final String kafka = streamsProperties.getProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); + final String processingGuarantee = streamsProperties.getProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG); + + if (kafka == null) { + System.err.println("No bootstrap kafka servers specified in " + StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); + Exit.exit(1); + } + + if ("process".equals(command)) { + if (!StreamsConfig.AT_LEAST_ONCE.equals(processingGuarantee) && + !StreamsConfig.EXACTLY_ONCE_V2.equals(processingGuarantee)) { + + System.err.println("processingGuarantee must be either " + StreamsConfig.AT_LEAST_ONCE + " or " + + StreamsConfig.EXACTLY_ONCE_V2); + + Exit.exit(1); + } + } + + System.out.println("StreamsTest instance started (StreamsSmokeTest)"); + System.out.println("command=" + command); + System.out.println("props=" + streamsProperties); + System.out.println("disableAutoTerminate=" + disableAutoTerminate); + + switch (command) { + case "run": + // this starts the driver (data generation and result verification) + final int numKeys = 10; + final int maxRecordsPerKey = 500; + if (disableAutoTerminate) { + generatePerpetually(kafka, numKeys, maxRecordsPerKey); + } else { + // slow down data production to span 30 seconds so that system tests have time to + // do their bounces, etc. + final Map> allData = + generate(kafka, numKeys, maxRecordsPerKey, Duration.ofSeconds(30)); + SmokeTestDriver.verify(kafka, allData, maxRecordsPerKey); + } + break; + case "process": + // this starts the stream processing app + new SmokeTestClient(UUID.randomUUID().toString()).start(streamsProperties); + break; + default: + System.out.println("unknown command: " + command); + } + } + +} diff --git a/streams/upgrade-system-tests-41/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java b/streams/upgrade-system-tests-41/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java new file mode 100644 index 00000000000..350169288e3 --- /dev/null +++ b/streams/upgrade-system-tests-41/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java @@ -0,0 +1,120 @@ +/* + * 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.streams.tests; + +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; + +import java.util.Properties; + +import static org.apache.kafka.streams.tests.SmokeTestUtil.intSerde; +import static org.apache.kafka.streams.tests.SmokeTestUtil.stringSerde; + + +public class StreamsUpgradeTest { + + @SuppressWarnings("unchecked") + public static void main(final String[] args) throws Exception { + if (args.length < 1) { + System.err.println("StreamsUpgradeTest requires one argument (properties-file) but provided none"); + } + final String propFileName = args[0]; + + final Properties streamsProperties = Utils.loadProps(propFileName); + + System.out.println("StreamsTest instance started (StreamsUpgradeTest v4.1)"); + System.out.println("props=" + streamsProperties); + + final StreamsBuilder builder = new StreamsBuilder(); + final KTable dataTable = builder.table( + "data", Consumed.with(stringSerde, intSerde)); + final KStream dataStream = dataTable.toStream(); + dataStream.process(printProcessorSupplier("data")); + dataStream.to("echo"); + + final boolean runFkJoin = Boolean.parseBoolean(streamsProperties.getProperty( + "test.run_fk_join", + "false")); + if (runFkJoin) { + try { + final KTable fkTable = builder.table( + "fk", Consumed.with(intSerde, stringSerde)); + buildFKTable(dataStream, fkTable); + } catch (final Exception e) { + System.err.println("Caught " + e.getMessage()); + } + } + + final Properties config = new Properties(); + config.setProperty( + StreamsConfig.APPLICATION_ID_CONFIG, + "StreamsUpgradeTest"); + config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); + config.putAll(streamsProperties); + + final KafkaStreams streams = new KafkaStreams(builder.build(), config); + streams.start(); + + Runtime.getRuntime().addShutdownHook(new Thread(() -> { + streams.close(); + System.out.println("UPGRADE-TEST-CLIENT-CLOSED"); + System.out.flush(); + })); + } + + private static void buildFKTable(final KStream primaryTable, + final KTable otherTable) { + final KStream kStream = primaryTable.toTable() + .join(otherTable, v -> v, (k0, v0) -> v0) + .toStream(); + kStream.process(printProcessorSupplier("fk")); + kStream.to("fk-result", Produced.with(stringSerde, stringSerde)); + } + + private static ProcessorSupplier printProcessorSupplier(final String topic) { + return () -> new ContextualProcessor() { + private int numRecordsProcessed = 0; + + @Override + public void init(final ProcessorContext context) { + System.out.println("[4.1] initializing processor: topic=" + topic + "taskId=" + context.taskId()); + numRecordsProcessed = 0; + } + + @Override + public void process(final Record record) { + numRecordsProcessed++; + if (numRecordsProcessed % 100 == 0) { + System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); + } + } + + @Override + public void close() {} + }; + } +} diff --git a/tests/kafkatest/tests/streams/streams_application_upgrade_test.py b/tests/kafkatest/tests/streams/streams_application_upgrade_test.py index 6831ec896e8..dcccc65f551 100644 --- a/tests/kafkatest/tests/streams/streams_application_upgrade_test.py +++ b/tests/kafkatest/tests/streams/streams_application_upgrade_test.py @@ -22,7 +22,7 @@ from kafkatest.services.kafka import KafkaService, quorum from kafkatest.services.streams import StreamsSmokeTestDriverService, StreamsSmokeTestJobRunnerService from kafkatest.version import LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, LATEST_2_8, \ LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, LATEST_3_8, \ - LATEST_3_9, LATEST_4_0, DEV_VERSION, KafkaVersion + LATEST_3_9, LATEST_4_0, LATEST_4_1, DEV_VERSION, KafkaVersion smoke_test_versions = [str(LATEST_2_4), str(LATEST_2_5), str(LATEST_2_6), @@ -30,7 +30,7 @@ smoke_test_versions = [str(LATEST_2_4), str(LATEST_2_5), str(LATEST_2_6), str(LATEST_3_1), str(LATEST_3_2), str(LATEST_3_3), str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7), str(LATEST_3_8), str(LATEST_3_9), - str(LATEST_4_0)] + str(LATEST_4_0), str(LATEST_4_1)] class StreamsUpgradeTest(Test): """ diff --git a/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py b/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py index 168da145876..b32187d84d4 100644 --- a/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py +++ b/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py @@ -22,7 +22,7 @@ from kafkatest.services.kafka import KafkaService, quorum from kafkatest.services.streams import StreamsBrokerCompatibilityService from kafkatest.services.verifiable_consumer import VerifiableConsumer from kafkatest.version import LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, \ - LATEST_3_7, LATEST_3_8, LATEST_3_9, LATEST_4_0, KafkaVersion + LATEST_3_7, LATEST_3_8, LATEST_3_9, LATEST_4_0, LATEST_4_1, KafkaVersion class StreamsBrokerCompatibility(Test): @@ -57,7 +57,7 @@ class StreamsBrokerCompatibility(Test): @cluster(num_nodes=4) @matrix(broker_version=[str(LATEST_3_0),str(LATEST_3_1),str(LATEST_3_2),str(LATEST_3_3), str(LATEST_3_4),str(LATEST_3_5),str(LATEST_3_6),str(LATEST_3_7), - str(LATEST_3_8),str(LATEST_3_9),str(LATEST_4_0)], + str(LATEST_3_8),str(LATEST_3_9),str(LATEST_4_0),str(LATEST_4_1)], metadata_quorum=[quorum.combined_kraft] ) def test_compatible_brokers_eos_disabled(self, broker_version, metadata_quorum): @@ -79,7 +79,7 @@ class StreamsBrokerCompatibility(Test): @cluster(num_nodes=4) @matrix(broker_version=[str(LATEST_3_0),str(LATEST_3_1),str(LATEST_3_2),str(LATEST_3_3), str(LATEST_3_4),str(LATEST_3_5),str(LATEST_3_6),str(LATEST_3_7), - str(LATEST_3_8),str(LATEST_3_9),str(LATEST_4_0)], + str(LATEST_3_8),str(LATEST_3_9),str(LATEST_4_0),str(LATEST_4_1)], metadata_quorum=[quorum.combined_kraft]) def test_compatible_brokers_eos_v2_enabled(self, broker_version, metadata_quorum): self.kafka.set_version(KafkaVersion(broker_version)) diff --git a/tests/kafkatest/tests/streams/streams_upgrade_test.py b/tests/kafkatest/tests/streams/streams_upgrade_test.py index 1bb223fcb2a..6b7041167ca 100644 --- a/tests/kafkatest/tests/streams/streams_upgrade_test.py +++ b/tests/kafkatest/tests/streams/streams_upgrade_test.py @@ -24,14 +24,14 @@ from kafkatest.services.streams import StreamsSmokeTestDriverService, StreamsSmo from kafkatest.tests.streams.utils import extract_generation_from_logs, extract_generation_id from kafkatest.version import LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, LATEST_2_8, \ LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, LATEST_3_8, LATEST_3_9, \ - LATEST_4_0, DEV_BRANCH, DEV_VERSION, KafkaVersion + LATEST_4_0, LATEST_4_1, DEV_BRANCH, DEV_VERSION, KafkaVersion # broker 0.10.0 is not compatible with newer Kafka Streams versions # broker 0.10.1 and 0.10.2 do not support headers, as required by suppress() (since v2.2.1) broker_upgrade_versions = [str(LATEST_2_8), str(LATEST_3_0), str(LATEST_3_1), str(LATEST_3_2), str(LATEST_3_3), str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7), str(LATEST_3_8), str(LATEST_3_9), str(LATEST_4_0), - str(DEV_BRANCH)] + str(LATEST_4_1), str(DEV_BRANCH)] metadata_2_versions = [str(LATEST_2_4), str(LATEST_2_5), str(LATEST_2_6), str(LATEST_2_7), str(LATEST_2_8), str(LATEST_3_0), str(LATEST_3_1), str(LATEST_3_2), str(LATEST_3_3)] @@ -39,7 +39,7 @@ metadata_2_versions = [str(LATEST_2_4), str(LATEST_2_5), str(LATEST_2_6), str(LA # -> https://issues.apache.org/jira/browse/KAFKA-14646 # thus, we cannot test two bounce rolling upgrade because we know it's broken # instead we add version 2.4...3.3 to the `metadata_2_versions` upgrade list -fk_join_versions = [str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7), str(LATEST_3_8), str(LATEST_3_9), str(LATEST_4_0)] +fk_join_versions = [str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7), str(LATEST_3_8), str(LATEST_3_9), str(LATEST_4_0), str(LATEST_4_1)] """ From a244565ed22fd41880c6468e3a84a637890f5486 Mon Sep 17 00:00:00 2001 From: Maros Orsak Date: Wed, 10 Sep 2025 16:50:08 +0200 Subject: [PATCH 014/100] KAFKA-18708: Move ScramPublisher to metadata module (#20468) Reviewers: Mickael Maison --- checkstyle/import-control-metadata.xml | 1 + checkstyle/import-control-server-common.xml | 1 + .../scala/kafka/server/BrokerServer.scala | 4 +- .../scala/kafka/server/ControllerServer.scala | 6 +- .../metadata/BrokerMetadataPublisher.scala | 2 +- .../server/metadata/ScramPublisher.scala | 71 ------------------- .../BrokerMetadataPublisherTest.scala | 2 +- .../metadata/publisher/ScramPublisher.java | 69 ++++++++++++++++++ .../kafka/security/CredentialProvider.java | 0 9 files changed, 78 insertions(+), 78 deletions(-) delete mode 100644 core/src/main/scala/kafka/server/metadata/ScramPublisher.scala create mode 100644 metadata/src/main/java/org/apache/kafka/metadata/publisher/ScramPublisher.java rename {server => server-common}/src/main/java/org/apache/kafka/security/CredentialProvider.java (100%) diff --git a/checkstyle/import-control-metadata.xml b/checkstyle/import-control-metadata.xml index 293801bd75f..d5b787ce24f 100644 --- a/checkstyle/import-control-metadata.xml +++ b/checkstyle/import-control-metadata.xml @@ -161,6 +161,7 @@ + diff --git a/checkstyle/import-control-server-common.xml b/checkstyle/import-control-server-common.xml index 8d85dffa341..21b13ed91d2 100644 --- a/checkstyle/import-control-server-common.xml +++ b/checkstyle/import-control-server-common.xml @@ -60,6 +60,7 @@ + diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 9af3492286e..dccd07b83c6 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -42,7 +42,7 @@ import org.apache.kafka.coordinator.share.{ShareCoordinator, ShareCoordinatorRec import org.apache.kafka.coordinator.transaction.ProducerIdManager import org.apache.kafka.image.publisher.{BrokerRegistrationTracker, MetadataPublisher} import org.apache.kafka.metadata.{BrokerState, ListenerInfo} -import org.apache.kafka.metadata.publisher.AclPublisher +import org.apache.kafka.metadata.publisher.{AclPublisher, ScramPublisher} import org.apache.kafka.security.CredentialProvider import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.common.{ApiMessageAndVersion, DirectoryEventHandler, NodeToControllerChannelManager, TopicIdPartition} @@ -497,7 +497,7 @@ class BrokerServer( quotaManagers, ), new ScramPublisher( - config, + config.nodeId, sharedServer.metadataPublishingFaultHandler, "broker", credentialProvider), diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index e8427fa7e53..bfcc9ed7c0b 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -22,7 +22,7 @@ import kafka.raft.KafkaRaftManager import kafka.server.QuotaFactory.QuotaManagers import scala.collection.immutable -import kafka.server.metadata.{ClientQuotaMetadataManager, DelegationTokenPublisher, DynamicClientQuotaPublisher, DynamicConfigPublisher, DynamicTopicClusterQuotaPublisher, KRaftMetadataCache, KRaftMetadataCachePublisher, ScramPublisher} +import kafka.server.metadata.{ClientQuotaMetadataManager, DelegationTokenPublisher, DynamicClientQuotaPublisher, DynamicConfigPublisher, DynamicTopicClusterQuotaPublisher, KRaftMetadataCache, KRaftMetadataCachePublisher} import kafka.utils.{CoreUtils, Logging} import org.apache.kafka.common.internals.Plugin import org.apache.kafka.common.message.ApiMessageType.ListenerType @@ -38,7 +38,7 @@ import org.apache.kafka.image.publisher.{ControllerRegistrationsPublisher, Metad import org.apache.kafka.metadata.{KafkaConfigSchema, ListenerInfo} import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer import org.apache.kafka.metadata.bootstrap.BootstrapMetadata -import org.apache.kafka.metadata.publisher.{AclPublisher, FeaturesPublisher} +import org.apache.kafka.metadata.publisher.{AclPublisher, FeaturesPublisher, ScramPublisher} import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.security.CredentialProvider import org.apache.kafka.server.{DelegationTokenManager, ProcessRole, SimpleApiVersionManager} @@ -350,7 +350,7 @@ class ControllerServer( // Set up the SCRAM publisher. metadataPublishers.add(new ScramPublisher( - config, + config.nodeId, sharedServer.metadataPublishingFaultHandler, "controller", credentialProvider diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala index 7876163f9ea..80a037ce4bd 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala @@ -33,7 +33,7 @@ import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.image.loader.LoaderManifest import org.apache.kafka.image.publisher.MetadataPublisher import org.apache.kafka.image.{MetadataDelta, MetadataImage, TopicDelta} -import org.apache.kafka.metadata.publisher.AclPublisher +import org.apache.kafka.metadata.publisher.{AclPublisher, ScramPublisher} import org.apache.kafka.server.common.MetadataVersion.MINIMUM_VERSION import org.apache.kafka.server.common.{FinalizedFeatures, RequestLocal, ShareVersion} import org.apache.kafka.server.fault.FaultHandler diff --git a/core/src/main/scala/kafka/server/metadata/ScramPublisher.scala b/core/src/main/scala/kafka/server/metadata/ScramPublisher.scala deleted file mode 100644 index 818e01fa5f8..00000000000 --- a/core/src/main/scala/kafka/server/metadata/ScramPublisher.scala +++ /dev/null @@ -1,71 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.server.metadata - -import kafka.server.KafkaConfig -import kafka.utils.Logging -import org.apache.kafka.image.loader.LoaderManifest -import org.apache.kafka.image.{MetadataDelta, MetadataImage} -import org.apache.kafka.security.CredentialProvider -import org.apache.kafka.server.fault.FaultHandler - - -class ScramPublisher( - conf: KafkaConfig, - faultHandler: FaultHandler, - nodeType: String, - credentialProvider: CredentialProvider, -) extends Logging with org.apache.kafka.image.publisher.MetadataPublisher { - logIdent = s"[${name()}] " - - override def name(): String = s"ScramPublisher $nodeType id=${conf.nodeId}" - - override def onMetadataUpdate( - delta: MetadataDelta, - newImage: MetadataImage, - manifest: LoaderManifest - ): Unit = { - onMetadataUpdate(delta, newImage) - } - - def onMetadataUpdate( - delta: MetadataDelta, - newImage: MetadataImage, - ): Unit = { - val deltaName = s"MetadataDelta up to ${newImage.highestOffsetAndEpoch().offset}" - try { - // Apply changes to SCRAM credentials. - Option(delta.scramDelta()).foreach { scramDelta => - scramDelta.changes().forEach { - case (mechanism, userChanges) => - userChanges.forEach { - case (userName, change) => - if (change.isPresent) { - credentialProvider.updateCredential(mechanism, userName, change.get().toCredential) - } else { - credentialProvider.removeCredentials(mechanism, userName) - } - } - } - } - } catch { - case t: Throwable => faultHandler.handleFault("Uncaught exception while " + - s"publishing SCRAM changes from $deltaName", t) - } - } -} 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 3bb3f5fc3f7..828ca0d7ad4 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala @@ -40,7 +40,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinator import org.apache.kafka.coordinator.share.ShareCoordinator import org.apache.kafka.image.{AclsImage, ClientQuotasImage, ClusterImageTest, ConfigurationsImage, DelegationTokenImage, FeaturesImage, MetadataDelta, MetadataImage, MetadataImageTest, MetadataProvenance, ProducerIdsImage, ScramImage, TopicsImage} import org.apache.kafka.image.loader.LogDeltaManifest -import org.apache.kafka.metadata.publisher.AclPublisher +import org.apache.kafka.metadata.publisher.{AclPublisher, ScramPublisher} import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.raft.LeaderAndEpoch import org.apache.kafka.server.common.{KRaftVersion, MetadataVersion, ShareVersion} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/publisher/ScramPublisher.java b/metadata/src/main/java/org/apache/kafka/metadata/publisher/ScramPublisher.java new file mode 100644 index 00000000000..7a286eb6351 --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/metadata/publisher/ScramPublisher.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.metadata.publisher; + +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.image.ScramDelta; +import org.apache.kafka.image.loader.LoaderManifest; +import org.apache.kafka.image.publisher.MetadataPublisher; +import org.apache.kafka.security.CredentialProvider; +import org.apache.kafka.server.fault.FaultHandler; + +public class ScramPublisher implements MetadataPublisher { + private final int nodeId; + private final FaultHandler faultHandler; + private final String nodeType; + private final CredentialProvider credentialProvider; + + public ScramPublisher(int nodeId, FaultHandler faultHandler, String nodeType, CredentialProvider credentialProvider) { + this.nodeId = nodeId; + this.faultHandler = faultHandler; + this.nodeType = nodeType; + this.credentialProvider = credentialProvider; + } + + @Override + public final String name() { + return "ScramPublisher " + nodeType + " id=" + nodeId; + } + + @Override + public void onMetadataUpdate(MetadataDelta delta, MetadataImage newImage, LoaderManifest manifest) { + onMetadataUpdate(delta, newImage); + } + + public void onMetadataUpdate(MetadataDelta delta, MetadataImage newImage) { + String deltaName = "MetadataDelta up to " + newImage.highestOffsetAndEpoch().offset(); + try { + // Apply changes to SCRAM credentials. + ScramDelta scramDelta = delta.scramDelta(); + if (scramDelta != null) { + scramDelta.changes().forEach((mechanism, userChanges) -> { + userChanges.forEach((userName, change) -> { + if (change.isPresent()) + credentialProvider.updateCredential(mechanism, userName, change.get().toCredential()); + else + credentialProvider.removeCredentials(mechanism, userName); + }); + }); + } + } catch (Throwable t) { + faultHandler.handleFault("Uncaught exception while publishing SCRAM changes from " + deltaName, t); + } + } +} diff --git a/server/src/main/java/org/apache/kafka/security/CredentialProvider.java b/server-common/src/main/java/org/apache/kafka/security/CredentialProvider.java similarity index 100% rename from server/src/main/java/org/apache/kafka/security/CredentialProvider.java rename to server-common/src/main/java/org/apache/kafka/security/CredentialProvider.java From 709c5fab22834b3d4bda5ab37e63b566621f22c1 Mon Sep 17 00:00:00 2001 From: Shashank Date: Wed, 10 Sep 2025 08:10:46 -0700 Subject: [PATCH 015/100] KAFKA-19666: Remove old restoration codepath from EosIntegrationTest [5/N] (#20499) clean up `EosIntegrationTest.java` Reviewers: Lucas Brutschy --- .../streams/integration/EosIntegrationTest.java | 15 +-------------- 1 file changed, 1 insertion(+), 14 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java index 95c19fd9cb8..d8743330218 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java @@ -788,19 +788,8 @@ public class EosIntegrationTest { @ParameterizedTest @ValueSource(booleans = {true, false}) - public void shouldCheckpointRestoredOffsetsWhenClosingCleanDuringRestoringStateUpdaterEnabled( + public void shouldCheckpointRestoredOffsetsWhenClosingCleanDuringRestoring( final boolean processingThreadsEnabled) throws Exception { - shouldCheckpointRestoredOffsetsWhenClosingCleanDuringRestoring(processingThreadsEnabled, true); - } - - @Test - public void shouldCheckpointRestoredOffsetsWhenClosingCleanDuringRestoringStateUpdaterDisabled() throws Exception { - shouldCheckpointRestoredOffsetsWhenClosingCleanDuringRestoring(false, false); - } - - private void shouldCheckpointRestoredOffsetsWhenClosingCleanDuringRestoring( - final boolean processingThreadsEnabled, - final boolean stateUpdaterEnabled) throws Exception { final Properties streamsConfiguration = new Properties(); streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); @@ -812,7 +801,6 @@ public class EosIntegrationTest { streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(applicationId).getPath()); streamsConfiguration.put(InternalConfig.PROCESSING_THREADS_ENABLED, processingThreadsEnabled); - streamsConfiguration.put(InternalConfig.STATE_UPDATER_ENABLED, stateUpdaterEnabled); streamsConfiguration.put(StreamsConfig.restoreConsumerPrefix(ConsumerConfig.MAX_POLL_RECORDS_CONFIG), 100); final String stateStoreName = "stateStore"; @@ -1223,7 +1211,6 @@ public class EosIntegrationTest { properties.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); properties.put(StreamsConfig.STATE_DIR_CONFIG, stateTmpDir + appDir); properties.put(StreamsConfig.APPLICATION_SERVER_CONFIG, dummyHostName + ":2142"); - properties.put(InternalConfig.STATE_UPDATER_ENABLED, processingThreadsEnabled); properties.put(InternalConfig.PROCESSING_THREADS_ENABLED, processingThreadsEnabled); final Properties config = StreamsTestUtils.getStreamsConfig( From 351203873dbe44048ed3c42a66416be2fa985116 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Wed, 10 Sep 2025 17:55:51 +0200 Subject: [PATCH 016/100] KAFKA-19661 [5/N]: Use below-quota as a condition for standby task assignment (#20458) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit In the original algorithm, standby tasks are assigned to a process that previously owned the task only if it is “load-balanced”, meaning the process has fewer tasks that members, or it is the least loaded process. This is strong requirement, and will cause standby tasks to often not get assigned to process that previously owned it. Furthermore, the condition “is the least loaded process” is hard to evaluate efficiently in this context. We propose to instead use the same “below-quota” condition as in active task assignment. We compute a quota for active and standby tasks, by definiing numOfTasks = numberOfActiveTasks+numberOfStandbyTasks and defining the quota as numOfTasks/numberOfMembers rounded up. Whenever a member becomes “full” (its assigned number of tasks is equal to numOfTasks) we deduct its tasks from numOfTasks and decrement numberOfMembers and recompute the quota, which means that the quota may be reduced by one during the assignment process, to avoid uneven assignments. A standby task can be assigned to a process that previously owned it, whenever the process has fewer than numOfMembersOfProcess*quota. This condition will, again, prioritize standby stickyness, and can be evaluated in constant time. In our worst-performing benchmark, this improves the runtime by 2.5x on top of the previous optimizations, but 5x on the more important incremental assignment case. Reviewers: Bill Bejeck --- .../group/streams/assignor/ProcessState.java | 6 - .../streams/assignor/StickyTaskAssignor.java | 130 ++++++++++-------- .../assignor/StickyTaskAssignorTest.java | 62 +++++++++ 3 files changed, 138 insertions(+), 60 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/ProcessState.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/ProcessState.java index d4dd2d4ba49..84c1b8a8207 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/ProcessState.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/ProcessState.java @@ -32,7 +32,6 @@ public class ProcessState { private int capacity; private double load; private int taskCount; - private int activeTaskCount; private final Map memberToTaskCounts; private final Map> assignedActiveTasks; private final Map> assignedStandbyTasks; @@ -65,10 +64,6 @@ public class ProcessState { return memberToTaskCounts; } - public int activeTaskCount() { - return activeTaskCount; - } - public Set assignedActiveTasks() { return assignedActiveTasks.values().stream() .flatMap(Set::stream) @@ -93,7 +88,6 @@ public class ProcessState { taskCount += 1; assignedTasks.add(taskId); if (isActive) { - activeTaskCount += 1; assignedActiveTasks.putIfAbsent(memberId, new HashSet<>()); assignedActiveTasks.get(memberId).add(taskId); } else { diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignor.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignor.java index fc29f93b883..1b3f08a7382 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignor.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignor.java @@ -57,13 +57,9 @@ public class StickyTaskAssignor implements TaskAssignor { final LinkedList activeTasks = taskIds(topologyDescriber, true); assignActive(activeTasks); - //standby - final int numStandbyReplicas = - groupSpec.assignmentConfigs().isEmpty() ? 0 - : Integer.parseInt(groupSpec.assignmentConfigs().get("num.standby.replicas")); - if (numStandbyReplicas > 0) { + if (localState.numStandbyReplicas > 0) { final LinkedList statefulTasks = taskIds(topologyDescriber, false); - assignStandby(statefulTasks, numStandbyReplicas); + assignStandby(statefulTasks); } return buildGroupAssignment(groupSpec.members().keySet()); @@ -84,13 +80,24 @@ public class StickyTaskAssignor implements TaskAssignor { private void initialize(final GroupSpec groupSpec, final TopologyDescriber topologyDescriber) { localState = new LocalState(); - localState.allTasks = 0; + localState.numStandbyReplicas = + groupSpec.assignmentConfigs().isEmpty() ? 0 + : Integer.parseInt(groupSpec.assignmentConfigs().get("num.standby.replicas")); + + // Helpers for computing active tasks per member, and tasks per member + localState.totalActiveTasks = 0; + localState.totalTasks = 0; for (final String subtopology : topologyDescriber.subtopologies()) { final int numberOfPartitions = topologyDescriber.maxNumInputPartitions(subtopology); - localState.allTasks += numberOfPartitions; + localState.totalTasks += numberOfPartitions; + localState.totalActiveTasks += numberOfPartitions; + if (topologyDescriber.isStateful(subtopology)) + localState.totalTasks += numberOfPartitions * localState.numStandbyReplicas; } - localState.totalCapacity = groupSpec.members().size(); - localState.tasksPerMember = computeTasksPerMember(localState.allTasks, localState.totalCapacity); + localState.totalMembersWithActiveTaskCapacity = groupSpec.members().size(); + localState.totalMembersWithTaskCapacity = groupSpec.members().size(); + localState.activeTasksPerMember = computeTasksPerMember(localState.totalActiveTasks, localState.totalMembersWithActiveTaskCapacity); + localState.tasksPerMember = computeTasksPerMember(localState.totalTasks, localState.totalMembersWithTaskCapacity); localState.processIdToState = new HashMap<>(); localState.activeTaskToPrevMember = new HashMap<>(); @@ -175,11 +182,13 @@ public class StickyTaskAssignor implements TaskAssignor { for (final Iterator it = activeTasks.iterator(); it.hasNext();) { final TaskId task = it.next(); final Member prevMember = localState.activeTaskToPrevMember.get(task); - if (prevMember != null && hasUnfulfilledQuota(prevMember)) { + if (prevMember != null) { final ProcessState processState = localState.processIdToState.get(prevMember.processId); - processState.addTask(prevMember.memberId, task, true); - maybeUpdateTasksPerMember(processState.activeTaskCount()); - it.remove(); + if (hasUnfulfilledActiveTaskQuota(processState, prevMember)) { + processState.addTask(prevMember.memberId, task, true); + maybeUpdateActiveTasksPerMember(processState.memberToTaskCounts().get(prevMember.memberId)); + it.remove(); + } } } @@ -188,11 +197,13 @@ public class StickyTaskAssignor implements TaskAssignor { final TaskId task = it.next(); final ArrayList prevMembers = localState.standbyTaskToPrevMember.get(task); final Member prevMember = findPrevMemberWithLeastLoad(prevMembers, null); - if (prevMember != null && hasUnfulfilledQuota(prevMember)) { + if (prevMember != null) { final ProcessState processState = localState.processIdToState.get(prevMember.processId); - processState.addTask(prevMember.memberId, task, true); - maybeUpdateTasksPerMember(processState.activeTaskCount()); - it.remove(); + if (hasUnfulfilledActiveTaskQuota(processState, prevMember)) { + processState.addTask(prevMember.memberId, task, true); + maybeUpdateActiveTasksPerMember(processState.memberToTaskCounts().get(prevMember.memberId)); + it.remove(); + } } } @@ -206,24 +217,32 @@ public class StickyTaskAssignor implements TaskAssignor { final TaskId task = it.next(); final ProcessState processWithLeastLoad = processByLoad.poll(); if (processWithLeastLoad == null) { - throw new TaskAssignorException("No process available to assign active task {}." + task); + throw new TaskAssignorException(String.format("No process available to assign active task %s.", task)); } final String member = memberWithLeastLoad(processWithLeastLoad); if (member == null) { - throw new TaskAssignorException("No member available to assign active task {}." + task); + throw new TaskAssignorException(String.format("No member available to assign active task %s.", task)); } processWithLeastLoad.addTask(member, task, true); it.remove(); - maybeUpdateTasksPerMember(processWithLeastLoad.activeTaskCount()); + maybeUpdateActiveTasksPerMember(processWithLeastLoad.memberToTaskCounts().get(member)); processByLoad.add(processWithLeastLoad); // Add it back to the queue after updating its state } } - private void maybeUpdateTasksPerMember(final int activeTasksNo) { - if (activeTasksNo == localState.tasksPerMember) { - localState.totalCapacity--; - localState.allTasks -= activeTasksNo; - localState.tasksPerMember = computeTasksPerMember(localState.allTasks, localState.totalCapacity); + private void maybeUpdateActiveTasksPerMember(final int activeTasksNo) { + if (activeTasksNo == localState.activeTasksPerMember) { + localState.totalMembersWithActiveTaskCapacity--; + localState.totalActiveTasks -= activeTasksNo; + localState.activeTasksPerMember = computeTasksPerMember(localState.totalActiveTasks, localState.totalMembersWithActiveTaskCapacity); + } + } + + private void maybeUpdateTasksPerMember(final int taskNo) { + if (taskNo == localState.tasksPerMember) { + localState.totalMembersWithTaskCapacity--; + localState.totalTasks -= taskNo; + localState.tasksPerMember = computeTasksPerMember(localState.totalTasks, localState.totalMembersWithTaskCapacity); } } @@ -298,43 +317,49 @@ public class StickyTaskAssignor implements TaskAssignor { return memberWithLeastLoad.orElse(null); } - private boolean hasUnfulfilledQuota(final Member member) { - return localState.processIdToState.get(member.processId).memberToTaskCounts().get(member.memberId) < localState.tasksPerMember; + private boolean hasUnfulfilledActiveTaskQuota(final ProcessState process, final Member member) { + return process.memberToTaskCounts().get(member.memberId) < localState.activeTasksPerMember; } - private void assignStandby(final LinkedList standbyTasks, int numStandbyReplicas) { - final ArrayList toLeastLoaded = new ArrayList<>(standbyTasks.size() * numStandbyReplicas); + private boolean hasUnfulfilledTaskQuota(final ProcessState process, final Member member) { + return process.memberToTaskCounts().get(member.memberId) < localState.tasksPerMember; + } + private void assignStandby(final LinkedList standbyTasks) { + final ArrayList toLeastLoaded = new ArrayList<>(standbyTasks.size() * localState.numStandbyReplicas); + // Assuming our current assignment is range-based, we want to sort by partition first. standbyTasks.sort(Comparator.comparing(TaskId::partition).thenComparing(TaskId::subtopologyId).reversed()); for (TaskId task : standbyTasks) { - for (int i = 0; i < numStandbyReplicas; i++) { + for (int i = 0; i < localState.numStandbyReplicas; i++) { // prev active task - final Member prevMember = localState.activeTaskToPrevMember.get(task); - if (prevMember != null) { - final ProcessState prevMemberProcessState = localState.processIdToState.get(prevMember.processId); - if (!prevMemberProcessState.hasTask(task) && isLoadBalanced(prevMemberProcessState)) { - prevMemberProcessState.addTask(prevMember.memberId, task, false); + final Member prevActiveMember = localState.activeTaskToPrevMember.get(task); + if (prevActiveMember != null) { + final ProcessState prevActiveMemberProcessState = localState.processIdToState.get(prevActiveMember.processId); + if (!prevActiveMemberProcessState.hasTask(task) && hasUnfulfilledTaskQuota(prevActiveMemberProcessState, prevActiveMember)) { + prevActiveMemberProcessState.addTask(prevActiveMember.memberId, task, false); + maybeUpdateTasksPerMember(prevActiveMemberProcessState.memberToTaskCounts().get(prevActiveMember.memberId)); continue; } } // prev standby tasks - final ArrayList prevMembers = localState.standbyTaskToPrevMember.get(task); - if (prevMembers != null && !prevMembers.isEmpty()) { - final Member prevMember2 = findPrevMemberWithLeastLoad(prevMembers, task); - if (prevMember2 != null) { - final ProcessState prevMemberProcessState = localState.processIdToState.get(prevMember2.processId); - if (isLoadBalanced(prevMemberProcessState)) { - prevMemberProcessState.addTask(prevMember2.memberId, task, false); + final ArrayList prevStandbyMembers = localState.standbyTaskToPrevMember.get(task); + if (prevStandbyMembers != null && !prevStandbyMembers.isEmpty()) { + final Member prevStandbyMember = findPrevMemberWithLeastLoad(prevStandbyMembers, task); + if (prevStandbyMember != null) { + final ProcessState prevStandbyMemberProcessState = localState.processIdToState.get(prevStandbyMember.processId); + if (hasUnfulfilledTaskQuota(prevStandbyMemberProcessState, prevStandbyMember)) { + prevStandbyMemberProcessState.addTask(prevStandbyMember.memberId, task, false); + maybeUpdateTasksPerMember(prevStandbyMemberProcessState.memberToTaskCounts().get(prevStandbyMember.memberId)); continue; } } } - toLeastLoaded.add(new StandbyToAssign(task, numStandbyReplicas - i)); + toLeastLoaded.add(new StandbyToAssign(task, localState.numStandbyReplicas - i)); break; } } @@ -350,7 +375,7 @@ public class StickyTaskAssignor implements TaskAssignor { if (!assignStandbyToMemberWithLeastLoad(processByLoad, toAssign.taskId)) { log.warn("{} There is not enough available capacity. " + "You should increase the number of threads and/or application instances to maintain the requested number of standby replicas.", - errorMessage(numStandbyReplicas, i, toAssign.taskId)); + errorMessage(localState.numStandbyReplicas, i, toAssign.taskId)); break; } } @@ -362,13 +387,6 @@ public class StickyTaskAssignor implements TaskAssignor { " of " + numStandbyReplicas + " standby tasks for task [" + task + "]."; } - private boolean isLoadBalanced(final ProcessState process) { - final double load = process.load(); - final boolean isLeastLoadedProcess = localState.processIdToState.values().stream() - .allMatch(p -> p.load() >= load); - return process.hasCapacity() || isLeastLoadedProcess; - } - private static int computeTasksPerMember(final int numberOfTasks, final int numberOfMembers) { if (numberOfMembers == 0) { return 0; @@ -406,8 +424,12 @@ public class StickyTaskAssignor implements TaskAssignor { Map> standbyTaskToPrevMember; Map processIdToState; - int allTasks; - int totalCapacity; + int numStandbyReplicas; + int totalActiveTasks; + int totalTasks; + int totalMembersWithActiveTaskCapacity; + int totalMembersWithTaskCapacity; + int activeTasksPerMember; int tasksPerMember; } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignorTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignorTest.java index 1e9d4115cb2..53318578432 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignorTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignorTest.java @@ -1233,6 +1233,68 @@ public class StickyTaskAssignorTest { assertEquals(1, testMember2.standbyTasks().get("test-subtopology2").size()); } + @Test + public void shouldAssignStandbyTaskToPreviousOwnerBasedOnBelowQuotaCondition() { + // This test demonstrates the change from "load-balanced" to "below-quota" condition for standby assignment. + // We create a scenario where: + // - Process1 previously owned standby task 1 and currently has 1 active task (task 0) + // - Process2 currently has 1 active task (task 1) + // - Process3 has no previous tasks (least loaded after assignment) + // + // Under the old "load-balanced" algorithm: Process1 might not get standby task 1 because + // Process3 could be considered least loaded. + // + // Under the new "below-quota" algorithm: Process1 gets standby task 1 because + // it previously owned it AND is below quota. + + // Set up previous task assignments: + // Process1: active=[0], standby=[1] (previously had both active and standby tasks) + // Process2: active=[1] (had the active task that process1 had as standby) + // Process3: no previous tasks + final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", + mkMap(mkEntry("test-subtopology", Sets.newSet(0))), + mkMap(mkEntry("test-subtopology", Sets.newSet(1)))); + final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", + mkMap(mkEntry("test-subtopology", Sets.newSet(1))), + Map.of()); + final AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process3"); + + final Map members = mkMap( + mkEntry("member1", memberSpec1), + mkEntry("member2", memberSpec2), + mkEntry("member3", memberSpec3)); + + // We have 2 active tasks + 1 standby replica = 4 total tasks + // Quota per process = 4 tasks / 3 processes = 1.33 -> 2 tasks per process + final GroupAssignment result = assignor.assign( + new GroupSpecImpl(members, mkMap(mkEntry(NUM_STANDBY_REPLICAS_CONFIG, "1"))), + new TopologyDescriberImpl(2, true, List.of("test-subtopology")) + ); + + // Verify that process1 gets the standby task 1 that it previously owned + // This should work under the new "below-quota" algorithm since process1 has only 1 active task + // which is below the quota of 2 tasks per process + final MemberAssignment member1 = result.members().get("member1"); + assertNotNull(member1); + + // Member1 should retain its active task 0 + assertTrue(member1.activeTasks().get("test-subtopology").contains(0)); + + // Member1 should get standby task 1 because it previously owned it and is below quota + assertNotNull(member1.standbyTasks().get("test-subtopology"), "Member1 should have standby tasks assigned"); + assertTrue(member1.standbyTasks().get("test-subtopology").contains(1), + "Member1 should have standby task 1, but has: " + member1.standbyTasks().get("test-subtopology")); + + // Verify that member1 doesn't have active task 1 (standby can't be same as active) + assertFalse(member1.activeTasks().get("test-subtopology").contains(1)); + + // Verify the process1's total task count is at or below quota + int member1ActiveCount = member1.activeTasks().get("test-subtopology").size(); + int member1StandbyCount = member1.standbyTasks().get("test-subtopology").size(); + int member1TotalTasks = member1ActiveCount + member1StandbyCount; + assertTrue(member1TotalTasks <= 2, "Member1 should have <= 2 total tasks (quota), but has " + member1TotalTasks); + } + private int getAllActiveTaskCount(GroupAssignment result, String... memberIds) { int size = 0; From ff5025a21c4e2b4d437f85aa75e1c3f7d4f09ef9 Mon Sep 17 00:00:00 2001 From: Sushant Mahajan Date: Wed, 10 Sep 2025 22:08:34 +0530 Subject: [PATCH 017/100] KAFKA-19695: Fix bug in redundant offset calculation. (#20516) * The `ShareCoordinatorShard` maintains the the record offset information for `SharePartitionKey`s in the `ShareCoordinatorOffsetsManager` class. * Replay of `ShareSnapshot`s in the shards are reflected in the offsets manager including records created due to delete state. * However, if the share partition delete is due to topic delete, no record will ever be written for the same `SharePartitionKey` post the delete tombstone (as topic id will not repeat). As a result the offset manager will always consider the deleted share partition's offset as the last redundant one. * The fix is to make the offset manager aware of the tombstone records and remove them from the redundant offset calculation. * Unit tests have been updated for the same. Reviewers: Andrew Schofield , Apoorv Mittal --- .../share/ShareCoordinatorOffsetsManager.java | 9 +- .../share/ShareCoordinatorShard.java | 2 +- .../ShareCoordinatorOffsetsManagerTest.java | 96 ++++++++++++++++--- 3 files changed, 90 insertions(+), 17 deletions(-) diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorOffsetsManager.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorOffsetsManager.java index 69070f65e93..0b3e5a5ff08 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorOffsetsManager.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorOffsetsManager.java @@ -60,13 +60,20 @@ public class ShareCoordinatorOffsetsManager { * * @param key - represents {@link SharePartitionKey} whose offset needs updating * @param offset - represents the latest partition offset for provided key + * @param isDelete - true if the offset is for a tombstone record */ - public void updateState(SharePartitionKey key, long offset) { + public void updateState(SharePartitionKey key, long offset, boolean isDelete) { lastRedundantOffset.set(Math.min(lastRedundantOffset.get(), offset)); offsets.put(key, offset); Optional redundantOffset = findRedundantOffset(); redundantOffset.ifPresent(lastRedundantOffset::set); + + // If the share partition is deleted, we should not hold onto its offset in our calculations + // as there is nothing beyond deletion which is going to update its state. + if (isDelete) { + offsets.remove(key); + } } private Optional findRedundantOffset() { diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java index 76a654de4c1..9d52780faa5 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java @@ -266,7 +266,7 @@ public class ShareCoordinatorShard implements CoordinatorShard expectedOffset; + final boolean isDelete; - private TestTuple(SharePartitionKey key, long offset, Optional expectedOffset) { + private TestTuple(SharePartitionKey key, long offset, Optional expectedOffset, boolean isDelete) { this.key = key; this.offset = offset; this.expectedOffset = expectedOffset; + this.isDelete = isDelete; } static TestTuple instance(SharePartitionKey key, long offset, Optional expectedOffset) { - return new TestTuple(key, offset, expectedOffset); + return new TestTuple(key, offset, expectedOffset, false); + } + + static TestTuple instance(SharePartitionKey key, long offset, Optional expectedOffset, boolean isDelete) { + return new TestTuple(key, offset, expectedOffset, isDelete); } } @@ -96,19 +106,35 @@ public class ShareCoordinatorOffsetsManagerTest { static Stream generateNoRedundantStateCases() { return Stream.of( new ShareOffsetTestHolder( - "no redundant state single key", + "no redundant state single key.", List.of( ShareOffsetTestHolder.TestTuple.instance(KEY1, 10L, Optional.of(10L)) ) ), new ShareOffsetTestHolder( - "no redundant state multiple keys", + "no redundant state single key with delete.", + List.of( + ShareOffsetTestHolder.TestTuple.instance(KEY1, 10L, Optional.of(10L), true) + ) + ), + + new ShareOffsetTestHolder( + "no redundant state multiple keys.", List.of( ShareOffsetTestHolder.TestTuple.instance(KEY1, 10L, Optional.of(10L)), ShareOffsetTestHolder.TestTuple.instance(KEY4, 11L, Optional.of(10L)), ShareOffsetTestHolder.TestTuple.instance(KEY2, 13L, Optional.of(10L)) ) + ), + + new ShareOffsetTestHolder( + "no redundant state multiple keys with delete.", + List.of( + ShareOffsetTestHolder.TestTuple.instance(KEY1, 10L, Optional.of(10L), true), + ShareOffsetTestHolder.TestTuple.instance(KEY4, 11L, Optional.of(11L), true), + ShareOffsetTestHolder.TestTuple.instance(KEY2, 13L, Optional.of(13L), true) + ) ) ); } @@ -116,7 +142,7 @@ public class ShareCoordinatorOffsetsManagerTest { static Stream generateRedundantStateCases() { return Stream.of( new ShareOffsetTestHolder( - "redundant state single key", + "redundant state single key.", List.of( ShareOffsetTestHolder.TestTuple.instance(KEY1, 10L, Optional.of(10L)), ShareOffsetTestHolder.TestTuple.instance(KEY1, 11L, Optional.of(11L)), @@ -125,7 +151,7 @@ public class ShareCoordinatorOffsetsManagerTest { ), new ShareOffsetTestHolder( - "redundant state multiple keys", + "redundant state multiple keys.", // KEY1: 10 17 // KEY2: 11 16 // KEY3: 15 @@ -136,6 +162,20 @@ public class ShareCoordinatorOffsetsManagerTest { ShareOffsetTestHolder.TestTuple.instance(KEY2, 16L, Optional.of(10L)), // KEY2 11 redundant but should not be returned ShareOffsetTestHolder.TestTuple.instance(KEY1, 17L, Optional.of(15L)) ) + ), + + new ShareOffsetTestHolder( + "redundant state multiple keys with delete.", + // KEY1: 10 17 + // KEY2: 11 16 + // KEY3: 15 + List.of( + ShareOffsetTestHolder.TestTuple.instance(KEY1, 10L, Optional.of(10L)), + ShareOffsetTestHolder.TestTuple.instance(KEY2, 11L, Optional.of(10L)), + ShareOffsetTestHolder.TestTuple.instance(KEY3, 15L, Optional.of(10L), true), + ShareOffsetTestHolder.TestTuple.instance(KEY2, 16L, Optional.of(10L)), // KEY2 11 redundant but should not be returned + ShareOffsetTestHolder.TestTuple.instance(KEY1, 17L, Optional.of(16L)) // Because we have removed KEY3 from calculation + ) ) ); @@ -144,7 +184,7 @@ public class ShareCoordinatorOffsetsManagerTest { static Stream generateComplexCases() { return Stream.of( new ShareOffsetTestHolder( - "redundant state reverse key order", + "redundant state reverse key order.", // Requests come in order KEY1, KEY2, KEY3, KEY3, KEY2, KEY1. List.of( ShareOffsetTestHolder.TestTuple.instance(KEY1, 10L, Optional.of(10L)), @@ -156,6 +196,18 @@ public class ShareCoordinatorOffsetsManagerTest { ) ), + new ShareOffsetTestHolder( + "redundant state reverse key order with delete.", + List.of( + ShareOffsetTestHolder.TestTuple.instance(KEY1, 10L, Optional.of(10L)), + ShareOffsetTestHolder.TestTuple.instance(KEY2, 11L, Optional.of(10L)), + ShareOffsetTestHolder.TestTuple.instance(KEY3, 15L, Optional.of(10L)), + ShareOffsetTestHolder.TestTuple.instance(KEY3, 18L, Optional.of(10L), true), + ShareOffsetTestHolder.TestTuple.instance(KEY2, 20L, Optional.of(10L), true), + ShareOffsetTestHolder.TestTuple.instance(KEY1, 25L, Optional.of(25L)) // Because KEY2 and KEY3 are gone. + ) + ), + new ShareOffsetTestHolder( "redundant state infrequently written partition.", List.of( @@ -170,6 +222,20 @@ public class ShareCoordinatorOffsetsManagerTest { ShareOffsetTestHolder.TestTuple.instance(KEY3, 28L, Optional.of(10L)), ShareOffsetTestHolder.TestTuple.instance(KEY1, 30L, Optional.of(27L)) ) + ), + + new ShareOffsetTestHolder( + "redundant state infrequently written partition with delete.", + List.of( + ShareOffsetTestHolder.TestTuple.instance(KEY1, 10L, Optional.of(10L)), + ShareOffsetTestHolder.TestTuple.instance(KEY2, 11L, Optional.of(10L)), + ShareOffsetTestHolder.TestTuple.instance(KEY3, 15L, Optional.of(10L)), + ShareOffsetTestHolder.TestTuple.instance(KEY2, 18L, Optional.of(10L)), + ShareOffsetTestHolder.TestTuple.instance(KEY3, 20L, Optional.of(10L), true), //KEY3 no longer party to calculation + ShareOffsetTestHolder.TestTuple.instance(KEY2, 22L, Optional.of(10L)), + ShareOffsetTestHolder.TestTuple.instance(KEY2, 27L, Optional.of(10L), true), //KEY2 no longer party to calculation + ShareOffsetTestHolder.TestTuple.instance(KEY1, 30L, Optional.of(30L)) + ) ) ); } @@ -179,7 +245,7 @@ public class ShareCoordinatorOffsetsManagerTest { public void testUpdateStateNoRedundantState(ShareOffsetTestHolder holder) { if (holder.shouldRun) { holder.tuples.forEach(tuple -> { - manager.updateState(tuple.key, tuple.offset); + manager.updateState(tuple.key, tuple.offset, tuple.isDelete); assertEquals(tuple.expectedOffset, manager.lastRedundantOffset(), holder.testName); }); } @@ -190,7 +256,7 @@ public class ShareCoordinatorOffsetsManagerTest { public void testUpdateStateRedundantState(ShareOffsetTestHolder holder) { if (holder.shouldRun) { holder.tuples.forEach(tuple -> { - manager.updateState(tuple.key, tuple.offset); + manager.updateState(tuple.key, tuple.offset, tuple.isDelete); assertEquals(tuple.expectedOffset, manager.lastRedundantOffset(), holder.testName); }); } @@ -201,9 +267,9 @@ public class ShareCoordinatorOffsetsManagerTest { public void testUpdateStateComplexCases(ShareOffsetTestHolder holder) { if (holder.shouldRun) { holder.tuples.forEach(tuple -> { - manager.updateState(tuple.key, tuple.offset); + manager.updateState(tuple.key, tuple.offset, tuple.isDelete); assertEquals(tuple.expectedOffset, manager.lastRedundantOffset(), holder.testName); }); } } -} +} \ No newline at end of file From 865beb6ede7aced951bf73c5e23d386dc6ab9094 Mon Sep 17 00:00:00 2001 From: Mickael Maison Date: Wed, 10 Sep 2025 20:47:11 +0200 Subject: [PATCH 018/100] MINOR: Remove explicit version list from upgrade.from docs (#20518) That config has a `Validator` so we already automatically print the valid values in the generated docs: https://kafka.apache.org/documentation/#streamsconfigs_upgrade.from That will be one less place to upgrade each time we make a new release. Reviewers: Chia-Ping Tsai --- .../main/java/org/apache/kafka/streams/StreamsConfig.java | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 7f498e32bdc..772efe6be99 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -836,13 +836,7 @@ public class StreamsConfig extends AbstractConfig { private static final String UPGRADE_FROM_DOC = "Allows live upgrading (and downgrading in some cases -- see upgrade guide) in a backward compatible way. Default is null. " + "Please refer to the Kafka Streams upgrade guide for instructions on how and when to use this config. " + "Note that when upgrading from 3.5 to a newer version it is never required to specify this config, " + - "while upgrading live directly to 4.0+ from 2.3 or below is no longer supported even with this config. " + - "Accepted values are \"" + UPGRADE_FROM_24 + "\", \"" + - UPGRADE_FROM_25 + "\", \"" + UPGRADE_FROM_26 + "\", \"" + UPGRADE_FROM_27 + "\", \"" + - UPGRADE_FROM_28 + "\", \"" + UPGRADE_FROM_30 + "\", \"" + UPGRADE_FROM_31 + "\", \"" + - UPGRADE_FROM_32 + "\", \"" + UPGRADE_FROM_33 + "\", \"" + UPGRADE_FROM_34 + "\", \"" + - UPGRADE_FROM_35 + "\", \"" + UPGRADE_FROM_36 + "\", \"" + UPGRADE_FROM_37 + "\", \"" + - UPGRADE_FROM_38 + "\", \"" + UPGRADE_FROM_39 + "\", \"" + "(for upgrading from the corresponding old version)."; + "while upgrading live directly to 4.0+ from 2.3 or below is no longer supported even with this config."; /** {@code topology.optimization} */ public static final String TOPOLOGY_OPTIMIZATION_CONFIG = "topology.optimization"; From 8a79ea2e5bde348e4ab3e45c28a7b2661fc6bfef Mon Sep 17 00:00:00 2001 From: jimmy Date: Thu, 11 Sep 2025 18:21:35 +0800 Subject: [PATCH 019/100] KAFKA-19676 EpochState should override close to avoid throwing IOException (#20508) Jira: [KAFKA-19676](https://issues.apache.org/jira/browse/KAFKA-19676) All subclasses of EpochState do not throw an IOException when closing, so catching it is unnecessary. We could override close to remove the IOException declaration. Reviewers: Jhen-Yung Hsu , TaiJuWu , Chia-Ping Tsai --- raft/src/main/java/org/apache/kafka/raft/EpochState.java | 7 +++++++ .../src/main/java/org/apache/kafka/raft/QuorumState.java | 9 +-------- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/EpochState.java b/raft/src/main/java/org/apache/kafka/raft/EpochState.java index 338f6603181..e4afba6893a 100644 --- a/raft/src/main/java/org/apache/kafka/raft/EpochState.java +++ b/raft/src/main/java/org/apache/kafka/raft/EpochState.java @@ -59,4 +59,11 @@ public interface EpochState extends Closeable { * User-friendly description of the state */ String name(); + + /** + * Since all subclasses implement the Closeable interface while none throw any IOException, + * this implementation is provided to eliminate the need for exception handling in the close operation. + */ + @Override + void close(); } diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index 1462b824fab..4ae24c84eaf 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -27,8 +27,6 @@ import org.apache.kafka.server.common.OffsetAndEpoch; import org.slf4j.Logger; -import java.io.IOException; -import java.io.UncheckedIOException; import java.util.List; import java.util.Optional; import java.util.OptionalInt; @@ -736,12 +734,7 @@ public class QuorumState { private void memoryTransitionTo(EpochState newState) { if (state != null) { - try { - state.close(); - } catch (IOException e) { - throw new UncheckedIOException( - "Failed to transition from " + state.name() + " to " + newState.name(), e); - } + state.close(); } EpochState from = state; From dd824a2e748d7a61f02d3647a6606b8395b9220d Mon Sep 17 00:00:00 2001 From: Shashank Date: Thu, 11 Sep 2025 07:06:25 -0700 Subject: [PATCH 020/100] KAFKA-19666: Remove old restoration codepath from RestoreIntegrationTest [4/N] (#20498) Clean up `RestoreIntegrationTest.java` Reviewers: Lucas Brutschy --- .../integration/RestoreIntegrationTest.java | 74 ++++++------------- 1 file changed, 21 insertions(+), 53 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java index 7370d488757..e85ac344157 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java @@ -43,7 +43,6 @@ import org.apache.kafka.streams.KafkaStreams.State; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.StreamsConfig.InternalConfig; import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; @@ -77,7 +76,6 @@ import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.TestInfo; 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; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -161,8 +159,8 @@ public class RestoreIntegrationTest { CLUSTER.createTopic(inputStream, 2, 1); } - private Properties props(final boolean stateUpdaterEnabled) { - return props(mkObjectProperties(mkMap(mkEntry(InternalConfig.STATE_UPDATER_ENABLED, stateUpdaterEnabled)))); + private Properties props() { + return props(mkObjectProperties(mkMap())); } private Properties props(final Properties extraProperties) { @@ -267,17 +265,12 @@ public class RestoreIntegrationTest { } @ParameterizedTest - @CsvSource({ - "true,true", - "true,false", - "false,true", - "false,false" - }) - public void shouldRestoreStateFromSourceTopicForReadOnlyStore(final boolean stateUpdaterEnabled, final boolean useNewProtocol) throws Exception { + @ValueSource(booleans = {true, false}) + public void shouldRestoreStateFromSourceTopicForReadOnlyStore(final boolean useNewProtocol) throws Exception { final AtomicInteger numReceived = new AtomicInteger(0); final Topology topology = new Topology(); - final Properties props = props(stateUpdaterEnabled); + final Properties props = props(); if (useNewProtocol) { props.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.name()); } @@ -338,17 +331,12 @@ public class RestoreIntegrationTest { } @ParameterizedTest - @CsvSource({ - "true,true", - "true,false", - "false,true", - "false,false" - }) - public void shouldRestoreStateFromSourceTopicForGlobalTable(final boolean stateUpdaterEnabled, final boolean useNewProtocol) throws Exception { + @ValueSource(booleans = {true, false}) + public void shouldRestoreStateFromSourceTopicForGlobalTable(final boolean useNewProtocol) throws Exception { final AtomicInteger numReceived = new AtomicInteger(0); final StreamsBuilder builder = new StreamsBuilder(); - final Properties props = props(stateUpdaterEnabled); + final Properties props = props(); props.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE); if (useNewProtocol) { props.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.name()); @@ -413,20 +401,15 @@ public class RestoreIntegrationTest { } @ParameterizedTest - @CsvSource({ - "true,true", - "true,false", - "false,true", - "false,false" - }) - public void shouldRestoreStateFromChangelogTopic(final boolean stateUpdaterEnabled, final boolean useNewProtocol) throws Exception { + @ValueSource(booleans = {true, false}) + public void shouldRestoreStateFromChangelogTopic(final boolean useNewProtocol) throws Exception { final String changelog = appId + "-store-changelog"; CLUSTER.createTopic(changelog, 2, 1); final AtomicInteger numReceived = new AtomicInteger(0); final StreamsBuilder builder = new StreamsBuilder(); - final Properties props = props(stateUpdaterEnabled); + final Properties props = props(); if (useNewProtocol) { props.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.name()); @@ -474,13 +457,8 @@ public class RestoreIntegrationTest { } @ParameterizedTest - @CsvSource({ - "true,true", - "true,false", - "false,true", - "false,false" - }) - public void shouldSuccessfullyStartWhenLoggingDisabled(final boolean stateUpdaterEnabled, final boolean useNewProtocol) throws InterruptedException { + @ValueSource(booleans = {true, false}) + public void shouldSuccessfullyStartWhenLoggingDisabled(final boolean useNewProtocol) throws InterruptedException { final StreamsBuilder builder = new StreamsBuilder(); final KStream stream = builder.stream(inputStream); @@ -490,7 +468,7 @@ public class RestoreIntegrationTest { Integer::sum, Materialized.>as("reduce-store").withLoggingDisabled() ); - final Properties props = props(stateUpdaterEnabled); + final Properties props = props(); if (useNewProtocol) { props.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.name()); } @@ -503,13 +481,8 @@ public class RestoreIntegrationTest { } @ParameterizedTest - @CsvSource({ - "true,true", - "true,false", - "false,true", - "false,false" - }) - public void shouldProcessDataFromStoresWithLoggingDisabled(final boolean stateUpdaterEnabled, final boolean useNewProtocol) throws InterruptedException { + @ValueSource(booleans = {true, false}) + public void shouldProcessDataFromStoresWithLoggingDisabled(final boolean useNewProtocol) throws InterruptedException { IntegrationTestUtils.produceKeyValuesSynchronously(inputStream, asList(KeyValue.pair(1, 1), KeyValue.pair(2, 2), @@ -537,7 +510,7 @@ public class RestoreIntegrationTest { final Topology topology = streamsBuilder.build(); - final Properties props = props(stateUpdaterEnabled); + final Properties props = props(); if (useNewProtocol) { props.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.name()); @@ -558,13 +531,8 @@ public class RestoreIntegrationTest { } @ParameterizedTest - @CsvSource({ - "true,true", - "true,false", - "false,true", - "false,false" - }) - public void shouldRecycleStateFromStandbyTaskPromotedToActiveTaskAndNotRestore(final boolean stateUpdaterEnabled, final boolean useNewProtocol) throws Exception { + @ValueSource(booleans = {true, false}) + public void shouldRecycleStateFromStandbyTaskPromotedToActiveTaskAndNotRestore(final boolean useNewProtocol) throws Exception { final StreamsBuilder builder = new StreamsBuilder(); builder.table( inputStream, @@ -576,7 +544,7 @@ public class RestoreIntegrationTest { CLUSTER.setGroupStandbyReplicas(appId, 1); } - final Properties props1 = props(stateUpdaterEnabled); + final Properties props1 = props(); props1.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1); props1.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId + "-1").getPath()); if (useNewProtocol) { @@ -585,7 +553,7 @@ public class RestoreIntegrationTest { purgeLocalStreamsState(props1); final KafkaStreams streams1 = new KafkaStreams(builder.build(), props1); - final Properties props2 = props(stateUpdaterEnabled); + final Properties props2 = props(); props2.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1); props2.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId + "-2").getPath()); if (useNewProtocol) { From 20268330d56a024610e561ff2b4a9539db072fa1 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Fri, 12 Sep 2025 10:31:19 +0200 Subject: [PATCH 021/100] MINOR: Deflake and improve SmokeTestDriverIntegrationTest (#20509) This improves the SmokeTestDriverIntegrationTest in three ways: 1) If a SmokeTestClient fails (enters a terminal ERROR state), the SmokeTestDriverIntegrationTest currently times out, because it keeps waiting for state NOT_RUNNING. This makes debugging quite difficult. This minor change makes sure to just fail the test immediately, if a SmokeTestClient enters the ERROR state. 2) If a test times out or fails prematurely, because a SmokeTestClient crashed, the SmokeTestClients aren't shut down correctly, which will affect the following test runs. Therefore, I am adding clean-up logic that running SmokeTestClients in `@AfterAll`. 3) Finally, I found that the processingThread variation of this thread triggers a subtle race condition. Since this features is currently not actively developed, I disabled those variations and created a ticket to reactivate the test. Reviewers: Matthias J. Sax , Chia-Ping Tsai , Bill Bejeck --------- Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- .../SmokeTestDriverIntegrationTest.java | 37 +++++++++++++------ .../kafka/streams/tests/SmokeTestClient.java | 9 +++++ 2 files changed, 35 insertions(+), 11 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java index 6d0c9f4762d..4e59e9523c4 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java @@ -27,6 +27,7 @@ import org.apache.kafka.streams.tests.SmokeTestClient; import org.apache.kafka.streams.tests.SmokeTestDriver; import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; @@ -46,22 +47,26 @@ import java.util.Set; import static org.apache.kafka.streams.tests.SmokeTestDriver.generate; import static org.apache.kafka.streams.tests.SmokeTestDriver.verify; import static org.apache.kafka.streams.utils.TestUtils.safeUniqueTestName; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @Timeout(600) @Tag("integration") public class SmokeTestDriverIntegrationTest { - public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(3); + private static EmbeddedKafkaCluster cluster = null; public TestInfo testInfo; + private ArrayList clients = new ArrayList<>(); @BeforeAll public static void startCluster() throws IOException { - CLUSTER.start(); + cluster = new EmbeddedKafkaCluster(3); + cluster.start(); } @AfterAll public static void closeCluster() { - CLUSTER.stop(); + cluster.stop(); + cluster = null; } @BeforeEach @@ -69,6 +74,16 @@ public class SmokeTestDriverIntegrationTest { this.testInfo = testInfo; } + @AfterEach + public void shutDown(final TestInfo testInfo) { + // Clean up clients in case the test failed or timed out + for (final SmokeTestClient client : clients) { + if (!client.closed() && !client.error()) { + client.close(); + } + } + } + private static class Driver extends Thread { private final String bootstrapServers; private final int numKeys; @@ -108,12 +123,11 @@ public class SmokeTestDriverIntegrationTest { // During the new stream added and old stream left, the stream process should still complete without issue. // We set 2 timeout condition to fail the test before passing the verification: // (1) 10 min timeout, (2) 30 tries of polling without getting any data + // The processing thread variations where disabled since they triggered a race condition, see KAFKA-19696 @ParameterizedTest @CsvSource({ "false, true", - "false, false", - "true, true", - "true, false" + "false, false" }) public void shouldWorkWithRebalance( final boolean processingThreadsEnabled, @@ -126,11 +140,10 @@ public class SmokeTestDriverIntegrationTest { throw new AssertionError("Test called halt(). code:" + statusCode + " message:" + message); }); int numClientsCreated = 0; - final ArrayList clients = new ArrayList<>(); - IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, SmokeTestDriver.topics()); + IntegrationTestUtils.cleanStateBeforeTest(cluster, SmokeTestDriver.topics()); - final String bootstrapServers = CLUSTER.bootstrapServers(); + final String bootstrapServers = cluster.bootstrapServers(); final Driver driver = new Driver(bootstrapServers, 10, 1000); driver.start(); System.out.println("started driver"); @@ -144,8 +157,8 @@ public class SmokeTestDriverIntegrationTest { if (streamsProtocolEnabled) { props.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.name().toLowerCase(Locale.getDefault())); // decrease the session timeout so that we can trigger the rebalance soon after old client left closed - CLUSTER.setGroupSessionTimeout(appId, 10000); - CLUSTER.setGroupHeartbeatTimeout(appId, 1000); + cluster.setGroupSessionTimeout(appId, 10000); + cluster.setGroupHeartbeatTimeout(appId, 1000); } else { // decrease the session timeout so that we can trigger the rebalance soon after old client left closed props.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 10000); @@ -167,6 +180,7 @@ public class SmokeTestDriverIntegrationTest { client.closeAsync(); while (!client.closed()) { + assertFalse(client.error(), "The streams application seems to have crashed."); Thread.sleep(100); } } @@ -184,6 +198,7 @@ public class SmokeTestDriverIntegrationTest { // then, wait for them to stop for (final SmokeTestClient client : clients) { while (!client.closed()) { + assertFalse(client.error(), "The streams application seems to have crashed."); Thread.sleep(100); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java index 9ca9db21a95..7f8057c5597 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -54,6 +54,7 @@ public class SmokeTestClient extends SmokeTestUtil { private KafkaStreams streams; private boolean uncaughtException = false; private volatile boolean closed; + private volatile boolean error; private static void addShutdownHook(final String name, final Runnable runnable) { if (name != null) { @@ -71,6 +72,10 @@ public class SmokeTestClient extends SmokeTestUtil { return closed; } + public boolean error() { + return error; + } + public void start(final Properties streamsProperties) { final Topology build = getTopology(); streams = new KafkaStreams(build, getStreamsConfig(streamsProperties)); @@ -85,6 +90,10 @@ public class SmokeTestClient extends SmokeTestUtil { if (newState == KafkaStreams.State.NOT_RUNNING) { closed = true; } + + if (newState == KafkaStreams.State.ERROR) { + error = true; + } }); streams.setUncaughtExceptionHandler(e -> { From 54b88f672172f0fe5cc22051e03ff9ff6c8e44d8 Mon Sep 17 00:00:00 2001 From: Maros Orsak Date: Fri, 12 Sep 2025 23:24:40 +0200 Subject: [PATCH 022/100] MINOR: Refactor on FeaturesPublisher and ScramPublisher (#20522) This PR is a follow-up from https://github.com/apache/kafka/pull/20468. Basically makes two things: 1. Moves the variable to the catch block as it is used only there. 2. Refactor FeaturesPublisher to handle exceptions the same as ScramPublisher or other publishers :) Reviewers: Chia-Ping Tsai --------- Signed-off-by: see-quick --- .../scala/kafka/server/ControllerServer.scala | 2 +- .../metadata/BrokerMetadataPublisher.scala | 2 +- .../metadata/publisher/FeaturesPublisher.java | 25 +++++++++++++------ .../metadata/publisher/ScramPublisher.java | 8 ++---- 4 files changed, 21 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index bfcc9ed7c0b..badcb9b2d8a 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -145,7 +145,7 @@ class ControllerServer( metadataCachePublisher = new KRaftMetadataCachePublisher(metadataCache) - featuresPublisher = new FeaturesPublisher(logContext) + featuresPublisher = new FeaturesPublisher(logContext, sharedServer.metadataPublishingFaultHandler) registrationsPublisher = new ControllerRegistrationsPublisher() diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala index 80a037ce4bd..30ea835b5be 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala @@ -224,7 +224,7 @@ class BrokerMetadataPublisher( dynamicTopicClusterQuotaPublisher.onMetadataUpdate(delta, newImage) // Apply SCRAM delta. - scramPublisher.onMetadataUpdate(delta, newImage) + scramPublisher.onMetadataUpdate(delta, newImage, manifest) // Apply DelegationToken delta. delegationTokenPublisher.onMetadataUpdate(delta, newImage) diff --git a/metadata/src/main/java/org/apache/kafka/metadata/publisher/FeaturesPublisher.java b/metadata/src/main/java/org/apache/kafka/metadata/publisher/FeaturesPublisher.java index 60fa202ee42..9ccddbe8109 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/publisher/FeaturesPublisher.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/publisher/FeaturesPublisher.java @@ -23,6 +23,7 @@ import org.apache.kafka.image.MetadataImage; import org.apache.kafka.image.loader.LoaderManifest; import org.apache.kafka.image.publisher.MetadataPublisher; import org.apache.kafka.server.common.FinalizedFeatures; +import org.apache.kafka.server.fault.FaultHandler; import org.slf4j.Logger; @@ -31,12 +32,15 @@ import static org.apache.kafka.server.common.MetadataVersion.MINIMUM_VERSION; public class FeaturesPublisher implements MetadataPublisher { private final Logger log; + private final FaultHandler faultHandler; private volatile FinalizedFeatures finalizedFeatures = FinalizedFeatures.fromKRaftVersion(MINIMUM_VERSION); public FeaturesPublisher( - LogContext logContext + LogContext logContext, + FaultHandler faultHandler ) { - log = logContext.logger(FeaturesPublisher.class); + this.log = logContext.logger(FeaturesPublisher.class); + this.faultHandler = faultHandler; } public FinalizedFeatures features() { @@ -54,15 +58,20 @@ public class FeaturesPublisher implements MetadataPublisher { MetadataImage newImage, LoaderManifest manifest ) { - if (delta.featuresDelta() != null) { - FinalizedFeatures newFinalizedFeatures = new FinalizedFeatures(newImage.features().metadataVersionOrThrow(), + try { + if (delta.featuresDelta() != null) { + FinalizedFeatures newFinalizedFeatures = new FinalizedFeatures(newImage.features().metadataVersionOrThrow(), newImage.features().finalizedVersions(), newImage.provenance().lastContainedOffset() - ); - if (!newFinalizedFeatures.equals(finalizedFeatures)) { - log.info("Loaded new metadata {}.", newFinalizedFeatures); - finalizedFeatures = newFinalizedFeatures; + ); + if (!newFinalizedFeatures.equals(finalizedFeatures)) { + log.info("Loaded new metadata {}.", newFinalizedFeatures); + finalizedFeatures = newFinalizedFeatures; + } } + } catch (Throwable t) { + faultHandler.handleFault("Uncaught exception while publishing SCRAM changes from MetadataDelta up to " + + newImage.highestOffsetAndEpoch().offset(), t); } } } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/publisher/ScramPublisher.java b/metadata/src/main/java/org/apache/kafka/metadata/publisher/ScramPublisher.java index 7a286eb6351..2621de77bf7 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/publisher/ScramPublisher.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/publisher/ScramPublisher.java @@ -44,11 +44,6 @@ public class ScramPublisher implements MetadataPublisher { @Override public void onMetadataUpdate(MetadataDelta delta, MetadataImage newImage, LoaderManifest manifest) { - onMetadataUpdate(delta, newImage); - } - - public void onMetadataUpdate(MetadataDelta delta, MetadataImage newImage) { - String deltaName = "MetadataDelta up to " + newImage.highestOffsetAndEpoch().offset(); try { // Apply changes to SCRAM credentials. ScramDelta scramDelta = delta.scramDelta(); @@ -63,7 +58,8 @@ public class ScramPublisher implements MetadataPublisher { }); } } catch (Throwable t) { - faultHandler.handleFault("Uncaught exception while publishing SCRAM changes from " + deltaName, t); + faultHandler.handleFault("Uncaught exception while publishing SCRAM changes from MetadataDelta up to " + + newImage.highestOffsetAndEpoch().offset(), t); } } } From af2a8db3c663895c47102ca623d877e484c9ab2f Mon Sep 17 00:00:00 2001 From: Chang-Chi Hsu Date: Fri, 12 Sep 2025 23:44:17 +0200 Subject: [PATCH 023/100] KAFKA-18105 Fix flaky PlaintextAdminIntegrationTest#testElectPreferredLeaders (#20068) ## Changes This PR improves the stability of the PlaintextAdminIntegrationTest.testElectPreferredLeaders test by introducing short Thread.sleep( ) delays before invoking: - changePreferredLeader( ) - waitForBrokersOutOfIsr( ) ## Reasons - Metadata propagation for partition2 : Kafka requires time to propagate the updated leader metadata across all brokers. Without waiting, metadataCache may return outdated leader information for partition2. - Eviction of broker1 from the ISR : To simulate a scenario where broker1 is no longer eligible as leader, the test relies on broker1 being removed from the ISR (e.g., due to intentional shutdown). This eviction is not instantaneous and requires a brief delay before Kafka reflects the change. Reviewers: PoAn Yang , TengYao Chi , TaiJuWu , Ken Huang , Chia-Ping Tsai --- .../api/PlaintextAdminIntegrationTest.scala | 30 ++++++++++++++++--- 1 file changed, 26 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 6b1e3dda727..1c3a7ed42e3 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -3061,6 +3061,20 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { } } + /** + * Waits until the metadata for the given partition has fully propagated and become consistent across all brokers. + * + * @param partition The partition whose leader metadata should be verified across all brokers. + */ + def waitForBrokerMetadataPropagation(partition: TopicPartition): Unit = { + while (brokers.exists(_.metadataCache.getPartitionLeaderEndpoint(partition.topic, partition.partition(), listenerName).isEmpty) || + brokers.map(_.metadataCache.getPartitionLeaderEndpoint(partition.topic, partition.partition(), listenerName)) + .filter(_.isPresent) + .map(_.get()) + .toSet.size != 1) + TimeUnit.MILLISECONDS.sleep(300) + } + @Test def testElectPreferredLeaders(): Unit = { client = createAdminClient @@ -3087,12 +3101,12 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val prior1 = brokers.head.metadataCache.getPartitionLeaderEndpoint(partition1.topic, partition1.partition(), listenerName).get.id() val prior2 = brokers.head.metadataCache.getPartitionLeaderEndpoint(partition2.topic, partition2.partition(), listenerName).get.id() - var m = Map.empty[TopicPartition, Optional[NewPartitionReassignment]] + var reassignmentMap = Map.empty[TopicPartition, Optional[NewPartitionReassignment]] if (prior1 != preferred) - m += partition1 -> Optional.of(new NewPartitionReassignment(newAssignment.map(Int.box).asJava)) + reassignmentMap += partition1 -> Optional.of(new NewPartitionReassignment(newAssignment.map(Int.box).asJava)) if (prior2 != preferred) - m += partition2 -> Optional.of(new NewPartitionReassignment(newAssignment.map(Int.box).asJava)) - client.alterPartitionReassignments(m.asJava).all().get() + reassignmentMap += partition2 -> Optional.of(new NewPartitionReassignment(newAssignment.map(Int.box).asJava)) + client.alterPartitionReassignments(reassignmentMap.asJava).all().get() TestUtils.waitUntilTrue( () => preferredLeader(partition1) == preferred && preferredLeader(partition2) == preferred, @@ -3120,6 +3134,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { TestUtils.assertLeader(client, partition2, 0) // Now change the preferred leader to 1 + waitForBrokerMetadataPropagation(partition1) + waitForBrokerMetadataPropagation(partition2) changePreferredLeader(prefer1) // meaningful election @@ -3158,6 +3174,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { TestUtils.assertLeader(client, partition2, 1) // Now change the preferred leader to 2 + waitForBrokerMetadataPropagation(partition1) + waitForBrokerMetadataPropagation(partition2) changePreferredLeader(prefer2) // mixed results @@ -3174,9 +3192,13 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { TestUtils.assertLeader(client, partition2, 2) // Now change the preferred leader to 1 + waitForBrokerMetadataPropagation(partition1) + waitForBrokerMetadataPropagation(partition2) changePreferredLeader(prefer1) // but shut it down... killBroker(1) + waitForBrokerMetadataPropagation(partition1) + waitForBrokerMetadataPropagation(partition2) TestUtils.waitForBrokersOutOfIsr(client, Set(partition1, partition2), Set(1)) def assertPreferredLeaderNotAvailable( From 026710cbb4377c5522afb7644ef3763ec738ef0c Mon Sep 17 00:00:00 2001 From: Logan Zhu Date: Sat, 13 Sep 2025 06:01:00 +0800 Subject: [PATCH 024/100] MINOR: Update ClusterTestExtensions Javadoc example (#20525) The previous Javadoc example used the deprecated ClusterType. It is now updated to use `types = {Type.KRAFT, Type.CO_KRAFT}` Reviewers: Chia-Ping Tsai --- .../kafka/common/test/junit/ClusterTestExtensions.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/ClusterTestExtensions.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/ClusterTestExtensions.java index f08abdb32aa..c827ef0ab3c 100644 --- a/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/ClusterTestExtensions.java +++ b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/ClusterTestExtensions.java @@ -78,14 +78,14 @@ import static org.junit.jupiter.api.Assertions.assertTrue; * *
  * class SomeIntegrationTest {
- *   @ClusterTest(brokers = 1, controllers = 1, clusterType = ClusterType.Both)
- *   def someTest(): Unit = {
+ *   @ClusterTest(brokers = 1, controllers = 1, types = {Type.KRAFT, Type.CO_KRAFT})
+ *   void someTest(ClusterInstance cluster) {
  *     assertTrue(condition)
  *   }
  * }
  * 
* - * will generate two invocations of "someTest" (since ClusterType.Both was given). For each invocation, the test class + * will generate two invocations of "someTest" (since two cluster types were specified). For each invocation, the test class * SomeIntegrationTest will be instantiated, lifecycle methods (before/after) will be run, and "someTest" will be invoked. * * A special system property "kafka.cluster.test.repeat" can be used to cause repeated invocation of the tests. From 962f4ada75cd1ac0a8d3bdd54fcd455a57c8943f Mon Sep 17 00:00:00 2001 From: Chang-Chi Hsu Date: Sun, 14 Sep 2025 01:20:01 +0200 Subject: [PATCH 025/100] KAFKA-19203 Replace `ApiError#exception` by `Error#exception` for KafkaAdminClient (#19623) This pull request addresses KAFKA-19203 by replacing `ApiError#exception` with `Error#exception` in `KafkaAdminClient`. The previous use of `ApiError#exception` was redundant, as we only need the exception without the additional wrapping of `ApiError`. ## Changes - Replaced some usages of `ApiError#exception` with `Error#exception` in `KafkaAdminClient`. - Simplified exception handling logic to reduce unnecessary layers. Reviewers: Chia-Ping Tsai --- .../kafka/clients/admin/KafkaAdminClient.java | 36 ++++++++----------- 1 file changed, 14 insertions(+), 22 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 90f83eac935..78a7f905319 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 @@ -2494,8 +2494,7 @@ public class KafkaAdminClient extends AdminClient { DescribeClusterResponse response = (DescribeClusterResponse) abstractResponse; Errors error = Errors.forCode(response.data().errorCode()); if (error != Errors.NONE) { - ApiError apiError = new ApiError(error, response.data().errorMessage()); - handleFailure(apiError.exception()); + handleFailure(error.exception(response.data().errorMessage())); return; } @@ -2691,10 +2690,9 @@ public class KafkaAdminClient extends AdminClient { } else { List filterResults = new ArrayList<>(); for (DeleteAclsMatchingAcl matchingAcl : filterResult.matchingAcls()) { - ApiError aclError = new ApiError(Errors.forCode(matchingAcl.errorCode()), - matchingAcl.errorMessage()); + Errors aclError = Errors.forCode(matchingAcl.errorCode()); AclBinding aclBinding = DeleteAclsResponse.aclBinding(matchingAcl); - filterResults.add(new FilterResult(aclBinding, aclError.exception())); + filterResults.add(new FilterResult(aclBinding, aclError.exception(matchingAcl.errorMessage()))); } future.complete(new FilterResults(filterResults)); } @@ -3995,7 +3993,7 @@ public class KafkaAdminClient extends AdminClient { for (ReassignablePartitionResponse partition : topicResponse.partitions()) { errors.put( new TopicPartition(topicName, partition.partitionIndex()), - new ApiError(topLevelError, response.data().errorMessage()).exception() + topLevelError.exception(response.data().errorMessage()) ); receivedResponsesCount += 1; } @@ -4035,7 +4033,7 @@ public class KafkaAdminClient extends AdminClient { if (partitionError == Errors.NONE) { errors.put(tp, null); } else { - errors.put(tp, new ApiError(partitionError, partResponse.errorMessage()).exception()); + errors.put(tp, partitionError.exception(partResponse.errorMessage())); } receivedResponsesCount += 1; } @@ -4111,7 +4109,7 @@ public class KafkaAdminClient extends AdminClient { handleNotControllerError(error); break; default: - partitionReassignmentsFuture.completeExceptionally(new ApiError(error, response.data().errorMessage()).exception()); + partitionReassignmentsFuture.completeExceptionally(error.exception(response.data().errorMessage())); break; } Map reassignmentMap = new HashMap<>(); @@ -4993,14 +4991,11 @@ public class KafkaAdminClient extends AdminClient { void handleResponse(AbstractResponse response) { handleNotControllerError(response); AddRaftVoterResponse addResponse = (AddRaftVoterResponse) response; - if (addResponse.data().errorCode() != Errors.NONE.code()) { - ApiError error = new ApiError( - addResponse.data().errorCode(), - addResponse.data().errorMessage()); - future.completeExceptionally(error.exception()); - } else { + Errors error = Errors.forCode(addResponse.data().errorCode()); + if (error != Errors.NONE) + future.completeExceptionally(error.exception(addResponse.data().errorMessage())); + else future.complete(null); - } } @Override @@ -5038,14 +5033,11 @@ public class KafkaAdminClient extends AdminClient { void handleResponse(AbstractResponse response) { handleNotControllerError(response); RemoveRaftVoterResponse addResponse = (RemoveRaftVoterResponse) response; - if (addResponse.data().errorCode() != Errors.NONE.code()) { - ApiError error = new ApiError( - addResponse.data().errorCode(), - addResponse.data().errorMessage()); - future.completeExceptionally(error.exception()); - } else { + Errors error = Errors.forCode(addResponse.data().errorCode()); + if (error != Errors.NONE) + future.completeExceptionally(error.exception(addResponse.data().errorMessage())); + else future.complete(null); - } } @Override From 374bc469c7ee7b01c11654bf509cc8df6f470a5e Mon Sep 17 00:00:00 2001 From: Mickael Maison Date: Sun, 14 Sep 2025 14:25:12 +0200 Subject: [PATCH 026/100] MINOR: Cleanups in ops docs (#20532) - Fix typo in `process.role` - Fix formatting of quorum description commands Reviewers: Lan Ding , Ken Huang , TengYao Chi --- docs/ops.html | 62 +++++++++++++++++++++------------------------------ 1 file changed, 26 insertions(+), 36 deletions(-) diff --git a/docs/ops.html b/docs/ops.html index d6c0982854a..803b429f18f 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -4082,42 +4082,32 @@ In the replica description 0@controller-0:1234:3Db5QLSqSZieL3rJBUUegA, 0 is the If you are not sure whether you are using static or dynamic quorums, you can determine this by running something like the following:

-


-  $ bin/kafka-features.sh --bootstrap-controller localhost:9093 describe
-

+

$ bin/kafka-features.sh --bootstrap-controller localhost:9093 describe
+

+ If the kraft.version field is level 0 or absent, you are using a static quorum. If + it is 1 or above, you are using a dynamic quorum. For example, here is an example of a static + quorum:

+

Feature: kraft.version  SupportedMinVersion: 0  SupportedMaxVersion: 1  FinalizedVersionLevel: 0 Epoch: 5
+Feature: metadata.version       SupportedMinVersion: 3.3-IV3    SupportedMaxVersion: 3.9-IV0 FinalizedVersionLevel: 3.9-IV0  Epoch: 5
+

+ Here is another example of a static quorum:

+

Feature: metadata.version       SupportedMinVersion: 3.3-IV3    SupportedMaxVersion: 3.8-IV0 FinalizedVersionLevel: 3.8-IV0  Epoch: 5
+

+ Here is an example of a dynamic quorum:

+

Feature: kraft.version  SupportedMinVersion: 0  SupportedMaxVersion: 1  FinalizedVersionLevel: 1 Epoch: 5
+Feature: metadata.version       SupportedMinVersion: 3.3-IV3    SupportedMaxVersion: 3.9-IV0 FinalizedVersionLevel: 3.9-IV0  Epoch: 5
+

+ The static versus dynamic nature of the quorum is determined at the time of formatting. + Specifically, the quorum will be formatted as dynamic if controller.quorum.voters is + not present, and if the software version is Apache Kafka 3.9 or newer. If you have + followed the instructions earlier in this document, you will get a dynamic quorum.

- If the kraft.version field is level 0 or absent, you are using a static quorum. If - it is 1 or above, you are using a dynamic quorum. For example, here is an example of a static - quorum:

-


-Feature: kraft.version  SupportedMinVersion: 0  SupportedMaxVersion: 1  FinalizedVersionLevel: 0 Epoch: 5
-Feature: metadata.version       SupportedMinVersion: 3.3-IV3    SupportedMaxVersion: 3.9-IV0 FinalizedVersionLevel: 3.9-IV0  Epoch: 5
-

- - Here is another example of a static quorum:

-


-Feature: metadata.version       SupportedMinVersion: 3.3-IV3    SupportedMaxVersion: 3.8-IV0 FinalizedVersionLevel: 3.8-IV0  Epoch: 5
-

- - Here is an example of a dynamic quorum:

-


-Feature: kraft.version  SupportedMinVersion: 0  SupportedMaxVersion: 1  FinalizedVersionLevel: 1 Epoch: 5
-Feature: metadata.version       SupportedMinVersion: 3.3-IV3    SupportedMaxVersion: 3.9-IV0 FinalizedVersionLevel: 3.9-IV0  Epoch: 5
-

- - The static versus dynamic nature of the quorum is determined at the time of formatting. - Specifically, the quorum will be formatted as dynamic if controller.quorum.voters is - not present, and if the software version is Apache Kafka 3.9 or newer. If you have - followed the instructions earlier in this document, you will get a dynamic quorum.

- - If you would like the formatting process to fail if a dynamic quorum cannot be achieved, format your - controllers using the --feature kraft.version=1. (Note that you should not supply - this flag when formatting brokers -- only when formatting controllers.)

- -


-  $ bin/kafka-storage.sh format -t KAFKA_CLUSTER_ID --feature kraft.version=1 -c controller.properties
-

+ If you would like the formatting process to fail if a dynamic quorum cannot be achieved, format your + controllers using the --feature kraft.version=1. (Note that you should not supply + this flag when formatting brokers -- only when formatting controllers.)

+

$ bin/kafka-storage.sh format -t KAFKA_CLUSTER_ID --feature kraft.version=1 -c controller.properties
+

Note: To migrate from static voter set to dynamic voter set, please refer to the Upgrade section.

Add New Controller
@@ -4169,7 +4159,7 @@ CurrentObservers: [{"id": 0, "directoryId": "3Db5QLSqSZieL3rJBUUegA"},
$ bin/kafka-dump-log.sh --cluster-metadata-decoder --files metadata_log_dir/__cluster_metadata-0/00000000000000000000.log
-

This command decodes and prints the records in the a cluster metadata snapshot:

+

This command decodes and prints the records in a cluster metadata snapshot:

$ bin/kafka-dump-log.sh --cluster-metadata-decoder --files metadata_log_dir/__cluster_metadata-0/00000000000000000100-0000000001.checkpoint
@@ -4199,7 +4189,7 @@ foo

Deploying Considerations

    -
  • Kafka server's process.role should be set to either broker or controller but not both. Combined mode can be used in development environments, but it should be avoided in critical deployment environments.
  • +
  • Kafka server's process.roles should be set to either broker or controller but not both. Combined mode can be used in development environments, but it should be avoided in critical deployment environments.
  • For redundancy, a Kafka cluster should use 3 or more controllers, depending on factors like cost and the number of concurrent failures your system should withstand without availability impact. For the KRaft controller cluster to withstand N concurrent failures the controller cluster must include 2N + 1 controllers.
  • The Kafka controllers store all the metadata for the cluster in memory and on disk. We believe that for a typical Kafka cluster 5GB of main memory and 5GB of disk space on the metadata log director is sufficient.
From a9e529236fa54c07d48d38275bf9ab1124db072d Mon Sep 17 00:00:00 2001 From: NICOLAS GUYOMAR Date: Mon, 15 Sep 2025 04:04:06 +0200 Subject: [PATCH 027/100] MINOR: increase Config change throwable log info to error (#14380) The ApiError.fromThrowable(t) is going to return a generic Errors.UNKNOWN_SERVER_ERROR back to the calling client (CLI for instance) (eg if the broker has an authZ issue with ZK) and such UnknownServerException should have a matching ERROR level log in the broker logs IHMO to make it easier to troubleshoot Reviewers: Chia-Ping Tsai --- core/src/main/scala/kafka/server/ConfigAdminManager.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/ConfigAdminManager.scala b/core/src/main/scala/kafka/server/ConfigAdminManager.scala index 45a68d2d036..7394d2cfc43 100644 --- a/core/src/main/scala/kafka/server/ConfigAdminManager.scala +++ b/core/src/main/scala/kafka/server/ConfigAdminManager.scala @@ -151,7 +151,7 @@ class ConfigAdminManager(nodeId: Int, } catch { case t: Throwable => val err = ApiError.fromThrowable(t) - info(s"Error preprocessing incrementalAlterConfigs request on $configResource", t) + error(s"Error preprocessing incrementalAlterConfigs request on $configResource", t) results.put(resource, err) } } @@ -252,7 +252,7 @@ class ConfigAdminManager(nodeId: Int, } catch { case t: Throwable => val err = ApiError.fromThrowable(t) - info(s"Error preprocessing alterConfigs request on $configResource: $err") + error(s"Error preprocessing alterConfigs request on ${configResource}: ${err}") results.put(resource, err) } } From 2fd54837f0d1277d18799b540add1e06e7886f3a Mon Sep 17 00:00:00 2001 From: keemsisi Date: Mon, 15 Sep 2025 03:13:49 +0100 Subject: [PATCH 028/100] MINOR: Update on fixing tag description missing in javadoc (#20380) * Added tag description to @throws in method javadoc * Added explicit throws IndexOffsetOverflowException to method signature Reviewers: Chia-Ping Tsai --- .../apache/kafka/storage/internals/log/AbstractIndex.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) 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 b1475884e3a..7e7c3e1cc63 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 @@ -301,8 +301,12 @@ public abstract class AbstractIndex implements Closeable { } /** - * Get offset relative to base offset of this index - * @throws IndexOffsetOverflowException + * Gets the offset relative to the {@code baseOffset} of this index. + * + * @param offset the absolute offset to be converted into a relative offset from {@code baseOffset} + * @return the relative offset as an {@code int} + * @throws IndexOffsetOverflowException if the input offset is lesser than the base offset + * or if the relative offset exceeds {@link Integer#MAX_VALUE} */ public int relativeOffset(long offset) { OptionalInt relativeOffset = toRelative(offset); From e1b7699975e787a78f5e9bca540b4aa19f0d419d Mon Sep 17 00:00:00 2001 From: Yunchi Pang Date: Mon, 15 Sep 2025 10:35:42 +0800 Subject: [PATCH 029/100] MINOR: Bump versions in CI (#20497) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit **Summary** This PR bumps several GitHub Actions and dependencies used in CI workflows to their latest stable versions. This ensures our CI environment remains consistent, secure, and aligned with upstream improvements. **Changes** - requests: 2.32.3 → 2.32.4 - actions/checkout: v4 → v5 - actions/setup-python: v5 → v6 - actions/setup-java: v4 → v5 - actions/download-artifact: v4 → v5 - actions/labeler: v5 → v6 related: https://github.com/apache/kafka/pull/19940/files#r2328391161 Reviewers: Ken Huang , TengYao Chi , Chia-Ping Tsai --- .github/actions/setup-gradle/action.yml | 2 +- .github/actions/setup-python/action.yml | 2 +- .github/scripts/requirements.txt | 4 +++- .github/workflows/build.yml | 20 +++++++++---------- .github/workflows/ci-complete.yml | 4 ++-- .github/workflows/deflake.yml | 2 +- .github/workflows/docker_build_and_test.yml | 4 ++-- .../docker_official_image_build_and_test.yml | 4 ++-- .github/workflows/docker_rc_release.yml | 4 ++-- .github/workflows/generate-reports.yml | 2 +- .github/workflows/pr-labeled.yml | 2 +- .github/workflows/pr-linter.yml | 4 ++-- .github/workflows/pr-update.yml | 4 ++-- .../prepare_docker_official_image_source.yml | 4 ++-- .github/workflows/workflow-requested.yml | 2 +- tests/setup.py | 1 + 16 files changed, 34 insertions(+), 31 deletions(-) diff --git a/.github/actions/setup-gradle/action.yml b/.github/actions/setup-gradle/action.yml index 1a5b0902703..fe456568066 100644 --- a/.github/actions/setup-gradle/action.yml +++ b/.github/actions/setup-gradle/action.yml @@ -37,7 +37,7 @@ runs: using: "composite" steps: - name: Setup Java - uses: actions/setup-java@v4 + uses: actions/setup-java@v5 with: distribution: temurin java-version: ${{ inputs.java-version }} diff --git a/.github/actions/setup-python/action.yml b/.github/actions/setup-python/action.yml index d7e326314c1..10c55f6e083 100644 --- a/.github/actions/setup-python/action.yml +++ b/.github/actions/setup-python/action.yml @@ -22,7 +22,7 @@ runs: using: "composite" steps: - name: Setup Python - uses: actions/setup-python@v5 + uses: actions/setup-python@v6 with: python-version: 3.12 - name: Pip install diff --git a/.github/scripts/requirements.txt b/.github/scripts/requirements.txt index d59455f79da..d3fcf50bb74 100644 --- a/.github/scripts/requirements.txt +++ b/.github/scripts/requirements.txt @@ -12,6 +12,8 @@ # 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. + +# Note: Ensure the 'requests' version here matches the version in tests/setup.py PyYAML~=6.0 pytz==2024.2 -requests==2.32.3 +requests==2.32.4 diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index 5137f8bf372..47bb2cbc31d 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -66,12 +66,12 @@ jobs: name: Load Test Catalog steps: - name: Checkout main - uses: actions/checkout@v4 + uses: actions/checkout@v5 with: persist-credentials: false - name: Checkout test-catalog - uses: actions/checkout@v4 + uses: actions/checkout@v5 with: ref: 'test-catalog' persist-credentials: false @@ -118,7 +118,7 @@ jobs: env: GITHUB_CONTEXT: ${{ toJson(github) }} - name: Checkout code - uses: actions/checkout@v4 + uses: actions/checkout@v5 with: persist-credentials: false ref: ${{ github.sha }} # this is the default, just being explicit. @@ -192,7 +192,7 @@ jobs: name: JUnit tests Java ${{ matrix.java }}${{ matrix.run-flaky == true && ' (flaky)' || '' }}${{ matrix.run-new == true && ' (new)' || '' }} steps: - name: Checkout code - uses: actions/checkout@v4 + uses: actions/checkout@v5 with: persist-credentials: false ref: ${{ needs.configure.outputs.sha }} @@ -210,7 +210,7 @@ jobs: # the overall workflow, so we'll continue here without a test catalog. - name: Load Test Catalog id: load-test-catalog - uses: actions/download-artifact@v4 + uses: actions/download-artifact@v5 continue-on-error: true with: name: combined-test-catalog @@ -282,11 +282,11 @@ jobs: uploaded-test-catalog: ${{ steps.archive-test-catalog.outcome == 'success' }} steps: - name: Checkout code - uses: actions/checkout@v4 + uses: actions/checkout@v5 with: persist-credentials: false - name: Download Thread Dumps - uses: actions/download-artifact@v4 + uses: actions/download-artifact@v5 with: pattern: junit-thread-dumps-24-* path: thread-dumps @@ -300,7 +300,7 @@ jobs: exit 1; fi - name: Download JUnit XMLs - uses: actions/download-artifact@v4 + uses: actions/download-artifact@v5 with: pattern: junit-xml-24-* # Only look at JDK 24 tests for the test catalog path: junit-xml @@ -334,7 +334,7 @@ jobs: contents: write steps: - name: Checkout Test Catalog - uses: actions/checkout@v4 + uses: actions/checkout@v5 with: persist-credentials: true # Needed to commit and push later ref: test-catalog @@ -342,7 +342,7 @@ jobs: run: | rm -rf test-catalog - name: Download Test Catalog - uses: actions/download-artifact@v4 + uses: actions/download-artifact@v5 with: name: test-catalog path: test-catalog diff --git a/.github/workflows/ci-complete.yml b/.github/workflows/ci-complete.yml index 44d4f5a9c1d..6b8492fb7c0 100644 --- a/.github/workflows/ci-complete.yml +++ b/.github/workflows/ci-complete.yml @@ -61,7 +61,7 @@ jobs: env: GITHUB_CONTEXT: ${{ toJson(github) }} - name: Checkout code - uses: actions/checkout@v4 + uses: actions/checkout@v5 with: persist-credentials: false @@ -72,7 +72,7 @@ jobs: develocity-access-key: ${{ secrets.DEVELOCITY_ACCESS_KEY }} - name: Download build scan archive id: download-build-scan - uses: actions/download-artifact@v4 + uses: actions/download-artifact@v5 continue-on-error: true # Don't want this step to fail the overall workflow with: github-token: ${{ github.token }} diff --git a/.github/workflows/deflake.yml b/.github/workflows/deflake.yml index f58408a0473..3a2fbb56345 100644 --- a/.github/workflows/deflake.yml +++ b/.github/workflows/deflake.yml @@ -42,7 +42,7 @@ jobs: name: Deflake JUnit tests steps: - name: Checkout code - uses: actions/checkout@v4 + uses: actions/checkout@v5 with: persist-credentials: false diff --git a/.github/workflows/docker_build_and_test.yml b/.github/workflows/docker_build_and_test.yml index 67acdf9fb74..6a1b2f7de25 100644 --- a/.github/workflows/docker_build_and_test.yml +++ b/.github/workflows/docker_build_and_test.yml @@ -32,9 +32,9 @@ jobs: build: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v5 - name: Set up Python 3.10 - uses: actions/setup-python@v5 + uses: actions/setup-python@v6 with: python-version: "3.10" - name: Setup Docker Compose diff --git a/.github/workflows/docker_official_image_build_and_test.yml b/.github/workflows/docker_official_image_build_and_test.yml index 58866a19d6c..1580ea1f744 100644 --- a/.github/workflows/docker_official_image_build_and_test.yml +++ b/.github/workflows/docker_official_image_build_and_test.yml @@ -31,9 +31,9 @@ jobs: build: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v5 - name: Set up Python 3.10 - uses: actions/setup-python@v5 + uses: actions/setup-python@v6 with: python-version: "3.10" - name: Setup Docker Compose diff --git a/.github/workflows/docker_rc_release.yml b/.github/workflows/docker_rc_release.yml index 1f824b39b97..da851f4a430 100644 --- a/.github/workflows/docker_rc_release.yml +++ b/.github/workflows/docker_rc_release.yml @@ -37,9 +37,9 @@ jobs: runs-on: ubuntu-latest steps: - name: Checkout - uses: actions/checkout@v4 + uses: actions/checkout@v5 - name: Set up Python 3.10 - uses: actions/setup-python@v5 + uses: actions/setup-python@v6 with: python-version: "3.10" - name: Install dependencies diff --git a/.github/workflows/generate-reports.yml b/.github/workflows/generate-reports.yml index a3ae34fcecb..dee7094c27c 100644 --- a/.github/workflows/generate-reports.yml +++ b/.github/workflows/generate-reports.yml @@ -32,7 +32,7 @@ jobs: env: GITHUB_CONTEXT: ${{ toJson(github) }} - name: Checkout code - uses: actions/checkout@v4 + uses: actions/checkout@v5 - name: Setup Python uses: ./.github/actions/setup-python - name: Run Report diff --git a/.github/workflows/pr-labeled.yml b/.github/workflows/pr-labeled.yml index 87b39a659ec..b5695825861 100644 --- a/.github/workflows/pr-labeled.yml +++ b/.github/workflows/pr-labeled.yml @@ -35,7 +35,7 @@ jobs: env: GITHUB_CONTEXT: ${{ toJson(github) }} - name: Checkout code - uses: actions/checkout@v4 + uses: actions/checkout@v5 with: persist-credentials: false diff --git a/.github/workflows/pr-linter.yml b/.github/workflows/pr-linter.yml index f19efbfabe2..d38a9659a01 100644 --- a/.github/workflows/pr-linter.yml +++ b/.github/workflows/pr-linter.yml @@ -31,12 +31,12 @@ jobs: env: GITHUB_CONTEXT: ${{ toJson(github) }} - name: Checkout code - uses: actions/checkout@v4 + uses: actions/checkout@v5 with: persist-credentials: false - name: Load PR Number id: load-pr-number - uses: actions/download-artifact@v4 + uses: actions/download-artifact@v5 with: github-token: ${{ github.token }} run-id: ${{ github.event.workflow_run.id }} diff --git a/.github/workflows/pr-update.yml b/.github/workflows/pr-update.yml index e1cd7214d6c..7b45a15d191 100644 --- a/.github/workflows/pr-update.yml +++ b/.github/workflows/pr-update.yml @@ -37,8 +37,8 @@ jobs: runs-on: ubuntu-latest steps: - name: Checkout code - uses: actions/checkout@v4 - - uses: actions/labeler@v5 + uses: actions/checkout@v5 + - uses: actions/labeler@v6 with: configuration-path: .github/configs/labeler.yml - name: check small label diff --git a/.github/workflows/prepare_docker_official_image_source.yml b/.github/workflows/prepare_docker_official_image_source.yml index 32f21a0afd0..82204b9b935 100644 --- a/.github/workflows/prepare_docker_official_image_source.yml +++ b/.github/workflows/prepare_docker_official_image_source.yml @@ -31,9 +31,9 @@ jobs: build: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v5 - name: Set up Python 3.10 - uses: actions/setup-python@v5 + uses: actions/setup-python@v6 with: python-version: "3.10" - name: Install dependencies diff --git a/.github/workflows/workflow-requested.yml b/.github/workflows/workflow-requested.yml index 6463c72ab0a..aae00f9ece7 100644 --- a/.github/workflows/workflow-requested.yml +++ b/.github/workflows/workflow-requested.yml @@ -38,7 +38,7 @@ jobs: env: GITHUB_CONTEXT: ${{ toJson(github) }} - name: Checkout code - uses: actions/checkout@v4 + uses: actions/checkout@v5 with: persist-credentials: false diff --git a/tests/setup.py b/tests/setup.py index 17476849d52..1a88ec5ee0e 100644 --- a/tests/setup.py +++ b/tests/setup.py @@ -40,6 +40,7 @@ class PyTest(Command): sys.exit(errno) # Note: when changing the version of ducktape, also revise tests/docker/Dockerfile +# Ensure the 'requests' version here matches the one specified in .github/scripts/requirements.txt setup(name="kafkatest", version=version, description="Apache Kafka System Tests", From 749c2d91d52036f789cacc7ee0e04d0bcada6813 Mon Sep 17 00:00:00 2001 From: Hong-Yi Chen Date: Mon, 15 Sep 2025 11:25:54 +0800 Subject: [PATCH 030/100] KAFKA-19609 Move TransactionLogTest to transaction-coordinator module (#20460) This PR migrates the `TransactionLogTest` from Scala to Java for better consistency with the rest of the test suite and to simplify future maintenance. Reviewers: Chia-Ping Tsai --- ...import-control-transaction-coordinator.xml | 1 + .../transaction/TransactionLog.scala | 4 +- .../transaction/TransactionLogTest.scala | 246 --------------- .../transaction/TransactionLogTest.java | 283 ++++++++++++++++++ 4 files changed, 286 insertions(+), 248 deletions(-) delete mode 100644 core/src/test/scala/unit/kafka/coordinator/transaction/TransactionLogTest.scala create mode 100644 transaction-coordinator/src/test/java/org/apache/kafka/coordinator/transaction/TransactionLogTest.java diff --git a/checkstyle/import-control-transaction-coordinator.xml b/checkstyle/import-control-transaction-coordinator.xml index a48100a9acc..810c127c95c 100644 --- a/checkstyle/import-control-transaction-coordinator.xml +++ b/checkstyle/import-control-transaction-coordinator.xml @@ -40,6 +40,7 @@ + diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala index a3e9eacb66f..f024e88aa8e 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala @@ -52,7 +52,7 @@ object TransactionLog { * * @return key bytes */ - private[transaction] def keyToBytes(transactionalId: String): Array[Byte] = { + def keyToBytes(transactionalId: String): Array[Byte] = { MessageUtil.toCoordinatorTypePrefixedBytes(new TransactionLogKey().setTransactionalId(transactionalId)) } @@ -61,7 +61,7 @@ object TransactionLog { * * @return value payload bytes */ - private[transaction] def valueToBytes(txnMetadata: TxnTransitMetadata, + def valueToBytes(txnMetadata: TxnTransitMetadata, transactionVersionLevel: TransactionVersion): Array[Byte] = { if (txnMetadata.txnState == TransactionState.EMPTY && !txnMetadata.topicPartitions.isEmpty) throw new IllegalStateException(s"Transaction is not expected to have any partitions since its state is ${txnMetadata.txnState}: $txnMetadata") diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionLogTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionLogTest.scala deleted file mode 100644 index 46116be6d7c..00000000000 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionLogTest.scala +++ /dev/null @@ -1,246 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package kafka.coordinator.transaction - -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.compress.Compression -import org.apache.kafka.common.protocol.{ByteBufferAccessor, MessageUtil} -import org.apache.kafka.common.protocol.types.Field.TaggedFieldsSection -import org.apache.kafka.common.protocol.types.{CompactArrayOf, Field, Schema, Struct, Type} -import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, SimpleRecord} -import org.apache.kafka.coordinator.transaction.{TransactionMetadata, TransactionState, TxnTransitMetadata} -import org.apache.kafka.coordinator.transaction.generated.{TransactionLogKey, TransactionLogValue} -import org.apache.kafka.server.common.TransactionVersion.{TV_0, TV_2} -import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue, fail} -import org.junit.jupiter.api.Test - -import java.nio.ByteBuffer -import java.util -import scala.collection.Seq -import scala.jdk.CollectionConverters._ - -class TransactionLogTest { - - val producerEpoch: Short = 0 - val transactionTimeoutMs: Int = 1000 - - val topicPartitions = util.Set.of(new TopicPartition("topic1", 0), - new TopicPartition("topic1", 1), - new TopicPartition("topic2", 0), - new TopicPartition("topic2", 1), - new TopicPartition("topic2", 2)) - - @Test - def shouldThrowExceptionWriteInvalidTxn(): Unit = { - val transactionalId = "transactionalId" - val producerId = 23423L - - val txnMetadata = new TransactionMetadata(transactionalId, producerId, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_ID, producerEpoch, - RecordBatch.NO_PRODUCER_EPOCH, transactionTimeoutMs, TransactionState.EMPTY, util.Set.of, 0, 0, TV_0) - txnMetadata.addPartitions(topicPartitions) - - assertThrows(classOf[IllegalStateException], () => TransactionLog.valueToBytes(txnMetadata.prepareNoTransit(), TV_2)) - } - - @Test - def shouldReadWriteMessages(): Unit = { - val pidMappings = Map[String, Long]("zero" -> 0L, - "one" -> 1L, - "two" -> 2L, - "three" -> 3L, - "four" -> 4L, - "five" -> 5L) - - val transactionStates = Map[Long, TransactionState](0L -> TransactionState.EMPTY, - 1L -> TransactionState.ONGOING, - 2L -> TransactionState.PREPARE_COMMIT, - 3L -> TransactionState.COMPLETE_COMMIT, - 4L -> TransactionState.PREPARE_ABORT, - 5L -> TransactionState.COMPLETE_ABORT) - - // generate transaction log messages - val txnRecords = pidMappings.map { case (transactionalId, producerId) => - val txnMetadata = new TransactionMetadata(transactionalId, producerId, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_ID, producerEpoch, - RecordBatch.NO_PRODUCER_EPOCH, transactionTimeoutMs, transactionStates(producerId), util.Set.of, 0, 0, TV_0) - - if (!txnMetadata.state.equals(TransactionState.EMPTY)) - txnMetadata.addPartitions(topicPartitions) - - val keyBytes = TransactionLog.keyToBytes(transactionalId) - val valueBytes = TransactionLog.valueToBytes(txnMetadata.prepareNoTransit(), TV_2) - - new SimpleRecord(keyBytes, valueBytes) - }.toSeq - - val records = MemoryRecords.withRecords(0, Compression.NONE, txnRecords: _*) - - var count = 0 - for (record <- records.records.asScala) { - TransactionLog.readTxnRecordKey(record.key) match { - case Left(version) => fail(s"Unexpected record version: $version") - case Right(transactionalId) => - val txnMetadata = TransactionLog.readTxnRecordValue(transactionalId, record.value).get - - assertEquals(pidMappings(transactionalId), txnMetadata.producerId) - assertEquals(producerEpoch, txnMetadata.producerEpoch) - assertEquals(transactionTimeoutMs, txnMetadata.txnTimeoutMs) - assertEquals(transactionStates(txnMetadata.producerId), txnMetadata.state) - - if (txnMetadata.state.equals(TransactionState.EMPTY)) - assertEquals(util.Set.of, txnMetadata.topicPartitions) - else - assertEquals(topicPartitions, txnMetadata.topicPartitions) - - count = count + 1 - } - } - - assertEquals(pidMappings.size, count) - } - - @Test - def testSerializeTransactionLogValueToHighestNonFlexibleVersion(): Unit = { - val txnTransitMetadata = new TxnTransitMetadata(1, 1, 1, 1, 1, 1000, TransactionState.COMPLETE_COMMIT, new util.HashSet(), 500, 500, TV_0) - val txnLogValueBuffer = ByteBuffer.wrap(TransactionLog.valueToBytes(txnTransitMetadata, TV_0)) - assertEquals(0, txnLogValueBuffer.getShort) - } - - @Test - def testSerializeTransactionLogValueToFlexibleVersion(): Unit = { - val txnTransitMetadata = new TxnTransitMetadata(1, 1, 1, 1, 1, 1000, TransactionState.COMPLETE_COMMIT, new util.HashSet(), 500, 500, TV_2) - val txnLogValueBuffer = ByteBuffer.wrap(TransactionLog.valueToBytes(txnTransitMetadata, TV_2)) - assertEquals(TransactionLogValue.HIGHEST_SUPPORTED_VERSION, txnLogValueBuffer.getShort) - } - - @Test - def testDeserializeHighestSupportedTransactionLogValue(): Unit = { - val txnPartitions = new TransactionLogValue.PartitionsSchema() - .setTopic("topic") - .setPartitionIds(util.List.of(0)) - - val txnLogValue = new TransactionLogValue() - .setProducerId(100) - .setProducerEpoch(50.toShort) - .setTransactionStatus(TransactionState.COMPLETE_COMMIT.id) - .setTransactionStartTimestampMs(750L) - .setTransactionLastUpdateTimestampMs(1000L) - .setTransactionTimeoutMs(500) - .setTransactionPartitions(util.List.of(txnPartitions)) - - val serialized = MessageUtil.toVersionPrefixedByteBuffer(1, txnLogValue) - val deserialized = TransactionLog.readTxnRecordValue("transactionId", serialized).get - - assertEquals(100, deserialized.producerId) - assertEquals(50, deserialized.producerEpoch) - assertEquals(TransactionState.COMPLETE_COMMIT, deserialized.state) - assertEquals(750L, deserialized.txnStartTimestamp) - assertEquals(1000L, deserialized.txnLastUpdateTimestamp) - assertEquals(500, deserialized.txnTimeoutMs) - - val actualTxnPartitions = deserialized.topicPartitions - assertEquals(1, actualTxnPartitions.size) - assertTrue(actualTxnPartitions.contains(new TopicPartition("topic", 0))) - } - - @Test - def testDeserializeFutureTransactionLogValue(): Unit = { - // Copy of TransactionLogValue.PartitionsSchema.SCHEMA_1 with a few - // additional tagged fields. - val futurePartitionsSchema = new Schema( - new Field("topic", Type.COMPACT_STRING, ""), - new Field("partition_ids", new CompactArrayOf(Type.INT32), ""), - TaggedFieldsSection.of( - Int.box(100), new Field("partition_foo", Type.STRING, ""), - Int.box(101), new Field("partition_foo", Type.INT32, "") - ) - ) - - // Create TransactionLogValue.PartitionsSchema with tagged fields - val txnPartitions = new Struct(futurePartitionsSchema) - txnPartitions.set("topic", "topic") - txnPartitions.set("partition_ids", Array(Integer.valueOf(1))) - val txnPartitionsTaggedFields = new util.TreeMap[Integer, Any]() - txnPartitionsTaggedFields.put(100, "foo") - txnPartitionsTaggedFields.put(101, 4000) - txnPartitions.set("_tagged_fields", txnPartitionsTaggedFields) - - // Copy of TransactionLogValue.SCHEMA_1 with a few - // additional tagged fields. - val futureTransactionLogValueSchema = new Schema( - new Field("producer_id", Type.INT64, ""), - new Field("producer_epoch", Type.INT16, ""), - new Field("transaction_timeout_ms", Type.INT32, ""), - new Field("transaction_status", Type.INT8, ""), - new Field("transaction_partitions", CompactArrayOf.nullable(futurePartitionsSchema), ""), - new Field("transaction_last_update_timestamp_ms", Type.INT64, ""), - new Field("transaction_start_timestamp_ms", Type.INT64, ""), - TaggedFieldsSection.of( - Int.box(100), new Field("txn_foo", Type.STRING, ""), - Int.box(101), new Field("txn_bar", Type.INT32, "") - ) - ) - - // Create TransactionLogValue with tagged fields - val transactionLogValue = new Struct(futureTransactionLogValueSchema) - transactionLogValue.set("producer_id", 1000L) - transactionLogValue.set("producer_epoch", 100.toShort) - transactionLogValue.set("transaction_timeout_ms", 1000) - transactionLogValue.set("transaction_status", TransactionState.COMPLETE_COMMIT.id) - transactionLogValue.set("transaction_partitions", Array(txnPartitions)) - transactionLogValue.set("transaction_last_update_timestamp_ms", 2000L) - transactionLogValue.set("transaction_start_timestamp_ms", 3000L) - val txnLogValueTaggedFields = new util.TreeMap[Integer, Any]() - txnLogValueTaggedFields.put(100, "foo") - txnLogValueTaggedFields.put(101, 4000) - transactionLogValue.set("_tagged_fields", txnLogValueTaggedFields) - - // Prepare the buffer. - val buffer = ByteBuffer.allocate(transactionLogValue.sizeOf() + 2) - buffer.put(0.toByte) - buffer.put(1.toByte) // Add 1 as version. - transactionLogValue.writeTo(buffer) - buffer.flip() - - // Read the buffer with the real schema and verify that tagged - // fields were read but ignored. - buffer.getShort() // Skip version. - val value = new TransactionLogValue(new ByteBufferAccessor(buffer), 1.toShort) - assertEquals(Seq(100, 101), value.unknownTaggedFields().asScala.map(_.tag)) - assertEquals(Seq(100, 101), value.transactionPartitions().get(0).unknownTaggedFields().asScala.map(_.tag)) - - // Read the buffer with readTxnRecordValue. - buffer.rewind() - val txnMetadata = TransactionLog.readTxnRecordValue("transaction-id", buffer).get - assertEquals(1000L, txnMetadata.producerId) - assertEquals(100, txnMetadata.producerEpoch) - assertEquals(1000L, txnMetadata.txnTimeoutMs) - assertEquals(TransactionState.COMPLETE_COMMIT, txnMetadata.state) - assertEquals(util.Set.of(new TopicPartition("topic", 1)), txnMetadata.topicPartitions) - assertEquals(2000L, txnMetadata.txnLastUpdateTimestamp) - assertEquals(3000L, txnMetadata.txnStartTimestamp) - } - - @Test - def testReadTxnRecordKeyCanReadUnknownMessage(): Unit = { - val record = new TransactionLogKey() - val unknownRecord = MessageUtil.toVersionPrefixedBytes(Short.MaxValue, record) - TransactionLog.readTxnRecordKey(ByteBuffer.wrap(unknownRecord)) match { - case Left(version) => assertEquals(Short.MaxValue, version) - case Right(_) => fail("Expected to read unknown message") - } - } -} diff --git a/transaction-coordinator/src/test/java/org/apache/kafka/coordinator/transaction/TransactionLogTest.java b/transaction-coordinator/src/test/java/org/apache/kafka/coordinator/transaction/TransactionLogTest.java new file mode 100644 index 00000000000..3295901a8ba --- /dev/null +++ b/transaction-coordinator/src/test/java/org/apache/kafka/coordinator/transaction/TransactionLogTest.java @@ -0,0 +1,283 @@ +/* + * 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.coordinator.transaction; + +import kafka.coordinator.transaction.TransactionLog; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.compress.Compression; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.protocol.MessageUtil; +import org.apache.kafka.common.protocol.types.CompactArrayOf; +import org.apache.kafka.common.protocol.types.Field; +import org.apache.kafka.common.protocol.types.RawTaggedField; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.protocol.types.Type; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.record.SimpleRecord; +import org.apache.kafka.coordinator.transaction.generated.TransactionLogKey; +import org.apache.kafka.coordinator.transaction.generated.TransactionLogValue; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; + +import java.nio.ByteBuffer; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.TreeMap; +import java.util.stream.Stream; + +import static java.nio.ByteBuffer.allocate; +import static java.nio.ByteBuffer.wrap; +import static org.apache.kafka.common.protocol.types.Field.TaggedFieldsSection; +import static org.apache.kafka.server.common.TransactionVersion.LATEST_PRODUCTION; +import static org.apache.kafka.server.common.TransactionVersion.TV_0; +import static org.apache.kafka.server.common.TransactionVersion.TV_2; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +class TransactionLogTest { + + private final Set topicPartitions = Set.of( + new TopicPartition("topic1", 0), + new TopicPartition("topic1", 1), + new TopicPartition("topic2", 0), + new TopicPartition("topic2", 1), + new TopicPartition("topic2", 2) + ); + + private sealed interface TxnKeyResult { + record UnknownVersion(short version) implements TxnKeyResult { } + record TransactionalId(String id) implements TxnKeyResult { } + } + + private static TxnKeyResult readTxnRecordKey(ByteBuffer buf) { + var e = TransactionLog.readTxnRecordKey(buf); + return e.isLeft() + ? new TxnKeyResult.UnknownVersion((Short) e.left().get()) + : new TxnKeyResult.TransactionalId(e.right().get()); + } + + private static TransactionMetadata TransactionMetadata(TransactionState state) { + return new TransactionMetadata( + "transactionalId", + 0L, + RecordBatch.NO_PRODUCER_ID, + RecordBatch.NO_PRODUCER_ID, + (short) 0, + RecordBatch.NO_PRODUCER_EPOCH, + 1000, + state, + Set.of(), + 0, + 0, + LATEST_PRODUCTION + ); + } + + private static Stream transactionStatesProvider() { + return Stream.of( + TransactionState.EMPTY, + TransactionState.ONGOING, + TransactionState.PREPARE_COMMIT, + TransactionState.COMPLETE_COMMIT, + TransactionState.PREPARE_ABORT, + TransactionState.COMPLETE_ABORT + ); + } + + @Test + void shouldThrowExceptionWriteInvalidTxn() { + var txnMetadata = TransactionMetadata(TransactionState.EMPTY); + txnMetadata.addPartitions(topicPartitions); + + var preparedMetadata = txnMetadata.prepareNoTransit(); + assertThrows(IllegalStateException.class, () -> TransactionLog.valueToBytes(preparedMetadata, TV_2)); + } + + @ParameterizedTest(name = "{0}") + @MethodSource("transactionStatesProvider") + void shouldReadWriteMessages(TransactionState state) { + var txnMetadata = TransactionMetadata(state); + if (state != TransactionState.EMPTY) { + txnMetadata.addPartitions(topicPartitions); + } + + var record = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord( + TransactionLog.keyToBytes(txnMetadata.transactionalId()), + TransactionLog.valueToBytes(txnMetadata.prepareNoTransit(), TV_2) + )).records().iterator().next(); + var txnIdResult = assertInstanceOf(TxnKeyResult.TransactionalId.class, readTxnRecordKey(record.key())); + var deserialized = TransactionLog.readTxnRecordValue(txnIdResult.id(), record.value()).get(); + + assertEquals(txnMetadata.producerId(), deserialized.producerId()); + assertEquals(txnMetadata.producerEpoch(), deserialized.producerEpoch()); + assertEquals(txnMetadata.txnTimeoutMs(), deserialized.txnTimeoutMs()); + assertEquals(txnMetadata.state(), deserialized.state()); + + if (txnMetadata.state() == TransactionState.EMPTY) { + assertEquals(Set.of(), deserialized.topicPartitions()); + } else { + assertEquals(topicPartitions, deserialized.topicPartitions()); + } + } + + @Test + void testSerializeTransactionLogValueToHighestNonFlexibleVersion() { + var txnTransitMetadata = new TxnTransitMetadata(1L, 1L, 1L, (short) 1, (short) 1, 1000, TransactionState.COMPLETE_COMMIT, new HashSet<>(), 500L, 500L, TV_0); + var txnLogValueBuffer = wrap(TransactionLog.valueToBytes(txnTransitMetadata, TV_0)); + assertEquals(TV_0.transactionLogValueVersion(), txnLogValueBuffer.getShort()); + } + + @Test + void testSerializeTransactionLogValueToFlexibleVersion() { + var txnTransitMetadata = new TxnTransitMetadata(1L, 1L, 1L, (short) 1, (short) 1, 1000, TransactionState.COMPLETE_COMMIT, new HashSet<>(), 500L, 500L, TV_2); + var txnLogValueBuffer = wrap(TransactionLog.valueToBytes(txnTransitMetadata, TV_2)); + assertEquals(TransactionLogValue.HIGHEST_SUPPORTED_VERSION, txnLogValueBuffer.getShort()); + } + + @Test + void testDeserializeHighestSupportedTransactionLogValue() { + var txnPartitions = new TransactionLogValue.PartitionsSchema() + .setTopic("topic") + .setPartitionIds(List.of(0)); + + var txnLogValue = new TransactionLogValue() + .setProducerId(100) + .setProducerEpoch((short) 50) + .setTransactionStatus(TransactionState.COMPLETE_COMMIT.id()) + .setTransactionStartTimestampMs(750L) + .setTransactionLastUpdateTimestampMs(1000L) + .setTransactionTimeoutMs(500) + .setTransactionPartitions(List.of(txnPartitions)); + + var serialized = MessageUtil.toVersionPrefixedByteBuffer((short) 1, txnLogValue); + var deserialized = TransactionLog.readTxnRecordValue("transactionId", serialized).get(); + + assertEquals(100, deserialized.producerId()); + assertEquals(50, deserialized.producerEpoch()); + assertEquals(TransactionState.COMPLETE_COMMIT, deserialized.state()); + assertEquals(750L, deserialized.txnStartTimestamp()); + assertEquals(1000L, deserialized.txnLastUpdateTimestamp()); + assertEquals(500, deserialized.txnTimeoutMs()); + + var actualTxnPartitions = deserialized.topicPartitions(); + assertEquals(1, actualTxnPartitions.size()); + assertTrue(actualTxnPartitions.contains(new TopicPartition("topic", 0))); + } + + @Test + void testDeserializeFutureTransactionLogValue() { + // Copy of TransactionLogValue.PartitionsSchema.SCHEMA_1 with a few + // additional tagged fields. + var futurePartitionsSchema = new Schema( + new Field("topic", Type.COMPACT_STRING, ""), + new Field("partition_ids", new CompactArrayOf(Type.INT32), ""), + TaggedFieldsSection.of( + 100, new Field("partition_foo", Type.STRING, ""), + 101, new Field("partition_foo", Type.INT32, "") + ) + ); + + // Create TransactionLogValue.PartitionsSchema with tagged fields + var txnPartitions = new Struct(futurePartitionsSchema); + txnPartitions.set("topic", "topic"); + txnPartitions.set("partition_ids", new Integer[]{1}); + var txnPartitionsTaggedFields = new TreeMap(); + txnPartitionsTaggedFields.put(100, "foo"); + txnPartitionsTaggedFields.put(101, 4000); + txnPartitions.set("_tagged_fields", txnPartitionsTaggedFields); + + // Copy of TransactionLogValue.SCHEMA_1 with a few + // additional tagged fields. + var futureTransactionLogValueSchema = new Schema( + new Field("producer_id", Type.INT64, ""), + new Field("producer_epoch", Type.INT16, ""), + new Field("transaction_timeout_ms", Type.INT32, ""), + new Field("transaction_status", Type.INT8, ""), + new Field("transaction_partitions", CompactArrayOf.nullable(futurePartitionsSchema), ""), + new Field("transaction_last_update_timestamp_ms", Type.INT64, ""), + new Field("transaction_start_timestamp_ms", Type.INT64, ""), + TaggedFieldsSection.of( + 100, new Field("txn_foo", Type.STRING, ""), + 101, new Field("txn_bar", Type.INT32, "") + ) + ); + + // Create TransactionLogValue with tagged fields + var transactionLogValue = new Struct(futureTransactionLogValueSchema); + transactionLogValue.set("producer_id", 1000L); + transactionLogValue.set("producer_epoch", (short) 100); + transactionLogValue.set("transaction_timeout_ms", 1000); + transactionLogValue.set("transaction_status", TransactionState.COMPLETE_COMMIT.id()); + transactionLogValue.set("transaction_partitions", new Struct[]{txnPartitions}); + transactionLogValue.set("transaction_last_update_timestamp_ms", 2000L); + transactionLogValue.set("transaction_start_timestamp_ms", 3000L); + var txnLogValueTaggedFields = new TreeMap(); + txnLogValueTaggedFields.put(100, "foo"); + txnLogValueTaggedFields.put(101, 4000); + transactionLogValue.set("_tagged_fields", txnLogValueTaggedFields); + + // Prepare the buffer. + var buffer = allocate(Short.BYTES + transactionLogValue.sizeOf()); + buffer.putShort((short) 1); // Add 1 as version. + transactionLogValue.writeTo(buffer); + buffer.flip(); + + // Read the buffer with the real schema and verify that tagged + // fields were read but ignored. + buffer.getShort(); // Skip version. + var value = new TransactionLogValue(new ByteBufferAccessor(buffer), (short) 1); + assertEquals(List.of(100, 101), value.unknownTaggedFields().stream().map(RawTaggedField::tag).toList()); + assertEquals(List.of(100, 101), value.transactionPartitions().get(0).unknownTaggedFields().stream().map(RawTaggedField::tag).toList()); + + // Read the buffer with readTxnRecordValue. + buffer.rewind(); + var txnMetadata = TransactionLog.readTxnRecordValue("transaction-id", buffer); + + assertFalse(txnMetadata.isEmpty(), "Expected transaction metadata but got none"); + + var metadata = txnMetadata.get(); + assertEquals(1000L, metadata.producerId()); + assertEquals(100, metadata.producerEpoch()); + assertEquals(1000, metadata.txnTimeoutMs()); + assertEquals(TransactionState.COMPLETE_COMMIT, metadata.state()); + assertEquals(Set.of(new TopicPartition("topic", 1)), metadata.topicPartitions()); + assertEquals(2000L, metadata.txnLastUpdateTimestamp()); + assertEquals(3000L, metadata.txnStartTimestamp()); + } + + @Test + void testReadTxnRecordKeyCanReadUnknownMessage() { + var unknownRecord = MessageUtil.toVersionPrefixedBytes(Short.MAX_VALUE, new TransactionLogKey()); + var result = readTxnRecordKey(wrap(unknownRecord)); + + var uv = assertInstanceOf(TxnKeyResult.UnknownVersion.class, result); + assertEquals(Short.MAX_VALUE, uv.version()); + } + + @Test + void shouldReturnEmptyWhenForTombstoneRecord() { + assertTrue(TransactionLog.readTxnRecordValue("transaction-id", null).isEmpty()); + } +} \ No newline at end of file From 8628d74c494d07ceca1b55f36274491c2dd01fcd Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Mon, 15 Sep 2025 17:19:53 +0200 Subject: [PATCH 031/100] KAFKA-19661 [6/N]: Use heaps also on the process-level (#20523) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit In the current solution, we only use a heap to select the right process, but resort to linear search for selecting a member within a process. This means use cases where a lot of threads run within the same process can yield slow assignment. The number of threads in a process shouldn’t scale arbitrarily (our assumed case for benchmarking of 50 threads in a single process seems quite extreme already), however, we can optimize for this case to reduce the runtime further. Other assignment algorithms assign directly on the member-level, but we cannot do this in Kafka Streams, since we cannot assign tasks to processes that already own the task. Defining a heap directly on members would mean that we may have to skip through 10s of member before finding one that does not belong to a process that does not yet own the member. Instead, we can define a separate heap for each process, which keeps the members of the process by load. We can only keep the heap as long as we are only changing the load of the top-most member (which we usually do). This means we keep track of a lot of heaps, but since heaps are backed by arrays in Java, this should not result in extreme memory inefficiencies. In our worst-performing benchmark, this improves the runtime by ~2x on top of the optimization above. Also piggybacked are some minor optimizations / clean-ups: - initialize HashMaps and ArrayLists with the right capacity - fix some comments - improve logging output Note that this is a pure performance change, so there are no changes to the unit tests. Reviewers: Bill Bejeck --- .../assignor/AssignmentMemberSpec.java | 6 +- .../group/streams/assignor/ProcessState.java | 66 ++++++++++++++++- .../streams/assignor/StickyTaskAssignor.java | 72 ++++++++----------- .../group/streams/assignor/TaskId.java | 5 ++ 4 files changed, 100 insertions(+), 49 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/AssignmentMemberSpec.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/AssignmentMemberSpec.java index 71ee8c4ff30..fa41b5a8a3b 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/AssignmentMemberSpec.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/AssignmentMemberSpec.java @@ -27,9 +27,9 @@ import java.util.Set; * * @param instanceId The instance ID if provided. * @param rackId The rack ID if provided. - * @param activeTasks Reconciled active tasks - * @param standbyTasks Reconciled standby tasks - * @param warmupTasks Reconciled warm-up tasks + * @param activeTasks Current target active tasks + * @param standbyTasks Current target standby tasks + * @param warmupTasks Current target warm-up tasks * @param processId The process ID. * @param clientTags The client tags for a rack-aware assignment. * @param taskOffsets The last received cumulative task offsets of assigned tasks or dormant tasks. diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/ProcessState.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/ProcessState.java index 84c1b8a8207..330a0cc0da3 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/ProcessState.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/ProcessState.java @@ -16,9 +16,11 @@ */ package org.apache.kafka.coordinator.group.streams.assignor; +import java.util.AbstractMap; import java.util.HashMap; import java.util.HashSet; import java.util.Map; +import java.util.PriorityQueue; import java.util.Set; import java.util.stream.Collectors; @@ -36,6 +38,7 @@ public class ProcessState { private final Map> assignedActiveTasks; private final Map> assignedStandbyTasks; private final Set assignedTasks; + private PriorityQueue> membersByLoad; ProcessState(final String processId) { this.processId = processId; @@ -45,9 +48,9 @@ public class ProcessState { this.assignedActiveTasks = new HashMap<>(); this.assignedStandbyTasks = new HashMap<>(); this.memberToTaskCounts = new HashMap<>(); + this.membersByLoad = null; } - public String processId() { return processId; } @@ -84,7 +87,26 @@ public class ProcessState { return assignedStandbyTasks; } - public void addTask(final String memberId, final TaskId taskId, final boolean isActive) { + /** + * Assigns a task to a member of this process. + * + * @param memberId The member to assign to. + * @param taskId The task to assign. + * @param isActive Whether the task is an active task (true) or a standby task (false). + * @return the number of tasks that `memberId` has assigned after adding the new task. + */ + public int addTask(final String memberId, final TaskId taskId, final boolean isActive) { + int newTaskCount = addTaskInternal(memberId, taskId, isActive); + // We cannot efficiently add a task to a specific member and keep the memberByLoad ordered correctly. + // So we just drop the heap here. + // + // The order in which addTask and addTaskToLeastLoadedMember is called ensures that the heaps are built at most + // twice (once for active, once for standby) + membersByLoad = null; + return newTaskCount; + } + + private int addTaskInternal(final String memberId, final TaskId taskId, final boolean isActive) { taskCount += 1; assignedTasks.add(taskId); if (isActive) { @@ -94,8 +116,46 @@ public class ProcessState { assignedStandbyTasks.putIfAbsent(memberId, new HashSet<>()); assignedStandbyTasks.get(memberId).add(taskId); } - memberToTaskCounts.put(memberId, memberToTaskCounts.get(memberId) + 1); + int newTaskCount = memberToTaskCounts.get(memberId) + 1; + memberToTaskCounts.put(memberId, newTaskCount); computeLoad(); + return newTaskCount; + } + + /** + * Assigns a task to the least loaded member of this process + * + * @param taskId The task to assign. + * @param isActive Whether the task is an active task (true) or a standby task (false). + * @return the number of tasks that `memberId` has assigned after adding the new task, or -1 if the + * task was not assigned to any member. + */ + public int addTaskToLeastLoadedMember(final TaskId taskId, final boolean isActive) { + if (memberToTaskCounts.isEmpty()) { + return -1; + } + if (memberToTaskCounts.size() == 1) { + return addTaskInternal(memberToTaskCounts.keySet().iterator().next(), taskId, isActive); + } + if (membersByLoad == null) { + membersByLoad = new PriorityQueue<>( + memberToTaskCounts.size(), + Map.Entry.comparingByValue() + ); + for (Map.Entry entry : memberToTaskCounts.entrySet()) { + // Copy here, since map entry objects are allowed to be reused by the underlying map implementation. + membersByLoad.add(new AbstractMap.SimpleEntry<>(entry.getKey(), entry.getValue())); + } + } + Map.Entry member = membersByLoad.poll(); + if (member != null) { + int newTaskCount = addTaskInternal(member.getKey(), taskId, isActive); + member.setValue(newTaskCount); + membersByLoad.add(member); // Reinsert the updated member back into the priority queue + return newTaskCount; + } else { + throw new TaskAssignorException("No members available to assign task " + taskId); + } } private void incrementCapacity() { diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignor.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignor.java index 1b3f08a7382..2278e627bcd 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignor.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignor.java @@ -27,7 +27,6 @@ import java.util.HashSet; import java.util.Iterator; import java.util.LinkedList; import java.util.Map; -import java.util.Optional; import java.util.PriorityQueue; import java.util.Set; import java.util.stream.Collectors; @@ -97,11 +96,11 @@ public class StickyTaskAssignor implements TaskAssignor { localState.totalMembersWithActiveTaskCapacity = groupSpec.members().size(); localState.totalMembersWithTaskCapacity = groupSpec.members().size(); localState.activeTasksPerMember = computeTasksPerMember(localState.totalActiveTasks, localState.totalMembersWithActiveTaskCapacity); - localState.tasksPerMember = computeTasksPerMember(localState.totalTasks, localState.totalMembersWithTaskCapacity); + localState.totalTasksPerMember = computeTasksPerMember(localState.totalTasks, localState.totalMembersWithTaskCapacity); - localState.processIdToState = new HashMap<>(); - localState.activeTaskToPrevMember = new HashMap<>(); - localState.standbyTaskToPrevMember = new HashMap<>(); + localState.processIdToState = new HashMap<>(localState.totalMembersWithActiveTaskCapacity); + localState.activeTaskToPrevMember = new HashMap<>(localState.totalActiveTasks); + localState.standbyTaskToPrevMember = new HashMap<>(localState.numStandbyReplicas > 0 ? (localState.totalTasks - localState.totalActiveTasks) / localState.numStandbyReplicas : 0); for (final Map.Entry memberEntry : groupSpec.members().entrySet()) { final String memberId = memberEntry.getKey(); final String processId = memberEntry.getValue().processId(); @@ -124,7 +123,7 @@ public class StickyTaskAssignor implements TaskAssignor { final Set partitionNoSet = entry.getValue(); for (final int partitionNo : partitionNoSet) { final TaskId taskId = new TaskId(entry.getKey(), partitionNo); - localState.standbyTaskToPrevMember.putIfAbsent(taskId, new ArrayList<>()); + localState.standbyTaskToPrevMember.putIfAbsent(taskId, new ArrayList<>(localState.numStandbyReplicas)); localState.standbyTaskToPrevMember.get(taskId).add(member); } } @@ -185,8 +184,9 @@ public class StickyTaskAssignor implements TaskAssignor { if (prevMember != null) { final ProcessState processState = localState.processIdToState.get(prevMember.processId); if (hasUnfulfilledActiveTaskQuota(processState, prevMember)) { - processState.addTask(prevMember.memberId, task, true); - maybeUpdateActiveTasksPerMember(processState.memberToTaskCounts().get(prevMember.memberId)); + int newActiveTasks = processState.addTask(prevMember.memberId, task, true); + maybeUpdateActiveTasksPerMember(newActiveTasks); + maybeUpdateTotalTasksPerMember(newActiveTasks); it.remove(); } } @@ -200,8 +200,9 @@ public class StickyTaskAssignor implements TaskAssignor { if (prevMember != null) { final ProcessState processState = localState.processIdToState.get(prevMember.processId); if (hasUnfulfilledActiveTaskQuota(processState, prevMember)) { - processState.addTask(prevMember.memberId, task, true); - maybeUpdateActiveTasksPerMember(processState.memberToTaskCounts().get(prevMember.memberId)); + int newActiveTasks = processState.addTask(prevMember.memberId, task, true); + maybeUpdateActiveTasksPerMember(newActiveTasks); + maybeUpdateTotalTasksPerMember(newActiveTasks); it.remove(); } } @@ -213,19 +214,18 @@ public class StickyTaskAssignor implements TaskAssignor { // 3. assign any remaining unassigned tasks final PriorityQueue processByLoad = new PriorityQueue<>(Comparator.comparingDouble(ProcessState::load)); processByLoad.addAll(localState.processIdToState.values()); - for (final Iterator it = activeTasks.iterator(); it.hasNext();) { - final TaskId task = it.next(); + for (final TaskId task: activeTasks) { final ProcessState processWithLeastLoad = processByLoad.poll(); if (processWithLeastLoad == null) { throw new TaskAssignorException(String.format("No process available to assign active task %s.", task)); } - final String member = memberWithLeastLoad(processWithLeastLoad); - if (member == null) { + final int newTaskCount = processWithLeastLoad.addTaskToLeastLoadedMember(task, true); + if (newTaskCount != -1) { + maybeUpdateActiveTasksPerMember(newTaskCount); + maybeUpdateTotalTasksPerMember(newTaskCount); + } else { throw new TaskAssignorException(String.format("No member available to assign active task %s.", task)); } - processWithLeastLoad.addTask(member, task, true); - it.remove(); - maybeUpdateActiveTasksPerMember(processWithLeastLoad.memberToTaskCounts().get(member)); processByLoad.add(processWithLeastLoad); // Add it back to the queue after updating its state } } @@ -238,11 +238,11 @@ public class StickyTaskAssignor implements TaskAssignor { } } - private void maybeUpdateTasksPerMember(final int taskNo) { - if (taskNo == localState.tasksPerMember) { + private void maybeUpdateTotalTasksPerMember(final int taskNo) { + if (taskNo == localState.totalTasksPerMember) { localState.totalMembersWithTaskCapacity--; localState.totalTasks -= taskNo; - localState.tasksPerMember = computeTasksPerMember(localState.totalTasks, localState.totalMembersWithTaskCapacity); + localState.totalTasksPerMember = computeTasksPerMember(localState.totalTasks, localState.totalMembersWithTaskCapacity); } } @@ -253,10 +253,10 @@ public class StickyTaskAssignor implements TaskAssignor { } boolean found = false; if (!processWithLeastLoad.hasTask(taskId)) { - final String memberId = memberWithLeastLoad(processWithLeastLoad); - if (memberId != null) { - processWithLeastLoad.addTask(memberId, taskId, false); + final int newTaskCount = processWithLeastLoad.addTaskToLeastLoadedMember(taskId, false); + if (newTaskCount != -1) { found = true; + maybeUpdateTotalTasksPerMember(newTaskCount); } } else if (!queue.isEmpty()) { found = assignStandbyToMemberWithLeastLoad(queue, taskId); @@ -303,26 +303,12 @@ public class StickyTaskAssignor implements TaskAssignor { return null; } - private String memberWithLeastLoad(final ProcessState processWithLeastLoad) { - final Map members = processWithLeastLoad.memberToTaskCounts(); - if (members.isEmpty()) { - return null; - } - if (members.size() == 1) { - return members.keySet().iterator().next(); - } - final Optional memberWithLeastLoad = processWithLeastLoad.memberToTaskCounts().entrySet().stream() - .min(Map.Entry.comparingByValue()) - .map(Map.Entry::getKey); - return memberWithLeastLoad.orElse(null); - } - private boolean hasUnfulfilledActiveTaskQuota(final ProcessState process, final Member member) { return process.memberToTaskCounts().get(member.memberId) < localState.activeTasksPerMember; } private boolean hasUnfulfilledTaskQuota(final ProcessState process, final Member member) { - return process.memberToTaskCounts().get(member.memberId) < localState.tasksPerMember; + return process.memberToTaskCounts().get(member.memberId) < localState.totalTasksPerMember; } private void assignStandby(final LinkedList standbyTasks) { @@ -339,8 +325,8 @@ public class StickyTaskAssignor implements TaskAssignor { if (prevActiveMember != null) { final ProcessState prevActiveMemberProcessState = localState.processIdToState.get(prevActiveMember.processId); if (!prevActiveMemberProcessState.hasTask(task) && hasUnfulfilledTaskQuota(prevActiveMemberProcessState, prevActiveMember)) { - prevActiveMemberProcessState.addTask(prevActiveMember.memberId, task, false); - maybeUpdateTasksPerMember(prevActiveMemberProcessState.memberToTaskCounts().get(prevActiveMember.memberId)); + int newTaskCount = prevActiveMemberProcessState.addTask(prevActiveMember.memberId, task, false); + maybeUpdateTotalTasksPerMember(newTaskCount); continue; } } @@ -352,8 +338,8 @@ public class StickyTaskAssignor implements TaskAssignor { if (prevStandbyMember != null) { final ProcessState prevStandbyMemberProcessState = localState.processIdToState.get(prevStandbyMember.processId); if (hasUnfulfilledTaskQuota(prevStandbyMemberProcessState, prevStandbyMember)) { - prevStandbyMemberProcessState.addTask(prevStandbyMember.memberId, task, false); - maybeUpdateTasksPerMember(prevStandbyMemberProcessState.memberToTaskCounts().get(prevStandbyMember.memberId)); + int newTaskCount = prevStandbyMemberProcessState.addTask(prevStandbyMember.memberId, task, false); + maybeUpdateTotalTasksPerMember(newTaskCount); continue; } } @@ -430,6 +416,6 @@ public class StickyTaskAssignor implements TaskAssignor { int totalMembersWithActiveTaskCapacity; int totalMembersWithTaskCapacity; int activeTasksPerMember; - int tasksPerMember; + int totalTasksPerMember; } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/TaskId.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/TaskId.java index 3a9e594a82b..979c042df00 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/TaskId.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/TaskId.java @@ -38,4 +38,9 @@ public record TaskId(String subtopologyId, int partition) implements Comparable< .compare(this, other); } + @Override + public String toString() { + return subtopologyId + '_' + partition; + } + } From 3fcc0c287795c6a591216617e0d5a02bd6fdbfc3 Mon Sep 17 00:00:00 2001 From: Alex Date: Tue, 16 Sep 2025 01:26:47 +0800 Subject: [PATCH 032/100] MINOR: Fix an off-by-one issue in ValuesTest (#20520) This test case ensures that the parser can convert ISO8601 correctly. However, when the local time falls on a different day than the UTC time, there will be an off-by-one issue. I changed the test to convert the local time and then compare it with the expected local time. This should fix the off-by-one issue. [Reference link](https://github.com/apache/kafka/pull/18611#discussion_r2318146619) Reviewers: Andrew Schofield --------- Signed-off-by: Alex --- .../test/java/org/apache/kafka/connect/data/ValuesTest.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java index 6be0b4a3772..d100be29b4d 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java @@ -30,7 +30,6 @@ import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.text.SimpleDateFormat; import java.time.Instant; -import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; import java.time.ZoneId; @@ -903,7 +902,7 @@ public class ValuesTest { // ISO8601 strings - accept a string matching pattern "yyyy-MM-dd" LocalDateTime localTimeTruncated = localTime.truncatedTo(ChronoUnit.DAYS); - java.util.Date d3 = Values.convertToDate(Date.SCHEMA, LocalDate.ofEpochDay(days).format(DateTimeFormatter.ISO_LOCAL_DATE)); + java.util.Date d3 = Values.convertToDate(Date.SCHEMA, localTime.format(DateTimeFormatter.ISO_LOCAL_DATE)); LocalDateTime date3 = LocalDateTime.ofInstant(Instant.ofEpochMilli(d3.getTime()), ZoneId.systemDefault()); assertEquals(localTimeTruncated, date3); From caeca090b860a200d75e0b9c7492f67259ec650a Mon Sep 17 00:00:00 2001 From: Lianet Magrans <98415067+lianetm@users.noreply.github.com> Date: Mon, 15 Sep 2025 13:28:27 -0400 Subject: [PATCH 033/100] MINOR: Improve producer docs and add tests around timeout behaviour on missing topic/partition (#20533) Clarify timeout errors received on send if the case is topic not in metadata vs partition not in metadata. Add integration tests showcases the difference Follow-up from 4.1 fix for misleading timeout error message (https://issues.apache.org/jira/browse/KAFKA-8862) Reviewers: TengYao Chi , Kuan-Po Tseng --- .../kafka/clients/producer/KafkaProducer.java | 25 ++++---- .../kafka/api/BaseProducerSendTest.scala | 12 ++-- .../kafka/api/PlaintextProducerSendTest.scala | 57 ++++++++++++++++++- 3 files changed, 78 insertions(+), 16 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 48be57f262b..6e656f590e4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -1027,15 +1027,15 @@ public class KafkaProducer implements Producer { * expensive callbacks it is recommended to use your own {@link java.util.concurrent.Executor} in the callback body * to parallelize processing. * - * @param record The record to send + * @param record The record to send * @param callback A user-supplied callback to execute when the record has been acknowledged by the server (null - * indicates no callback) - * - * @throws IllegalStateException if a transactional.id has been configured and no transaction has been started, or - * when send is invoked after producer has been closed. - * @throws InterruptException If the thread is interrupted while blocked + * indicates no callback) + * @throws IllegalStateException if a transactional.id has been configured and no transaction has been started, or + * when send is invoked after producer has been closed. + * @throws TimeoutException if the topic or the partition specified in the record cannot be found in metadata within {@code max.block.ms} + * @throws InterruptException If the thread is interrupted while blocked * @throws SerializationException If the key or value are not valid objects given the configured serializers - * @throws KafkaException If a Kafka related error occurs that does not belong to the public API exceptions. + * @throws KafkaException If a Kafka related error occurs that does not belong to the public API exceptions. */ @Override public Future send(ProducerRecord record, Callback callback) { @@ -1335,11 +1335,14 @@ public class KafkaProducer implements Producer { /** * Get the partition metadata for the given topic. This can be used for custom partitioning. + *

+ * This will attempt to refresh metadata until it finds the topic in it, or the configured {@link ProducerConfig#MAX_BLOCK_MS_CONFIG} expires. + * * @throws AuthenticationException if authentication fails. See the exception for more details - * @throws AuthorizationException if not authorized to the specified topic. See the exception for more details - * @throws InterruptException if the thread is interrupted while blocked - * @throws TimeoutException if metadata could not be refreshed within {@code max.block.ms} - * @throws KafkaException for all Kafka-related exceptions, including the case where this method is called after producer close + * @throws AuthorizationException if not authorized to the specified topic. See the exception for more details + * @throws InterruptException if the thread is interrupted while blocked + * @throws TimeoutException if the topic cannot be found in metadata within {@code max.block.ms} + * @throws KafkaException for all Kafka-related exceptions, including the case where this method is called after producer close */ @Override public List partitionsFor(String topic) { diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index 98155438549..add18b260cd 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -48,16 +48,20 @@ import scala.jdk.javaapi.OptionConverters abstract class BaseProducerSendTest extends KafkaServerTestHarness { def generateConfigs: scala.collection.Seq[KafkaConfig] = { - val overridingProps = new Properties() val numServers = 2 - overridingProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, 2.toShort) - overridingProps.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, 4.toString) TestUtils.createBrokerConfigs( numServers, interBrokerSecurityProtocol = Some(securityProtocol), trustStoreFile = trustStoreFile, saslProperties = serverSaslProperties - ).map(KafkaConfig.fromProps(_, overridingProps)) + ).map(KafkaConfig.fromProps(_, brokerOverrides)) + } + + protected def brokerOverrides: Properties = { + val overridingProps = new Properties() + overridingProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, 2.toShort) + overridingProps.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, 4.toString) + overridingProps } private var consumer: Consumer[Array[Byte], Array[Byte]] = _ diff --git a/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala index 6e3bbf4aed7..dc8b9423304 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala @@ -28,7 +28,7 @@ import org.apache.kafka.common.record.{DefaultRecord, DefaultRecordBatch, Record import org.apache.kafka.common.serialization.ByteArraySerializer import org.apache.kafka.server.config.ServerLogConfigs import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.Timeout +import org.junit.jupiter.api.{BeforeEach, TestInfo, Timeout} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{Arguments, MethodSource} @@ -37,6 +37,22 @@ import java.nio.charset.StandardCharsets class PlaintextProducerSendTest extends BaseProducerSendTest { + // topic auto creation is enabled by default, only some tests disable it + var disableAutoTopicCreation = false + + override def brokerOverrides: Properties = { + val props = super.brokerOverrides + if (disableAutoTopicCreation) { + props.put("auto.create.topics.enable", "false") + } + props + } + @BeforeEach + override def setUp(testInfo: TestInfo): Unit = { + disableAutoTopicCreation = testInfo.getDisplayName.contains("autoCreateTopicsEnabled=false") + super.setUp(testInfo) + } + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames) @MethodSource(Array("getTestGroupProtocolParametersAll")) def testWrongSerializer(groupProtocol: String): Unit = { @@ -121,6 +137,39 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { } } + /** + * Test error message received when send fails waiting on metadata for a topic that does not exist. + * No need to run this for both rebalance protocols. + */ + @ParameterizedTest(name = "groupProtocol={0}.autoCreateTopicsEnabled={1}") + @MethodSource(Array("protocolAndAutoCreateTopicProviders")) + def testSendTimeoutErrorMessageWhenTopicDoesNotExist(groupProtocol: String, autoCreateTopicsEnabled: String): Unit = { + val producer = createProducer(maxBlockMs = 500) + val record = new ProducerRecord(topic, null, "key".getBytes, "value".getBytes) + val exception = assertThrows(classOf[ExecutionException], () => producer.send(record).get) + assertInstanceOf(classOf[TimeoutException], exception.getCause) + assertEquals("Topic topic not present in metadata after 500 ms.", exception.getCause.getMessage) + } + + /** + * Test error message received when send fails waiting on metadata for a partition that does not exist (topic exists). + * No need to run this for both rebalance protocols. + */ + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames) + @MethodSource(Array("getTestGroupProtocolParametersClassicGroupProtocolOnly")) + def testSendTimeoutErrorWhenPartitionDoesNotExist(groupProtocol: String): Unit = { + val producer = createProducer(maxBlockMs = 500) + // Send a message to auto-create the topic + var record = new ProducerRecord(topic, null, "key".getBytes, "value".getBytes) + assertEquals(0L, producer.send(record).get.offset, "Should have offset 0") + + // Send another message to the topic that exists but to a partition that does not + record = new ProducerRecord(topic, 10, "key".getBytes, "value".getBytes) + val exception = assertThrows(classOf[ExecutionException], () => producer.send(record).get) + assertInstanceOf(classOf[TimeoutException], exception.getCause) + assertEquals("Partition 10 of topic topic with partition count 4 is not present in metadata after 500 ms.", exception.getCause.getMessage) + } + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames) @MethodSource(Array("timestampConfigProvider")) def testSendWithInvalidBeforeAndAfterTimestamp(groupProtocol: String, messageTimeStampConfig: String, recordTimestamp: Long): Unit = { @@ -285,4 +334,10 @@ object PlaintextProducerSendTest { } data.stream() } + + def protocolAndAutoCreateTopicProviders: java.util.stream.Stream[Arguments] = { + val data = new java.util.ArrayList[Arguments]() + data.add(Arguments.of("classic", "false")) + data.stream() + } } \ No newline at end of file From 3cbb2a0aaf71898a63d30fb3690ea34d655c529d Mon Sep 17 00:00:00 2001 From: Mickael Maison Date: Mon, 15 Sep 2025 21:56:11 +0200 Subject: [PATCH 034/100] MINOR: Small cleanups in clients (#20530) - Fix non-constant calls to logging - Fix assertEquals order - Fix javadoc Reviewers: TengYao Chi , Chia-Ping Tsai --- .../admin/DeleteShareGroupsOptions.java | 2 +- .../admin/DeleteShareGroupsResult.java | 2 +- .../admin/DeleteStreamsGroupsOptions.java | 2 +- .../admin/ListStreamsGroupOffsetsSpec.java | 1 + .../DeleteShareGroupOffsetsHandler.java | 14 ++++++------- .../admin/internals/ListOffsetsHandler.java | 2 +- .../internals/AbstractMembershipManager.java | 2 +- .../internals/AbstractStickyAssignor.java | 4 ++-- .../internals/CommitRequestManager.java | 2 +- .../StreamsGroupHeartbeatRequestManager.java | 2 +- .../internals/StreamsMembershipManager.java | 20 +++++++++---------- .../apache/kafka/common/metrics/Metrics.java | 6 +++--- .../common/utils/ByteBufferUnmapper.java | 4 ++-- .../kafka/common/utils/ConfigUtils.java | 3 ++- .../common/utils/LoggingSignalHandler.java | 2 +- .../kafka/common/utils/SecurityUtils.java | 2 +- .../org/apache/kafka/common/utils/Utils.java | 4 ++-- .../apache/kafka/common/KafkaFutureTest.java | 2 +- .../org/apache/kafka/common/UuidTest.java | 10 +++++----- .../common/config/AbstractConfigTest.java | 4 ++-- .../common/network/ChannelBuildersTest.java | 18 ++++++++--------- .../kafka/common/protocol/ErrorsTest.java | 2 +- .../kafka/common/record/FileRecordsTest.java | 4 ++-- .../common/replica/ReplicaSelectorTest.java | 6 +++--- 24 files changed, 61 insertions(+), 59 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DeleteShareGroupsOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/DeleteShareGroupsOptions.java index a41ec6d00b3..80fd55c7323 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DeleteShareGroupsOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DeleteShareGroupsOptions.java @@ -22,7 +22,7 @@ import org.apache.kafka.common.annotation.InterfaceStability; import java.util.Collection; /** - * Options for the {@link Admin#deleteShareGroups(Collection , DeleteShareGroupsOptions)} call. + * Options for the {@link Admin#deleteShareGroups(Collection, DeleteShareGroupsOptions)} call. *

* The API of this class is evolving, see {@link Admin} for details. */ diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DeleteShareGroupsResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/DeleteShareGroupsResult.java index c2791e681f7..ff53da08df8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DeleteShareGroupsResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DeleteShareGroupsResult.java @@ -25,7 +25,7 @@ import java.util.HashMap; import java.util.Map; /** - * The result of the {@link Admin#deleteShareGroups(Collection , DeleteShareGroupsOptions)} call. + * The result of the {@link Admin#deleteShareGroups(Collection, DeleteShareGroupsOptions)} call. *

* The API of this class is evolving, see {@link Admin} for details. */ diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DeleteStreamsGroupsOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/DeleteStreamsGroupsOptions.java index 6cd14797122..6ca2ec66a27 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DeleteStreamsGroupsOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DeleteStreamsGroupsOptions.java @@ -21,7 +21,7 @@ import org.apache.kafka.common.annotation.InterfaceStability; import java.util.Collection; /** - * Options for the {@link Admin#deleteStreamsGroups(Collection, DeleteStreamsGroupsOptions)} call. + * Options for the {@link Admin#deleteStreamsGroups(Collection, DeleteStreamsGroupsOptions)} call. *

* The API of this class is evolving, see {@link Admin} for details. */ diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ListStreamsGroupOffsetsSpec.java b/clients/src/main/java/org/apache/kafka/clients/admin/ListStreamsGroupOffsetsSpec.java index ce2d1552a53..4f5380f7491 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/ListStreamsGroupOffsetsSpec.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ListStreamsGroupOffsetsSpec.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.annotation.InterfaceStability; import java.util.Collection; +import java.util.Map; /** * Specification of streams group offsets to list using {@link Admin#listStreamsGroupOffsets(Map, ListStreamsGroupOffsetsOptions)}. diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteShareGroupOffsetsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteShareGroupOffsetsHandler.java index 9f5742937c8..9f0ab59e32c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteShareGroupOffsetsHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteShareGroupOffsetsHandler.java @@ -122,8 +122,8 @@ public class DeleteShareGroupOffsetsHandler extends AdminApiHandler.Batched handleUnsupportedVersionException( int brokerId, UnsupportedVersionException exception, Set keys ) { - log.warn("Broker " + brokerId + " does not support MAX_TIMESTAMP offset specs"); + log.warn("Broker {} does not support MAX_TIMESTAMP offset specs", brokerId); Map maxTimestampPartitions = new HashMap<>(); for (TopicPartition topicPartition : keys) { Long offsetTimestamp = offsetTimestampsByPartition.get(topicPartition); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java index 74ecf7f9bb8..ffe01c089e7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java @@ -988,7 +988,7 @@ public abstract class AbstractMembershipManager impl String reason = rejoinedWhileReconciliationInProgress ? "the member has re-joined the group" : "the member already transitioned out of the reconciling state into " + state; - log.info("Interrupting reconciliation that is not relevant anymore because " + reason); + log.info("Interrupting reconciliation that is not relevant anymore because {}", reason); markReconciliationCompleted(); } return shouldAbort; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignor.java index 4ac1513ede5..c38b5859f5f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignor.java @@ -965,8 +965,8 @@ public abstract class AbstractStickyAssignor extends AbstractPartitionAssignor { super(partitionsPerTopic, rackInfo, currentAssignment); this.subscriptions = subscriptions; - topic2AllPotentialConsumers = new HashMap<>(partitionsPerTopic.keySet().size()); - consumer2AllPotentialTopics = new HashMap<>(subscriptions.keySet().size()); + topic2AllPotentialConsumers = new HashMap<>(partitionsPerTopic.size()); + consumer2AllPotentialTopics = new HashMap<>(subscriptions.size()); // initialize topic2AllPotentialConsumers and consumer2AllPotentialTopics partitionsPerTopic.keySet().forEach( diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java index b6a6a9f14cf..fe4d3806f2a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java @@ -541,7 +541,7 @@ public class CommitRequestManager implements RequestManager, MemberStateListener boolean inflightRemoved = pendingRequests.inflightOffsetFetches.remove(fetchRequest); if (!inflightRemoved) { log.warn("A duplicated, inflight, request was identified, but unable to find it in the " + - "outbound buffer:" + fetchRequest); + "outbound buffer: {}", fetchRequest); } if (error == null) { maybeUpdateLastSeenEpochIfNewer(res); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java index 7e83bc7a1e4..ceeeb6c1916 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java @@ -287,7 +287,7 @@ public class StreamsGroupHeartbeatRequestManager implements RequestManager { private final HeartbeatMetricsManager metricsManager; - private StreamsRebalanceData streamsRebalanceData; + private final StreamsRebalanceData streamsRebalanceData; /** * Timer for tracking the time since the last consumer poll. If the timer expires, the consumer will stop diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsMembershipManager.java index 08d03a68980..4c0562b674e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsMembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsMembershipManager.java @@ -1010,8 +1010,8 @@ public class StreamsMembershipManager implements RequestManager { return; } if (reconciliationInProgress) { - log.trace("Ignoring reconciliation attempt. Another reconciliation is already in progress. Assignment " + - targetAssignment + " will be handled in the next reconciliation loop."); + log.trace("Ignoring reconciliation attempt. Another reconciliation is already in progress. Assignment {}" + + " will be handled in the next reconciliation loop.", targetAssignment); return; } @@ -1131,12 +1131,12 @@ public class StreamsMembershipManager implements RequestManager { ); final SortedSet partitionsToAssign = topicPartitionsForActiveTasks(activeTasksToAssign); - final SortedSet partitionsToAssigneNotPreviouslyOwned = + final SortedSet partitionsToAssignNotPreviouslyOwned = partitionsToAssignNotPreviouslyOwned(partitionsToAssign, topicPartitionsForActiveTasks(ownedActiveTasks)); subscriptionState.assignFromSubscribedAwaitingCallback( partitionsToAssign, - partitionsToAssigneNotPreviouslyOwned + partitionsToAssignNotPreviouslyOwned ); notifyAssignmentChange(partitionsToAssign); @@ -1152,10 +1152,10 @@ public class StreamsMembershipManager implements RequestManager { if (callbackError == null) { subscriptionState.enablePartitionsAwaitingCallback(partitionsToAssign); } else { - if (!partitionsToAssigneNotPreviouslyOwned.isEmpty()) { + if (!partitionsToAssignNotPreviouslyOwned.isEmpty()) { log.warn("Leaving newly assigned partitions {} marked as non-fetchable and not " + "requiring initializing positions after onTasksAssigned callback failed.", - partitionsToAssigneNotPreviouslyOwned, callbackError); + partitionsToAssignNotPreviouslyOwned, callbackError); } } }); @@ -1205,9 +1205,9 @@ public class StreamsMembershipManager implements RequestManager { Stream.concat( streamsRebalanceData.subtopologies().get(task.subtopologyId()).sourceTopics().stream(), streamsRebalanceData.subtopologies().get(task.subtopologyId()).repartitionSourceTopics().keySet().stream() - ).forEach(topic -> { - topicPartitions.add(new TopicPartition(topic, task.partitionId())); - }) + ).forEach(topic -> + topicPartitions.add(new TopicPartition(topic, task.partitionId())) + ) ); return topicPartitions; } @@ -1223,7 +1223,7 @@ public class StreamsMembershipManager implements RequestManager { String reason = rejoinedWhileReconciliationInProgress ? "the member has re-joined the group" : "the member already transitioned out of the reconciling state into " + state; - log.info("Interrupting reconciliation that is not relevant anymore because " + reason); + log.info("Interrupting reconciliation that is not relevant anymore because {}", reason); markReconciliationCompleted(); } return shouldAbort; diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java index 23644784087..182a8c74849 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java @@ -553,7 +553,7 @@ public final class Metrics implements Closeable { try { reporter.metricRemoval(metric); } catch (Exception e) { - log.error("Error when removing metric from " + reporter.getClass().getName(), e); + log.error("Error when removing metric from {}", reporter.getClass().getName(), e); } } log.trace("Removed metric named {}", metricName); @@ -596,7 +596,7 @@ public final class Metrics implements Closeable { try { reporter.metricChange(metric); } catch (Exception e) { - log.error("Error when registering metric on " + reporter.getClass().getName(), e); + log.error("Error when registering metric on {}", reporter.getClass().getName(), e); } } log.trace("Registered metric named {}", metricName); @@ -688,7 +688,7 @@ public final class Metrics implements Closeable { log.info("Closing reporter {}", reporter.getClass().getName()); reporter.close(); } catch (Exception e) { - log.error("Error when closing " + reporter.getClass().getName(), e); + log.error("Error when closing {}", reporter.getClass().getName(), e); } } log.info("Metrics reporters closed"); diff --git a/clients/src/main/java/org/apache/kafka/common/utils/ByteBufferUnmapper.java b/clients/src/main/java/org/apache/kafka/common/utils/ByteBufferUnmapper.java index a760f817b81..ba51a8dd491 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/ByteBufferUnmapper.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/ByteBufferUnmapper.java @@ -40,7 +40,7 @@ public final class ByteBufferUnmapper { private static final RuntimeException UNMAP_NOT_SUPPORTED_EXCEPTION; static { - Object unmap = null; + MethodHandle unmap = null; RuntimeException exception = null; try { unmap = lookupUnmapMethodHandle(); @@ -48,7 +48,7 @@ public final class ByteBufferUnmapper { exception = e; } if (unmap != null) { - UNMAP = (MethodHandle) unmap; + UNMAP = unmap; UNMAP_NOT_SUPPORTED_EXCEPTION = null; } else { UNMAP = null; diff --git a/clients/src/main/java/org/apache/kafka/common/utils/ConfigUtils.java b/clients/src/main/java/org/apache/kafka/common/utils/ConfigUtils.java index 397ab623b52..2c3702af131 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/ConfigUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/ConfigUtils.java @@ -75,7 +75,8 @@ public class ConfigUtils { } else if (value instanceof String) { return Boolean.parseBoolean((String) value); } else { - log.error("Invalid value (" + value + ") on configuration '" + key + "'. The default value '" + defaultValue + "' will be used instead. Please specify a true/false value."); + log.error("Invalid value ({}) on configuration '{}'. The default value '{}' will be used instead. Please specify a true/false value.", + value, key, defaultValue); return defaultValue; } } diff --git a/clients/src/main/java/org/apache/kafka/common/utils/LoggingSignalHandler.java b/clients/src/main/java/org/apache/kafka/common/utils/LoggingSignalHandler.java index 86ae7bd78cd..e0a8f89e848 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/LoggingSignalHandler.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/LoggingSignalHandler.java @@ -69,7 +69,7 @@ public class LoggingSignalHandler { for (String signal : SIGNALS) { register(signal, jvmSignalHandlers); } - log.info("Registered signal handlers for " + String.join(", ", SIGNALS)); + log.info("Registered signal handlers for {}", String.join(", ", SIGNALS)); } private Object createSignalHandler(final Map jvmSignalHandlers) { diff --git a/clients/src/main/java/org/apache/kafka/common/utils/SecurityUtils.java b/clients/src/main/java/org/apache/kafka/common/utils/SecurityUtils.java index 47a280fee58..0c8876ebd6a 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/SecurityUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/SecurityUtils.java @@ -33,7 +33,7 @@ import java.util.Map; public class SecurityUtils { - private static final Logger LOGGER = LoggerFactory.getLogger(SecurityConfig.class); + private static final Logger LOGGER = LoggerFactory.getLogger(SecurityUtils.class); private static final Map NAME_TO_RESOURCE_TYPES; private static final Map NAME_TO_OPERATIONS; diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index f60bc03bff9..dc7b0e7625a 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -857,7 +857,7 @@ public final class Utils { public static void delete(final File rootFile) throws IOException { if (rootFile == null) return; - Files.walkFileTree(rootFile.toPath(), new SimpleFileVisitor() { + Files.walkFileTree(rootFile.toPath(), new SimpleFileVisitor<>() { @Override public FileVisitResult visitFileFailed(Path path, IOException exc) throws IOException { if (exc instanceof NoSuchFileException) { @@ -1403,7 +1403,7 @@ public final class Utils { * @return new Collector, M, M> */ public static > Collector, M, M> entriesToMap(final Supplier mapSupplier) { - return new Collector, M, M>() { + return new Collector<>() { @Override public Supplier supplier() { return mapSupplier; diff --git a/clients/src/test/java/org/apache/kafka/common/KafkaFutureTest.java b/clients/src/test/java/org/apache/kafka/common/KafkaFutureTest.java index ceb819dee8f..5df435149de 100644 --- a/clients/src/test/java/org/apache/kafka/common/KafkaFutureTest.java +++ b/clients/src/test/java/org/apache/kafka/common/KafkaFutureTest.java @@ -318,7 +318,7 @@ public class KafkaFutureTest { awaitAndAssertResult(future, 21, null); Throwable cause = awaitAndAssertFailure(dependantFuture, CompletionException.class, "java.lang.RuntimeException: We require more vespene gas"); assertInstanceOf(RuntimeException.class, cause.getCause()); - assertEquals(cause.getCause().getMessage(), "We require more vespene gas"); + assertEquals("We require more vespene gas", cause.getCause().getMessage()); } @Test diff --git a/clients/src/test/java/org/apache/kafka/common/UuidTest.java b/clients/src/test/java/org/apache/kafka/common/UuidTest.java index 65316469c69..9acc8145be8 100644 --- a/clients/src/test/java/org/apache/kafka/common/UuidTest.java +++ b/clients/src/test/java/org/apache/kafka/common/UuidTest.java @@ -35,8 +35,8 @@ public class UuidTest { public void testSignificantBits() { Uuid id = new Uuid(34L, 98L); - assertEquals(id.getMostSignificantBits(), 34L); - assertEquals(id.getLeastSignificantBits(), 98L); + assertEquals(34L, id.getMostSignificantBits()); + assertEquals(98L, id.getLeastSignificantBits()); } @Test @@ -74,15 +74,15 @@ public class UuidTest { String zeroIdString = Uuid.ZERO_UUID.toString(); - assertEquals(Uuid.fromString(zeroIdString), Uuid.ZERO_UUID); + assertEquals(Uuid.ZERO_UUID, Uuid.fromString(zeroIdString)); } @RepeatedTest(value = 100, name = RepeatedTest.LONG_DISPLAY_NAME) public void testRandomUuid() { Uuid randomID = Uuid.randomUuid(); - assertNotEquals(randomID, Uuid.ZERO_UUID); - assertNotEquals(randomID, Uuid.METADATA_TOPIC_ID); + assertNotEquals(Uuid.ZERO_UUID, randomID); + assertNotEquals(Uuid.METADATA_TOPIC_ID, randomID); assertFalse(randomID.toString().startsWith("-")); } diff --git a/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java b/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java index 8669156d7c3..b517f55534e 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java @@ -379,8 +379,8 @@ public class AbstractConfigTest { Properties props = new Properties(); props.put("config.providers", "file"); TestIndirectConfigResolution config = new TestIndirectConfigResolution(props); - assertEquals(config.originals().get("config.providers"), "file"); - assertEquals(config.originals(Collections.singletonMap("config.providers", "file2")).get("config.providers"), "file2"); + assertEquals("file", config.originals().get("config.providers")); + assertEquals("file2", config.originals(Collections.singletonMap("config.providers", "file2")).get("config.providers")); } @Test diff --git a/clients/src/test/java/org/apache/kafka/common/network/ChannelBuildersTest.java b/clients/src/test/java/org/apache/kafka/common/network/ChannelBuildersTest.java index 01936f457c6..9aa90811b95 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/ChannelBuildersTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/ChannelBuildersTest.java @@ -62,10 +62,10 @@ public class ChannelBuildersTest { assertNull(configs.get("listener.name.listener1.gssapi.sasl.kerberos.service.name")); assertFalse(securityConfig.unused().contains("listener.name.listener1.gssapi.sasl.kerberos.service.name")); - assertEquals(configs.get("gssapi.sasl.kerberos.service.name"), "testkafka"); + assertEquals("testkafka", configs.get("gssapi.sasl.kerberos.service.name")); assertFalse(securityConfig.unused().contains("gssapi.sasl.kerberos.service.name")); - assertEquals(configs.get("sasl.kerberos.service.name"), "testkafkaglobal"); + assertEquals("testkafkaglobal", configs.get("sasl.kerberos.service.name")); assertFalse(securityConfig.unused().contains("sasl.kerberos.service.name")); assertNull(configs.get("listener.name.listener1.sasl.kerberos.service.name")); @@ -74,35 +74,35 @@ public class ChannelBuildersTest { assertNull(configs.get("plain.sasl.server.callback.handler.class")); assertFalse(securityConfig.unused().contains("plain.sasl.server.callback.handler.class")); - assertEquals(configs.get("listener.name.listener1.gssapi.config1.key"), "custom.config1"); + assertEquals("custom.config1", configs.get("listener.name.listener1.gssapi.config1.key")); assertFalse(securityConfig.unused().contains("listener.name.listener1.gssapi.config1.key")); - assertEquals(configs.get("custom.config2.key"), "custom.config2"); + assertEquals("custom.config2", configs.get("custom.config2.key")); assertFalse(securityConfig.unused().contains("custom.config2.key")); // test configs without listener prefix securityConfig = new TestSecurityConfig(props); configs = ChannelBuilders.channelBuilderConfigs(securityConfig, null); - assertEquals(configs.get("listener.name.listener1.gssapi.sasl.kerberos.service.name"), "testkafka"); + assertEquals("testkafka", configs.get("listener.name.listener1.gssapi.sasl.kerberos.service.name")); assertFalse(securityConfig.unused().contains("listener.name.listener1.gssapi.sasl.kerberos.service.name")); assertNull(configs.get("gssapi.sasl.kerberos.service.name")); assertFalse(securityConfig.unused().contains("gssapi.sasl.kerberos.service.name")); - assertEquals(configs.get("listener.name.listener1.sasl.kerberos.service.name"), "testkafkaglobal"); + assertEquals("testkafkaglobal", configs.get("listener.name.listener1.sasl.kerberos.service.name")); assertFalse(securityConfig.unused().contains("listener.name.listener1.sasl.kerberos.service.name")); assertNull(configs.get("sasl.kerberos.service.name")); assertFalse(securityConfig.unused().contains("sasl.kerberos.service.name")); - assertEquals(configs.get("plain.sasl.server.callback.handler.class"), "callback"); + assertEquals("callback", configs.get("plain.sasl.server.callback.handler.class")); assertFalse(securityConfig.unused().contains("plain.sasl.server.callback.handler.class")); - assertEquals(configs.get("listener.name.listener1.gssapi.config1.key"), "custom.config1"); + assertEquals("custom.config1", configs.get("listener.name.listener1.gssapi.config1.key")); assertFalse(securityConfig.unused().contains("listener.name.listener1.gssapi.config1.key")); - assertEquals(configs.get("custom.config2.key"), "custom.config2"); + assertEquals("custom.config2", configs.get("custom.config2.key")); assertFalse(securityConfig.unused().contains("custom.config2.key")); } diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/ErrorsTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/ErrorsTest.java index 23c49412ab9..528fefed3f3 100644 --- a/clients/src/test/java/org/apache/kafka/common/protocol/ErrorsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/protocol/ErrorsTest.java @@ -53,7 +53,7 @@ public class ErrorsTest { public void testExceptionsAreNotGeneric() { for (Errors error : Errors.values()) { if (error != Errors.NONE) - assertNotEquals(error.exception().getClass(), ApiException.class, "Generic ApiException should not be used"); + assertNotEquals(ApiException.class, error.exception().getClass(), "Generic ApiException should not be used"); } } diff --git a/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java index 04590ccbe7b..a9d12285f12 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java @@ -197,7 +197,7 @@ public class FileRecordsTest { * Test a simple append and read. */ @Test - public void testRead() throws IOException { + public void testRead() { FileRecords read = fileRecords.slice(0, fileRecords.sizeInBytes()); assertEquals(fileRecords.sizeInBytes(), read.sizeInBytes()); TestUtils.checkEquals(fileRecords.batches(), read.batches()); @@ -279,7 +279,7 @@ public class FileRecordsTest { * Test that the message set iterator obeys start and end slicing */ @Test - public void testIteratorWithLimits() throws IOException { + public void testIteratorWithLimits() { RecordBatch batch = batches(fileRecords).get(1); int start = fileRecords.searchForOffsetFromPosition(1, 0).position; int size = batch.sizeInBytes(); diff --git a/clients/src/test/java/org/apache/kafka/common/replica/ReplicaSelectorTest.java b/clients/src/test/java/org/apache/kafka/common/replica/ReplicaSelectorTest.java index c66b0411fc5..d9d42d4d922 100644 --- a/clients/src/test/java/org/apache/kafka/common/replica/ReplicaSelectorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/replica/ReplicaSelectorTest.java @@ -46,8 +46,8 @@ public class ReplicaSelectorTest { ReplicaSelector selector = new RackAwareReplicaSelector(); Optional selected = selector.select(tp, metadata("rack-b"), partitionView); assertOptional(selected, replicaInfo -> { - assertEquals(replicaInfo.endpoint().rack(), "rack-b", "Expect replica to be in rack-b"); - assertEquals(replicaInfo.endpoint().id(), 3, "Expected replica 3 since it is more caught-up"); + assertEquals("rack-b", replicaInfo.endpoint().rack(), "Expect replica to be in rack-b"); + assertEquals(3, replicaInfo.endpoint().id(), "Expected replica 3 since it is more caught-up"); }); selected = selector.select(tp, metadata("not-a-rack"), partitionView); @@ -57,7 +57,7 @@ public class ReplicaSelectorTest { selected = selector.select(tp, metadata("rack-a"), partitionView); assertOptional(selected, replicaInfo -> { - assertEquals(replicaInfo.endpoint().rack(), "rack-a", "Expect replica to be in rack-a"); + assertEquals("rack-a", replicaInfo.endpoint().rack(), "Expect replica to be in rack-a"); assertEquals(replicaInfo, leader, "Expect the leader since it's in rack-a"); }); From dddb619177a83d4127861f5dedb84ddeec18da1e Mon Sep 17 00:00:00 2001 From: Jhen-Yung Hsu Date: Tue, 16 Sep 2025 16:19:42 +0800 Subject: [PATCH 035/100] MINOR: Move RaftManager interface to raft module (#20366) - Move the `RaftManager` interface to raft module, and remove the `register` and `leaderAndEpoch` methods since they are already part of the RaftClient APIs. - Rename RaftManager.scala to KafkaRaftManager.scala. Reviewers: Ken Huang , TengYao Chi , Chia-Ping Tsai --- checkstyle/import-control-metadata.xml | 1 + ...ftManager.scala => KafkaRaftManager.scala} | 45 +--------------- .../scala/kafka/server/ControllerApis.scala | 4 +- .../NodeToControllerChannelManager.scala | 8 +-- .../scala/kafka/server/SharedServer.scala | 2 +- .../kafka/tools/TestRaftRequestHandler.scala | 2 +- .../scala/kafka/tools/TestRaftServer.scala | 6 +-- .../AllocateProducerIdsRequestTest.scala | 4 +- .../kafka/server/ControllerApisTest.scala | 3 +- .../epoch/LeaderEpochIntegrationTest.scala | 2 +- .../kafka/controller/MockRaftClient.java | 7 +++ .../image/publisher/SnapshotEmitterTest.java | 7 +++ .../apache/kafka/raft/KafkaRaftClient.java | 1 + .../org/apache/kafka/raft/RaftClient.java | 11 ++++ .../org/apache/kafka/raft/RaftManager.java | 54 +++++++++++++++++++ 15 files changed, 98 insertions(+), 59 deletions(-) rename core/src/main/scala/kafka/raft/{RaftManager.scala => KafkaRaftManager.scala} (89%) create mode 100644 raft/src/main/java/org/apache/kafka/raft/RaftManager.java diff --git a/checkstyle/import-control-metadata.xml b/checkstyle/import-control-metadata.xml index d5b787ce24f..773635cec8e 100644 --- a/checkstyle/import-control-metadata.xml +++ b/checkstyle/import-control-metadata.xml @@ -107,6 +107,7 @@ + diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/KafkaRaftManager.scala similarity index 89% rename from core/src/main/scala/kafka/raft/RaftManager.scala rename to core/src/main/scala/kafka/raft/KafkaRaftManager.scala index 9e8ea38f8fd..86950e1ce25 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/KafkaRaftManager.scala @@ -20,16 +20,13 @@ import java.io.File import java.net.InetSocketAddress import java.nio.file.Files import java.nio.file.Paths -import java.util.OptionalInt +import java.util.{OptionalInt, Collection => JCollection, Map => JMap} import java.util.concurrent.CompletableFuture -import java.util.{Map => JMap} -import java.util.{Collection => JCollection} import kafka.server.KafkaConfig import kafka.utils.CoreUtils import kafka.utils.Logging import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, MetadataRecoveryStrategy, NetworkClient} import org.apache.kafka.common.KafkaException -import org.apache.kafka.common.Node import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.Uuid import org.apache.kafka.common.metrics.Metrics @@ -40,7 +37,7 @@ import org.apache.kafka.common.requests.RequestHeader import org.apache.kafka.common.security.JaasContext import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.{LogContext, Time, Utils} -import org.apache.kafka.raft.{Endpoints, ExternalKRaftMetrics, FileQuorumStateStore, KafkaNetworkChannel, KafkaRaftClient, KafkaRaftClientDriver, LeaderAndEpoch, MetadataLogConfig, QuorumConfig, RaftClient, ReplicatedLog, TimingWheelExpirationService} +import org.apache.kafka.raft.{Endpoints, ExternalKRaftMetrics, FileQuorumStateStore, KafkaNetworkChannel, KafkaRaftClient, KafkaRaftClientDriver, MetadataLogConfig, QuorumConfig, RaftManager, ReplicatedLog, TimingWheelExpirationService} import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.common.Feature import org.apache.kafka.server.common.serialization.RecordSerde @@ -50,7 +47,6 @@ import org.apache.kafka.server.util.timer.SystemTimer import org.apache.kafka.storage.internals.log.{LogManager, UnifiedLog} import scala.jdk.CollectionConverters._ -import scala.jdk.OptionConverters._ object KafkaRaftManager { private def createLogDirectory(logDir: File, logDirName: String): File = { @@ -85,29 +81,6 @@ object KafkaRaftManager { } } -trait RaftManager[T] { - def handleRequest( - context: RequestContext, - header: RequestHeader, - request: ApiMessage, - createdTimeMs: Long - ): CompletableFuture[ApiMessage] - - def register( - listener: RaftClient.Listener[T] - ): Unit - - def leaderAndEpoch: LeaderAndEpoch - - def client: RaftClient[T] - - def replicatedLog: ReplicatedLog - - def voterNode(id: Int, listener: ListenerName): Option[Node] - - def recordSerde: RecordSerde[T] -} - class KafkaRaftManager[T]( clusterId: String, config: KafkaConfig, @@ -178,12 +151,6 @@ class KafkaRaftManager[T]( CoreUtils.swallow(dataDirLock.foreach(_.destroy()), this) } - override def register( - listener: RaftClient.Listener[T] - ): Unit = { - client.register(listener) - } - override def handleRequest( context: RequestContext, header: RequestHeader, @@ -292,13 +259,5 @@ class KafkaRaftManager[T]( (controllerListenerName, networkClient) } - override def leaderAndEpoch: LeaderAndEpoch = { - client.leaderAndEpoch - } - - override def voterNode(id: Int, listener: ListenerName): Option[Node] = { - client.voterNode(id, listener).toScala - } - override def recordSerde: RecordSerde[T] = serde } diff --git a/core/src/main/scala/kafka/server/ControllerApis.scala b/core/src/main/scala/kafka/server/ControllerApis.scala index cc9a12c9aa3..ac9a2d9eff1 100644 --- a/core/src/main/scala/kafka/server/ControllerApis.scala +++ b/core/src/main/scala/kafka/server/ControllerApis.scala @@ -24,7 +24,6 @@ import java.util.Map.Entry import java.util.concurrent.CompletableFuture import java.util.function.Consumer import kafka.network.RequestChannel -import kafka.raft.RaftManager import kafka.server.QuotaFactory.QuotaManagers import kafka.server.logger.RuntimeLoggerManager import kafka.server.metadata.KRaftMetadataCache @@ -55,6 +54,7 @@ import org.apache.kafka.image.publisher.ControllerRegistrationsPublisher import org.apache.kafka.metadata.{BrokerHeartbeatReply, BrokerRegistrationReply} import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.auth.SecurityProtocol +import org.apache.kafka.raft.RaftManager import org.apache.kafka.server.{ApiVersionManager, DelegationTokenManager, ProcessRole} import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.common.{ApiMessageAndVersion, RequestLocal} @@ -1070,7 +1070,7 @@ class ControllerApis( EndpointType.CONTROLLER, clusterId, () => registrationsPublisher.describeClusterControllers(request.context.listenerName()), - () => raftManager.leaderAndEpoch.leaderId().orElse(-1) + () => raftManager.client.leaderAndEpoch.leaderId().orElse(-1) ) requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => new DescribeClusterResponse(response.setThrottleTimeMs(requestThrottleMs))) diff --git a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala index cd6b8e1d134..0caa03ec052 100644 --- a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala +++ b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala @@ -17,7 +17,6 @@ package kafka.server -import kafka.raft.RaftManager import kafka.utils.Logging import org.apache.kafka.clients._ import org.apache.kafka.common.metrics.Metrics @@ -28,6 +27,7 @@ import org.apache.kafka.common.security.JaasContext import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.{LogContext, Time} import org.apache.kafka.common.{Node, Reconfigurable} +import org.apache.kafka.raft.RaftManager import org.apache.kafka.server.common.{ApiMessageAndVersion, ControllerRequestCompletionHandler, NodeToControllerChannelManager} import org.apache.kafka.server.util.{InterBrokerSendThread, RequestAndCompletionHandler} @@ -37,7 +37,7 @@ import java.util.concurrent.LinkedBlockingDeque import java.util.concurrent.atomic.AtomicReference import scala.collection.Seq import scala.jdk.CollectionConverters._ -import scala.jdk.OptionConverters.{RichOption, RichOptionalInt} +import scala.jdk.OptionConverters.{RichOption, RichOptional, RichOptionalInt} case class ControllerInformation( node: Option[Node], @@ -79,10 +79,10 @@ class RaftControllerNodeProvider( val saslMechanism: String ) extends ControllerNodeProvider with Logging { - private def idToNode(id: Int): Option[Node] = raftManager.voterNode(id, listenerName) + private def idToNode(id: Int): Option[Node] = raftManager.client.voterNode(id, listenerName).toScala override def getControllerInfo(): ControllerInformation = - ControllerInformation(raftManager.leaderAndEpoch.leaderId.toScala.flatMap(idToNode), + ControllerInformation(raftManager.client.leaderAndEpoch.leaderId.toScala.flatMap(idToNode), listenerName, securityProtocol, saslMechanism) } diff --git a/core/src/main/scala/kafka/server/SharedServer.scala b/core/src/main/scala/kafka/server/SharedServer.scala index 03f1c9b929e..aba9035cb7e 100644 --- a/core/src/main/scala/kafka/server/SharedServer.scala +++ b/core/src/main/scala/kafka/server/SharedServer.scala @@ -342,7 +342,7 @@ class SharedServer( throw new RuntimeException("Unable to install metadata publishers.", t) } } - _raftManager.register(loader) + _raftManager.client.register(loader) debug("Completed SharedServer startup.") started = true } catch { diff --git a/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala b/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala index 2e9d8e2bb8a..081fbec3c95 100644 --- a/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala +++ b/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala @@ -18,7 +18,6 @@ package kafka.tools import kafka.network.RequestChannel -import kafka.raft.RaftManager import kafka.server.ApiRequestHandler import kafka.utils.Logging import org.apache.kafka.common.internals.FatalExitError @@ -26,6 +25,7 @@ import org.apache.kafka.common.message.{BeginQuorumEpochResponseData, EndQuorumE import org.apache.kafka.common.protocol.{ApiKeys, ApiMessage} import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, BeginQuorumEpochResponse, EndQuorumEpochResponse, FetchResponse, FetchSnapshotResponse, VoteResponse} import org.apache.kafka.common.utils.Time +import org.apache.kafka.raft.RaftManager import org.apache.kafka.server.ApiVersionManager import org.apache.kafka.server.common.RequestLocal diff --git a/core/src/main/scala/kafka/tools/TestRaftServer.scala b/core/src/main/scala/kafka/tools/TestRaftServer.scala index 2b3183be80b..48e101443a1 100644 --- a/core/src/main/scala/kafka/tools/TestRaftServer.scala +++ b/core/src/main/scala/kafka/tools/TestRaftServer.scala @@ -22,7 +22,7 @@ import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} import java.util.concurrent.{CompletableFuture, CountDownLatch, LinkedBlockingDeque, TimeUnit} import joptsimple.{OptionException, OptionSpec} import kafka.network.SocketServer -import kafka.raft.{DefaultExternalKRaftMetrics, KafkaRaftManager, RaftManager} +import kafka.raft.{DefaultExternalKRaftMetrics, KafkaRaftManager} import kafka.server.{KafkaConfig, KafkaRequestHandlerPool} import kafka.utils.{CoreUtils, Logging} import org.apache.kafka.common.message.ApiMessageType.ListenerType @@ -36,7 +36,7 @@ import org.apache.kafka.common.security.token.delegation.internals.DelegationTok import org.apache.kafka.common.utils.{Exit, Time, Utils} import org.apache.kafka.common.{TopicPartition, Uuid, protocol} import org.apache.kafka.raft.errors.NotLeaderException -import org.apache.kafka.raft.{Batch, BatchReader, Endpoints, LeaderAndEpoch, QuorumConfig, RaftClient} +import org.apache.kafka.raft.{Batch, BatchReader, Endpoints, LeaderAndEpoch, QuorumConfig, RaftClient, RaftManager} import org.apache.kafka.security.CredentialProvider import org.apache.kafka.server.SimpleApiVersionManager import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion} @@ -180,7 +180,7 @@ class TestRaftServer( private var claimedEpoch: Option[Int] = None - raftManager.register(this) + raftManager.client.register(this) override def handleLeaderChange(newLeaderAndEpoch: LeaderAndEpoch): Unit = { if (newLeaderAndEpoch.isLeader(config.nodeId)) { diff --git a/core/src/test/scala/unit/kafka/server/AllocateProducerIdsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AllocateProducerIdsRequestTest.scala index aa399985f83..16a82fdca8b 100644 --- a/core/src/test/scala/unit/kafka/server/AllocateProducerIdsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AllocateProducerIdsRequestTest.scala @@ -34,7 +34,7 @@ class AllocateProducerIdsRequestTest(cluster: ClusterInstance) { def testAllocateProducersIdSentToController(): Unit = { val sourceBroker = cluster.brokers.values().stream().findFirst().get().asInstanceOf[BrokerServer] - val controllerId = sourceBroker.raftManager.leaderAndEpoch.leaderId().getAsInt + val controllerId = sourceBroker.raftManager.client.leaderAndEpoch.leaderId().getAsInt val controllerServer = cluster.controllers.values().stream() .filter(_.config.nodeId == controllerId) .findFirst() @@ -50,7 +50,7 @@ class AllocateProducerIdsRequestTest(cluster: ClusterInstance) { def testAllocateProducersIdSentToNonController(): Unit = { val sourceBroker = cluster.brokers.values().stream().findFirst().get().asInstanceOf[BrokerServer] - val controllerId = sourceBroker.raftManager.leaderAndEpoch.leaderId().getAsInt + val controllerId = sourceBroker.raftManager.client.leaderAndEpoch.leaderId().getAsInt val controllerServer = cluster.controllers().values().stream() .filter(_.config.nodeId != controllerId) .findFirst() diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala index 901dbe74356..43c7d5aecf4 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala @@ -18,7 +18,6 @@ package kafka.server import kafka.network.RequestChannel -import kafka.raft.RaftManager import kafka.server.QuotaFactory.QuotaManagers import kafka.server.metadata.KRaftMetadataCache import org.apache.kafka.clients.admin.AlterConfigOp @@ -56,7 +55,7 @@ import org.apache.kafka.image.publisher.ControllerRegistrationsPublisher import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.metrics.RequestChannelMetrics import org.apache.kafka.network.Session -import org.apache.kafka.raft.QuorumConfig +import org.apache.kafka.raft.{QuorumConfig, RaftManager} import org.apache.kafka.server.SimpleApiVersionManager import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult, Authorizer} import org.apache.kafka.server.common.{ApiMessageAndVersion, FinalizedFeatures, KRaftVersion, MetadataVersion, ProducerIdsBlock, RequestLocal} diff --git a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala index d678a497b53..f1ba2c7ac5e 100644 --- a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala @@ -293,7 +293,7 @@ class LeaderEpochIntegrationTest extends QuorumTestHarness with Logging { } private def waitUntilQuorumLeaderElected(controllerServer: ControllerServer, timeout: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Int = { - val (leaderAndEpoch, _) = computeUntilTrue(controllerServer.raftManager.leaderAndEpoch, waitTime = timeout)(_.leaderId().isPresent) + val (leaderAndEpoch, _) = computeUntilTrue(controllerServer.raftManager.client.leaderAndEpoch, waitTime = timeout)(_.leaderId().isPresent) leaderAndEpoch.leaderId().orElseThrow(() => new AssertionError(s"Quorum Controller leader not elected after $timeout ms")) } diff --git a/metadata/src/test/java/org/apache/kafka/controller/MockRaftClient.java b/metadata/src/test/java/org/apache/kafka/controller/MockRaftClient.java index 21e3a645530..8bf6d9543e7 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/MockRaftClient.java +++ b/metadata/src/test/java/org/apache/kafka/controller/MockRaftClient.java @@ -17,6 +17,8 @@ package org.apache.kafka.controller; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.protocol.ObjectSerializationCache; import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.common.utils.LogContext; @@ -758,6 +760,11 @@ public final class MockRaftClient implements RaftClient, A return OptionalInt.of(nodeId); } + @Override + public Optional voterNode(int id, ListenerName listenerName) { + return Optional.empty(); + } + public List> listeners() { final CompletableFuture>> future = new CompletableFuture<>(); eventQueue.append(() -> diff --git a/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java index cf123078eae..57fe845c98f 100644 --- a/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java @@ -17,6 +17,8 @@ package org.apache.kafka.image.publisher; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.image.FakeSnapshotWriter; import org.apache.kafka.image.MetadataImageTest; @@ -73,6 +75,11 @@ public class SnapshotEmitterTest { return OptionalInt.empty(); } + @Override + public Optional voterNode(int id, ListenerName listenerName) { + return Optional.empty(); + } + @Override public long prepareAppend(int epoch, List records) { return 0; 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 343f4db6b02..6dc20026ca7 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -3793,6 +3793,7 @@ public final class KafkaRaftClient implements RaftClient { } } + @Override public Optional voterNode(int id, ListenerName listenerName) { return partitionState.lastVoterSet().voterNode(id, listenerName); } diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java index d2d59888178..95ed1905b67 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java @@ -16,7 +16,9 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.ApiException; +import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.raft.errors.BufferAllocationException; import org.apache.kafka.raft.errors.NotLeaderException; import org.apache.kafka.server.common.KRaftVersion; @@ -129,6 +131,15 @@ public interface RaftClient extends AutoCloseable { */ OptionalInt nodeId(); + /** + * Returns the node information for a given voter id and listener. + * + * @param id the id of the voter + * @param listenerName the name of the listener + * @return the node information if it exists, otherwise {@code Optional.empty()} + */ + Optional voterNode(int id, ListenerName listenerName); + /** * Prepare a list of records to be appended to the log. * diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftManager.java b/raft/src/main/java/org/apache/kafka/raft/RaftManager.java new file mode 100644 index 00000000000..1e69c365840 --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/raft/RaftManager.java @@ -0,0 +1,54 @@ +/* + * 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.protocol.ApiMessage; +import org.apache.kafka.common.requests.RequestContext; +import org.apache.kafka.common.requests.RequestHeader; +import org.apache.kafka.server.common.serialization.RecordSerde; + +import java.util.concurrent.CompletableFuture; + +public interface RaftManager { + + CompletableFuture handleRequest( + RequestContext context, + RequestHeader header, + ApiMessage request, + long createdTimeMs + ); + + /** + * Returns a Raft client. + *

+ * Always returns the same instance. Callers must NOT close it. + */ + RaftClient client(); + + /** + * Returns a replicated log. + *

+ * Always returns the same instance. Callers must NOT close it. + */ + ReplicatedLog replicatedLog(); + + /** + * Returns the record Serde. + */ + RecordSerde recordSerde(); +} From daa7aae0c1c7f3fa4de84023c9e70bd7c6d61eb5 Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Tue, 16 Sep 2025 16:34:14 +0800 Subject: [PATCH 036/100] KAFKA-19604 Document controller.quorum.auto.join.enable config in upgrade.html (#20409) Document controller.quorum.auto.join.enable config in upgrade.html Reviewers: Chia-Ping Tsai --- docs/upgrade.html | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/upgrade.html b/docs/upgrade.html index c093a6eb0d7..ec417e88c6b 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -142,6 +142,11 @@ +

  • + The controller.quorum.auto.join.enable has been added to QuorumConfig, enabling KRaft controllers to automatically join the cluster's voter set, + and defaults to false. + For further details, please refer to KIP-853. +
  • Upgrading to 4.1.0

    From 2c347380b7c2c83562bf6c30e99f04149f92f0b3 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Tue, 16 Sep 2025 16:32:47 +0200 Subject: [PATCH 037/100] KAFKA-19694: Trigger StreamsRebalanceListener in Consumer.close (#20511) In the consumer, we invoke the consumer rebalance onPartitionRevoked or onPartitionLost callbacks, when the consumer closes. The point is that the application may want to commit, or wipe the state if we are closing unsuccessfully. In the StreamsRebalanceListener, we did not implement this behavior, which means when closing the consumer we may lose some progress, and in the worst case also miss that we have to wipe our local state state since we got fenced. In this PR we implement StreamsRebalanceListenerInvoker, very similarly to ConsumerRebalanceListenerInvoker and invoke it in Consumer.close. Reviewers: Lianet Magrans , Matthias J. Sax , TengYao Chi , Uladzislau Blok <123193120+UladzislauBlok@users.noreply.github.com> --------- Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- .../internals/AsyncKafkaConsumer.java | 103 +++--- .../StreamsRebalanceListenerInvoker.java | 117 +++++++ .../internals/AsyncKafkaConsumerTest.java | 67 ++++ .../StreamsRebalanceListenerInvokerTest.java | 293 ++++++++++++++++++ .../DefaultStreamsRebalanceListener.java | 2 + .../DefaultStreamsRebalanceListenerTest.java | 100 +++--- 6 files changed, 579 insertions(+), 103 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvoker.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvokerTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 5c72c2babbb..938ae909027 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -187,25 +187,6 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { */ private class BackgroundEventProcessor implements EventProcessor { - private Optional streamsRebalanceListener = Optional.empty(); - private final Optional streamsRebalanceData; - - public BackgroundEventProcessor() { - this.streamsRebalanceData = Optional.empty(); - } - - public BackgroundEventProcessor(final Optional streamsRebalanceData) { - this.streamsRebalanceData = streamsRebalanceData; - } - - private void setStreamsRebalanceListener(final StreamsRebalanceListener streamsRebalanceListener) { - if (streamsRebalanceData.isEmpty()) { - throw new IllegalStateException("Background event processor was not created to be used with Streams " + - "rebalance protocol events"); - } - this.streamsRebalanceListener = Optional.of(streamsRebalanceListener); - } - @Override public void process(final BackgroundEvent event) { switch (event.type()) { @@ -278,44 +259,26 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { private StreamsOnTasksRevokedCallbackCompletedEvent invokeOnTasksRevokedCallback(final Set activeTasksToRevoke, final CompletableFuture future) { - final Optional exceptionFromCallback = streamsRebalanceListener().onTasksRevoked(activeTasksToRevoke); + final Optional exceptionFromCallback = Optional.ofNullable(streamsRebalanceListenerInvoker().invokeTasksRevoked(activeTasksToRevoke)); final Optional error = exceptionFromCallback.map(e -> ConsumerUtils.maybeWrapAsKafkaException(e, "Task revocation callback throws an error")); return new StreamsOnTasksRevokedCallbackCompletedEvent(future, error); } private StreamsOnTasksAssignedCallbackCompletedEvent invokeOnTasksAssignedCallback(final StreamsRebalanceData.Assignment assignment, final CompletableFuture future) { - final Optional error; - final Optional exceptionFromCallback = streamsRebalanceListener().onTasksAssigned(assignment); - if (exceptionFromCallback.isPresent()) { - error = Optional.of(ConsumerUtils.maybeWrapAsKafkaException(exceptionFromCallback.get(), "Task assignment callback throws an error")); - } else { - error = Optional.empty(); - streamsRebalanceData().setReconciledAssignment(assignment); - } + final Optional exceptionFromCallback = Optional.ofNullable(streamsRebalanceListenerInvoker().invokeTasksAssigned(assignment)); + final Optional error = exceptionFromCallback.map(e -> ConsumerUtils.maybeWrapAsKafkaException(e, "Task assignment callback throws an error")); return new StreamsOnTasksAssignedCallbackCompletedEvent(future, error); } private StreamsOnAllTasksLostCallbackCompletedEvent invokeOnAllTasksLostCallback(final CompletableFuture future) { - final Optional error; - final Optional exceptionFromCallback = streamsRebalanceListener().onAllTasksLost(); - if (exceptionFromCallback.isPresent()) { - error = Optional.of(ConsumerUtils.maybeWrapAsKafkaException(exceptionFromCallback.get(), "All tasks lost callback throws an error")); - } else { - error = Optional.empty(); - streamsRebalanceData().setReconciledAssignment(StreamsRebalanceData.Assignment.EMPTY); - } + final Optional exceptionFromCallback = Optional.ofNullable(streamsRebalanceListenerInvoker().invokeAllTasksLost()); + final Optional error = exceptionFromCallback.map(e -> ConsumerUtils.maybeWrapAsKafkaException(e, "All tasks lost callback throws an error")); return new StreamsOnAllTasksLostCallbackCompletedEvent(future, error); } - private StreamsRebalanceData streamsRebalanceData() { - return streamsRebalanceData.orElseThrow( - () -> new IllegalStateException("Background event processor was not created to be used with Streams " + - "rebalance protocol events")); - } - - private StreamsRebalanceListener streamsRebalanceListener() { - return streamsRebalanceListener.orElseThrow( + private StreamsRebalanceListenerInvoker streamsRebalanceListenerInvoker() { + return streamsRebalanceListenerInvoker.orElseThrow( () -> new IllegalStateException("Background event processor was not created to be used with Streams " + "rebalance protocol events")); } @@ -367,6 +330,7 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { private final WakeupTrigger wakeupTrigger = new WakeupTrigger(); private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; + private final Optional streamsRebalanceListenerInvoker; // Last triggered async commit future. Used to wait until all previous async commits are completed. // We only need to keep track of the last one, since they are guaranteed to complete in order. private CompletableFuture> lastPendingAsyncCommit = null; @@ -517,7 +481,9 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { time, new RebalanceCallbackMetricsManager(metrics) ); - this.backgroundEventProcessor = new BackgroundEventProcessor(streamsRebalanceData); + this.streamsRebalanceListenerInvoker = streamsRebalanceData.map(s -> + new StreamsRebalanceListenerInvoker(logContext, s)); + this.backgroundEventProcessor = new BackgroundEventProcessor(); this.backgroundEventReaper = backgroundEventReaperFactory.build(logContext); // The FetchCollector is only used on the application thread. @@ -577,6 +543,7 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { this.time = time; this.backgroundEventQueue = backgroundEventQueue; this.rebalanceListenerInvoker = rebalanceListenerInvoker; + this.streamsRebalanceListenerInvoker = Optional.empty(); this.backgroundEventProcessor = new BackgroundEventProcessor(); this.backgroundEventReaper = backgroundEventReaper; this.metrics = metrics; @@ -699,6 +666,7 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { networkClientDelegateSupplier, requestManagersSupplier, asyncConsumerMetrics); + this.streamsRebalanceListenerInvoker = Optional.empty(); this.backgroundEventProcessor = new BackgroundEventProcessor(); this.backgroundEventReaper = new CompletableEventReaper(logContext); } @@ -1477,7 +1445,7 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { () -> autoCommitOnClose(closeTimer), firstException); swallow(log, Level.ERROR, "Failed to stop finding coordinator", this::stopFindCoordinatorOnClose, firstException); - swallow(log, Level.ERROR, "Failed to release group assignment", + swallow(log, Level.ERROR, "Failed to run rebalance callbacks", this::runRebalanceCallbacksOnClose, firstException); swallow(log, Level.ERROR, "Failed to leave group while closing consumer", () -> leaveGroupOnClose(closeTimer, membershipOperation), firstException); @@ -1527,26 +1495,39 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { } private void runRebalanceCallbacksOnClose() { - if (groupMetadata.get().isEmpty() || rebalanceListenerInvoker == null) + if (groupMetadata.get().isEmpty()) return; int memberEpoch = groupMetadata.get().get().generationId(); - Set assignedPartitions = groupAssignmentSnapshot.get(); + Exception error = null; - if (assignedPartitions.isEmpty()) - // Nothing to revoke. - return; + if (streamsRebalanceListenerInvoker != null && streamsRebalanceListenerInvoker.isPresent()) { - SortedSet droppedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); - droppedPartitions.addAll(assignedPartitions); + if (memberEpoch > 0) { + error = streamsRebalanceListenerInvoker.get().invokeAllTasksRevoked(); + } else { + error = streamsRebalanceListenerInvoker.get().invokeAllTasksLost(); + } - final Exception error; + } else if (rebalanceListenerInvoker != null) { - if (memberEpoch > 0) - error = rebalanceListenerInvoker.invokePartitionsRevoked(droppedPartitions); - else - error = rebalanceListenerInvoker.invokePartitionsLost(droppedPartitions); + Set assignedPartitions = groupAssignmentSnapshot.get(); + + if (assignedPartitions.isEmpty()) + // Nothing to revoke. + return; + + SortedSet droppedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); + droppedPartitions.addAll(assignedPartitions); + + if (memberEpoch > 0) { + error = rebalanceListenerInvoker.invokePartitionsRevoked(droppedPartitions); + } else { + error = rebalanceListenerInvoker.invokePartitionsLost(droppedPartitions); + } + + } if (error != null) throw ConsumerUtils.maybeWrapAsKafkaException(error); @@ -1963,8 +1944,12 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { } public void subscribe(Collection topics, StreamsRebalanceListener streamsRebalanceListener) { + + streamsRebalanceListenerInvoker + .orElseThrow(() -> new IllegalStateException("Consumer was not created to be used with Streams rebalance protocol events")) + .setRebalanceListener(streamsRebalanceListener); + subscribeInternal(topics, Optional.empty()); - backgroundEventProcessor.setStreamsRebalanceListener(streamsRebalanceListener); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvoker.java new file mode 100644 index 00000000000..f4c5aa4addc --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvoker.java @@ -0,0 +1,117 @@ +/* + * 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.clients.consumer.internals; + +import org.apache.kafka.common.errors.InterruptException; +import org.apache.kafka.common.errors.WakeupException; +import org.apache.kafka.common.utils.LogContext; + +import org.slf4j.Logger; + +import java.util.Objects; +import java.util.Optional; +import java.util.Set; + +/** + * This class encapsulates the invocation of the callback methods defined in the {@link StreamsRebalanceListener} + * interface. When streams group task assignment changes, these methods are invoked. This class wraps those + * callback calls with some logging and error handling. + */ +public class StreamsRebalanceListenerInvoker { + + private final Logger log; + + private final StreamsRebalanceData streamsRebalanceData; + private Optional listener; + + StreamsRebalanceListenerInvoker(LogContext logContext, StreamsRebalanceData streamsRebalanceData) { + this.log = logContext.logger(getClass()); + this.listener = Optional.empty(); + this.streamsRebalanceData = streamsRebalanceData; + } + + public void setRebalanceListener(StreamsRebalanceListener streamsRebalanceListener) { + Objects.requireNonNull(streamsRebalanceListener, "StreamsRebalanceListener cannot be null"); + this.listener = Optional.of(streamsRebalanceListener); + } + + public Exception invokeAllTasksRevoked() { + if (listener.isEmpty()) { + throw new IllegalStateException("StreamsRebalanceListener is not defined"); + } + return invokeTasksRevoked(streamsRebalanceData.reconciledAssignment().activeTasks()); + } + + public Exception invokeTasksAssigned(final StreamsRebalanceData.Assignment assignment) { + if (listener.isEmpty()) { + throw new IllegalStateException("StreamsRebalanceListener is not defined"); + } + log.info("Invoking tasks assigned callback for new assignment: {}", assignment); + try { + listener.get().onTasksAssigned(assignment); + } catch (WakeupException | InterruptException e) { + throw e; + } catch (Exception e) { + log.error( + "Streams rebalance listener failed on invocation of onTasksAssigned for tasks {}", + assignment, + e + ); + return e; + } + return null; + } + + public Exception invokeTasksRevoked(final Set tasks) { + if (listener.isEmpty()) { + throw new IllegalStateException("StreamsRebalanceListener is not defined"); + } + log.info("Invoking task revoked callback for revoked active tasks {}", tasks); + try { + listener.get().onTasksRevoked(tasks); + } catch (WakeupException | InterruptException e) { + throw e; + } catch (Exception e) { + log.error( + "Streams rebalance listener failed on invocation of onTasksRevoked for tasks {}", + tasks, + e + ); + return e; + } + return null; + } + + public Exception invokeAllTasksLost() { + if (listener.isEmpty()) { + throw new IllegalStateException("StreamsRebalanceListener is not defined"); + } + log.info("Invoking tasks lost callback for all tasks"); + try { + listener.get().onAllTasksLost(); + } catch (WakeupException | InterruptException e) { + throw e; + } catch (Exception e) { + log.error( + "Streams rebalance listener failed on invocation of onTasksLost.", + e + ); + return e; + } + return null; + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index dcf604d6b81..9aef4cf7f52 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -2210,6 +2210,73 @@ public class AsyncKafkaConsumerTest { }).when(applicationEventHandler).add(ArgumentMatchers.isA(CommitEvent.class)); } + @Test + public void testCloseInvokesStreamsRebalanceListenerOnTasksRevokedWhenMemberEpochPositive() { + final String groupId = "streamsGroup"; + final StreamsRebalanceData streamsRebalanceData = new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of()); + + try (final MockedStatic requestManagers = mockStatic(RequestManagers.class)) { + consumer = newConsumerWithStreamRebalanceData(requiredConsumerConfigAndGroupId(groupId), streamsRebalanceData); + StreamsRebalanceListener mockStreamsListener = mock(StreamsRebalanceListener.class); + when(mockStreamsListener.onTasksRevoked(any())).thenReturn(Optional.empty()); + consumer.subscribe(singletonList("topic"), mockStreamsListener); + final MemberStateListener groupMetadataUpdateListener = captureGroupMetadataUpdateListener(requestManagers); + final int memberEpoch = 42; + final String memberId = "memberId"; + groupMetadataUpdateListener.onMemberEpochUpdated(Optional.of(memberEpoch), memberId); + + consumer.close(CloseOptions.timeout(Duration.ZERO)); + + verify(mockStreamsListener).onTasksRevoked(any()); + } + } + + @Test + public void testCloseInvokesStreamsRebalanceListenerOnAllTasksLostWhenMemberEpochZeroOrNegative() { + final String groupId = "streamsGroup"; + final StreamsRebalanceData streamsRebalanceData = new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of()); + + try (final MockedStatic requestManagers = mockStatic(RequestManagers.class)) { + consumer = newConsumerWithStreamRebalanceData(requiredConsumerConfigAndGroupId(groupId), streamsRebalanceData); + StreamsRebalanceListener mockStreamsListener = mock(StreamsRebalanceListener.class); + when(mockStreamsListener.onAllTasksLost()).thenReturn(Optional.empty()); + consumer.subscribe(singletonList("topic"), mockStreamsListener); + final MemberStateListener groupMetadataUpdateListener = captureGroupMetadataUpdateListener(requestManagers); + final int memberEpoch = 0; + final String memberId = "memberId"; + groupMetadataUpdateListener.onMemberEpochUpdated(Optional.of(memberEpoch), memberId); + + consumer.close(CloseOptions.timeout(Duration.ZERO)); + + verify(mockStreamsListener).onAllTasksLost(); + } + } + + @Test + public void testCloseWrapsStreamsRebalanceListenerException() { + final String groupId = "streamsGroup"; + final StreamsRebalanceData streamsRebalanceData = new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of()); + + try (final MockedStatic requestManagers = mockStatic(RequestManagers.class)) { + consumer = newConsumerWithStreamRebalanceData(requiredConsumerConfigAndGroupId(groupId), streamsRebalanceData); + StreamsRebalanceListener mockStreamsListener = mock(StreamsRebalanceListener.class); + RuntimeException testException = new RuntimeException("Test streams listener exception"); + doThrow(testException).when(mockStreamsListener).onTasksRevoked(any()); + consumer.subscribe(singletonList("topic"), mockStreamsListener); + final MemberStateListener groupMetadataUpdateListener = captureGroupMetadataUpdateListener(requestManagers); + final int memberEpoch = 1; + final String memberId = "memberId"; + groupMetadataUpdateListener.onMemberEpochUpdated(Optional.of(memberEpoch), memberId); + + KafkaException thrownException = assertThrows(KafkaException.class, + () -> consumer.close(CloseOptions.timeout(Duration.ZERO))); + + assertInstanceOf(RuntimeException.class, thrownException.getCause()); + assertTrue(thrownException.getCause().getMessage().contains("Test streams listener exception")); + verify(mockStreamsListener).onTasksRevoked(any()); + } + } + private void markReconcileAndAutoCommitCompleteForPollEvent() { doAnswer(invocation -> { PollEvent event = invocation.getArgument(0); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvokerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvokerTest.java new file mode 100644 index 00000000000..2f3e5ab0523 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvokerTest.java @@ -0,0 +1,293 @@ +/* + * 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.clients.consumer.internals; + +import org.apache.kafka.common.errors.InterruptException; +import org.apache.kafka.common.errors.WakeupException; +import org.apache.kafka.common.utils.LogContext; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.mockito.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; + +import java.util.Optional; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +@MockitoSettings(strictness = Strictness.STRICT_STUBS) +public class StreamsRebalanceListenerInvokerTest { + + @Mock + private StreamsRebalanceListener mockListener; + + @Mock + private StreamsRebalanceData streamsRebalanceData; + + private StreamsRebalanceListenerInvoker invoker; + private final LogContext logContext = new LogContext(); + + @BeforeEach + public void setup() { + invoker = new StreamsRebalanceListenerInvoker(logContext, streamsRebalanceData); + } + + @Test + public void testSetRebalanceListenerWithNull() { + NullPointerException exception = assertThrows(NullPointerException.class, + () -> invoker.setRebalanceListener(null)); + assertEquals("StreamsRebalanceListener cannot be null", exception.getMessage()); + } + + @Test + public void testSetRebalanceListenerOverwritesExisting() { + StreamsRebalanceListener firstListener = org.mockito.Mockito.mock(StreamsRebalanceListener.class); + StreamsRebalanceListener secondListener = org.mockito.Mockito.mock(StreamsRebalanceListener.class); + + StreamsRebalanceData.Assignment mockAssignment = createMockAssignment(); + when(streamsRebalanceData.reconciledAssignment()).thenReturn(mockAssignment); + when(secondListener.onTasksRevoked(any())).thenReturn(Optional.empty()); + + // Set first listener + invoker.setRebalanceListener(firstListener); + + // Overwrite with second listener + invoker.setRebalanceListener(secondListener); + + // Should use second listener + invoker.invokeAllTasksRevoked(); + verify(firstListener, never()).onTasksRevoked(any()); + verify(secondListener).onTasksRevoked(eq(mockAssignment.activeTasks())); + } + + @Test + public void testInvokeMethodsWithNoListener() { + IllegalStateException exception1 = assertThrows(IllegalStateException.class, + () -> invoker.invokeAllTasksRevoked()); + assertEquals("StreamsRebalanceListener is not defined", exception1.getMessage()); + + IllegalStateException exception2 = assertThrows(IllegalStateException.class, + () -> invoker.invokeTasksAssigned(createMockAssignment())); + assertEquals("StreamsRebalanceListener is not defined", exception2.getMessage()); + + IllegalStateException exception3 = assertThrows(IllegalStateException.class, + () -> invoker.invokeTasksRevoked(createMockTasks())); + assertEquals("StreamsRebalanceListener is not defined", exception3.getMessage()); + + IllegalStateException exception4 = assertThrows(IllegalStateException.class, + () -> invoker.invokeAllTasksLost()); + assertEquals("StreamsRebalanceListener is not defined", exception4.getMessage()); + } + + @Test + public void testInvokeAllTasksRevokedWithListener() { + invoker.setRebalanceListener(mockListener); + + StreamsRebalanceData.Assignment mockAssignment = createMockAssignment(); + when(streamsRebalanceData.reconciledAssignment()).thenReturn(mockAssignment); + when(mockListener.onTasksRevoked(any())).thenReturn(Optional.empty()); + + Exception result = invoker.invokeAllTasksRevoked(); + + assertNull(result); + verify(mockListener).onTasksRevoked(eq(mockAssignment.activeTasks())); + } + + @Test + public void testInvokeTasksAssignedWithListener() { + invoker.setRebalanceListener(mockListener); + StreamsRebalanceData.Assignment assignment = createMockAssignment(); + when(mockListener.onTasksAssigned(assignment)).thenReturn(Optional.empty()); + + Exception result = invoker.invokeTasksAssigned(assignment); + + assertNull(result); + verify(mockListener).onTasksAssigned(eq(assignment)); + } + + @Test + public void testInvokeTasksAssignedWithWakeupException() { + invoker.setRebalanceListener(mockListener); + StreamsRebalanceData.Assignment assignment = createMockAssignment(); + WakeupException wakeupException = new WakeupException(); + doThrow(wakeupException).when(mockListener).onTasksAssigned(assignment); + + WakeupException thrownException = assertThrows(WakeupException.class, + () -> invoker.invokeTasksAssigned(assignment)); + + assertEquals(wakeupException, thrownException); + verify(mockListener).onTasksAssigned(eq(assignment)); + } + + @Test + public void testInvokeTasksAssignedWithInterruptException() { + invoker.setRebalanceListener(mockListener); + StreamsRebalanceData.Assignment assignment = createMockAssignment(); + InterruptException interruptException = new InterruptException("Test interrupt"); + doThrow(interruptException).when(mockListener).onTasksAssigned(assignment); + + InterruptException thrownException = assertThrows(InterruptException.class, + () -> invoker.invokeTasksAssigned(assignment)); + + assertEquals(interruptException, thrownException); + verify(mockListener).onTasksAssigned(eq(assignment)); + } + + @Test + public void testInvokeTasksAssignedWithOtherException() { + invoker.setRebalanceListener(mockListener); + StreamsRebalanceData.Assignment assignment = createMockAssignment(); + RuntimeException runtimeException = new RuntimeException("Test exception"); + doThrow(runtimeException).when(mockListener).onTasksAssigned(assignment); + + Exception result = invoker.invokeTasksAssigned(assignment); + + assertEquals(runtimeException, result); + verify(mockListener).onTasksAssigned(eq(assignment)); + } + + @Test + public void testInvokeTasksRevokedWithListener() { + invoker.setRebalanceListener(mockListener); + Set tasks = createMockTasks(); + when(mockListener.onTasksRevoked(tasks)).thenReturn(Optional.empty()); + + Exception result = invoker.invokeTasksRevoked(tasks); + + assertNull(result); + verify(mockListener).onTasksRevoked(eq(tasks)); + } + + @Test + public void testInvokeTasksRevokedWithWakeupException() { + invoker.setRebalanceListener(mockListener); + Set tasks = createMockTasks(); + WakeupException wakeupException = new WakeupException(); + doThrow(wakeupException).when(mockListener).onTasksRevoked(tasks); + + WakeupException thrownException = assertThrows(WakeupException.class, + () -> invoker.invokeTasksRevoked(tasks)); + + assertEquals(wakeupException, thrownException); + verify(mockListener).onTasksRevoked(eq(tasks)); + } + + @Test + public void testInvokeTasksRevokedWithInterruptException() { + invoker.setRebalanceListener(mockListener); + Set tasks = createMockTasks(); + InterruptException interruptException = new InterruptException("Test interrupt"); + doThrow(interruptException).when(mockListener).onTasksRevoked(tasks); + + InterruptException thrownException = assertThrows(InterruptException.class, + () -> invoker.invokeTasksRevoked(tasks)); + + assertEquals(interruptException, thrownException); + verify(mockListener).onTasksRevoked(eq(tasks)); + } + + @Test + public void testInvokeTasksRevokedWithOtherException() { + invoker.setRebalanceListener(mockListener); + Set tasks = createMockTasks(); + RuntimeException runtimeException = new RuntimeException("Test exception"); + doThrow(runtimeException).when(mockListener).onTasksRevoked(tasks); + + Exception result = invoker.invokeTasksRevoked(tasks); + + assertEquals(runtimeException, result); + verify(mockListener).onTasksRevoked(eq(tasks)); + } + + @Test + public void testInvokeAllTasksLostWithListener() { + invoker.setRebalanceListener(mockListener); + when(mockListener.onAllTasksLost()).thenReturn(Optional.empty()); + + Exception result = invoker.invokeAllTasksLost(); + + assertNull(result); + verify(mockListener).onAllTasksLost(); + } + + @Test + public void testInvokeAllTasksLostWithWakeupException() { + invoker.setRebalanceListener(mockListener); + WakeupException wakeupException = new WakeupException(); + doThrow(wakeupException).when(mockListener).onAllTasksLost(); + + WakeupException thrownException = assertThrows(WakeupException.class, + () -> invoker.invokeAllTasksLost()); + + assertEquals(wakeupException, thrownException); + verify(mockListener).onAllTasksLost(); + } + + @Test + public void testInvokeAllTasksLostWithInterruptException() { + invoker.setRebalanceListener(mockListener); + InterruptException interruptException = new InterruptException("Test interrupt"); + doThrow(interruptException).when(mockListener).onAllTasksLost(); + + InterruptException thrownException = assertThrows(InterruptException.class, + () -> invoker.invokeAllTasksLost()); + + assertEquals(interruptException, thrownException); + verify(mockListener).onAllTasksLost(); + } + + @Test + public void testInvokeAllTasksLostWithOtherException() { + invoker.setRebalanceListener(mockListener); + RuntimeException runtimeException = new RuntimeException("Test exception"); + doThrow(runtimeException).when(mockListener).onAllTasksLost(); + + Exception result = invoker.invokeAllTasksLost(); + + assertEquals(runtimeException, result); + verify(mockListener).onAllTasksLost(); + } + + private StreamsRebalanceData.Assignment createMockAssignment() { + Set activeTasks = createMockTasks(); + Set standbyTasks = Set.of(); + Set warmupTasks = Set.of(); + + return new StreamsRebalanceData.Assignment(activeTasks, standbyTasks, warmupTasks); + } + + private Set createMockTasks() { + return Set.of( + new StreamsRebalanceData.TaskId("subtopology1", 0), + new StreamsRebalanceData.TaskId("subtopology1", 1) + ); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListener.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListener.java index dcc4821f2a8..a95fcef5a6c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListener.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListener.java @@ -89,6 +89,7 @@ public class DefaultStreamsRebalanceListener implements StreamsRebalanceListener taskManager.handleAssignment(activeTasksWithPartitions, standbyTasksWithPartitions); streamThread.setState(StreamThread.State.PARTITIONS_ASSIGNED); taskManager.handleRebalanceComplete(); + streamsRebalanceData.setReconciledAssignment(assignment); } catch (final Exception exception) { return Optional.of(exception); } @@ -99,6 +100,7 @@ public class DefaultStreamsRebalanceListener implements StreamsRebalanceListener public Optional onAllTasksLost() { try { taskManager.handleLostAll(); + streamsRebalanceData.setReconciledAssignment(StreamsRebalanceData.Assignment.EMPTY); } catch (final Exception exception) { return Optional.of(exception); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListenerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListenerTest.java index 66cb8e5185b..1297df7b1ee 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListenerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListenerTest.java @@ -118,49 +118,46 @@ public class DefaultStreamsRebalanceListenerTest { @Test void testOnTasksAssigned() { - createRebalanceListenerWithRebalanceData(new StreamsRebalanceData( - UUID.randomUUID(), - Optional.empty(), - Map.of( - "1", - new StreamsRebalanceData.Subtopology( - Set.of("source1"), - Set.of(), - Map.of("repartition1", new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.of((short) 1), Map.of())), - Map.of(), - Set.of() - ), - "2", - new StreamsRebalanceData.Subtopology( - Set.of("source2"), - Set.of(), - Map.of("repartition2", new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.of((short) 1), Map.of())), - Map.of(), - Set.of() - ), - "3", - new StreamsRebalanceData.Subtopology( - Set.of("source3"), - Set.of(), - Map.of("repartition3", new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.of((short) 1), Map.of())), - Map.of(), - Set.of() - ) + final StreamsRebalanceData streamsRebalanceData = mock(StreamsRebalanceData.class); + when(streamsRebalanceData.subtopologies()).thenReturn(Map.of( + "1", + new StreamsRebalanceData.Subtopology( + Set.of("source1"), + Set.of(), + Map.of("repartition1", new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.of((short) 1), Map.of())), + Map.of(), + Set.of() ), - Map.of() - )); - - final Optional result = defaultStreamsRebalanceListener.onTasksAssigned( - new StreamsRebalanceData.Assignment( - Set.of(new StreamsRebalanceData.TaskId("1", 0)), - Set.of(new StreamsRebalanceData.TaskId("2", 0)), - Set.of(new StreamsRebalanceData.TaskId("3", 0)) + "2", + new StreamsRebalanceData.Subtopology( + Set.of("source2"), + Set.of(), + Map.of("repartition2", new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.of((short) 1), Map.of())), + Map.of(), + Set.of() + ), + "3", + new StreamsRebalanceData.Subtopology( + Set.of("source3"), + Set.of(), + Map.of("repartition3", new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.of((short) 1), Map.of())), + Map.of(), + Set.of() ) + )); + createRebalanceListenerWithRebalanceData(streamsRebalanceData); + + final StreamsRebalanceData.Assignment assignment = new StreamsRebalanceData.Assignment( + Set.of(new StreamsRebalanceData.TaskId("1", 0)), + Set.of(new StreamsRebalanceData.TaskId("2", 0)), + Set.of(new StreamsRebalanceData.TaskId("3", 0)) ); + final Optional result = defaultStreamsRebalanceListener.onTasksAssigned(assignment); + assertTrue(result.isEmpty()); - final InOrder inOrder = inOrder(taskManager, streamThread); + final InOrder inOrder = inOrder(taskManager, streamThread, streamsRebalanceData); inOrder.verify(taskManager).handleAssignment( Map.of(new TaskId(1, 0), Set.of(new TopicPartition("source1", 0), new TopicPartition("repartition1", 0))), Map.of( @@ -170,6 +167,7 @@ public class DefaultStreamsRebalanceListenerTest { ); inOrder.verify(streamThread).setState(StreamThread.State.PARTITIONS_ASSIGNED); inOrder.verify(taskManager).handleRebalanceComplete(); + inOrder.verify(streamsRebalanceData).setReconciledAssignment(assignment); } @Test @@ -177,21 +175,32 @@ public class DefaultStreamsRebalanceListenerTest { final Exception exception = new RuntimeException("sample exception"); doThrow(exception).when(taskManager).handleAssignment(any(), any()); - createRebalanceListenerWithRebalanceData(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())); - final Optional result = defaultStreamsRebalanceListener.onTasksAssigned(new StreamsRebalanceData.Assignment(Set.of(), Set.of(), Set.of())); - assertTrue(defaultStreamsRebalanceListener.onAllTasksLost().isEmpty()); + final StreamsRebalanceData streamsRebalanceData = mock(StreamsRebalanceData.class); + when(streamsRebalanceData.subtopologies()).thenReturn(Map.of()); + createRebalanceListenerWithRebalanceData(streamsRebalanceData); + + final Optional result = defaultStreamsRebalanceListener.onTasksAssigned( + new StreamsRebalanceData.Assignment(Set.of(), Set.of(), Set.of()) + ); assertTrue(result.isPresent()); assertEquals(exception, result.get()); - verify(taskManager).handleLostAll(); + verify(taskManager).handleAssignment(any(), any()); verify(streamThread, never()).setState(StreamThread.State.PARTITIONS_ASSIGNED); verify(taskManager, never()).handleRebalanceComplete(); + verify(streamsRebalanceData, never()).setReconciledAssignment(any()); } @Test void testOnAllTasksLost() { - createRebalanceListenerWithRebalanceData(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())); + final StreamsRebalanceData streamsRebalanceData = mock(StreamsRebalanceData.class); + when(streamsRebalanceData.subtopologies()).thenReturn(Map.of()); + createRebalanceListenerWithRebalanceData(streamsRebalanceData); + assertTrue(defaultStreamsRebalanceListener.onAllTasksLost().isEmpty()); - verify(taskManager).handleLostAll(); + + final InOrder inOrder = inOrder(taskManager, streamsRebalanceData); + inOrder.verify(taskManager).handleLostAll(); + inOrder.verify(streamsRebalanceData).setReconciledAssignment(StreamsRebalanceData.Assignment.EMPTY); } @Test @@ -199,10 +208,13 @@ public class DefaultStreamsRebalanceListenerTest { final Exception exception = new RuntimeException("sample exception"); doThrow(exception).when(taskManager).handleLostAll(); - createRebalanceListenerWithRebalanceData(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())); + final StreamsRebalanceData streamsRebalanceData = mock(StreamsRebalanceData.class); + when(streamsRebalanceData.subtopologies()).thenReturn(Map.of()); + createRebalanceListenerWithRebalanceData(streamsRebalanceData); final Optional result = defaultStreamsRebalanceListener.onAllTasksLost(); assertTrue(result.isPresent()); assertEquals(exception, result.get()); verify(taskManager).handleLostAll(); + verify(streamsRebalanceData, never()).setReconciledAssignment(any()); } } From 9f657abf3a2b9a97f93548c41f1ee290110af2cb Mon Sep 17 00:00:00 2001 From: Lianet Magrans <98415067+lianetm@users.noreply.github.com> Date: Tue, 16 Sep 2025 11:12:19 -0400 Subject: [PATCH 038/100] MINOR: Improve consumer rebalance callbacks docs (#20528) Clarify rebalance callbacks behaviour (got some questions for onPartitionsAssigned, docs where indeed confusing about the partitions received in params). Reviewed all rebalance callbacks with it. Reviewers: Bill Bejeck --- .../PlaintextConsumerCallbackTest.java | 84 +++++++++++++++++++ .../consumer/ConsumerRebalanceListener.java | 57 +++++++++---- .../kafka/clients/consumer/KafkaConsumer.java | 9 +- 3 files changed, 129 insertions(+), 21 deletions(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCallbackTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCallbackTest.java index c81a3cd1667..0e09d62033b 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCallbackTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCallbackTest.java @@ -27,7 +27,9 @@ import java.util.Collection; import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import static org.apache.kafka.clients.ClientsTestUtils.consumeAndVerifyRecords; @@ -166,6 +168,80 @@ public class PlaintextConsumerCallbackTest { testGetPositionOfNewlyAssignedPartitionOnPartitionsAssignedCallback(CLASSIC); } + @ClusterTest + public void testOnPartitionsAssignedCalledWithNewPartitionsOnlyForClassicCooperative() throws InterruptedException { + try (var consumer = createClassicConsumerCooperativeProtocol()) { + testOnPartitionsAssignedCalledWithExpectedPartitions(consumer, true); + } + } + + @ClusterTest + public void testOnPartitionsAssignedCalledWithNewPartitionsOnlyForAsyncConsumer() throws InterruptedException { + try (var consumer = createConsumer(CONSUMER)) { + testOnPartitionsAssignedCalledWithExpectedPartitions(consumer, true); + } + } + + @ClusterTest + public void testOnPartitionsAssignedCalledWithNewPartitionsOnlyForClassicEager() throws InterruptedException { + try (var consumer = createConsumer(CLASSIC)) { + testOnPartitionsAssignedCalledWithExpectedPartitions(consumer, false); + } + } + + private void testOnPartitionsAssignedCalledWithExpectedPartitions( + Consumer consumer, + boolean expectNewPartitionsOnlyInCallback) throws InterruptedException { + subscribeAndExpectOnPartitionsAssigned(consumer, List.of(topic), List.of(tp)); + assertEquals(Set.of(tp), consumer.assignment()); + + // Add a new partition assignment while keeping the previous one + String newTopic = "newTopic"; + TopicPartition addedPartition = new TopicPartition(newTopic, 0); + List expectedPartitionsInCallback; + if (expectNewPartitionsOnlyInCallback) { + expectedPartitionsInCallback = List.of(addedPartition); + } else { + expectedPartitionsInCallback = List.of(tp, addedPartition); + } + + // Change subscription to keep the previous one and add a new topic. Assignment should be updated + // to contain partitions from both topics, but the onPartitionsAssigned parameters may containing + // the full new assignment or just the newly added partitions depending on the case. + subscribeAndExpectOnPartitionsAssigned( + consumer, + List.of(topic, newTopic), + expectedPartitionsInCallback); + assertEquals(Set.of(tp, addedPartition), consumer.assignment()); + } + + private void subscribeAndExpectOnPartitionsAssigned(Consumer consumer, List topics, Collection expectedPartitionsInCallback) throws InterruptedException { + var partitionsAssigned = new AtomicBoolean(false); + AtomicReference> partitionsFromCallback = new AtomicReference<>(); + consumer.subscribe(topics, new ConsumerRebalanceListener() { + @Override + public void onPartitionsAssigned(Collection partitions) { + if (partitions.containsAll(expectedPartitionsInCallback)) { + partitionsFromCallback.set(partitions); + partitionsAssigned.set(true); + } + } + + @Override + public void onPartitionsRevoked(Collection partitions) { + // noop + } + }); + ClientsTestUtils.pollUntilTrue( + consumer, + partitionsAssigned::get, + "Timed out before expected rebalance completed" + ); + // These are different types, so comparing values instead + assertTrue(expectedPartitionsInCallback.containsAll(partitionsFromCallback.get()) && partitionsFromCallback.get().containsAll(expectedPartitionsInCallback), + "Expected partitions " + expectedPartitionsInCallback + " as parameter for onPartitionsAssigned, but got " + partitionsFromCallback.get()); + } + @ClusterTest public void testAsyncConsumerGetPositionOfNewlyAssignedPartitionOnPartitionsAssignedCallback() throws InterruptedException { testGetPositionOfNewlyAssignedPartitionOnPartitionsAssignedCallback(CONSUMER); @@ -284,4 +360,12 @@ public class PlaintextConsumerCallbackTest { ENABLE_AUTO_COMMIT_CONFIG, "false" )); } + + private Consumer createClassicConsumerCooperativeProtocol() { + return cluster.consumer(Map.of( + GROUP_PROTOCOL_CONFIG, CLASSIC.name.toLowerCase(Locale.ROOT), + ENABLE_AUTO_COMMIT_CONFIG, "false", + ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "org.apache.kafka.clients.consumer.CooperativeStickyAssignor" + )); + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceListener.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceListener.java index c49b2c8045a..23e045b7600 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceListener.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceListener.java @@ -50,7 +50,7 @@ import java.util.Collection; * Under normal conditions, if a partition is reassigned from one consumer to another, then the old consumer will * always invoke {@link #onPartitionsRevoked(Collection) onPartitionsRevoked} for that partition prior to the new consumer * invoking {@link #onPartitionsAssigned(Collection) onPartitionsAssigned} for the same partition. So if offsets or other state is saved in the - * {@link #onPartitionsRevoked(Collection) onPartitionsRevoked} call by one consumer member, it will be always accessible by the time the + * {@link #onPartitionsRevoked(Collection) onPartitionsRevoked} call by one consumer member, it will always be accessible by the time the * other consumer member taking over that partition and triggering its {@link #onPartitionsAssigned(Collection) onPartitionsAssigned} callback to load the state. *

    * You can think of revocation as a graceful way to give up ownership of a partition. In some cases, the consumer may not have an opportunity to do so. @@ -120,13 +120,31 @@ public interface ConsumerRebalanceListener { /** * A callback method the user can implement to provide handling of offset commits to a customized store. * This method will be called during a rebalance operation when the consumer has to give up some partitions. - * It can also be called when consumer is being closed ({@link KafkaConsumer#close(CloseOptions option)}) - * or is unsubscribing ({@link KafkaConsumer#unsubscribe()}). + * The consumer may need to give up some partitions (thus this callback executed) under the following scenarios: + *

      + *
    • If the consumer assignment changes
    • + *
    • If the consumer is being closed ({@link KafkaConsumer#close(CloseOptions option)})
    • + *
    • If the consumer is unsubscribing ({@link KafkaConsumer#unsubscribe()})
    • + *
    * It is recommended that offsets should be committed in this callback to either Kafka or a * custom offset store to prevent duplicate data. *

    - * In eager rebalancing, it will always be called at the start of a rebalance and after the consumer stops fetching data. - * In cooperative rebalancing, it will be called at the end of a rebalance on the set of partitions being revoked iff the set is non-empty. + * This callback is always called before re-assigning the partitions. + * If the consumer is using the {@link GroupProtocol#CLASSIC} rebalance protocol: + *

      + *
    • + * In eager rebalancing, onPartitionsRevoked will be called with the full set of assigned partitions as a parameter (all partitions are revoked). + * It will be called even if there are no partitions to revoke. + *
    • + *
    • + * In cooperative rebalancing, onPartitionsRevoked will be called with the set of partitions to revoke, + * iff the set is non-empty. + *
    • + *
    + * If the consumer is using the {@link GroupProtocol#CONSUMER} rebalance protocol, this callback will be called + * with the set of partitions to revoke iff the set is non-empty + * (same behavior as the {@link GroupProtocol#CLASSIC} rebalance protocol with Cooperative mode). + *

    * For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer}. *

    * It is common for the revocation callback to use the consumer instance in order to commit offsets. It is possible @@ -135,8 +153,9 @@ public interface ConsumerRebalanceListener { * invocation of {@link KafkaConsumer#poll(java.time.Duration)} in which this callback is being executed. This means it is not * necessary to catch these exceptions and re-attempt to wakeup or interrupt the consumer thread. * - * @param partitions The list of partitions that were assigned to the consumer and now need to be revoked (may not - * include all currently assigned partitions, i.e. there may still be some partitions left) + * @param partitions The list of partitions that were assigned to the consumer and now need to be revoked. This will + * include the full assignment under the Classic/Eager protocol, given that it revokes all partitions. + * It will only include the subset to revoke under the Classic/Cooperative and Consumer protocols. * @throws org.apache.kafka.common.errors.WakeupException If raised from a nested call to {@link KafkaConsumer} * @throws org.apache.kafka.common.errors.InterruptException If raised from a nested call to {@link KafkaConsumer} */ @@ -144,12 +163,13 @@ public interface ConsumerRebalanceListener { /** * A callback method the user can implement to provide handling of customized offsets on completion of a successful - * partition re-assignment. This method will be called after the partition re-assignment completes and before the - * consumer starts fetching data, and only as the result of a {@link Consumer#poll(java.time.Duration) poll(long)} call. + * partition re-assignment. This method will be called after the partition re-assignment completes (even if no new + * partitions were assigned to the consumer), and before the consumer starts fetching data, + * and only as the result of a {@link Consumer#poll(java.time.Duration) poll(long)} call. *

    * It is guaranteed that under normal conditions all the processes in a consumer group will execute their - * {@link #onPartitionsRevoked(Collection)} callback before any instance executes its - * {@link #onPartitionsAssigned(Collection)} callback. During exceptional scenarios, partitions may be migrated + * {@link #onPartitionsRevoked(Collection)} callback before any instance executes this onPartitionsAssigned callback. + * During exceptional scenarios, partitions may be migrated * without the old owner being notified (i.e. their {@link #onPartitionsRevoked(Collection)} callback not triggered), * and later when the old owner consumer realized this event, the {@link #onPartitionsLost(Collection)} callback * will be triggered by the consumer then. @@ -160,9 +180,11 @@ public interface ConsumerRebalanceListener { * invocation of {@link KafkaConsumer#poll(java.time.Duration)} in which this callback is being executed. This means it is not * necessary to catch these exceptions and re-attempt to wakeup or interrupt the consumer thread. * - * @param partitions The list of partitions that are now assigned to the consumer (previously owned partitions will - * NOT be included, i.e. this list will only include newly added partitions) - * @throws org.apache.kafka.common.errors.WakeupException If raised from a nested call to {@link KafkaConsumer} + * @param partitions Partitions that have been added to the assignment as a result of the rebalance. + * Note that partitions that were already owned by this consumer and remain assigned are not + * included in this list under the Classic/Cooperative or Consumer protocols. THe full assignment + * will be received under the Classic/Eager protocol. + * @throws org.apache.kafka.common.errors.WakeupException If raised from a nested call to {@link KafkaConsumer} * @throws org.apache.kafka.common.errors.InterruptException If raised from a nested call to {@link KafkaConsumer} */ void onPartitionsAssigned(Collection partitions); @@ -187,10 +209,9 @@ public interface ConsumerRebalanceListener { * necessary to catch these exceptions and re-attempt to wakeup or interrupt the consumer thread. * * @param partitions The list of partitions that were assigned to the consumer and now have been reassigned - * to other consumers. With the current protocol this will always include all of the consumer's - * previously assigned partitions, but this may change in future protocols (ie there would still - * be some partitions left) - * @throws org.apache.kafka.common.errors.WakeupException If raised from a nested call to {@link KafkaConsumer} + * to other consumers. With both, the Classic and Consumer protocols, this will always include + * all partitions that were previously assigned to the consumer. + * @throws org.apache.kafka.common.errors.WakeupException If raised from a nested call to {@link KafkaConsumer} * @throws org.apache.kafka.common.errors.InterruptException If raised from a nested call to {@link KafkaConsumer} */ default void onPartitionsLost(Collection partitions) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index e74cf0414a8..9f1992d6568 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -661,7 +661,7 @@ public class KafkaConsumer implements Consumer { * If the given list of topics is empty, it is treated the same as {@link #unsubscribe()}. * *

    - * As part of group management, the consumer will keep track of the list of consumers that belong to a particular + * As part of group management, the group coordinator will keep track of the list of consumers that belong to a particular * group and will trigger a rebalance operation if any one of the following events are triggered: *

      *
    • Number of partitions change for any of the subscribed topics @@ -670,8 +670,11 @@ public class KafkaConsumer implements Consumer { *
    • A new member is added to the consumer group *
    *

    - * When any of these events are triggered, the provided listener will be invoked first to indicate that - * the consumer's assignment has been revoked, and then again when the new assignment has been received. + * When any of these events are triggered, the provided listener will be invoked in this way: + *

      + *
    • {@link ConsumerRebalanceListener#onPartitionsRevoked(Collection)} will be invoked with the partitions to revoke, before re-assigning those partitions to another consumer.
    • + *
    • {@link ConsumerRebalanceListener#onPartitionsAssigned(Collection)} will be invoked when the rebalance completes (even if no new partitions are assigned to the consumer)
    • + *
    * Note that rebalances will only occur during an active call to {@link #poll(Duration)}, so callbacks will * also only be invoked during that time. * From b043ca207424b201bdce8224870925a4b0eb0628 Mon Sep 17 00:00:00 2001 From: Shashank Date: Tue, 16 Sep 2025 11:46:20 -0700 Subject: [PATCH 039/100] KAFKA-19683: Remove dead tests and modify tests in TaskManagerTest [1/N] (#20501) This is the first part of cleaning up of the tests in `TaskManagerTest` - Removed dead tests - Added new tests as suggested earlier Reviewers: Lucas Brutschy --- .../processor/internals/TaskManagerTest.java | 101 +++++++----------- 1 file changed, 39 insertions(+), 62 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java index 26a1523131b..8d83d1e99fa 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java @@ -152,6 +152,7 @@ public class TaskManagerTest { private final TopicPartition t1p1 = new TopicPartition(topic1, 1); private final TopicPartition t2p2 = new TopicPartition(topic2, 1); private final TopicPartition t1p1changelog = new TopicPartition("changelog", 1); + private final TopicPartition t1p1changelog2 = new TopicPartition("changelog2", 1); private final Set taskId01Partitions = Set.of(t1p1); private final Set taskId01ChangelogPartitions = Set.of(t1p1changelog); private final Map> taskId01Assignment = singletonMap(taskId01, taskId01Partitions); @@ -218,6 +219,10 @@ public class TaskManagerTest { taskManager = setUpTaskManager(StreamsConfigUtils.ProcessingMode.AT_LEAST_ONCE, null, false); } + private TaskManager setUpTaskManager(final ProcessingMode processingMode, final TasksRegistry tasks) { + return setUpTaskManager(processingMode, tasks, false); + } + private TaskManager setUpTaskManager(final ProcessingMode processingMode, final boolean stateUpdaterEnabled) { return setUpTaskManager(processingMode, null, stateUpdaterEnabled, false); } @@ -249,52 +254,6 @@ public class TaskManagerTest { return taskManager; } - @Test - public void shouldClassifyExistingTasksWithoutStateUpdater() { - final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, false); - final Map> runningActiveTasks = mkMap(mkEntry(taskId01, Set.of(t1p1))); - final Map> standbyTasks = mkMap(mkEntry(taskId02, Set.of(t2p2))); - final Map> restoringActiveTasks = mkMap(mkEntry(taskId03, Set.of(t1p3))); - final Map> activeTasks = new HashMap<>(runningActiveTasks); - activeTasks.putAll(restoringActiveTasks); - handleAssignment(runningActiveTasks, standbyTasks, restoringActiveTasks); - - taskManager.handleAssignment(activeTasks, standbyTasks); - - verifyNoInteractions(stateUpdater); - } - - @Test - public void shouldNotUpdateExistingStandbyTaskIfStandbyIsReassignedWithSameInputPartitionWithoutStateUpdater() { - final StandbyTask standbyTask = standbyTask(taskId03, taskId03ChangelogPartitions) - .inState(State.RUNNING) - .withInputPartitions(taskId03Partitions).build(); - updateExistingStandbyTaskIfStandbyIsReassignedWithoutStateUpdater(standbyTask, taskId03Partitions); - verify(standbyTask, never()).updateInputPartitions(eq(taskId03Partitions), any()); - } - - @Test - public void shouldUpdateExistingStandbyTaskIfStandbyIsReassignedWithDifferentInputPartitionWithoutStateUpdater() { - final StandbyTask standbyTask = standbyTask(taskId03, taskId03ChangelogPartitions) - .inState(State.RUNNING) - .withInputPartitions(taskId03Partitions).build(); - updateExistingStandbyTaskIfStandbyIsReassignedWithoutStateUpdater(standbyTask, taskId04Partitions); - verify(standbyTask).updateInputPartitions(eq(taskId04Partitions), any()); - } - - private void updateExistingStandbyTaskIfStandbyIsReassignedWithoutStateUpdater(final Task standbyTask, - final Set newInputPartition) { - final TasksRegistry tasks = mock(TasksRegistry.class); - when(tasks.allTasks()).thenReturn(Set.of(standbyTask)); - final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, false); - - taskManager.handleAssignment( - Collections.emptyMap(), - mkMap(mkEntry(standbyTask.id(), newInputPartition)) - ); - - verify(standbyTask).resume(); - } @Test public void shouldLockAllTasksOnCorruptionWithProcessingThreads() { @@ -1853,14 +1812,20 @@ public class TaskManagerTest { } @Test - public void shouldReportLatestOffsetAsOffsetSumForRunningTask() throws Exception { - final Map changelogOffsets = mkMap( - mkEntry(new TopicPartition("changelog", 0), Task.LATEST_OFFSET), - mkEntry(new TopicPartition("changelog", 1), Task.LATEST_OFFSET) - ); - final Map expectedOffsetSums = mkMap(mkEntry(taskId00, Task.LATEST_OFFSET)); + public void shouldComputeOffsetSumForRunningStatefulTask() { + final StreamTask runningStatefulTask = statefulTask(taskId00, taskId00ChangelogPartitions) + .inState(State.RUNNING).build(); + final long changelogOffsetOfRunningTask = Task.LATEST_OFFSET; + when(runningStatefulTask.changelogOffsets()) + .thenReturn(mkMap(mkEntry(t1p0changelog, changelogOffsetOfRunningTask))); + final TasksRegistry tasks = mock(TasksRegistry.class); + final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks); + when(tasks.allTasksPerId()).thenReturn(mkMap(mkEntry(taskId00, runningStatefulTask))); - computeOffsetSumAndVerify(changelogOffsets, expectedOffsetSums); + assertThat( + taskManager.taskOffsetSums(), + is(mkMap(mkEntry(taskId00, changelogOffsetOfRunningTask))) + ); } @Test @@ -1911,14 +1876,14 @@ public class TaskManagerTest { } @Test - public void shouldComputeOffsetSumForRunningStatefulTaskAndRestoringTaskWithStateUpdater() { + public void shouldComputeOffsetSumForRunningStatefulTaskAndRestoringTask() { final StreamTask runningStatefulTask = statefulTask(taskId00, taskId00ChangelogPartitions) .inState(State.RUNNING).build(); final StreamTask restoringStatefulTask = statefulTask(taskId01, taskId01ChangelogPartitions) .inState(State.RESTORING).build(); final StandbyTask restoringStandbyTask = standbyTask(taskId02, taskId02ChangelogPartitions) .inState(State.RUNNING).build(); - final long changelogOffsetOfRunningTask = 42L; + final long changelogOffsetOfRunningTask = Task.LATEST_OFFSET; final long changelogOffsetOfRestoringStatefulTask = 24L; final long changelogOffsetOfRestoringStandbyTask = 84L; when(runningStatefulTask.changelogOffsets()) @@ -1943,14 +1908,26 @@ public class TaskManagerTest { } @Test - public void shouldSkipUnknownOffsetsWhenComputingOffsetSum() throws Exception { - final Map changelogOffsets = mkMap( - mkEntry(new TopicPartition("changelog", 0), OffsetCheckpoint.OFFSET_UNKNOWN), - mkEntry(new TopicPartition("changelog", 1), 10L) - ); - final Map expectedOffsetSums = mkMap(mkEntry(taskId00, 10L)); + public void shouldSkipUnknownOffsetsWhenComputingOffsetSum() { + final StreamTask restoringStatefulTask = statefulTask(taskId01, taskId01ChangelogPartitions) + .inState(State.RESTORING).build(); + final long changelogOffsetOfRestoringStandbyTask = 84L; + when(restoringStatefulTask.changelogOffsets()) + .thenReturn(mkMap( + mkEntry(t1p1changelog, changelogOffsetOfRestoringStandbyTask), + mkEntry(t1p1changelog2, OffsetCheckpoint.OFFSET_UNKNOWN) + )); + final TasksRegistry tasks = mock(TasksRegistry.class); + final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks); + when(tasks.allTasksPerId()).thenReturn(mkMap(mkEntry(taskId01, restoringStatefulTask))); + when(stateUpdater.tasks()).thenReturn(Set.of(restoringStatefulTask)); - computeOffsetSumAndVerify(changelogOffsets, expectedOffsetSums); + assertThat( + taskManager.taskOffsetSums(), + is(mkMap( + mkEntry(taskId01, changelogOffsetOfRestoringStandbyTask) + )) + ); } private void computeOffsetSumAndVerify(final Map changelogOffsets, From 8ba41a2d0dfac115422ff8eb485a7ce00dd7ff84 Mon Sep 17 00:00:00 2001 From: Jinhe Zhang Date: Tue, 16 Sep 2025 14:52:39 -0400 Subject: [PATCH 040/100] MINOR: Expose internal topic creation errors to the user (#20325) This PR introduces an ExpiringErrorCache that temporarily stores topic creation errors, allowing the system to provide detailed failure reasons in subsequent heartbeat responses. Key Designs: Time-based expiration: Errors are cached with a TTL based on the streams group heartbeat interval (2x heartbeat interval). This ensures errors remain available for at least one retry cycle while preventing unbounded growth. 2. Priority queue for efficient expiry: Uses a min-heap to track entries by expiration time, enabling efficient cleanup of expired entries during cache operations. 3. Capacity enforcement: Limits cache size to prevent memory issues under high error rates. When capacity is exceeded, oldest entries are evicted first. 4. Reference equality checks: Uses eq for object identity comparison when cleaning up stale entries, avoiding expensive value comparisons while correctly handling entry updates. Reviewers: Lucas Brutschy --- .../server/AutoTopicCreationManager.scala | 190 ++++++++- .../scala/kafka/server/BrokerServer.scala | 5 +- .../main/scala/kafka/server/KafkaApis.scala | 29 +- .../server/AutoTopicCreationManagerTest.scala | 246 ++++++++++- .../kafka/server/ExpiringErrorCacheTest.scala | 400 ++++++++++++++++++ .../unit/kafka/server/KafkaApisTest.scala | 62 ++- 6 files changed, 913 insertions(+), 19 deletions(-) create mode 100644 core/src/test/scala/unit/kafka/server/ExpiringErrorCacheTest.scala diff --git a/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala b/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala index d23fc7db88a..5da400e29de 100644 --- a/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala +++ b/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala @@ -18,6 +18,7 @@ package kafka.server import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.locks.ReentrantLock import java.util.{Collections, Properties} import kafka.coordinator.transaction.TransactionCoordinator import kafka.utils.Logging @@ -35,6 +36,7 @@ import org.apache.kafka.coordinator.share.ShareCoordinator import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, NodeToControllerChannelManager} import org.apache.kafka.server.quota.ControllerMutationQuota +import org.apache.kafka.common.utils.Time import scala.collection.{Map, Seq, Set, mutable} import scala.jdk.CollectionConverters._ @@ -50,21 +52,96 @@ trait AutoTopicCreationManager { def createStreamsInternalTopics( topics: Map[String, CreatableTopic], - requestContext: RequestContext + requestContext: RequestContext, + timeoutMs: Long ): Unit + def getStreamsInternalTopicCreationErrors( + topicNames: Set[String], + currentTimeMs: Long + ): Map[String, String] + + def close(): Unit = {} + } +/** + * Thread-safe cache that stores topic creation errors with per-entry expiration. + * - Expiration: maintained by a min-heap (priority queue) on expiration time + * - Capacity: enforced by evicting entries with earliest expiration time (not LRU) + * - Updates: old entries remain in queue but are ignored via reference equality check + */ +private[server] class ExpiringErrorCache(maxSize: Int, time: Time) { + + private case class Entry(topicName: String, errorMessage: String, expirationTimeMs: Long) + + private val byTopic = new ConcurrentHashMap[String, Entry]() + private val expiryQueue = new java.util.PriorityQueue[Entry](11, new java.util.Comparator[Entry] { + override def compare(a: Entry, b: Entry): Int = java.lang.Long.compare(a.expirationTimeMs, b.expirationTimeMs) + }) + private val lock = new ReentrantLock() + + def put(topicName: String, errorMessage: String, ttlMs: Long): Unit = { + lock.lock() + try { + val currentTimeMs = time.milliseconds() + val expirationTimeMs = currentTimeMs + ttlMs + val entry = Entry(topicName, errorMessage, expirationTimeMs) + byTopic.put(topicName, entry) + expiryQueue.add(entry) + + // Clean up expired entries and enforce capacity + while (!expiryQueue.isEmpty && + (expiryQueue.peek().expirationTimeMs <= currentTimeMs || byTopic.size() > maxSize)) { + val evicted = expiryQueue.poll() + val current = byTopic.get(evicted.topicName) + if (current != null && (current eq evicted)) { + byTopic.remove(evicted.topicName) + } + } + } finally { + lock.unlock() + } + } + + def getErrorsForTopics(topicNames: Set[String], currentTimeMs: Long): Map[String, String] = { + val result = mutable.Map.empty[String, String] + topicNames.foreach { topicName => + val entry = byTopic.get(topicName) + if (entry != null && entry.expirationTimeMs > currentTimeMs) { + result.put(topicName, entry.errorMessage) + } + } + result.toMap + } + + private[server] def clear(): Unit = { + lock.lock() + try { + byTopic.clear() + expiryQueue.clear() + } finally { + lock.unlock() + } + } +} + + class DefaultAutoTopicCreationManager( config: KafkaConfig, channelManager: NodeToControllerChannelManager, groupCoordinator: GroupCoordinator, txnCoordinator: TransactionCoordinator, - shareCoordinator: ShareCoordinator + shareCoordinator: ShareCoordinator, + time: Time, + topicErrorCacheCapacity: Int = 1000 ) extends AutoTopicCreationManager with Logging { private val inflightTopics = Collections.newSetFromMap(new ConcurrentHashMap[String, java.lang.Boolean]()) + // Hardcoded default capacity; can be overridden in tests via constructor param + private val topicCreationErrorCache = new ExpiringErrorCache(topicErrorCacheCapacity, time) + /** * Initiate auto topic creation for the given topics. * @@ -93,13 +170,21 @@ class DefaultAutoTopicCreationManager( override def createStreamsInternalTopics( topics: Map[String, CreatableTopic], - requestContext: RequestContext + requestContext: RequestContext, + timeoutMs: Long ): Unit = { if (topics.nonEmpty) { - sendCreateTopicRequest(topics, Some(requestContext)) + sendCreateTopicRequestWithErrorCaching(topics, Some(requestContext), timeoutMs) } } + override def getStreamsInternalTopicCreationErrors( + topicNames: Set[String], + currentTimeMs: Long + ): Map[String, String] = { + topicCreationErrorCache.getErrorsForTopics(topicNames, currentTimeMs) + } + private def sendCreateTopicRequest( creatableTopics: Map[String, CreatableTopic], requestContext: Option[RequestContext] @@ -264,4 +349,101 @@ class DefaultAutoTopicCreationManager( (creatableTopics, uncreatableTopics) } + + private def sendCreateTopicRequestWithErrorCaching( + creatableTopics: Map[String, CreatableTopic], + requestContext: Option[RequestContext], + timeoutMs: Long + ): Seq[MetadataResponseTopic] = { + val topicsToCreate = new CreateTopicsRequestData.CreatableTopicCollection(creatableTopics.size) + topicsToCreate.addAll(creatableTopics.values.asJavaCollection) + + val createTopicsRequest = new CreateTopicsRequest.Builder( + new CreateTopicsRequestData() + .setTimeoutMs(config.requestTimeoutMs) + .setTopics(topicsToCreate) + ) + + val requestCompletionHandler = new ControllerRequestCompletionHandler { + override def onTimeout(): Unit = { + clearInflightRequests(creatableTopics) + debug(s"Auto topic creation timed out for ${creatableTopics.keys}.") + cacheTopicCreationErrors(creatableTopics.keys.toSet, "Auto topic creation timed out.", timeoutMs) + } + + override def onComplete(response: ClientResponse): Unit = { + clearInflightRequests(creatableTopics) + if (response.authenticationException() != null) { + val authException = response.authenticationException() + warn(s"Auto topic creation failed for ${creatableTopics.keys} with authentication exception: ${authException.getMessage}") + cacheTopicCreationErrors(creatableTopics.keys.toSet, authException.getMessage, timeoutMs) + } else if (response.versionMismatch() != null) { + val versionException = response.versionMismatch() + warn(s"Auto topic creation failed for ${creatableTopics.keys} with version mismatch exception: ${versionException.getMessage}") + cacheTopicCreationErrors(creatableTopics.keys.toSet, versionException.getMessage, timeoutMs) + } else { + response.responseBody() match { + case createTopicsResponse: CreateTopicsResponse => + cacheTopicCreationErrorsFromResponse(createTopicsResponse, timeoutMs) + case _ => + debug(s"Auto topic creation completed for ${creatableTopics.keys} with response ${response.responseBody}.") + } + } + } + } + + val request = requestContext.map { context => + val requestVersion = + channelManager.controllerApiVersions.toScala match { + case None => + // We will rely on the Metadata request to be retried in the case + // that the latest version is not usable by the controller. + ApiKeys.CREATE_TOPICS.latestVersion() + case Some(nodeApiVersions) => + nodeApiVersions.latestUsableVersion(ApiKeys.CREATE_TOPICS) + } + + // Borrow client information such as client id and correlation id from the original request, + // in order to correlate the create request with the original metadata request. + val requestHeader = new RequestHeader(ApiKeys.CREATE_TOPICS, + requestVersion, + context.clientId, + context.correlationId) + ForwardingManager.buildEnvelopeRequest(context, + createTopicsRequest.build(requestVersion).serializeWithHeader(requestHeader)) + }.getOrElse(createTopicsRequest) + + channelManager.sendRequest(request, requestCompletionHandler) + + val creatableTopicResponses = creatableTopics.keySet.toSeq.map { topic => + new MetadataResponseTopic() + .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code) + .setName(topic) + .setIsInternal(Topic.isInternal(topic)) + } + + creatableTopicResponses + } + + private def cacheTopicCreationErrors(topicNames: Set[String], errorMessage: String, ttlMs: Long): Unit = { + topicNames.foreach { topicName => + topicCreationErrorCache.put(topicName, errorMessage, ttlMs) + } + } + + private def cacheTopicCreationErrorsFromResponse(response: CreateTopicsResponse, ttlMs: Long): Unit = { + response.data().topics().forEach { topicResult => + if (topicResult.errorCode() != Errors.NONE.code()) { + val errorMessage = Option(topicResult.errorMessage()) + .filter(_.nonEmpty) + .getOrElse(Errors.forCode(topicResult.errorCode()).message()) + topicCreationErrorCache.put(topicResult.name(), errorMessage, ttlMs) + debug(s"Cached topic creation error for ${topicResult.name()}: $errorMessage") + } + } + } + + override def close(): Unit = { + topicCreationErrorCache.clear() + } } diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index dccd07b83c6..3ded033020b 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -387,7 +387,7 @@ class BrokerServer( autoTopicCreationManager = new DefaultAutoTopicCreationManager( config, clientToControllerChannelManager, groupCoordinator, - transactionCoordinator, shareCoordinator) + transactionCoordinator, shareCoordinator, time) dynamicConfigHandlers = Map[ConfigType, ConfigHandler]( ConfigType.TOPIC -> new TopicConfigHandler(replicaManager, config, quotaManagers), @@ -780,6 +780,9 @@ class BrokerServer( if (shareCoordinator != null) CoreUtils.swallow(shareCoordinator.shutdown(), this) + if (autoTopicCreationManager != null) + CoreUtils.swallow(autoTopicCreationManager.close(), this) + if (assignmentsManager != null) CoreUtils.swallow(assignmentsManager.close(), this) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 1276d6e72a9..d3935c8e507 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -2812,10 +2812,35 @@ class KafkaApis(val requestChannel: RequestChannel, ) } } else { - autoTopicCreationManager.createStreamsInternalTopics(topicsToCreate, requestContext); + // Compute group-specific timeout for caching errors (2 * heartbeat interval) + val heartbeatIntervalMs = Option(groupConfigManager.groupConfig(streamsGroupHeartbeatRequest.data.groupId).orElse(null)) + .map(_.streamsHeartbeatIntervalMs().toLong) + .getOrElse(config.groupCoordinatorConfig.streamsGroupHeartbeatIntervalMs().toLong) + val timeoutMs = heartbeatIntervalMs * 2 + + autoTopicCreationManager.createStreamsInternalTopics(topicsToCreate, requestContext, timeoutMs) + + // Check for cached topic creation errors only if there's already a MISSING_INTERNAL_TOPICS status + val hasMissingInternalTopicsStatus = responseData.status() != null && + responseData.status().stream().anyMatch(s => s.statusCode() == StreamsGroupHeartbeatResponse.Status.MISSING_INTERNAL_TOPICS.code()) + + if (hasMissingInternalTopicsStatus) { + val currentTimeMs = time.milliseconds() + val cachedErrors = autoTopicCreationManager.getStreamsInternalTopicCreationErrors(topicsToCreate.keys.toSet, currentTimeMs) + if (cachedErrors.nonEmpty) { + val missingInternalTopicStatus = + responseData.status().stream().filter(x => x.statusCode() == StreamsGroupHeartbeatResponse.Status.MISSING_INTERNAL_TOPICS.code()).findFirst() + val creationErrorDetails = cachedErrors.map { case (topic, error) => s"$topic ($error)" }.mkString(", ") + if (missingInternalTopicStatus.isPresent) { + val existingDetail = Option(missingInternalTopicStatus.get().statusDetail()).getOrElse("") + missingInternalTopicStatus.get().setStatusDetail( + existingDetail + s"; Creation failed: $creationErrorDetails." + ) + } + } + } } } - requestHelper.sendMaybeThrottle(request, new StreamsGroupHeartbeatResponse(responseData)) } } diff --git a/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala b/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala index 3065588a87a..cd17d7df2b3 100644 --- a/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala @@ -35,6 +35,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, ByteBufferAccessor, Errors} import org.apache.kafka.common.requests._ import org.apache.kafka.common.security.auth.{KafkaPrincipal, KafkaPrincipalSerde, SecurityProtocol} import org.apache.kafka.common.utils.{SecurityUtils, Utils} +import org.apache.kafka.server.util.MockTime import org.apache.kafka.coordinator.group.{GroupCoordinator, GroupCoordinatorConfig} import org.apache.kafka.coordinator.share.{ShareCoordinator, ShareCoordinatorConfig} import org.apache.kafka.metadata.MetadataCache @@ -45,14 +46,15 @@ import org.apache.kafka.server.quota.ControllerMutationQuota import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue} import org.junit.jupiter.api.{BeforeEach, Test} import org.mockito.ArgumentMatchers.any -import org.mockito.Mockito.never import org.mockito.{ArgumentCaptor, ArgumentMatchers, Mockito} +import org.mockito.Mockito.never import scala.collection.{Map, Seq} class AutoTopicCreationManagerTest { private val requestTimeout = 100 + private val testCacheCapacity = 3 private var config: KafkaConfig = _ private val metadataCache = Mockito.mock(classOf[MetadataCache]) private val brokerToController = Mockito.mock(classOf[NodeToControllerChannelManager]) @@ -60,6 +62,7 @@ class AutoTopicCreationManagerTest { private val transactionCoordinator = Mockito.mock(classOf[TransactionCoordinator]) private val shareCoordinator = Mockito.mock(classOf[ShareCoordinator]) private var autoTopicCreationManager: AutoTopicCreationManager = _ + private val mockTime = new MockTime(0L, 0L) private val internalTopicPartitions = 2 private val internalTopicReplicationFactor: Short = 2 @@ -76,6 +79,8 @@ class AutoTopicCreationManagerTest { props.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, internalTopicReplicationFactor.toString) props.setProperty(TransactionLogConfig.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, internalTopicReplicationFactor.toString) props.setProperty(ShareCoordinatorConfig.STATE_TOPIC_NUM_PARTITIONS_CONFIG, internalTopicReplicationFactor.toString) + // Set a short group max session timeout for testing TTL (1 second) + props.setProperty(GroupCoordinatorConfig.GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG, "1000") config = KafkaConfig.fromProps(props) val aliveBrokers = util.List.of(new Node(0, "host0", 0), new Node(1, "host1", 1)) @@ -115,7 +120,9 @@ class AutoTopicCreationManagerTest { brokerToController, groupCoordinator, transactionCoordinator, - shareCoordinator) + shareCoordinator, + mockTime, + topicErrorCacheCapacity = testCacheCapacity) val topicsCollection = new CreateTopicsRequestData.CreatableTopicCollection topicsCollection.add(getNewTopic(topicName, numPartitions, replicationFactor)) @@ -231,9 +238,11 @@ class AutoTopicCreationManagerTest { brokerToController, groupCoordinator, transactionCoordinator, - shareCoordinator) + shareCoordinator, + mockTime, + topicErrorCacheCapacity = testCacheCapacity) - autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext) + autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext, config.groupCoordinatorConfig.streamsGroupHeartbeatIntervalMs() * 2) val argumentCaptor = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]) Mockito.verify(brokerToController).sendRequest( @@ -267,9 +276,11 @@ class AutoTopicCreationManagerTest { brokerToController, groupCoordinator, transactionCoordinator, - shareCoordinator) + shareCoordinator, + mockTime, + topicErrorCacheCapacity = testCacheCapacity) - autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext) + autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext, config.groupCoordinatorConfig.streamsGroupHeartbeatIntervalMs() * 2) Mockito.verify(brokerToController, never()).sendRequest( any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]), @@ -288,9 +299,11 @@ class AutoTopicCreationManagerTest { brokerToController, groupCoordinator, transactionCoordinator, - shareCoordinator) + shareCoordinator, + mockTime, + topicErrorCacheCapacity = testCacheCapacity) - autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext) + autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext, config.groupCoordinatorConfig.streamsGroupHeartbeatIntervalMs() * 2) val argumentCaptor = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]) Mockito.verify(brokerToController).sendRequest( @@ -319,7 +332,9 @@ class AutoTopicCreationManagerTest { brokerToController, groupCoordinator, transactionCoordinator, - shareCoordinator) + shareCoordinator, + mockTime, + topicErrorCacheCapacity = testCacheCapacity) val createTopicApiVersion = new ApiVersionsResponseData.ApiVersion() .setApiKey(ApiKeys.CREATE_TOPICS.id) @@ -356,4 +371,217 @@ class AutoTopicCreationManagerTest { .setNumPartitions(numPartitions) .setReplicationFactor(replicationFactor) } + + @Test + def testTopicCreationErrorCaching(): Unit = { + autoTopicCreationManager = new DefaultAutoTopicCreationManager( + config, + brokerToController, + groupCoordinator, + transactionCoordinator, + shareCoordinator, + mockTime, + topicErrorCacheCapacity = testCacheCapacity) + + val topics = Map( + "test-topic-1" -> new CreatableTopic().setName("test-topic-1").setNumPartitions(1).setReplicationFactor(1) + ) + val requestContext = initializeRequestContextWithUserPrincipal() + + autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext, config.groupCoordinatorConfig.streamsGroupHeartbeatIntervalMs() * 2) + + val argumentCaptor = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) + Mockito.verify(brokerToController).sendRequest( + any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]), + argumentCaptor.capture()) + + // Simulate a CreateTopicsResponse with errors + val createTopicsResponseData = new org.apache.kafka.common.message.CreateTopicsResponseData() + val topicResult = new org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult() + .setName("test-topic-1") + .setErrorCode(Errors.TOPIC_ALREADY_EXISTS.code()) + .setErrorMessage("Topic 'test-topic-1' already exists.") + createTopicsResponseData.topics().add(topicResult) + + val createTopicsResponse = new CreateTopicsResponse(createTopicsResponseData) + val header = new RequestHeader(ApiKeys.CREATE_TOPICS, 0, "client", 1) + val clientResponse = new ClientResponse(header, null, null, + 0, 0, false, null, null, createTopicsResponse) + + // Trigger the completion handler + argumentCaptor.getValue.onComplete(clientResponse) + + // Verify that the error was cached + val cachedErrors = autoTopicCreationManager.getStreamsInternalTopicCreationErrors(Set("test-topic-1"), mockTime.milliseconds()) + assertEquals(1, cachedErrors.size) + assertTrue(cachedErrors.contains("test-topic-1")) + assertEquals("Topic 'test-topic-1' already exists.", cachedErrors("test-topic-1")) + } + + @Test + def testGetTopicCreationErrorsWithMultipleTopics(): Unit = { + autoTopicCreationManager = new DefaultAutoTopicCreationManager( + config, + brokerToController, + groupCoordinator, + transactionCoordinator, + shareCoordinator, + mockTime, + topicErrorCacheCapacity = testCacheCapacity) + + val topics = Map( + "success-topic" -> new CreatableTopic().setName("success-topic").setNumPartitions(1).setReplicationFactor(1), + "failed-topic" -> new CreatableTopic().setName("failed-topic").setNumPartitions(1).setReplicationFactor(1) + ) + val requestContext = initializeRequestContextWithUserPrincipal() + autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext, config.groupCoordinatorConfig.streamsGroupHeartbeatIntervalMs() * 2) + + val argumentCaptor = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) + Mockito.verify(brokerToController).sendRequest( + any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]), + argumentCaptor.capture()) + + // Simulate mixed response - one success, one failure + val createTopicsResponseData = new org.apache.kafka.common.message.CreateTopicsResponseData() + createTopicsResponseData.topics().add( + new org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult() + .setName("success-topic") + .setErrorCode(Errors.NONE.code()) + ) + createTopicsResponseData.topics().add( + new org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult() + .setName("failed-topic") + .setErrorCode(Errors.POLICY_VIOLATION.code()) + .setErrorMessage("Policy violation") + ) + + val createTopicsResponse = new CreateTopicsResponse(createTopicsResponseData) + val header = new RequestHeader(ApiKeys.CREATE_TOPICS, 0, "client", 1) + val clientResponse = new ClientResponse(header, null, null, + 0, 0, false, null, null, createTopicsResponse) + + argumentCaptor.getValue.onComplete(clientResponse) + + // Only the failed topic should be cached + val cachedErrors = autoTopicCreationManager.getStreamsInternalTopicCreationErrors(Set("success-topic", "failed-topic", "nonexistent-topic"), mockTime.milliseconds()) + assertEquals(1, cachedErrors.size) + assertTrue(cachedErrors.contains("failed-topic")) + assertEquals("Policy violation", cachedErrors("failed-topic")) + } + + @Test + def testErrorCacheTTL(): Unit = { + autoTopicCreationManager = new DefaultAutoTopicCreationManager( + config, + brokerToController, + groupCoordinator, + transactionCoordinator, + shareCoordinator, + mockTime, + topicErrorCacheCapacity = testCacheCapacity) + + + // First cache an error by simulating topic creation failure + val topics = Map( + "test-topic" -> new CreatableTopic().setName("test-topic").setNumPartitions(1).setReplicationFactor(1) + ) + val requestContext = initializeRequestContextWithUserPrincipal() + val shortTtlMs = 1000L // Use 1 second TTL for faster testing + autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext, shortTtlMs) + + val argumentCaptor = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) + Mockito.verify(brokerToController).sendRequest( + any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]), + argumentCaptor.capture()) + + // Simulate a CreateTopicsResponse with error + val createTopicsResponseData = new org.apache.kafka.common.message.CreateTopicsResponseData() + val topicResult = new org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult() + .setName("test-topic") + .setErrorCode(Errors.INVALID_REPLICATION_FACTOR.code()) + .setErrorMessage("Invalid replication factor") + createTopicsResponseData.topics().add(topicResult) + + val createTopicsResponse = new CreateTopicsResponse(createTopicsResponseData) + val header = new RequestHeader(ApiKeys.CREATE_TOPICS, 0, "client", 1) + val clientResponse = new ClientResponse(header, null, null, + 0, 0, false, null, null, createTopicsResponse) + + // Cache the error at T0 + argumentCaptor.getValue.onComplete(clientResponse) + + // Verify error is cached and accessible within TTL + val cachedErrors = autoTopicCreationManager.getStreamsInternalTopicCreationErrors(Set("test-topic"), mockTime.milliseconds()) + assertEquals(1, cachedErrors.size) + assertEquals("Invalid replication factor", cachedErrors("test-topic")) + + // Advance time beyond TTL + mockTime.sleep(shortTtlMs + 100) // T0 + 1.1 seconds + + // Verify error is now expired and proactively cleaned up + val expiredErrors = autoTopicCreationManager.getStreamsInternalTopicCreationErrors(Set("test-topic"), mockTime.milliseconds()) + assertTrue(expiredErrors.isEmpty, "Expired errors should be proactively cleaned up") + } + + @Test + def testErrorCacheExpirationBasedEviction(): Unit = { + // Create manager with small cache size for testing + autoTopicCreationManager = new DefaultAutoTopicCreationManager( + config, + brokerToController, + groupCoordinator, + transactionCoordinator, + shareCoordinator, + mockTime, + topicErrorCacheCapacity = 3) + + val requestContext = initializeRequestContextWithUserPrincipal() + + // Create 5 topics to exceed the cache size of 3 + val topicNames = (1 to 5).map(i => s"test-topic-$i") + + // Add errors for all 5 topics to the cache + topicNames.zipWithIndex.foreach { case (topicName, idx) => + val topics = Map( + topicName -> new CreatableTopic().setName(topicName).setNumPartitions(1).setReplicationFactor(1) + ) + + autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext, config.groupCoordinatorConfig.streamsGroupHeartbeatIntervalMs() * 2) + + val argumentCaptor = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) + Mockito.verify(brokerToController, Mockito.atLeastOnce()).sendRequest( + any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]), + argumentCaptor.capture()) + + // Simulate error response for this topic + val createTopicsResponseData = new org.apache.kafka.common.message.CreateTopicsResponseData() + val topicResult = new org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult() + .setName(topicName) + .setErrorCode(Errors.TOPIC_ALREADY_EXISTS.code()) + .setErrorMessage(s"Topic '$topicName' already exists.") + createTopicsResponseData.topics().add(topicResult) + + val createTopicsResponse = new CreateTopicsResponse(createTopicsResponseData) + val header = new RequestHeader(ApiKeys.CREATE_TOPICS, 0, "client", 1) + val clientResponse = new ClientResponse(header, null, null, + 0, 0, false, null, null, createTopicsResponse) + + argumentCaptor.getValue.onComplete(clientResponse) + + // Advance time slightly between additions to ensure different timestamps + mockTime.sleep(10) + + } + + // With cache size of 3, topics 1 and 2 should have been evicted + val cachedErrors = autoTopicCreationManager.getStreamsInternalTopicCreationErrors(topicNames.toSet, mockTime.milliseconds()) + + // Only the last 3 topics should be in the cache (topics 3, 4, 5) + assertEquals(3, cachedErrors.size, "Cache should contain only the most recent 3 entries") + assertTrue(cachedErrors.contains("test-topic-3"), "test-topic-3 should be in cache") + assertTrue(cachedErrors.contains("test-topic-4"), "test-topic-4 should be in cache") + assertTrue(cachedErrors.contains("test-topic-5"), "test-topic-5 should be in cache") + assertTrue(!cachedErrors.contains("test-topic-1"), "test-topic-1 should have been evicted") + assertTrue(!cachedErrors.contains("test-topic-2"), "test-topic-2 should have been evicted") + } } diff --git a/core/src/test/scala/unit/kafka/server/ExpiringErrorCacheTest.scala b/core/src/test/scala/unit/kafka/server/ExpiringErrorCacheTest.scala new file mode 100644 index 00000000000..be02f95374c --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ExpiringErrorCacheTest.scala @@ -0,0 +1,400 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import org.apache.kafka.server.util.MockTime +import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.{BeforeEach, Test} +import scala.concurrent.Future +import scala.concurrent.ExecutionContext.Implicits.global +import scala.util.Random +import java.util.concurrent.{CountDownLatch, TimeUnit} + +class ExpiringErrorCacheTest { + + private var mockTime: MockTime = _ + private var cache: ExpiringErrorCache = _ + + @BeforeEach + def setUp(): Unit = { + mockTime = new MockTime() + } + + // Basic Functionality Tests + + @Test + def testPutAndGet(): Unit = { + cache = new ExpiringErrorCache(10, mockTime) + + cache.put("topic1", "error1", 1000L) + cache.put("topic2", "error2", 2000L) + + val errors = cache.getErrorsForTopics(Set("topic1", "topic2"), mockTime.milliseconds()) + assertEquals(2, errors.size) + assertEquals("error1", errors("topic1")) + assertEquals("error2", errors("topic2")) + } + + @Test + def testGetNonExistentTopic(): Unit = { + cache = new ExpiringErrorCache(10, mockTime) + + cache.put("topic1", "error1", 1000L) + + val errors = cache.getErrorsForTopics(Set("topic1", "topic2"), mockTime.milliseconds()) + assertEquals(1, errors.size) + assertEquals("error1", errors("topic1")) + assertFalse(errors.contains("topic2")) + } + + @Test + def testUpdateExistingEntry(): Unit = { + cache = new ExpiringErrorCache(10, mockTime) + + cache.put("topic1", "error1", 1000L) + assertEquals("error1", cache.getErrorsForTopics(Set("topic1"), mockTime.milliseconds())("topic1")) + + // Update with new error + cache.put("topic1", "error2", 2000L) + assertEquals("error2", cache.getErrorsForTopics(Set("topic1"), mockTime.milliseconds())("topic1")) + } + + @Test + def testGetMultipleTopics(): Unit = { + cache = new ExpiringErrorCache(10, mockTime) + + cache.put("topic1", "error1", 1000L) + cache.put("topic2", "error2", 1000L) + cache.put("topic3", "error3", 1000L) + + val errors = cache.getErrorsForTopics(Set("topic1", "topic3", "topic4"), mockTime.milliseconds()) + assertEquals(2, errors.size) + assertEquals("error1", errors("topic1")) + assertEquals("error3", errors("topic3")) + assertFalse(errors.contains("topic2")) + assertFalse(errors.contains("topic4")) + } + + // Expiration Tests + + @Test + def testExpiredEntryNotReturned(): Unit = { + cache = new ExpiringErrorCache(10, mockTime) + + cache.put("topic1", "error1", 1000L) + + // Entry should be available before expiration + assertEquals(1, cache.getErrorsForTopics(Set("topic1"), mockTime.milliseconds()).size) + + // Advance time past expiration + mockTime.sleep(1001L) + + // Entry should not be returned after expiration + assertTrue(cache.getErrorsForTopics(Set("topic1"), mockTime.milliseconds()).isEmpty) + } + + @Test + def testExpiredEntriesCleanedOnPut(): Unit = { + cache = new ExpiringErrorCache(10, mockTime) + + // Add entries with different TTLs + cache.put("topic1", "error1", 1000L) + cache.put("topic2", "error2", 2000L) + + // Advance time to expire topic1 but not topic2 + mockTime.sleep(1500L) + + // Add a new entry - this should trigger cleanup + cache.put("topic3", "error3", 1000L) + + // Verify only non-expired entries remain + val errors = cache.getErrorsForTopics(Set("topic1", "topic2", "topic3"), mockTime.milliseconds()) + assertEquals(2, errors.size) + assertFalse(errors.contains("topic1")) + assertEquals("error2", errors("topic2")) + assertEquals("error3", errors("topic3")) + } + + @Test + def testMixedExpiredAndValidEntries(): Unit = { + cache = new ExpiringErrorCache(10, mockTime) + + cache.put("topic1", "error1", 500L) + cache.put("topic2", "error2", 1000L) + cache.put("topic3", "error3", 1500L) + + // Advance time to expire only topic1 + mockTime.sleep(600L) + + val errors = cache.getErrorsForTopics(Set("topic1", "topic2", "topic3"), mockTime.milliseconds()) + assertEquals(2, errors.size) + assertFalse(errors.contains("topic1")) + assertTrue(errors.contains("topic2")) + assertTrue(errors.contains("topic3")) + } + + // Capacity Enforcement Tests + + @Test + def testCapacityEnforcement(): Unit = { + cache = new ExpiringErrorCache(3, mockTime) + + // Add 5 entries, exceeding capacity of 3 + for (i <- 1 to 5) { + cache.put(s"topic$i", s"error$i", 1000L) + // Small time advance between entries to ensure different insertion order + mockTime.sleep(10L) + } + + val errors = cache.getErrorsForTopics((1 to 5).map(i => s"topic$i").toSet, mockTime.milliseconds()) + assertEquals(3, errors.size) + + // The cache evicts by earliest expiration time + // Since all have same TTL, earliest inserted (topic1, topic2) should be evicted + assertFalse(errors.contains("topic1")) + assertFalse(errors.contains("topic2")) + assertTrue(errors.contains("topic3")) + assertTrue(errors.contains("topic4")) + assertTrue(errors.contains("topic5")) + } + + @Test + def testEvictionOrder(): Unit = { + cache = new ExpiringErrorCache(3, mockTime) + + // Add entries with different TTLs + cache.put("topic1", "error1", 3000L) // Expires at 3000 + mockTime.sleep(100L) + cache.put("topic2", "error2", 1000L) // Expires at 1100 + mockTime.sleep(100L) + cache.put("topic3", "error3", 2000L) // Expires at 2200 + mockTime.sleep(100L) + cache.put("topic4", "error4", 500L) // Expires at 800 + + // With capacity 3, topic4 (earliest expiration) should be evicted + val errors = cache.getErrorsForTopics(Set("topic1", "topic2", "topic3", "topic4"), mockTime.milliseconds()) + assertEquals(3, errors.size) + assertTrue(errors.contains("topic1")) + assertTrue(errors.contains("topic2")) + assertTrue(errors.contains("topic3")) + assertFalse(errors.contains("topic4")) + } + + @Test + def testCapacityWithDifferentTTLs(): Unit = { + cache = new ExpiringErrorCache(2, mockTime) + + cache.put("topic1", "error1", 5000L) // Long TTL + cache.put("topic2", "error2", 100L) // Short TTL + cache.put("topic3", "error3", 3000L) // Medium TTL + + // topic2 has earliest expiration, so it should be evicted + val errors = cache.getErrorsForTopics(Set("topic1", "topic2", "topic3"), mockTime.milliseconds()) + assertEquals(2, errors.size) + assertTrue(errors.contains("topic1")) + assertFalse(errors.contains("topic2")) + assertTrue(errors.contains("topic3")) + } + + // Update and Stale Entry Tests + + @Test + def testUpdateDoesNotLeaveStaleEntries(): Unit = { + cache = new ExpiringErrorCache(3, mockTime) + + // Fill cache to capacity + cache.put("topic1", "error1", 1000L) + cache.put("topic2", "error2", 1000L) + cache.put("topic3", "error3", 1000L) + + // Update topic2 with longer TTL + cache.put("topic2", "error2_updated", 5000L) + + // Add new entry to trigger eviction + cache.put("topic4", "error4", 1000L) + + // Should evict topic1 or topic3 (earliest expiration), not the updated topic2 + val errors = cache.getErrorsForTopics(Set("topic1", "topic2", "topic3", "topic4"), mockTime.milliseconds()) + assertEquals(3, errors.size) + assertTrue(errors.contains("topic2")) + assertEquals("error2_updated", errors("topic2")) + } + + @Test + def testStaleEntriesInQueueHandledCorrectly(): Unit = { + cache = new ExpiringErrorCache(10, mockTime) + + // Add and update same topic multiple times + cache.put("topic1", "error1", 1000L) + cache.put("topic1", "error2", 2000L) + cache.put("topic1", "error3", 3000L) + + // Only latest value should be returned + val errors = cache.getErrorsForTopics(Set("topic1"), mockTime.milliseconds()) + assertEquals(1, errors.size) + assertEquals("error3", errors("topic1")) + + // Advance time to expire first two entries + mockTime.sleep(2500L) + + // Force cleanup by adding new entry + cache.put("topic2", "error_new", 1000L) + + // topic1 should still be available with latest value + val errorsAfterCleanup = cache.getErrorsForTopics(Set("topic1"), mockTime.milliseconds()) + assertEquals(1, errorsAfterCleanup.size) + assertEquals("error3", errorsAfterCleanup("topic1")) + } + + // Edge Cases + + @Test + def testEmptyCache(): Unit = { + cache = new ExpiringErrorCache(10, mockTime) + + val errors = cache.getErrorsForTopics(Set("topic1", "topic2"), mockTime.milliseconds()) + assertTrue(errors.isEmpty) + } + + @Test + def testSingleEntryCache(): Unit = { + cache = new ExpiringErrorCache(1, mockTime) + + cache.put("topic1", "error1", 1000L) + cache.put("topic2", "error2", 1000L) + + // Only most recent should remain + val errors = cache.getErrorsForTopics(Set("topic1", "topic2"), mockTime.milliseconds()) + assertEquals(1, errors.size) + assertFalse(errors.contains("topic1")) + assertTrue(errors.contains("topic2")) + } + + @Test + def testZeroTTL(): Unit = { + cache = new ExpiringErrorCache(10, mockTime) + + cache.put("topic1", "error1", 0L) + + // Entry expires immediately + assertTrue(cache.getErrorsForTopics(Set("topic1"), mockTime.milliseconds()).isEmpty) + } + + @Test + def testClearOperation(): Unit = { + cache = new ExpiringErrorCache(10, mockTime) + + cache.put("topic1", "error1", 1000L) + cache.put("topic2", "error2", 1000L) + + assertEquals(2, cache.getErrorsForTopics(Set("topic1", "topic2"), mockTime.milliseconds()).size) + + cache.clear() + + assertTrue(cache.getErrorsForTopics(Set("topic1", "topic2"), mockTime.milliseconds()).isEmpty) + } + + // Concurrent Access Tests + + @Test + def testConcurrentPutOperations(): Unit = { + cache = new ExpiringErrorCache(100, mockTime) + val numThreads = 10 + val numTopicsPerThread = 20 + val latch = new CountDownLatch(numThreads) + + (1 to numThreads).foreach { threadId => + Future { + try { + for (i <- 1 to numTopicsPerThread) { + cache.put(s"topic_${threadId}_$i", s"error_${threadId}_$i", 1000L) + } + } finally { + latch.countDown() + } + } + } + + assertTrue(latch.await(5, TimeUnit.SECONDS)) + + // Verify all entries were added + val allTopics = (1 to numThreads).flatMap { threadId => + (1 to numTopicsPerThread).map(i => s"topic_${threadId}_$i") + }.toSet + + val errors = cache.getErrorsForTopics(allTopics, mockTime.milliseconds()) + assertEquals(100, errors.size) // Limited by cache capacity + } + + @Test + def testConcurrentPutAndGet(): Unit = { + cache = new ExpiringErrorCache(100, mockTime) + val numOperations = 1000 + val random = new Random() + val topics = (1 to 50).map(i => s"topic$i").toArray + + val futures = (1 to numOperations).map { _ => + Future { + if (random.nextBoolean()) { + // Put operation + val topic = topics(random.nextInt(topics.length)) + cache.put(topic, s"error_${random.nextInt()}", 1000L) + } else { + // Get operation + val topicsToGet = Set(topics(random.nextInt(topics.length))) + cache.getErrorsForTopics(topicsToGet, mockTime.milliseconds()) + } + } + } + + // Wait for all operations to complete + Future.sequence(futures).map(_ => ()) + } + + @Test + def testConcurrentUpdates(): Unit = { + cache = new ExpiringErrorCache(50, mockTime) + val numThreads = 10 + val numUpdatesPerThread = 100 + val sharedTopics = (1 to 10).map(i => s"shared_topic$i").toArray + val latch = new CountDownLatch(numThreads) + + (1 to numThreads).foreach { threadId => + Future { + try { + val random = new Random() + for (i <- 1 to numUpdatesPerThread) { + val topic = sharedTopics(random.nextInt(sharedTopics.length)) + cache.put(topic, s"error_thread${threadId}_update$i", 1000L) + } + } finally { + latch.countDown() + } + } + } + + assertTrue(latch.await(5, TimeUnit.SECONDS)) + + // Verify all shared topics have some value + val errors = cache.getErrorsForTopics(sharedTopics.toSet, mockTime.milliseconds()) + sharedTopics.foreach { topic => + assertTrue(errors.contains(topic), s"Topic $topic should have a value") + assertTrue(errors(topic).startsWith("error_thread"), s"Value should be from one of the threads") + } + } +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 42cf9a30ff5..bdd62291407 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -168,7 +168,8 @@ class KafkaApisTest extends Logging { authorizer: Option[Authorizer] = None, configRepository: ConfigRepository = new MockConfigRepository(), overrideProperties: Map[String, String] = Map.empty, - featureVersions: Seq[FeatureVersion] = Seq.empty + featureVersions: Seq[FeatureVersion] = Seq.empty, + autoTopicCreationManager: Option[AutoTopicCreationManager] = None ): KafkaApis = { val properties = TestUtils.createBrokerConfig(brokerId) @@ -194,7 +195,7 @@ class KafkaApisTest extends Logging { groupCoordinator = groupCoordinator, txnCoordinator = txnCoordinator, shareCoordinator = shareCoordinator, - autoTopicCreationManager = autoTopicCreationManager, + autoTopicCreationManager = autoTopicCreationManager.getOrElse(this.autoTopicCreationManager), brokerId = brokerId, config = config, configRepository = configRepository, @@ -10887,7 +10888,7 @@ class KafkaApisTest extends Logging { future.complete(new StreamsGroupHeartbeatResult(streamsGroupHeartbeatResponse, missingTopics.asJava)) val response = verifyNoThrottling[StreamsGroupHeartbeatResponse](requestChannelRequest) assertEquals(streamsGroupHeartbeatResponse, response.data) - verify(autoTopicCreationManager).createStreamsInternalTopics(missingTopics, requestChannelRequest.context) + verify(autoTopicCreationManager).createStreamsInternalTopics(any(), any(), anyLong()) } @Test @@ -10947,6 +10948,61 @@ class KafkaApisTest extends Logging { ) } + @Test + def testStreamsGroupHeartbeatRequestWithCachedTopicCreationErrors(): Unit = { + val features = mock(classOf[FinalizedFeatures]) + when(features.finalizedFeatures()).thenReturn(util.Map.of(StreamsVersion.FEATURE_NAME, 1.toShort)) + + metadataCache = mock(classOf[KRaftMetadataCache]) + when(metadataCache.features()).thenReturn(features) + + val streamsGroupHeartbeatRequest = new StreamsGroupHeartbeatRequestData().setGroupId("group") + val requestChannelRequest = buildRequest(new StreamsGroupHeartbeatRequest.Builder(streamsGroupHeartbeatRequest, true).build()) + + val future = new CompletableFuture[StreamsGroupHeartbeatResult]() + when(groupCoordinator.streamsGroupHeartbeat( + requestChannelRequest.context, + streamsGroupHeartbeatRequest + )).thenReturn(future) + + // Mock AutoTopicCreationManager to return cached errors + val mockAutoTopicCreationManager = mock(classOf[AutoTopicCreationManager]) + when(mockAutoTopicCreationManager.getStreamsInternalTopicCreationErrors(ArgumentMatchers.eq(Set("test-topic")), any())) + .thenReturn(Map("test-topic" -> "INVALID_REPLICATION_FACTOR")) + // Mock the createStreamsInternalTopics method to do nothing (simulate topic creation attempt) + doNothing().when(mockAutoTopicCreationManager).createStreamsInternalTopics(any(), any(), anyLong()) + + kafkaApis = createKafkaApis(autoTopicCreationManager = Some(mockAutoTopicCreationManager)) + kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) + + // Group coordinator returns MISSING_INTERNAL_TOPICS status and topics to create + val missingTopics = util.Map.of("test-topic", new CreatableTopic()) + val streamsGroupHeartbeatResponse = new StreamsGroupHeartbeatResponseData() + .setMemberId("member") + .setStatus(util.List.of( + new StreamsGroupHeartbeatResponseData.Status() + .setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_INTERNAL_TOPICS.code()) + .setStatusDetail("Internal topics are missing: [test-topic]") + )) + + future.complete(new StreamsGroupHeartbeatResult(streamsGroupHeartbeatResponse, missingTopics)) + val response = verifyNoThrottling[StreamsGroupHeartbeatResponse](requestChannelRequest) + + assertEquals(Errors.NONE.code, response.data.errorCode()) + assertEquals(null, response.data.errorMessage()) + + // Verify that the cached error was appended to the existing status detail + assertEquals(1, response.data.status().size()) + val status = response.data.status().get(0) + assertEquals(StreamsGroupHeartbeatResponse.Status.MISSING_INTERNAL_TOPICS.code(), status.statusCode()) + assertTrue(status.statusDetail().contains("Internal topics are missing: [test-topic]")) + assertTrue(status.statusDetail().contains("Creation failed: test-topic (INVALID_REPLICATION_FACTOR)")) + + // Verify that createStreamsInternalTopics was called + verify(mockAutoTopicCreationManager).createStreamsInternalTopics(any(), any(), anyLong()) + verify(mockAutoTopicCreationManager).getStreamsInternalTopicCreationErrors(ArgumentMatchers.eq(Set("test-topic")), any()) + } + @ParameterizedTest @ValueSource(booleans = Array(true, false)) def testConsumerGroupDescribe(includeAuthorizedOperations: Boolean): Unit = { From bbbc0cf79313a083b40cd206c132fd66dccd3ff5 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Wed, 17 Sep 2025 11:13:28 +0200 Subject: [PATCH 041/100] MINOR: Fix format in CoordinatorLoaderImpl (#20538) The format of the code in `CoordinatorLoaderImpl` in inconsistent with the rest of the code in the package. This small PR fixes it. Reviewers: Ken Huang , TengYao Chi , Andrew Schofield , Sean Quah , Chia-Ping Tsai --- .../common/runtime/CoordinatorLoaderImpl.java | 79 +++++++++---------- 1 file changed, 38 insertions(+), 41 deletions(-) diff --git a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImpl.java b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImpl.java index 078dad36ef8..6613ce25fc8 100644 --- a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImpl.java +++ b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImpl.java @@ -62,11 +62,11 @@ public class CoordinatorLoaderImpl implements CoordinatorLoader { private final KafkaScheduler scheduler = new KafkaScheduler(1); public CoordinatorLoaderImpl( - Time time, - Function> partitionLogSupplier, - Function> partitionLogEndOffsetSupplier, - Deserializer deserializer, - int loadBufferSize + Time time, + Function> partitionLogSupplier, + Function> partitionLogEndOffsetSupplier, + Deserializer deserializer, + int loadBufferSize ) { this.time = time; this.partitionLogSupplier = partitionLogSupplier; @@ -89,7 +89,7 @@ public class CoordinatorLoaderImpl implements CoordinatorLoader { long startTimeMs = time.milliseconds(); try { ScheduledFuture result = scheduler.scheduleOnce(String.format("Load coordinator from %s", tp), - () -> doLoad(tp, coordinator, future, startTimeMs)); + () -> doLoad(tp, coordinator, future, startTimeMs)); if (result.isCancelled()) { future.completeExceptionally(new RuntimeException("Coordinator loader is closed.")); } @@ -100,17 +100,17 @@ public class CoordinatorLoaderImpl implements CoordinatorLoader { } private void doLoad( - TopicPartition tp, - CoordinatorPlayback coordinator, - CompletableFuture future, - long startTimeMs + TopicPartition tp, + CoordinatorPlayback coordinator, + CompletableFuture future, + long startTimeMs ) { long schedulerQueueTimeMs = time.milliseconds() - startTimeMs; try { Optional logOpt = partitionLogSupplier.apply(tp); if (logOpt.isEmpty()) { future.completeExceptionally(new NotLeaderOrFollowerException( - "Could not load records from " + tp + " because the log does not exist.")); + "Could not load records from " + tp + " because the log does not exist.")); return; } @@ -142,8 +142,7 @@ public class CoordinatorLoaderImpl implements CoordinatorLoader { if (logEndOffset(tp) == -1L) { future.completeExceptionally(new NotLeaderOrFollowerException( - String.format("Stopped loading records from %s because the partition is not online or is no longer the leader.", tp) - )); + String.format("Stopped loading records from %s because the partition is not online or is no longer the leader.", tp))); } else if (isRunning.get()) { future.complete(new LoadSummary(startTimeMs, endTimeMs, schedulerQueueTimeMs, stats.numRecords, stats.numBytes)); } else { @@ -186,7 +185,7 @@ public class CoordinatorLoaderImpl implements CoordinatorLoader { if (buffer.capacity() < bytesNeeded) { if (loadBufferSize < bytesNeeded) { LOG.warn("Loaded metadata from {} with buffer larger ({} bytes) than" + - " configured buffer size ({} bytes).", tp, bytesNeeded, loadBufferSize); + " configured buffer size ({} bytes).", tp, bytesNeeded, loadBufferSize); } buffer = ByteBuffer.allocate(bytesNeeded); @@ -202,15 +201,14 @@ public class CoordinatorLoaderImpl implements CoordinatorLoader { } private ReplayResult processMemoryRecords( - TopicPartition tp, - UnifiedLog log, - MemoryRecords memoryRecords, - CoordinatorPlayback coordinator, - LoadStats loadStats, - long currentOffset, - long previousHighWatermark + TopicPartition tp, + UnifiedLog log, + MemoryRecords memoryRecords, + CoordinatorPlayback coordinator, + LoadStats loadStats, + long currentOffset, + long previousHighWatermark ) { - for (MutableRecordBatch batch : memoryRecords.batches()) { if (batch.isControlBatch()) { for (Record record : batch) { @@ -220,8 +218,8 @@ public class CoordinatorLoaderImpl implements CoordinatorLoader { if (controlRecord == ControlRecordType.COMMIT) { if (LOG.isTraceEnabled()) { LOG.trace("Replaying end transaction marker from {} at offset {} to commit" + - " transaction with producer id {} and producer epoch {}.", - tp, record.offset(), batch.producerId(), batch.producerEpoch()); + " transaction with producer id {} and producer epoch {}.", + tp, record.offset(), batch.producerId(), batch.producerEpoch()); } coordinator.replayEndTransactionMarker( batch.producerId(), @@ -231,8 +229,8 @@ public class CoordinatorLoaderImpl implements CoordinatorLoader { } else if (controlRecord == ControlRecordType.ABORT) { if (LOG.isTraceEnabled()) { LOG.trace("Replaying end transaction marker from {} at offset {} to abort" + - " transaction with producer id {} and producer epoch {}.", - tp, record.offset(), batch.producerId(), batch.producerEpoch()); + " transaction with producer id {} and producer epoch {}.", + tp, record.offset(), batch.producerId(), batch.producerEpoch()); } coordinator.replayEndTransactionMarker( batch.producerId(), @@ -250,7 +248,7 @@ public class CoordinatorLoaderImpl implements CoordinatorLoader { coordinatorRecordOpt = Optional.ofNullable(deserializer.deserialize(record.key(), record.value())); } catch (Deserializer.UnknownRecordTypeException ex) { LOG.warn("Unknown record type {} while loading offsets and group metadata from {}." + - " Ignoring it. It could be a left over from an aborted upgrade.", ex.unknownType(), tp); + " Ignoring it. It could be a left over from an aborted upgrade.", ex.unknownType(), tp); } catch (RuntimeException ex) { String msg = String.format("Deserializing record %s from %s failed.", record, tp); LOG.error(msg, ex); @@ -261,18 +259,18 @@ public class CoordinatorLoaderImpl implements CoordinatorLoader { try { if (LOG.isTraceEnabled()) { LOG.trace("Replaying record {} from {} at offset {} with producer id {}" + - " and producer epoch {}.", coordinatorRecord, tp, record.offset(), batch.producerId(), batch.producerEpoch()); + " and producer epoch {}.", coordinatorRecord, tp, record.offset(), batch.producerId(), batch.producerEpoch()); } coordinator.replay( - record.offset(), - batch.producerId(), - batch.producerEpoch(), - coordinatorRecord + record.offset(), + batch.producerId(), + batch.producerEpoch(), + coordinatorRecord ); } catch (RuntimeException ex) { String msg = String.format("Replaying record %s from %s at offset %d with producer id %d and" + - " producer epoch %d failed.", coordinatorRecord, tp, record.offset(), - batch.producerId(), batch.producerEpoch()); + " producer epoch %d failed.", coordinatorRecord, tp, record.offset(), + batch.producerId(), batch.producerEpoch()); LOG.error(msg, ex); throw new RuntimeException(msg, ex); } @@ -320,14 +318,13 @@ public class CoordinatorLoaderImpl implements CoordinatorLoader { @Override public String toString() { - return "LoadStats{" + - "numRecords=" + numRecords + - ", numBytes=" + numBytes + - ", readAtLeastOneRecord=" + readAtLeastOneRecord + - '}'; + return "LoadStats(" + + "numRecords=" + numRecords + + ", numBytes=" + numBytes + + ", readAtLeastOneRecord=" + readAtLeastOneRecord + + ')'; } } - private record ReplayResult(long nextOffset, long highWatermark) { - } + private record ReplayResult(long nextOffset, long highWatermark) { } } From 3bc50f937cb3acd5b814587da5518d8fb5e5ec21 Mon Sep 17 00:00:00 2001 From: Shivsundar R Date: Wed, 17 Sep 2025 10:28:20 -0400 Subject: [PATCH 042/100] KAFKA-19623: Implement KIP-1147 for console producer/consumer/share-consumer. (#20479) *What* https://issues.apache.org/jira/browse/KAFKA-19623 - The PR implements KIP-1147 (https://cwiki.apache.org/confluence/display/KAFKA/KIP-1147%3A+Improve+consistency+of+command-line+arguments) for the console tools i.e. `ConsoleProducer`, `ConsoleConsumer` and `ConsoleShareConsumer`. - Currently the previous names for the options are still usable but there will be warning message stating those are deprecated and will be removed in a future version. - I have added unit tests and also manually verified using the console tools that things are working as expected. Reviewers: Andrew Schofield , Jhen-Yung Hsu , Jimmy Wang <48462172+JimmyWang6@users.noreply.github.com> --- docker/examples/README.md | 8 +- docker/test/docker_sanity_test.py | 16 +- docs/security.html | 4 +- .../KStreamAggregationIntegrationTest.java | 14 +- tests/kafkatest/services/console_consumer.py | 19 +- .../services/console_share_consumer.py | 21 +- tests/kafkatest/version.py | 14 + .../apache/kafka/tools/ConsoleProducer.java | 40 +- .../consumer/ConsoleConsumerOptions.java | 71 +++- .../consumer/ConsoleShareConsumerOptions.java | 74 +++- .../kafka/tools/ConsoleProducerTest.java | 113 +++++- .../consumer/ConsoleConsumerOptionsTest.java | 364 ++++++++++++++---- .../ConsoleShareConsumerOptionsTest.java | 359 ++++++++++++++--- 13 files changed, 930 insertions(+), 187 deletions(-) diff --git a/docker/examples/README.md b/docker/examples/README.md index e76247bdb54..162e27c711a 100644 --- a/docker/examples/README.md +++ b/docker/examples/README.md @@ -147,7 +147,7 @@ Single Node - To produce messages using client scripts (Ensure that java version >= 17): ``` # Run from root of the repo - $ bin/kafka-console-producer.sh --topic test --bootstrap-server localhost:9093 --producer.config ./docker/examples/fixtures/client-secrets/client-ssl.properties + $ bin/kafka-console-producer.sh --topic test --bootstrap-server localhost:9093 --command-config ./docker/examples/fixtures/client-secrets/client-ssl.properties ``` - File Input: - Here ssl configs are provided via file input. @@ -167,7 +167,7 @@ Single Node - To produce messages using client scripts (Ensure that java version >= 17): ``` # Run from root of the repo - $ bin/kafka-console-producer.sh --topic test --bootstrap-server localhost:9093 --producer.config ./docker/examples/fixtures/client-secrets/client-ssl.properties + $ bin/kafka-console-producer.sh --topic test --bootstrap-server localhost:9093 --command-config ./docker/examples/fixtures/client-secrets/client-ssl.properties ``` Multi Node Cluster @@ -219,7 +219,7 @@ Multi Node Cluster - To produce messages using client scripts (Ensure that java version >= 17): ``` # Run from root of the repo - $ bin/kafka-console-producer.sh --topic test --bootstrap-server localhost:29093 --producer.config ./docker/examples/fixtures/client-secrets/client-ssl.properties + $ bin/kafka-console-producer.sh --topic test --bootstrap-server localhost:29093 --command-config ./docker/examples/fixtures/client-secrets/client-ssl.properties ``` - Isolated: - Examples are present in `docker-compose-files/cluster/isolated` directory. @@ -258,7 +258,7 @@ Multi Node Cluster - To produce messages using client scripts (Ensure that java version >= 17): ``` # Run from root of the repo - $ bin/kafka-console-producer.sh --topic test --bootstrap-server localhost:29093 --producer.config ./docker/examples/fixtures/client-secrets/client-ssl.properties + $ bin/kafka-console-producer.sh --topic test --bootstrap-server localhost:29093 --command-config ./docker/examples/fixtures/client-secrets/client-ssl.properties ``` - Note that the examples are meant to be tried one at a time, make sure you close an example server before trying out the other to avoid conflicts. diff --git a/docker/test/docker_sanity_test.py b/docker/test/docker_sanity_test.py index d2135fb0295..0d21bf47fee 100644 --- a/docker/test/docker_sanity_test.py +++ b/docker/test/docker_sanity_test.py @@ -65,7 +65,7 @@ class DockerSanityTest(unittest.TestCase): subprocess.run(["bash", "-c", " ".join(command)]) def consume_message(self, topic, consumer_config): - command = [f"{self.FIXTURES_DIR}/{constants.KAFKA_CONSOLE_CONSUMER}", "--topic", topic, "--property", "'print.key=true'", "--property", "'key.separator=:'", "--from-beginning", "--max-messages", "1", "--timeout-ms", f"{constants.CLIENT_TIMEOUT}"] + command = [f"{self.FIXTURES_DIR}/{constants.KAFKA_CONSOLE_CONSUMER}", "--topic", topic, "--formatter-property", "'print.key=true'", "--formatter-property", "'key.separator=:'", "--from-beginning", "--max-messages", "1", "--timeout-ms", f"{constants.CLIENT_TIMEOUT}"] command.extend(consumer_config) message = subprocess.check_output(["bash", "-c", " ".join(command)]) return message.decode("utf-8").strip() @@ -93,9 +93,9 @@ class DockerSanityTest(unittest.TestCase): errors.append(constants.BROKER_METRICS_ERROR_PREFIX + str(e)) return errors - producer_config = ["--bootstrap-server", "localhost:9092", "--property", "client.id=host"] + producer_config = ["--bootstrap-server", "localhost:9092", "--command-property", "client.id=host"] self.produce_message(constants.BROKER_METRICS_TEST_TOPIC, producer_config, "key", "message") - consumer_config = ["--bootstrap-server", "localhost:9092", "--property", "auto.offset.reset=earliest"] + consumer_config = ["--bootstrap-server", "localhost:9092", "--command-property", "auto.offset.reset=earliest"] message = self.consume_message(constants.BROKER_METRICS_TEST_TOPIC, consumer_config) try: self.assertEqual(message, "key:message") @@ -129,13 +129,13 @@ class DockerSanityTest(unittest.TestCase): return errors producer_config = ["--bootstrap-server", ssl_broker_port, - "--producer.config", f"{self.FIXTURES_DIR}/{constants.SSL_CLIENT_CONFIG}"] + "--command-config", f"{self.FIXTURES_DIR}/{constants.SSL_CLIENT_CONFIG}"] self.produce_message(topic, producer_config, "key", "message") consumer_config = [ "--bootstrap-server", ssl_broker_port, - "--property", "auto.offset.reset=earliest", - "--consumer.config", f"{self.FIXTURES_DIR}/{constants.SSL_CLIENT_CONFIG}", + "--command-property", "auto.offset.reset=earliest", + "--command-config", f"{self.FIXTURES_DIR}/{constants.SSL_CLIENT_CONFIG}", ] message = self.consume_message(topic, consumer_config) try: @@ -155,7 +155,7 @@ class DockerSanityTest(unittest.TestCase): errors.append(constants.BROKER_RESTART_ERROR_PREFIX + str(e)) return errors - producer_config = ["--bootstrap-server", "localhost:9092", "--property", "client.id=host"] + producer_config = ["--bootstrap-server", "localhost:9092", "--command-property", "client.id=host"] self.produce_message(constants.BROKER_RESTART_TEST_TOPIC, producer_config, "key", "message") print("Stopping Container") @@ -163,7 +163,7 @@ class DockerSanityTest(unittest.TestCase): print("Resuming Container") self.resume_container() - consumer_config = ["--bootstrap-server", "localhost:9092", "--property", "auto.offset.reset=earliest"] + consumer_config = ["--bootstrap-server", "localhost:9092", "--command-property", "auto.offset.reset=earliest"] message = self.consume_message(constants.BROKER_RESTART_TEST_TOPIC, consumer_config) try: self.assertEqual(message, "key:message") diff --git a/docs/security.html b/docs/security.html index 5940fc3cda6..9364a05e40a 100644 --- a/docs/security.html +++ b/docs/security.html @@ -510,8 +510,8 @@ ssl.key.password=test1234
    Examples using console-producer and console-consumer: -
    $ bin/kafka-console-producer.sh --bootstrap-server localhost:9093 --topic test --producer.config client-ssl.properties
    -$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9093 --topic test --consumer.config client-ssl.properties
    +
    $ bin/kafka-console-producer.sh --bootstrap-server localhost:9093 --topic test --command-config client-ssl.properties
    +$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9093 --topic test --command-config client-ssl.properties

    7.4 Authentication using SASL

    diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java index 7037ce1368d..aa283726134 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java @@ -1122,15 +1122,15 @@ public class KStreamAggregationIntegrationTest { final String[] args = new String[] { "--bootstrap-server", CLUSTER.bootstrapServers(), "--from-beginning", - "--property", "print.key=true", - "--property", "print.timestamp=" + printTimestamp, + "--formatter-property", "print.key=true", + "--formatter-property", "print.timestamp=" + printTimestamp, "--topic", outputTopic, "--max-messages", String.valueOf(numMessages), - "--property", "key.deserializer=" + keyDeserializer.getClass().getName(), - "--property", "value.deserializer=" + valueDeserializer.getClass().getName(), - "--property", "key.separator=" + keySeparator, - "--property", "key.deserializer." + TimeWindowedDeserializer.WINDOWED_INNER_DESERIALIZER_CLASS + "=" + Serdes.serdeFrom(innerClass).getClass().getName(), - "--property", "key.deserializer.window.size.ms=500", + "--formatter-property", "key.deserializer=" + keyDeserializer.getClass().getName(), + "--formatter-property", "value.deserializer=" + valueDeserializer.getClass().getName(), + "--formatter-property", "key.separator=" + keySeparator, + "--formatter-property", "key.deserializer." + TimeWindowedDeserializer.WINDOWED_INNER_DESERIALIZER_CLASS + "=" + Serdes.serdeFrom(innerClass).getClass().getName(), + "--formatter-property", "key.deserializer.window.size.ms=500", }; ConsoleConsumer.run(new ConsoleConsumerOptions(args)); diff --git a/tests/kafkatest/services/console_consumer.py b/tests/kafkatest/services/console_consumer.py index 9755faa1969..0f95a0f7c79 100644 --- a/tests/kafkatest/services/console_consumer.py +++ b/tests/kafkatest/services/console_consumer.py @@ -88,7 +88,7 @@ class ConsoleConsumer(KafkaPathResolverMixin, JmxMixin, BackgroundThreadService) jaas_override_variables A dict of variables to be used in the jaas.conf template file kafka_opts_override Override parameters of the KAFKA_OPTS environment variable client_prop_file_override Override client.properties file used by the consumer - consumer_properties A dict of values to pass in as --consumer-property key=value + consumer_properties A dict of values to pass in as --command-property key=value. For versions older than KAFKA_4_2_0, these will be passed as --consumer-property key=value """ JmxMixin.__init__(self, num_nodes=num_nodes, jmx_object_names=jmx_object_names, jmx_attributes=(jmx_attributes or []), root=ConsoleConsumer.PERSISTENT_ROOT) @@ -163,8 +163,11 @@ class ConsoleConsumer(KafkaPathResolverMixin, JmxMixin, BackgroundThreadService) "export KAFKA_LOG4J_OPTS=\"%(log4j_param)s%(log4j_config)s\"; " \ "export KAFKA_OPTS=%(kafka_opts)s; " \ "%(console_consumer)s " \ - "--topic %(topic)s " \ - "--consumer.config %(config_file)s " % args + "--topic %(topic)s " % args + + version = get_version(node) + command_config_arg = "--command-config" if version.supports_command_config() else "--consumer.config" + cmd += "%s %s" % (command_config_arg, args['config_file']) cmd += " --bootstrap-server %(broker_list)s" % args cmd += " --isolation-level %s" % self.isolation_level @@ -176,14 +179,15 @@ class ConsoleConsumer(KafkaPathResolverMixin, JmxMixin, BackgroundThreadService) # This will be added in the properties file instead cmd += " --timeout-ms %s" % self.consumer_timeout_ms + formatter_property_arg = "--formatter-property" if version.supports_formatter_property else "--property" if self.print_timestamp: - cmd += " --property print.timestamp=true" + cmd += " %s print.timestamp=true" % formatter_property_arg if self.print_key: - cmd += " --property print.key=true" + cmd += " %s print.key=true" % formatter_property_arg if self.print_partition: - cmd += " --property print.partition=true" + cmd += " %s print.partition=true" % formatter_property_arg # LoggingMessageFormatter was introduced after 0.9 if node.version > LATEST_3_7: @@ -194,9 +198,10 @@ class ConsoleConsumer(KafkaPathResolverMixin, JmxMixin, BackgroundThreadService) if self.enable_systest_events: cmd += " --enable-systest-events" + command_property_arg = "--command-property" if version.supports_command_property() else "--consumer-property" if self.consumer_properties is not None: for k, v in self.consumer_properties.items(): - cmd += " --consumer-property %s=%s" % (k, v) + cmd += " %s %s=%s" % (command_property_arg, k, v) cmd += " 2>> %(stderr)s | tee -a %(stdout)s &" % args return cmd diff --git a/tests/kafkatest/services/console_share_consumer.py b/tests/kafkatest/services/console_share_consumer.py index 03fbaeaf5a5..2d7da50fe30 100644 --- a/tests/kafkatest/services/console_share_consumer.py +++ b/tests/kafkatest/services/console_share_consumer.py @@ -21,7 +21,7 @@ from ducktape.utils.util import wait_until from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin from kafkatest.services.monitor.jmx import JmxMixin -from kafkatest.version import DEV_BRANCH, LATEST_4_1 +from kafkatest.version import DEV_BRANCH, LATEST_4_1, get_version from kafkatest.services.kafka.util import fix_opts_for_new_jvm, get_log4j_config_param, get_log4j_config_for_tools """ @@ -84,7 +84,7 @@ class ConsoleShareConsumer(KafkaPathResolverMixin, JmxMixin, BackgroundThreadSer jaas_override_variables A dict of variables to be used in the jaas.conf template file kafka_opts_override Override parameters of the KAFKA_OPTS environment variable client_prop_file_override Override client.properties file used by the consumer - share_consumer_properties A dict of values to pass in as --consumer-property key=value + share_consumer_properties A dict of values to pass in as --command-property key=value. For versions older than KAFKA_4_2_0, these will be passed as --consumer-property key=value """ JmxMixin.__init__(self, num_nodes=num_nodes, jmx_object_names=jmx_object_names, jmx_attributes=(jmx_attributes or []), root=ConsoleShareConsumer.PERSISTENT_ROOT) @@ -156,31 +156,36 @@ class ConsoleShareConsumer(KafkaPathResolverMixin, JmxMixin, BackgroundThreadSer "export KAFKA_LOG4J_OPTS=\"%(log4j_param)s%(log4j_config)s\"; " \ "export KAFKA_OPTS=%(kafka_opts)s; " \ "%(console_share_consumer)s " \ - "--topic %(topic)s " \ - "--consumer-config %(config_file)s " % args + "--topic %(topic)s " % args + + version = get_version(node) + command_config_arg = "--command-config" if version.supports_command_config() else "--consumer-config" + cmd += "%s %s" % (command_config_arg, args['config_file']) cmd += " --bootstrap-server %(broker_list)s" % args if self.share_consumer_timeout_ms is not None: # This will be added in the properties file instead cmd += " --timeout-ms %s" % self.share_consumer_timeout_ms + formatter_property_arg = "--formatter-property" if version.supports_formatter_property else "--property" if self.print_timestamp: - cmd += " --property print.timestamp=true" + cmd += " %s print.timestamp=true" % formatter_property_arg if self.print_key: - cmd += " --property print.key=true" + cmd += " %s print.key=true" % formatter_property_arg if self.print_partition: - cmd += " --property print.partition=true" + cmd += " %s print.partition=true" % formatter_property_arg cmd += " --formatter org.apache.kafka.tools.consumer.LoggingMessageFormatter" if self.enable_systest_events: cmd += " --enable-systest-events" + command_property_arg = "--command-property" if version.supports_command_property() else "--consumer-property" if self.share_consumer_properties is not None: for k, v in self.share_consumer_properties.items(): - cmd += " --consumer-property %s=%s" % (k, v) + cmd += " %s %s=%s" % (command_property_arg, k, v) cmd += " 2>> %(stderr)s | tee -a %(stdout)s &" % args return cmd diff --git a/tests/kafkatest/version.py b/tests/kafkatest/version.py index e4d600ac403..8114d51861e 100644 --- a/tests/kafkatest/version.py +++ b/tests/kafkatest/version.py @@ -104,6 +104,20 @@ class KafkaVersion(LooseVersion): # - For older versions, continue using --producer.config or --consumer.config return self >= V_4_2_0 + def supports_command_property(self): + # According to KIP-1147, --producer-property and --consumer-property have been deprecated and will be removed in future versions + # For backward compatibility, we select the configuration based on node version: + # - For versions 4.2.0 and above, use --command-property + # - For older versions, continue using --producer-property or --consumer-property + return self >= V_4_2_0 + + def supports_formatter_property(self): + # According to KIP-1147, --property has been deprecated and will be removed in future versions + # For backward compatibility, we select the configuration based on node version: + # - For versions 4.2.0 and above, use --formatter-property + # - For older versions, continue using --property + return self >= V_4_2_0 + def get_version(node=None): """Return the version attached to the given node. Default to DEV_BRANCH if node or node.version is undefined (aka None) diff --git a/tools/src/main/java/org/apache/kafka/tools/ConsoleProducer.java b/tools/src/main/java/org/apache/kafka/tools/ConsoleProducer.java index 93e81bd5a43..3e9e34f4210 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ConsoleProducer.java +++ b/tools/src/main/java/org/apache/kafka/tools/ConsoleProducer.java @@ -131,8 +131,12 @@ public class ConsoleProducer { private final OptionSpec socketBufferSizeOpt; private final OptionSpec propertyOpt; private final OptionSpec readerConfigOpt; + @Deprecated(since = "4.2", forRemoval = true) private final OptionSpec producerPropertyOpt; + private OptionSpec commandPropertyOpt; + @Deprecated(since = "4.2", forRemoval = true) private final OptionSpec producerConfigOpt; + private OptionSpec commandConfigOpt; public ConsoleProducerOptions(String[] args) { super(args); @@ -250,11 +254,20 @@ public class ConsoleProducer { .withRequiredArg() .describedAs("config file") .ofType(String.class); - producerPropertyOpt = parser.accepts("producer-property", "A mechanism to pass user-defined properties in the form key=value to the producer. ") + producerPropertyOpt = parser.accepts("producer-property", "(DEPRECATED) A mechanism to pass user-defined properties in the form key=value to the producer." + + "This option will be removed in a future version. Use --command-property instead.") .withRequiredArg() .describedAs("producer_prop") .ofType(String.class); - producerConfigOpt = parser.accepts("producer.config", "Producer config properties file. Note that " + producerPropertyOpt + " takes precedence over this config.") + commandPropertyOpt = parser.accepts("command-property", "A mechanism to pass user-defined properties in the form key=value to the producer.") + .withRequiredArg() + .describedAs("producer_prop") + .ofType(String.class); + producerConfigOpt = parser.accepts("producer.config", "(DEPRECATED) Producer config properties file. Note that " + commandPropertyOpt + " takes precedence over this config. This option will be removed in a future version. Use --command-config instead.") + .withRequiredArg() + .describedAs("config file") + .ofType(String.class); + commandConfigOpt = parser.accepts("command-config", "Producer config properties file. Note that " + commandPropertyOpt + " takes precedence over this config.") .withRequiredArg() .describedAs("config file") .ofType(String.class); @@ -273,6 +286,23 @@ public class ConsoleProducer { CommandLineUtils.checkRequiredArgs(parser, options, topicOpt); + if (options.has(commandConfigOpt) && options.has(producerConfigOpt)) { + CommandLineUtils.printUsageAndExit(parser, "Options --command-config and --producer.config cannot be specified together."); + } + if (options.has(commandPropertyOpt) && options.has(producerPropertyOpt)) { + CommandLineUtils.printUsageAndExit(parser, "Options --command-property and --producer-property cannot be specified together."); + } + + if (options.has(producerPropertyOpt)) { + System.out.println("Warning: --producer-property is deprecated and will be removed in a future version. Use --command-property instead."); + commandPropertyOpt = producerPropertyOpt; + } + + if (options.has(producerConfigOpt)) { + System.out.println("Warning: --producer.config is deprecated and will be removed in a future version. Use --command-config instead."); + commandConfigOpt = producerConfigOpt; + } + try { ToolsUtils.validateBootstrapServer(options.valueOf(bootstrapServerOpt)); } catch (IllegalArgumentException e) { @@ -314,11 +344,11 @@ public class ConsoleProducer { Properties producerProps() throws IOException { Properties props = new Properties(); - if (options.has(producerConfigOpt)) { - props.putAll(loadProps(options.valueOf(producerConfigOpt))); + if (options.has(commandConfigOpt)) { + props.putAll(loadProps(options.valueOf(commandConfigOpt))); } - props.putAll(parseKeyValueArgs(options.valuesOf(producerPropertyOpt))); + props.putAll(parseKeyValueArgs(options.valuesOf(commandPropertyOpt))); props.put(BOOTSTRAP_SERVERS_CONFIG, options.valueOf(bootstrapServerOpt)); props.put(COMPRESSION_TYPE_CONFIG, compressionCodec()); diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java index fe33bfe6c68..abe6322fd97 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java @@ -48,7 +48,9 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions { private final OptionSpec partitionIdOpt; private final OptionSpec offsetOpt; private final OptionSpec messageFormatterOpt; - private final OptionSpec messageFormatterArgOpt; + @Deprecated(since = "4.2", forRemoval = true) + private final OptionSpec messageFormatterArgOptDeprecated; + private OptionSpec messageFormatterArgOpt; private final OptionSpec messageFormatterConfigOpt; private final OptionSpec resetBeginningOpt; private final OptionSpec maxMessagesOpt; @@ -66,6 +68,7 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions { private final long timeoutMs; private final MessageFormatter formatter; + @SuppressWarnings("deprecation") public ConsoleConsumerOptions(String[] args) throws IOException { super(args); topicOpt = parser.accepts("topic", "The topic to consume on.") @@ -87,11 +90,23 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions { .describedAs("consume offset") .ofType(String.class) .defaultsTo("latest"); - OptionSpec consumerPropertyOpt = parser.accepts("consumer-property", "A mechanism to pass user-defined properties in the form key=value to the consumer.") + @Deprecated(since = "4.2", forRemoval = true) + OptionSpec consumerPropertyOpt = parser.accepts("consumer-property", "(DEPRECATED) A mechanism to pass user-defined properties in the form key=value to the consumer. " + + "This option will be removed in a future version. Use --command-property instead.") .withRequiredArg() .describedAs("consumer_prop") .ofType(String.class); - OptionSpec consumerConfigOpt = parser.accepts("consumer.config", "Consumer config properties file. Note that " + consumerPropertyOpt + " takes precedence over this config.") + OptionSpec commandPropertyOpt = parser.accepts("command-property", "A mechanism to pass user-defined properties in the form key=value to the consumer.") + .withRequiredArg() + .describedAs("consumer_prop") + .ofType(String.class); + @Deprecated(since = "4.2", forRemoval = true) + OptionSpec consumerConfigOpt = parser.accepts("consumer.config", "(DEPRECATED) Consumer config properties file. Note that " + commandPropertyOpt + " takes precedence over this config. " + + "This option will be removed in a future version. Use --command-config instead.") + .withRequiredArg() + .describedAs("config file") + .ofType(String.class); + OptionSpec commandConfigOpt = parser.accepts("command-config", "Consumer config properties file. Note that " + commandPropertyOpt + " takes precedence over this config.") .withRequiredArg() .describedAs("config file") .ofType(String.class); @@ -100,7 +115,28 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions { .describedAs("class") .ofType(String.class) .defaultsTo(DefaultMessageFormatter.class.getName()); - messageFormatterArgOpt = parser.accepts("property", + messageFormatterArgOptDeprecated = parser.accepts("property", + "(DEPRECATED) The properties to initialize the message formatter. Default properties include: \n" + + " print.timestamp=true|false\n" + + " print.key=true|false\n" + + " print.offset=true|false\n" + + " print.epoch=true|false\n" + + " print.partition=true|false\n" + + " print.headers=true|false\n" + + " print.value=true|false\n" + + " key.separator=\n" + + " line.separator=\n" + + " headers.separator=\n" + + " null.literal=\n" + + " key.deserializer=\n" + + " value.deserializer=\n" + + " header.deserializer=\n" + + "\nUsers can also pass in customized properties for their formatter; more specifically, users can pass in properties keyed with 'key.deserializer.', 'value.deserializer.' and 'headers.deserializer.' prefixes to configure their deserializers. " + + "\nThis option will be removed in a future version. Use --formatter-property instead.") + .withRequiredArg() + .describedAs("prop") + .ofType(String.class); + messageFormatterArgOpt = parser.accepts("formatter-property", "The properties to initialize the message formatter. Default properties include: \n" + " print.timestamp=true|false\n" + " print.key=true|false\n" + @@ -170,11 +206,25 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions { CommandLineUtils.maybePrintHelpOrVersion(this, "This tool helps to read data from Kafka topics and outputs it to standard output."); checkRequiredArgs(); + if (options.has(consumerPropertyOpt) && options.has(commandPropertyOpt)) { + CommandLineUtils.printUsageAndExit(parser, "Options --consumer-property and --command-property cannot be specified together."); + } + if (options.has(consumerConfigOpt) && options.has(commandConfigOpt)) { + CommandLineUtils.printUsageAndExit(parser, "Options --consumer.config and --command-config cannot be specified together."); + } - Properties consumerPropsFromFile = options.has(consumerConfigOpt) - ? Utils.loadProps(options.valueOf(consumerConfigOpt)) + if (options.has(consumerPropertyOpt)) { + System.out.println("Option --consumer-property is deprecated and will be removed in a future version. Use --command-property instead."); + commandPropertyOpt = consumerPropertyOpt; + } + if (options.has(consumerConfigOpt)) { + System.out.println("Option --consumer.config is deprecated and will be removed in a future version. Use --command-config instead."); + commandConfigOpt = consumerConfigOpt; + } + Properties consumerPropsFromFile = options.has(commandConfigOpt) + ? Utils.loadProps(options.valueOf(commandConfigOpt)) : new Properties(); - Properties extraConsumerProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(consumerPropertyOpt)); + Properties extraConsumerProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(commandPropertyOpt)); Set groupIdsProvided = checkConsumerGroup(consumerPropsFromFile, extraConsumerProps); consumerProps = buildConsumerProps(consumerPropsFromFile, extraConsumerProps, groupIdsProvided); offset = parseOffset(); @@ -323,6 +373,13 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions { Class messageFormatterClass = Class.forName(options.valueOf(messageFormatterOpt)); formatter = (MessageFormatter) messageFormatterClass.getDeclaredConstructor().newInstance(); + if (options.has(messageFormatterArgOpt) && options.has(messageFormatterArgOptDeprecated)) { + CommandLineUtils.printUsageAndExit(parser, "Options --property and --formatter-property cannot be specified together."); + } + if (options.has(messageFormatterArgOptDeprecated)) { + System.out.println("Option --property is deprecated and will be removed in a future version. Use --formatter-property instead."); + messageFormatterArgOpt = messageFormatterArgOptDeprecated; + } Properties formatterArgs = formatterArgs(); Map formatterConfigs = new HashMap<>(); for (final String name : formatterArgs.stringPropertyNames()) { diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptions.java b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptions.java index f00407b25eb..3472d07afa5 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptions.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptions.java @@ -37,7 +37,9 @@ import joptsimple.OptionSpec; public final class ConsoleShareConsumerOptions extends CommandDefaultOptions { private final OptionSpec messageFormatterOpt; private final OptionSpec messageFormatterConfigOpt; - private final OptionSpec messageFormatterArgOpt; + @Deprecated(since = "4.2", forRemoval = true) + private final OptionSpec messageFormatterArgOptDeprecated; + private OptionSpec messageFormatterArgOpt; private final OptionSpec keyDeserializerOpt; private final OptionSpec valueDeserializerOpt; private final OptionSpec maxMessagesOpt; @@ -52,17 +54,30 @@ public final class ConsoleShareConsumerOptions extends CommandDefaultOptions { private final MessageFormatter formatter; private final OptionSpec enableSystestEventsLoggingOpt; + @SuppressWarnings("deprecation") public ConsoleShareConsumerOptions(String[] args) throws IOException { super(args); topicOpt = parser.accepts("topic", "The topic to consume from.") .withRequiredArg() .describedAs("topic") .ofType(String.class); - OptionSpec consumerPropertyOpt = parser.accepts("consumer-property", "A mechanism to pass user-defined properties in the form key=value to the consumer.") + @Deprecated(since = "4.2", forRemoval = true) + OptionSpec consumerPropertyOpt = parser.accepts("consumer-property", "(DEPRECATED) A mechanism to pass user-defined properties in the form key=value to the consumer. " + + "This option will be removed in a future version. Use --command-property instead.") .withRequiredArg() .describedAs("consumer_prop") .ofType(String.class); - OptionSpec consumerConfigOpt = parser.accepts("consumer-config", "Consumer config properties file. Note that " + consumerPropertyOpt + " takes precedence over this config.") + OptionSpec commandPropertyOpt = parser.accepts("command-property", "A mechanism to pass user-defined properties in the form key=value to the consumer.") + .withRequiredArg() + .describedAs("consumer_prop") + .ofType(String.class); + @Deprecated(since = "4.2", forRemoval = true) + OptionSpec consumerConfigOpt = parser.accepts("consumer-config", "(DEPRECATED) Consumer config properties file. Note that " + commandPropertyOpt + " takes precedence over this config. " + + "This option will be removed in a future version. Use --command-config instead.") + .withRequiredArg() + .describedAs("config file") + .ofType(String.class); + OptionSpec commandConfigOpt = parser.accepts("command-config", "Consumer config properties file. Note that " + commandPropertyOpt + " takes precedence over this config.") .withRequiredArg() .describedAs("config file") .ofType(String.class); @@ -71,7 +86,29 @@ public final class ConsoleShareConsumerOptions extends CommandDefaultOptions { .describedAs("class") .ofType(String.class) .defaultsTo(DefaultMessageFormatter.class.getName()); - messageFormatterArgOpt = parser.accepts("property", + messageFormatterArgOptDeprecated = parser.accepts("property", + "(DEPRECATED) The properties to initialize the message formatter. Default properties include: \n" + + " print.timestamp=true|false\n" + + " print.key=true|false\n" + + " print.offset=true|false\n" + + " print.delivery=true|false\n" + + " print.epoch=true|false\n" + + " print.partition=true|false\n" + + " print.headers=true|false\n" + + " print.value=true|false\n" + + " key.separator=\n" + + " line.separator=\n" + + " headers.separator=\n" + + " null.literal=\n" + + " key.deserializer=\n" + + " value.deserializer=\n" + + " header.deserializer=\n" + + "\nUsers can also pass in customized properties for their formatter; more specifically, users can pass in properties keyed with 'key.deserializer.', 'value.deserializer.' and 'headers.deserializer.' prefixes to configure their deserializers. " + + "\nThis option will be removed in a future version. Use --formatter-property instead.") + .withRequiredArg() + .describedAs("prop") + .ofType(String.class); + messageFormatterArgOpt = parser.accepts("formatter-property", "The properties to initialize the message formatter. Default properties include: \n" + " print.timestamp=true|false\n" + " print.key=true|false\n" + @@ -141,10 +178,26 @@ public final class ConsoleShareConsumerOptions extends CommandDefaultOptions { CommandLineUtils.printUsageAndExit(parser, "At most one of --reject and --release may be specified."); } - Properties consumerPropsFromFile = options.has(consumerConfigOpt) - ? Utils.loadProps(options.valueOf(consumerConfigOpt)) + if (options.has(consumerPropertyOpt) && options.has(commandPropertyOpt)) { + CommandLineUtils.printUsageAndExit(parser, "Options --consumer-property and --command-property cannot be specified together."); + } + if (options.has(consumerConfigOpt) && options.has(commandConfigOpt)) { + CommandLineUtils.printUsageAndExit(parser, "Options --consumer-config and --command-config cannot be specified together."); + } + + if (options.has(consumerPropertyOpt)) { + System.out.println("Option --consumer-property is deprecated and will be removed in a future version. Use --command-property instead."); + commandPropertyOpt = consumerPropertyOpt; + } + if (options.has(consumerConfigOpt)) { + System.out.println("Option --consumer-config is deprecated and will be removed in a future version. Use --command-config instead."); + commandConfigOpt = consumerConfigOpt; + } + + Properties consumerPropsFromFile = options.has(commandConfigOpt) + ? Utils.loadProps(options.valueOf(commandConfigOpt)) : new Properties(); - Properties extraConsumerProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(consumerPropertyOpt)); + Properties extraConsumerProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(commandPropertyOpt)); Set groupIdsProvided = checkShareGroup(consumerPropsFromFile, extraConsumerProps); consumerProps = buildConsumerProps(consumerPropsFromFile, extraConsumerProps, groupIdsProvided); @@ -203,6 +256,13 @@ public final class ConsoleShareConsumerOptions extends CommandDefaultOptions { Class messageFormatterClass = Class.forName(options.valueOf(messageFormatterOpt)); formatter = (MessageFormatter) messageFormatterClass.getDeclaredConstructor().newInstance(); + if (options.has(messageFormatterArgOpt) && options.has(messageFormatterArgOptDeprecated)) { + CommandLineUtils.printUsageAndExit(parser, "Options --property and --formatter-property cannot be specified together."); + } + if (options.has(messageFormatterArgOptDeprecated)) { + System.out.println("Option --property is deprecated and will be removed in a future version. Use --formatter-property instead."); + messageFormatterArgOpt = messageFormatterArgOptDeprecated; + } Properties formatterArgs = formatterArgs(); Map formatterConfigs = new HashMap<>(); for (final String name : formatterArgs.stringPropertyNames()) { diff --git a/tools/src/test/java/org/apache/kafka/tools/ConsoleProducerTest.java b/tools/src/test/java/org/apache/kafka/tools/ConsoleProducerTest.java index 6752aef29c7..e1a519ac6de 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ConsoleProducerTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ConsoleProducerTest.java @@ -34,6 +34,7 @@ import java.io.InputStream; import java.io.OutputStream; import java.nio.file.Files; import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -57,11 +58,16 @@ public class ConsoleProducerTest { "--bootstrap-server", "localhost:1002", "--topic", "t3", }; - private static final String[] CLIENT_ID_OVERRIDE = new String[]{ + private static final String[] CLIENT_ID_OVERRIDE_DEPRECATED = new String[]{ "--bootstrap-server", "localhost:1001", "--topic", "t3", "--producer-property", "client.id=producer-1" }; + private static final String[] CLIENT_ID_OVERRIDE = new String[]{ + "--bootstrap-server", "localhost:1001", + "--topic", "t3", + "--command-property", "client.id=producer-1" + }; private static final String[] BATCH_SIZE_OVERRIDDEN_BY_MAX_PARTITION_MEMORY_BYTES_VALUE = new String[]{ "--bootstrap-server", "localhost:1002", "--topic", "t3", @@ -151,8 +157,8 @@ public class ConsoleProducerTest { } @Test - public void testClientIdOverride() throws IOException { - ConsoleProducerOptions opts = new ConsoleProducerOptions(CLIENT_ID_OVERRIDE); + public void testClientIdOverrideDeprecated() throws IOException { + ConsoleProducerOptions opts = new ConsoleProducerOptions(CLIENT_ID_OVERRIDE_DEPRECATED); ProducerConfig producerConfig = new ProducerConfig(opts.producerProps()); assertEquals("producer-1", producerConfig.getString(ProducerConfig.CLIENT_ID_CONFIG)); @@ -222,6 +228,107 @@ public class ConsoleProducerTest { assertEquals(1, reader.closeCount()); } + @Test + public void shouldExitOnBothProducerPropertyAndCommandProperty() { + Exit.setExitProcedure((code, message) -> { + throw new IllegalArgumentException(message); + }); + + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--producer-property", "acks=all", + "--command-property", "batch.size=16384" + }; + + try { + assertThrows(IllegalArgumentException.class, () -> new ConsoleProducerOptions(args)); + } finally { + Exit.resetExitProcedure(); + } + } + + @Test + public void shouldExitOnBothProducerConfigAndCommandConfig() throws IOException { + Exit.setExitProcedure((code, message) -> { + throw new IllegalArgumentException(message); + }); + + Map configs = new HashMap<>(); + configs.put("acks", "all"); + File propsFile = ToolsTestUtils.tempPropertiesFile(configs); + + Map configs2 = new HashMap<>(); + configs2.put("batch.size", "16384"); + File propsFile2 = ToolsTestUtils.tempPropertiesFile(configs2); + + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--producer.config", propsFile.getAbsolutePath(), + "--command-config", propsFile2.getAbsolutePath() + }; + + try { + assertThrows(IllegalArgumentException.class, () -> new ConsoleProducerOptions(args)); + } finally { + Exit.resetExitProcedure(); + } + } + + @Test + public void testClientIdOverrideUsingCommandProperty() throws IOException { + ConsoleProducerOptions opts = new ConsoleProducerOptions(CLIENT_ID_OVERRIDE); + ProducerConfig producerConfig = new ProducerConfig(opts.producerProps()); + + assertEquals("producer-1", producerConfig.getString(ProducerConfig.CLIENT_ID_CONFIG)); + } + + @Test + public void testProducerConfigFromFileUsingCommandConfig() throws IOException { + Map configs = new HashMap<>(); + configs.put("acks", "all"); + configs.put("batch.size", "32768"); + File propsFile = ToolsTestUtils.tempPropertiesFile(configs); + + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--command-config", propsFile.getAbsolutePath() + }; + + ConsoleProducerOptions opts = new ConsoleProducerOptions(args); + ProducerConfig producerConfig = new ProducerConfig(opts.producerProps()); + + // "all" gets converted to "-1" internally by ProducerConfig + assertEquals("-1", producerConfig.getString(ProducerConfig.ACKS_CONFIG)); + assertEquals(32768, producerConfig.getInt(ProducerConfig.BATCH_SIZE_CONFIG)); + } + + @Test + public void testCommandPropertyOverridesConfig() throws IOException { + Map configs = new HashMap<>(); + configs.put("acks", "1"); + configs.put("batch.size", "16384"); + File propsFile = ToolsTestUtils.tempPropertiesFile(configs); + + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--command-config", propsFile.getAbsolutePath(), + "--command-property", "acks=all" + }; + + ConsoleProducerOptions opts = new ConsoleProducerOptions(args); + ProducerConfig producerConfig = new ProducerConfig(opts.producerProps()); + + // Command property should override the config file value + // "all" gets converted to "-1" internally by ProducerConfig + assertEquals("-1", producerConfig.getString(ProducerConfig.ACKS_CONFIG)); + // Config file value should still be present + assertEquals(16384, producerConfig.getInt(ProducerConfig.BATCH_SIZE_CONFIG)); + } + public static class TestRecordReader implements RecordReader { private int configureCount = 0; private int closeCount = 0; diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java index 02ded3d5ca5..4639ff63a85 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java @@ -172,7 +172,7 @@ public class ConsoleConsumerOptionsTest { } @Test - public void shouldParseValidConsumerConfigWithAutoOffsetResetLatest() throws IOException { + public void shouldParseValidConsumerConfigWithAutoOffsetResetLatestDeprecated() throws IOException { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", @@ -189,7 +189,7 @@ public class ConsoleConsumerOptionsTest { } @Test - public void shouldParseValidConsumerConfigWithAutoOffsetResetEarliest() throws IOException { + public void shouldParseValidConsumerConfigWithAutoOffsetResetEarliestDeprecated() throws IOException { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", @@ -206,7 +206,7 @@ public class ConsoleConsumerOptionsTest { } @Test - public void shouldParseValidConsumerConfigWithAutoOffsetResetAndMatchingFromBeginning() throws IOException { + public void shouldParseValidConsumerConfigWithAutoOffsetResetAndMatchingFromBeginningDeprecated() throws IOException { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", @@ -240,7 +240,7 @@ public class ConsoleConsumerOptionsTest { } @Test - public void shouldExitOnInvalidConfigWithAutoOffsetResetAndConflictingFromBeginning() { + public void shouldExitOnInvalidConfigWithAutoOffsetResetAndConflictingFromBeginningDeprecated() { Exit.setExitProcedure((code, message) -> { throw new IllegalArgumentException(message); }); @@ -259,7 +259,7 @@ public class ConsoleConsumerOptionsTest { } @Test - public void shouldParseConfigsFromFile() throws IOException { + public void shouldParseConfigsFromFileDeprecated() throws IOException { Map configs = new HashMap<>(); configs.put("request.timeout.ms", "1000"); configs.put("group.id", "group1"); @@ -276,80 +276,82 @@ public class ConsoleConsumerOptionsTest { } @Test - public void groupIdsProvidedInDifferentPlacesMustMatch() throws IOException { + public void groupIdsProvidedInDifferentPlacesMustMatchDeprecated() throws IOException { Exit.setExitProcedure((code, message) -> { throw new IllegalArgumentException(message); }); + try { - // different in all three places - File propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); - final String[] args = new String[]{ - "--bootstrap-server", "localhost:9092", - "--topic", "test", - "--group", "group-from-arguments", - "--consumer-property", "group.id=group-from-properties", - "--consumer.config", propsFile.getAbsolutePath() - }; + // different in all three places + File propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); + final String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--group", "group-from-arguments", + "--consumer-property", "group.id=group-from-properties", + "--consumer.config", propsFile.getAbsolutePath() + }; - assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args)); + assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args)); - // the same in all three places - propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "test-group")); - final String[] args1 = new String[]{ - "--bootstrap-server", "localhost:9092", - "--topic", "test", - "--group", "test-group", - "--consumer-property", "group.id=test-group", - "--consumer.config", propsFile.getAbsolutePath() - }; + // the same in all three places + propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "test-group")); + final String[] args1 = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--group", "test-group", + "--consumer-property", "group.id=test-group", + "--consumer.config", propsFile.getAbsolutePath() + }; - ConsoleConsumerOptions config = new ConsoleConsumerOptions(args1); - Properties props = config.consumerProps(); - assertEquals("test-group", props.getProperty("group.id")); + ConsoleConsumerOptions config = new ConsoleConsumerOptions(args1); + Properties props = config.consumerProps(); + assertEquals("test-group", props.getProperty("group.id")); - // different via --consumer-property and --consumer.config - propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); - final String[] args2 = new String[]{ - "--bootstrap-server", "localhost:9092", - "--topic", "test", - "--consumer-property", "group.id=group-from-properties", - "--consumer.config", propsFile.getAbsolutePath() - }; + // different via --consumer-property and --consumer.config + propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); + final String[] args2 = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--consumer-property", "group.id=group-from-properties", + "--consumer.config", propsFile.getAbsolutePath() + }; - assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args2)); + assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args2)); - // different via --consumer-property and --group - final String[] args3 = new String[]{ - "--bootstrap-server", "localhost:9092", - "--topic", "test", - "--group", "group-from-arguments", - "--consumer-property", "group.id=group-from-properties" - }; + // different via --consumer-property and --group + final String[] args3 = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--group", "group-from-arguments", + "--consumer-property", "group.id=group-from-properties" + }; - assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args3)); + assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args3)); - // different via --group and --consumer.config - propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); - final String[] args4 = new String[]{ - "--bootstrap-server", "localhost:9092", - "--topic", "test", - "--group", "group-from-arguments", - "--consumer.config", propsFile.getAbsolutePath() - }; - assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args4)); + // different via --group and --consumer.config + propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); + final String[] args4 = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--group", "group-from-arguments", + "--consumer.config", propsFile.getAbsolutePath() + }; + assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args4)); - // via --group only - final String[] args5 = new String[]{ - "--bootstrap-server", "localhost:9092", - "--topic", "test", - "--group", "group-from-arguments" - }; + // via --group only + final String[] args5 = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--group", "group-from-arguments" + }; - config = new ConsoleConsumerOptions(args5); - props = config.consumerProps(); - assertEquals("group-from-arguments", props.getProperty("group.id")); - - Exit.resetExitProcedure(); + config = new ConsoleConsumerOptions(args5); + props = config.consumerProps(); + assertEquals("group-from-arguments", props.getProperty("group.id")); + } finally { + Exit.resetExitProcedure(); + } } @Test @@ -508,7 +510,7 @@ public class ConsoleConsumerOptionsTest { } @Test - public void testClientIdOverride() throws IOException { + public void testClientIdOverrideDeprecated() throws IOException { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", @@ -618,4 +620,234 @@ public class ConsoleConsumerOptionsTest { "--formatter", formatter, }; } + + @Test + public void shouldExitOnBothConsumerPropertyAndCommandProperty() { + Exit.setExitProcedure((code, message) -> { + throw new IllegalArgumentException(message); + }); + + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--consumer-property", "auto.offset.reset=latest", + "--command-property", "session.timeout.ms=10000" + }; + + try { + assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args)); + } finally { + Exit.resetExitProcedure(); + } + } + + @Test + public void shouldExitOnBothConsumerConfigAndCommandConfig() throws IOException { + Exit.setExitProcedure((code, message) -> { + throw new IllegalArgumentException(message); + }); + + Map configs = new HashMap<>(); + configs.put("request.timeout.ms", "1000"); + File propsFile = ToolsTestUtils.tempPropertiesFile(configs); + + Map configs2 = new HashMap<>(); + configs2.put("session.timeout.ms", "10000"); + File propsFile2 = ToolsTestUtils.tempPropertiesFile(configs2); + + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--consumer.config", propsFile.getAbsolutePath(), + "--command-config", propsFile2.getAbsolutePath() + }; + + try { + assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args)); + } finally { + Exit.resetExitProcedure(); + } + } + + @Test + public void shouldParseValidConsumerConfigWithAutoOffsetResetLatestUsingCommandProperty() throws IOException { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--command-property", "auto.offset.reset=latest" + }; + + ConsoleConsumerOptions config = new ConsoleConsumerOptions(args); + Properties consumerProperties = config.consumerProps(); + + assertEquals("localhost:9092", config.bootstrapServer()); + assertEquals("test", config.topicArg().orElse("")); + assertFalse(config.fromBeginning()); + assertEquals("latest", consumerProperties.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)); + } + + @Test + public void shouldParseValidConsumerConfigWithAutoOffsetResetEarliestUsingCommandProperty() throws IOException { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--command-property", "auto.offset.reset=earliest" + }; + + ConsoleConsumerOptions config = new ConsoleConsumerOptions(args); + Properties consumerProperties = config.consumerProps(); + + assertEquals("localhost:9092", config.bootstrapServer()); + assertEquals("test", config.topicArg().orElse("")); + assertFalse(config.fromBeginning()); + assertEquals("earliest", consumerProperties.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)); + } + + @Test + public void shouldParseValidConsumerConfigWithAutoOffsetResetAndMatchingFromBeginningUsingCommandProperty() throws IOException { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--command-property", "auto.offset.reset=earliest", + "--from-beginning" + }; + + ConsoleConsumerOptions config = new ConsoleConsumerOptions(args); + Properties consumerProperties = config.consumerProps(); + + assertEquals("localhost:9092", config.bootstrapServer()); + assertEquals("test", config.topicArg().orElse("")); + assertTrue(config.fromBeginning()); + assertEquals("earliest", consumerProperties.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)); + } + + @Test + public void shouldExitOnInvalidConfigWithAutoOffsetResetAndConflictingFromBeginningUsingCommandProperty() { + Exit.setExitProcedure((code, message) -> { + throw new IllegalArgumentException(message); + }); + + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--command-property", "auto.offset.reset=latest", + "--from-beginning" + }; + try { + assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args)); + } finally { + Exit.resetExitProcedure(); + } + } + + @Test + public void shouldParseConfigsFromFileUsingCommandConfig() throws IOException { + Map configs = new HashMap<>(); + configs.put("request.timeout.ms", "1000"); + configs.put("group.id", "group1"); + File propsFile = ToolsTestUtils.tempPropertiesFile(configs); + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--command-config", propsFile.getAbsolutePath() + }; + + ConsoleConsumerOptions config = new ConsoleConsumerOptions(args); + assertEquals("1000", config.consumerProps().get("request.timeout.ms")); + assertEquals("group1", config.consumerProps().get("group.id")); + } + + @Test + public void groupIdsProvidedInDifferentPlacesMustMatchUsingCommandConfig() throws IOException { + Exit.setExitProcedure((code, message) -> { + throw new IllegalArgumentException(message); + }); + + try { + // different in all three places + File propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); + final String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--group", "group-from-arguments", + "--command-property", "group.id=group-from-properties", + "--command-config", propsFile.getAbsolutePath() + }; + + assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args)); + + // the same in all three places + propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "test-group")); + final String[] args1 = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--group", "test-group", + "--command-property", "group.id=test-group", + "--command-config", propsFile.getAbsolutePath() + }; + + ConsoleConsumerOptions config = new ConsoleConsumerOptions(args1); + Properties props = config.consumerProps(); + assertEquals("test-group", props.getProperty("group.id")); + + // different via --command-property and --command-config + propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); + final String[] args2 = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--command-property", "group.id=group-from-properties", + "--command-config", propsFile.getAbsolutePath() + }; + + assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args2)); + + // different via --command-property and --group + final String[] args3 = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--group", "group-from-arguments", + "--command-property", "group.id=group-from-properties" + }; + + assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args3)); + + // different via --group and --command-config + propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); + final String[] args4 = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--group", "group-from-arguments", + "--command-config", propsFile.getAbsolutePath() + }; + assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args4)); + + // via --group only + final String[] args5 = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--group", "group-from-arguments" + }; + + config = new ConsoleConsumerOptions(args5); + props = config.consumerProps(); + assertEquals("group-from-arguments", props.getProperty("group.id")); + } finally { + Exit.resetExitProcedure(); + } + } + + @Test + public void testClientIdOverrideUsingCommandProperty() throws IOException { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--from-beginning", + "--command-property", "client.id=consumer-1" + }; + + ConsoleConsumerOptions config = new ConsoleConsumerOptions(args); + Properties consumerProperties = config.consumerProps(); + + assertEquals("consumer-1", consumerProperties.getProperty(ConsumerConfig.CLIENT_ID_CONFIG)); + } } diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptionsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptionsTest.java index c2b7b73c1a5..fecf53dbbec 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptionsTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptionsTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.clients.consumer.AcknowledgeType; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.test.MockDeserializer; import org.apache.kafka.tools.ToolsTestUtils; import org.junit.jupiter.api.Test; @@ -32,7 +33,9 @@ import java.util.Properties; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; public class ConsoleShareConsumerOptionsTest { @@ -72,7 +75,7 @@ public class ConsoleShareConsumerOptionsTest { } @Test - public void shouldParseValidConsumerConfigWithSessionTimeout() throws IOException { + public void shouldParseValidConsumerConfigWithSessionTimeoutDeprecated() throws IOException { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", @@ -88,7 +91,7 @@ public class ConsoleShareConsumerOptionsTest { } @Test - public void shouldParseConfigsFromFile() throws IOException { + public void shouldParseConfigsFromFileDeprecated() throws IOException { Map configs = new HashMap<>(); configs.put("request.timeout.ms", "1000"); configs.put("group.id", "group1"); @@ -109,80 +112,82 @@ public class ConsoleShareConsumerOptionsTest { } @Test - public void groupIdsProvidedInDifferentPlacesMustMatch() throws IOException { + public void groupIdsProvidedInDifferentPlacesMustMatchDeprecated() throws IOException { Exit.setExitProcedure((code, message) -> { throw new IllegalArgumentException(message); }); - // different in all three places - File propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); - final String[] args = new String[]{ - "--bootstrap-server", "localhost:9092", - "--topic", "test", - "--group", "group-from-arguments", - "--consumer-property", "group.id=group-from-properties", - "--consumer-config", propsFile.getAbsolutePath() - }; + try { + // different in all three places + File propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); + final String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--group", "group-from-arguments", + "--consumer-property", "group.id=group-from-properties", + "--consumer-config", propsFile.getAbsolutePath() + }; - assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args)); + assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args)); - // the same in all three places - propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "test-group")); - final String[] args1 = new String[]{ - "--bootstrap-server", "localhost:9092", - "--topic", "test", - "--group", "test-group", - "--consumer-property", "group.id=test-group", - "--consumer-config", propsFile.getAbsolutePath() - }; + // the same in all three places + propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "test-group")); + final String[] args1 = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--group", "test-group", + "--consumer-property", "group.id=test-group", + "--consumer-config", propsFile.getAbsolutePath() + }; - ConsoleShareConsumerOptions config = new ConsoleShareConsumerOptions(args1); - Properties props = config.consumerProps(); - assertEquals("test-group", props.getProperty("group.id")); + ConsoleShareConsumerOptions config = new ConsoleShareConsumerOptions(args1); + Properties props = config.consumerProps(); + assertEquals("test-group", props.getProperty("group.id")); - // different via --consumer-property and --consumer-config - propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); - final String[] args2 = new String[]{ - "--bootstrap-server", "localhost:9092", - "--topic", "test", - "--consumer-property", "group.id=group-from-properties", - "--consumer-config", propsFile.getAbsolutePath() - }; + // different via --consumer-property and --consumer-config + propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); + final String[] args2 = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--consumer-property", "group.id=group-from-properties", + "--consumer-config", propsFile.getAbsolutePath() + }; - assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args2)); + assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args2)); - // different via --consumer-property and --group - final String[] args3 = new String[]{ - "--bootstrap-server", "localhost:9092", - "--topic", "test", - "--group", "group-from-arguments", - "--consumer-property", "group.id=group-from-properties" - }; + // different via --consumer-property and --group + final String[] args3 = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--group", "group-from-arguments", + "--consumer-property", "group.id=group-from-properties" + }; - assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args3)); + assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args3)); - // different via --group and --consumer-config - propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); - final String[] args4 = new String[]{ - "--bootstrap-server", "localhost:9092", - "--topic", "test", - "--group", "group-from-arguments", - "--consumer-config", propsFile.getAbsolutePath() - }; - assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args4)); + // different via --group and --consumer-config + propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); + final String[] args4 = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--group", "group-from-arguments", + "--consumer-config", propsFile.getAbsolutePath() + }; + assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args4)); - // via --group only - final String[] args5 = new String[]{ - "--bootstrap-server", "localhost:9092", - "--topic", "test", - "--group", "group-from-arguments" - }; + // via --group only + final String[] args5 = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--group", "group-from-arguments" + }; - config = new ConsoleShareConsumerOptions(args5); - props = config.consumerProps(); - assertEquals("group-from-arguments", props.getProperty("group.id")); - - Exit.resetExitProcedure(); + config = new ConsoleShareConsumerOptions(args5); + props = config.consumerProps(); + assertEquals("group-from-arguments", props.getProperty("group.id")); + } finally { + Exit.resetExitProcedure(); + } } @Test @@ -203,7 +208,7 @@ public class ConsoleShareConsumerOptionsTest { } @Test - public void testClientIdOverride() throws IOException { + public void testClientIdOverrideDeprecated() throws IOException { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", @@ -216,6 +221,56 @@ public class ConsoleShareConsumerOptionsTest { assertEquals("consumer-1", consumerProperties.getProperty(ConsumerConfig.CLIENT_ID_CONFIG)); } + @Test + public void testCustomPropertyShouldBePassedToConfigureMethod() throws Exception { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--property", "print.key=true", + "--property", "key.deserializer=org.apache.kafka.test.MockDeserializer", + "--property", "key.deserializer.my-props=abc" + }; + + ConsoleShareConsumerOptions config = new ConsoleShareConsumerOptions(args); + + assertInstanceOf(DefaultMessageFormatter.class, config.formatter()); + assertTrue(config.formatterArgs().containsKey("key.deserializer.my-props")); + DefaultMessageFormatter formatter = (DefaultMessageFormatter) config.formatter(); + assertTrue(formatter.keyDeserializer().isPresent()); + assertInstanceOf(MockDeserializer.class, formatter.keyDeserializer().get()); + MockDeserializer keyDeserializer = (MockDeserializer) formatter.keyDeserializer().get(); + assertEquals(1, keyDeserializer.configs.size()); + assertEquals("abc", keyDeserializer.configs.get("my-props")); + assertTrue(keyDeserializer.isKey); + } + + @Test + public void testCustomConfigShouldBePassedToConfigureMethod() throws Exception { + Map configs = new HashMap<>(); + configs.put("key.deserializer.my-props", "abc"); + configs.put("print.key", "false"); + File propsFile = ToolsTestUtils.tempPropertiesFile(configs); + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--property", "print.key=true", + "--property", "key.deserializer=org.apache.kafka.test.MockDeserializer", + "--formatter-config", propsFile.getAbsolutePath() + }; + + ConsoleShareConsumerOptions config = new ConsoleShareConsumerOptions(args); + + assertInstanceOf(DefaultMessageFormatter.class, config.formatter()); + assertTrue(config.formatterArgs().containsKey("key.deserializer.my-props")); + DefaultMessageFormatter formatter = (DefaultMessageFormatter) config.formatter(); + assertTrue(formatter.keyDeserializer().isPresent()); + assertInstanceOf(MockDeserializer.class, formatter.keyDeserializer().get()); + MockDeserializer keyDeserializer = (MockDeserializer) formatter.keyDeserializer().get(); + assertEquals(1, keyDeserializer.configs.size()); + assertEquals("abc", keyDeserializer.configs.get("my-props")); + assertTrue(keyDeserializer.isKey); + } + @Test public void testDefaultClientId() throws IOException { String[] args = new String[]{ @@ -271,4 +326,182 @@ public class ConsoleShareConsumerOptionsTest { Exit.resetExitProcedure(); } } + + @Test + public void shouldExitOnBothConsumerPropertyAndCommandProperty() { + Exit.setExitProcedure((code, message) -> { + throw new IllegalArgumentException(message); + }); + + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--consumer-property", "session.timeout.ms=10000", + "--command-property", "request.timeout.ms=30000" + }; + + try { + assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args)); + } finally { + Exit.resetExitProcedure(); + } + } + + @Test + public void shouldExitOnBothConsumerConfigAndCommandConfig() throws IOException { + Exit.setExitProcedure((code, message) -> { + throw new IllegalArgumentException(message); + }); + + Map configs = new HashMap<>(); + configs.put("request.timeout.ms", "1000"); + File propsFile = ToolsTestUtils.tempPropertiesFile(configs); + + Map configs2 = new HashMap<>(); + configs2.put("session.timeout.ms", "10000"); + File propsFile2 = ToolsTestUtils.tempPropertiesFile(configs2); + + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--consumer-config", propsFile.getAbsolutePath(), + "--command-config", propsFile2.getAbsolutePath() + }; + + try { + assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args)); + } finally { + Exit.resetExitProcedure(); + } + } + + @Test + public void shouldParseValidConsumerConfigWithSessionTimeout() throws IOException { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--command-property", "session.timeout.ms=10000" + }; + + ConsoleShareConsumerOptions config = new ConsoleShareConsumerOptions(args); + Properties consumerProperties = config.consumerProps(); + + assertEquals("localhost:9092", config.bootstrapServer()); + assertEquals("test", config.topicArg()); + assertEquals("10000", consumerProperties.getProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG)); + } + + @Test + public void shouldParseConfigsFromFile() throws IOException { + Map configs = new HashMap<>(); + configs.put("request.timeout.ms", "1000"); + configs.put("group.id", "group1"); + File propsFile = ToolsTestUtils.tempPropertiesFile(configs); + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--command-config", propsFile.getAbsolutePath() + }; + + ConsoleShareConsumerOptions config = new ConsoleShareConsumerOptions(args); + + // KafkaShareConsumer uses Utils.propsToMap to convert the properties to a map, + // so using the same method to check the map has the expected values + Map configMap = Utils.propsToMap(config.consumerProps()); + assertEquals("1000", configMap.get("request.timeout.ms")); + assertEquals("group1", configMap.get("group.id")); + } + + @Test + public void groupIdsProvidedInDifferentPlacesMustMatch() throws IOException { + Exit.setExitProcedure((code, message) -> { + throw new IllegalArgumentException(message); + }); + + try { + // different in all three places + File propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); + final String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--group", "group-from-arguments", + "--command-property", "group.id=group-from-properties", + "--command-config", propsFile.getAbsolutePath() + }; + + assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args)); + + // the same in all three places + propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "test-group")); + final String[] args1 = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--group", "test-group", + "--command-property", "group.id=test-group", + "--command-config", propsFile.getAbsolutePath() + }; + + ConsoleShareConsumerOptions config = new ConsoleShareConsumerOptions(args1); + Properties props = config.consumerProps(); + assertEquals("test-group", props.getProperty("group.id")); + + // different via --command-property and --command-config + propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); + final String[] args2 = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--command-property", "group.id=group-from-properties", + "--command-config", propsFile.getAbsolutePath() + }; + + assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args2)); + + // different via --command-property and --group + final String[] args3 = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--group", "group-from-arguments", + "--command-property", "group.id=group-from-properties" + }; + + assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args3)); + + // different via --group and --command-config + propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); + final String[] args4 = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--group", "group-from-arguments", + "--command-config", propsFile.getAbsolutePath() + }; + assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args4)); + + // via --group only + final String[] args5 = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--group", "group-from-arguments" + }; + + config = new ConsoleShareConsumerOptions(args5); + props = config.consumerProps(); + assertEquals("group-from-arguments", props.getProperty("group.id")); + } finally { + Exit.resetExitProcedure(); + } + } + + @Test + public void testClientIdOverride() throws IOException { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--command-property", "client.id=consumer-1" + }; + + ConsoleShareConsumerOptions config = new ConsoleShareConsumerOptions(args); + Properties consumerProperties = config.consumerProps(); + + assertEquals("consumer-1", consumerProperties.getProperty(ConsumerConfig.CLIENT_ID_CONFIG)); + } } From 9a32a71e76da604f3e84734737fdf5e3fe9823f6 Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Wed, 17 Sep 2025 23:49:42 +0800 Subject: [PATCH 043/100] KAFKA-19699 improve the documentation of `RecordsToDelete` (#20527) document the behavior of "-1" (HIGH_WATERMARK) Reviewers: Ken Huang , TengYao Chi , Chia-Ping Tsai --- .../org/apache/kafka/clients/admin/RecordsToDelete.java | 6 ++++-- .../main/resources/common/message/DeleteRecordsRequest.json | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/RecordsToDelete.java b/clients/src/main/java/org/apache/kafka/clients/admin/RecordsToDelete.java index d3da26b03bb..57421e3568b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/RecordsToDelete.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/RecordsToDelete.java @@ -33,14 +33,16 @@ public class RecordsToDelete { /** * Delete all the records before the given {@code offset} * - * @param offset the offset before which all records will be deleted + * @param offset The offset before which all records will be deleted. + * Use {@code -1} to truncate to the high watermark. */ public static RecordsToDelete beforeOffset(long offset) { return new RecordsToDelete(offset); } /** - * The offset before which all records will be deleted + * The offset before which all records will be deleted. + * Use {@code -1} to truncate to the high watermark. */ public long beforeOffset() { return offset; diff --git a/clients/src/main/resources/common/message/DeleteRecordsRequest.json b/clients/src/main/resources/common/message/DeleteRecordsRequest.json index fc697944a02..969efd63e95 100644 --- a/clients/src/main/resources/common/message/DeleteRecordsRequest.json +++ b/clients/src/main/resources/common/message/DeleteRecordsRequest.json @@ -33,7 +33,7 @@ { "name": "PartitionIndex", "type": "int32", "versions": "0+", "about": "The partition index." }, { "name": "Offset", "type": "int64", "versions": "0+", - "about": "The deletion offset." } + "about": "The deletion offset. -1 means that records should be truncated to the high watermark." } ]} ]}, { "name": "TimeoutMs", "type": "int32", "versions": "0+", From e647bdcee5684154dcb569b758b0846628f4694e Mon Sep 17 00:00:00 2001 From: Shivsundar R Date: Thu, 18 Sep 2025 02:23:12 -0400 Subject: [PATCH 044/100] MINOR : Fix parantheses in console_consumer.py and console_share_consumer.py (#20552) *What* We were missing a parantheses when we invoked a method `supports_formatter_property()`. This would mean we would get the object not call the function. PR fixes this by including parantheses and invoking the actual function. Reviewers: Manikumar Reddy --- tests/kafkatest/services/console_consumer.py | 2 +- tests/kafkatest/services/console_share_consumer.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/kafkatest/services/console_consumer.py b/tests/kafkatest/services/console_consumer.py index 0f95a0f7c79..467e7c3247c 100644 --- a/tests/kafkatest/services/console_consumer.py +++ b/tests/kafkatest/services/console_consumer.py @@ -179,7 +179,7 @@ class ConsoleConsumer(KafkaPathResolverMixin, JmxMixin, BackgroundThreadService) # This will be added in the properties file instead cmd += " --timeout-ms %s" % self.consumer_timeout_ms - formatter_property_arg = "--formatter-property" if version.supports_formatter_property else "--property" + formatter_property_arg = "--formatter-property" if version.supports_formatter_property() else "--property" if self.print_timestamp: cmd += " %s print.timestamp=true" % formatter_property_arg diff --git a/tests/kafkatest/services/console_share_consumer.py b/tests/kafkatest/services/console_share_consumer.py index 2d7da50fe30..0761acddcc8 100644 --- a/tests/kafkatest/services/console_share_consumer.py +++ b/tests/kafkatest/services/console_share_consumer.py @@ -167,7 +167,7 @@ class ConsoleShareConsumer(KafkaPathResolverMixin, JmxMixin, BackgroundThreadSer # This will be added in the properties file instead cmd += " --timeout-ms %s" % self.share_consumer_timeout_ms - formatter_property_arg = "--formatter-property" if version.supports_formatter_property else "--property" + formatter_property_arg = "--formatter-property" if version.supports_formatter_property() else "--property" if self.print_timestamp: cmd += " %s print.timestamp=true" % formatter_property_arg From 04b4a8f5717c7433e1178b22846ee2415b50020e Mon Sep 17 00:00:00 2001 From: Jinhe Zhang Date: Thu, 18 Sep 2025 03:41:52 -0400 Subject: [PATCH 045/100] KAFKA-19705: Enable streams rebalance protocol in IQv2 integration test (#20541) Update IQv2 Integration tests for streams group protocol Reviewers: Lucas Brutschy --- .../integration/IQv2IntegrationTest.java | 75 +++++++++++++------ .../integration/IQv2StoreIntegrationTest.java | 18 +++-- .../IQv2VersionedStoreIntegrationTest.java | 67 ++++++++++++----- 3 files changed, 112 insertions(+), 48 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/IQv2IntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/IQv2IntegrationTest.java index dcd711a35c5..20fc7f47236 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/IQv2IntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/IQv2IntegrationTest.java @@ -57,11 +57,12 @@ import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import java.io.IOException; import java.lang.reflect.Field; @@ -76,6 +77,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.stream.Stream; import static java.util.Collections.singleton; import static org.apache.kafka.streams.query.StateQueryRequest.inStore; @@ -99,6 +101,7 @@ public class IQv2IntegrationTest { public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS); private KafkaStreams kafkaStreams; + private String groupProtocol; @BeforeAll public static void before() @@ -149,8 +152,8 @@ public class IQv2IntegrationTest { )); } - @BeforeEach - public void beforeTest(final TestInfo testInfo) { + private void setup(final String groupProtocol, final TestInfo testInfo) { + this.groupProtocol = groupProtocol; final StreamsBuilder builder = new StreamsBuilder(); builder.table( @@ -159,7 +162,6 @@ public class IQv2IntegrationTest { Materialized.as(STORE_NAME) ); - final String safeTestName = safeUniqueTestName(testInfo); kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration(safeTestName)); kafkaStreams.cleanUp(); @@ -167,8 +169,10 @@ public class IQv2IntegrationTest { @AfterEach public void afterTest() { - kafkaStreams.close(Duration.ofSeconds(60)); - kafkaStreams.cleanUp(); + if (kafkaStreams != null) { + kafkaStreams.close(Duration.ofSeconds(60)); + kafkaStreams.cleanUp(); + } } @AfterAll @@ -176,8 +180,10 @@ public class IQv2IntegrationTest { CLUSTER.stop(); } - @Test - public void shouldFailUnknownStore() { + @ParameterizedTest(name = "{1}") + @MethodSource("groupProtocolParameters") + public void shouldFailUnknownStore(final String groupProtocol, final String testName, final TestInfo testInfo) { + setup(groupProtocol, testInfo); final KeyQuery> query = KeyQuery.withKey(1); final StateQueryRequest> request = inStore("unknown-store").withQuery(query); @@ -185,8 +191,10 @@ public class IQv2IntegrationTest { assertThrows(UnknownStateStoreException.class, () -> kafkaStreams.query(request)); } - @Test - public void shouldFailNotStarted() { + @ParameterizedTest(name = "{1}") + @MethodSource("groupProtocolParameters") + public void shouldFailNotStarted(final String groupProtocol, final String testName, final TestInfo testInfo) { + setup(groupProtocol, testInfo); final KeyQuery> query = KeyQuery.withKey(1); final StateQueryRequest> request = inStore(STORE_NAME).withQuery(query); @@ -194,8 +202,10 @@ public class IQv2IntegrationTest { assertThrows(StreamsNotStartedException.class, () -> kafkaStreams.query(request)); } - @Test - public void shouldFailStopped() { + @ParameterizedTest(name = "{1}") + @MethodSource("groupProtocolParameters") + public void shouldFailStopped(final String groupProtocol, final String testName, final TestInfo testInfo) { + setup(groupProtocol, testInfo); final KeyQuery> query = KeyQuery.withKey(1); final StateQueryRequest> request = inStore(STORE_NAME).withQuery(query); @@ -205,9 +215,11 @@ public class IQv2IntegrationTest { assertThrows(StreamsStoppedException.class, () -> kafkaStreams.query(request)); } - @Test - public void shouldRejectNonRunningActive() + @ParameterizedTest(name = "{1}") + @MethodSource("groupProtocolParameters") + public void shouldRejectNonRunningActive(final String groupProtocol, final String testName, final TestInfo testInfo) throws NoSuchFieldException, IllegalAccessException { + setup(groupProtocol, testInfo); final KeyQuery> query = KeyQuery.withKey(1); final StateQueryRequest> request = inStore(STORE_NAME).withQuery(query).requireActive(); @@ -261,8 +273,10 @@ public class IQv2IntegrationTest { } } - @Test - public void shouldFetchFromPartition() { + @ParameterizedTest(name = "{1}") + @MethodSource("groupProtocolParameters") + public void shouldFetchFromPartition(final String groupProtocol, final String testName, final TestInfo testInfo) { + setup(groupProtocol, testInfo); final KeyQuery> query = KeyQuery.withKey(1); final int partition = 1; final Set partitions = singleton(partition); @@ -276,8 +290,10 @@ public class IQv2IntegrationTest { assertThat(result.getPartitionResults().keySet(), equalTo(partitions)); } - @Test - public void shouldFetchExplicitlyFromAllPartitions() { + @ParameterizedTest(name = "{1}") + @MethodSource("groupProtocolParameters") + public void shouldFetchExplicitlyFromAllPartitions(final String groupProtocol, final String testName, final TestInfo testInfo) { + setup(groupProtocol, testInfo); final KeyQuery> query = KeyQuery.withKey(1); final Set partitions = Set.of(0, 1); final StateQueryRequest> request = @@ -290,8 +306,10 @@ public class IQv2IntegrationTest { assertThat(result.getPartitionResults().keySet(), equalTo(partitions)); } - @Test - public void shouldNotRequireQueryHandler(final TestInfo testInfo) { + @ParameterizedTest(name = "{1}") + @MethodSource("groupProtocolParameters") + public void shouldNotRequireQueryHandler(final String groupProtocol, final String testName, final TestInfo testInfo) { + setup(groupProtocol, testInfo); final KeyQuery> query = KeyQuery.withKey(1); final int partition = 1; final Set partitions = singleton(partition); @@ -423,8 +441,11 @@ public class IQv2IntegrationTest { ); // Discard the basic streams and replace with test-specific topology - kafkaStreams.close(); + if (kafkaStreams != null) { + kafkaStreams.close(); + } final String safeTestName = safeUniqueTestName(testInfo); + this.groupProtocol = groupProtocol; kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration(safeTestName)); kafkaStreams.cleanUp(); @@ -446,7 +467,7 @@ public class IQv2IntegrationTest { private Properties streamsConfiguration(final String safeTestName) { final Properties config = new Properties(); config.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE); - config.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName); + config.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName + "-" + groupProtocol); config.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "localhost:" + (++port)); config.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); config.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); @@ -458,6 +479,14 @@ public class IQv2IntegrationTest { config.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000); config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); config.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1); + config.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol); return config; } + + private static Stream groupProtocolParameters() { + return Stream.of( + Arguments.of("classic", "CLASSIC protocol"), + Arguments.of("streams", "STREAMS protocol") + ); + } } diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/IQv2StoreIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/IQv2StoreIntegrationTest.java index 963183fd665..649799c7976 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/IQv2StoreIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/IQv2StoreIntegrationTest.java @@ -360,7 +360,9 @@ public class IQv2StoreIntegrationTest { for (final boolean logEnabled : Arrays.asList(true, false)) { for (final StoresToTest toTest : StoresToTest.values()) { for (final String kind : Arrays.asList("DSL", "PAPI")) { - values.add(Arguments.of(cacheEnabled, logEnabled, toTest.name(), kind)); + for (final String groupProtocol : Arrays.asList("classic", "streams")) { + values.add(Arguments.of(cacheEnabled, logEnabled, toTest.name(), kind, groupProtocol)); + } } } } @@ -426,13 +428,14 @@ public class IQv2StoreIntegrationTest { )); } - public void setup(final boolean cache, final boolean log, final StoresToTest storeToTest, final String kind) { + public void setup(final boolean cache, final boolean log, final StoresToTest storeToTest, final String kind, final String groupProtocol) { final StoreSupplier supplier = storeToTest.supplier(); final Properties streamsConfig = streamsConfiguration( cache, log, storeToTest.name(), - kind + kind, + groupProtocol ); final StreamsBuilder builder = new StreamsBuilder(); @@ -765,8 +768,8 @@ public class IQv2StoreIntegrationTest { @ParameterizedTest @MethodSource("data") - public void verifyStore(final boolean cache, final boolean log, final StoresToTest storeToTest, final String kind) { - setup(cache, log, storeToTest, kind); + public void verifyStore(final boolean cache, final boolean log, final StoresToTest storeToTest, final String kind, final String groupProtocol) { + setup(cache, log, storeToTest, kind, groupProtocol); try { if (storeToTest.global()) { // See KAFKA-13523 @@ -2030,10 +2033,10 @@ public class IQv2StoreIntegrationTest { } private static Properties streamsConfiguration(final boolean cache, final boolean log, - final String supplier, final String kind) { + final String supplier, final String kind, final String groupProtocol) { final String safeTestName = IQv2StoreIntegrationTest.class.getName() + "-" + cache + "-" + log + "-" + supplier - + "-" + kind + "-" + RANDOM.nextInt(); + + "-" + kind + "-" + groupProtocol + "-" + RANDOM.nextInt(); final Properties config = new Properties(); config.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE); config.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName); @@ -2048,6 +2051,7 @@ public class IQv2StoreIntegrationTest { config.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000); config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); config.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1); + config.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol); return config; } } \ No newline at end of file diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/IQv2VersionedStoreIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/IQv2VersionedStoreIntegrationTest.java index 26a5aed5a52..bb1f3c20a87 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/IQv2VersionedStoreIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/IQv2VersionedStoreIntegrationTest.java @@ -47,7 +47,10 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import java.time.Duration; import java.time.Instant; @@ -57,7 +60,9 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Optional; import java.util.Properties; +import java.util.stream.Stream; +import static org.apache.kafka.streams.utils.TestUtils.safeUniqueTestName; import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.empty; @@ -83,16 +88,25 @@ public class IQv2VersionedStoreIntegrationTest { private static final Long[] RECORD_TIMESTAMPS = {BASE_TIMESTAMP_LONG, BASE_TIMESTAMP_LONG + 10, BASE_TIMESTAMP_LONG + 20, BASE_TIMESTAMP_LONG + 30}; private static final int RECORD_NUMBER = RECORD_VALUES.length; private static final int LAST_INDEX = RECORD_NUMBER - 1; - private static final Position INPUT_POSITION = Position.emptyPosition(); + private Position inputPosition; public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS, Utils.mkProperties(Collections.singletonMap("auto.create.topics.enable", "true"))); private KafkaStreams kafkaStreams; + private String groupProtocol; @BeforeAll - public static void before() throws Exception { + public static void beforeAll() throws Exception { CLUSTER.start(); - + } + + @BeforeEach + public void beforeEach() throws Exception { + // Delete and recreate the topic to ensure clean state for each test + CLUSTER.deleteTopic(INPUT_TOPIC_NAME); + CLUSTER.createTopic(INPUT_TOPIC_NAME, 1, 1); + + // Set up fresh test data final Properties producerProps = new Properties(); producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class); @@ -103,19 +117,21 @@ public class IQv2VersionedStoreIntegrationTest { producer.send(new ProducerRecord<>(INPUT_TOPIC_NAME, 0, RECORD_TIMESTAMPS[2], RECORD_KEY, RECORD_VALUES[2])).get(); producer.send(new ProducerRecord<>(INPUT_TOPIC_NAME, 0, RECORD_TIMESTAMPS[3], RECORD_KEY, RECORD_VALUES[3])).get(); } - INPUT_POSITION.withComponent(INPUT_TOPIC_NAME, 0, 3); + inputPosition = Position.emptyPosition().withComponent(INPUT_TOPIC_NAME, 0, 3); } - @BeforeEach - public void beforeTest() { + private void setup(final String groupProtocol, final TestInfo testInfo) { + this.groupProtocol = groupProtocol; final StreamsBuilder builder = new StreamsBuilder(); builder.table(INPUT_TOPIC_NAME, Materialized.as(Stores.persistentVersionedKeyValueStore(STORE_NAME, HISTORY_RETENTION, SEGMENT_INTERVAL))); final Properties configs = new Properties(); - configs.put(StreamsConfig.APPLICATION_ID_CONFIG, "app"); + final String safeTestName = safeUniqueTestName(testInfo); + configs.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName); configs.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); configs.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class.getName()); configs.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class.getName()); + configs.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol); kafkaStreams = IntegrationTestUtils.getStartedStreams(configs, builder, true); } @@ -132,8 +148,19 @@ public class IQv2VersionedStoreIntegrationTest { CLUSTER.stop(); } - @Test - public void verifyStore() { + private static Stream groupProtocolParameters() { + return Stream.of( + Arguments.of("classic", "CLASSIC protocol"), + Arguments.of("streams", "STREAMS protocol") + ); + } + + @ParameterizedTest(name = "{1}") + @MethodSource("groupProtocolParameters") + public void verifyStore(final String groupProtocol, final String testName, final TestInfo testInfo) throws Exception { + // Set up streams + setup(groupProtocol, testInfo); + /* Test Versioned Key Queries */ // retrieve the latest value shouldHandleVersionedKeyQuery(Optional.empty(), RECORD_VALUES[3], RECORD_TIMESTAMPS[3], Optional.empty()); @@ -255,7 +282,10 @@ public class IQv2VersionedStoreIntegrationTest { private void shouldHandleRaceCondition() { final MultiVersionedKeyQuery query = defineQuery(RECORD_KEY, Optional.empty(), Optional.empty(), ResultOrder.ANY); - final Map>> partitionResults = sendRequestAndReceiveResults(query, kafkaStreams); + // For race condition test, we don't use position bounds since we're testing concurrent updates + final StateQueryRequest> request = StateQueryRequest.inStore(STORE_NAME).withQuery(query); + final StateQueryResult> result = IntegrationTestUtils.iqv2WaitForResult(kafkaStreams, request); + final Map>> partitionResults = result.getPartitionResults(); // verify results in two steps for (final Entry>> partitionResultsEntry : partitionResults.entrySet()) { @@ -327,14 +357,14 @@ public class IQv2VersionedStoreIntegrationTest { return query; } - private static Map>> sendRequestAndReceiveResults(final MultiVersionedKeyQuery query, final KafkaStreams kafkaStreams) { - final StateQueryRequest> request = StateQueryRequest.inStore(STORE_NAME).withQuery(query).withPositionBound(PositionBound.at(INPUT_POSITION)); + private Map>> sendRequestAndReceiveResults(final MultiVersionedKeyQuery query, final KafkaStreams kafkaStreams) { + final StateQueryRequest> request = StateQueryRequest.inStore(STORE_NAME).withQuery(query).withPositionBound(PositionBound.at(inputPosition)); final StateQueryResult> result = IntegrationTestUtils.iqv2WaitForResult(kafkaStreams, request); return result.getPartitionResults(); } - private static QueryResult> sendRequestAndReceiveResults(final VersionedKeyQuery query, final KafkaStreams kafkaStreams) { - final StateQueryRequest> request = StateQueryRequest.inStore(STORE_NAME).withQuery(query).withPositionBound(PositionBound.at(INPUT_POSITION)); + private QueryResult> sendRequestAndReceiveResults(final VersionedKeyQuery query, final KafkaStreams kafkaStreams) { + final StateQueryRequest> request = StateQueryRequest.inStore(STORE_NAME).withQuery(query).withPositionBound(PositionBound.at(inputPosition)); final StateQueryResult> result = IntegrationTestUtils.iqv2WaitForResult(kafkaStreams, request); return result.getOnlyPartitionResult(); } @@ -352,7 +382,7 @@ public class IQv2VersionedStoreIntegrationTest { /** * This method inserts a new value (999999) for the key in the oldest timestamp (RECORD_TIMESTAMPS[0]). */ - private static void updateRecordValue() { + private void updateRecordValue() { // update the record value at RECORD_TIMESTAMPS[0] final Properties producerProps = new Properties(); producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); @@ -361,8 +391,9 @@ public class IQv2VersionedStoreIntegrationTest { try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { producer.send(new ProducerRecord<>(INPUT_TOPIC_NAME, 0, RECORD_TIMESTAMPS[0], RECORD_KEY, 999999)); } - INPUT_POSITION.withComponent(INPUT_TOPIC_NAME, 0, 4); - assertThat(INPUT_POSITION, equalTo(Position.emptyPosition().withComponent(INPUT_TOPIC_NAME, 0, 4))); + + inputPosition = inputPosition.withComponent(INPUT_TOPIC_NAME, 0, 4); + assertThat(inputPosition, equalTo(Position.emptyPosition().withComponent(INPUT_TOPIC_NAME, 0, 4))); // make sure that the new value is picked up by the store final Properties consumerProps = new Properties(); From d6fdbfcf15adc279c6aabd6f630eada66f2dec41 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Thu, 18 Sep 2025 10:22:35 +0200 Subject: [PATCH 046/100] MINOR: Fix typos in CoordinatorRecordTypeGenerator (#20549) This patch fixes a few typos in CoordinatorRecordTypeGenerator. Reviewers: Chia-Ping Tsai , TengYao Chi , Sean Quah --- .../apache/kafka/message/CoordinatorRecordTypeGenerator.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/generator/src/main/java/org/apache/kafka/message/CoordinatorRecordTypeGenerator.java b/generator/src/main/java/org/apache/kafka/message/CoordinatorRecordTypeGenerator.java index 5a7ce9c1307..5e7089d280d 100644 --- a/generator/src/main/java/org/apache/kafka/message/CoordinatorRecordTypeGenerator.java +++ b/generator/src/main/java/org/apache/kafka/message/CoordinatorRecordTypeGenerator.java @@ -124,11 +124,11 @@ public class CoordinatorRecordTypeGenerator implements TypeClassGenerator { for (Map.Entry entry : records.entrySet()) { MessageSpec key = entry.getValue().key; if (key == null) { - throw new RuntimeException("Coordinator record " + entry.getKey() + " has not key."); + throw new RuntimeException("Coordinator record " + entry.getKey() + " has no key."); } MessageSpec value = entry.getValue().value; if (value == null) { - throw new RuntimeException("Coordinator record " + entry.getKey() + " has not key."); + throw new RuntimeException("Coordinator record " + entry.getKey() + " has no value."); } String name = cleanName(key.name()); numProcessed++; From 8c8e93c4a14ec56bedd19e5c719c9622ae8ae4e1 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Thu, 18 Sep 2025 11:10:35 +0200 Subject: [PATCH 047/100] MINOR: Remove metrics attribute from ConsumerGroup (#20542) The `metrics` attribute in `ConsumerGroup` is not used anymore. This patch removes it. Reviewers: Lianet Magrans , Chia-Ping Tsai , TengYao Chi , Dongnuo Lyu --- .../group/GroupMetadataManager.java | 9 +++-- .../group/modern/consumer/ConsumerGroup.java | 14 ++------ .../group/GroupCoordinatorShardTest.java | 6 ++-- .../group/classic/ClassicGroupTest.java | 8 ++--- .../modern/consumer/ConsumerGroupTest.java | 34 +++++-------------- 5 files changed, 18 insertions(+), 53 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 f87af4897a7..477e0499565 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 @@ -813,10 +813,10 @@ public class GroupMetadataManager { } if (group == null) { - return new ConsumerGroup(snapshotRegistry, groupId, metrics); + return new ConsumerGroup(snapshotRegistry, groupId); } else if (createIfNotExists && maybeDeleteEmptyClassicGroup(group, records)) { log.info("[GroupId {}] Converted the empty classic group to a consumer group.", groupId); - return new ConsumerGroup(snapshotRegistry, groupId, metrics); + return new ConsumerGroup(snapshotRegistry, groupId); } else { if (group.type() == CONSUMER) { return (ConsumerGroup) group; @@ -980,7 +980,7 @@ public class GroupMetadataManager { } if (group == null) { - ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, groupId, metrics); + ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, groupId); groups.put(groupId, consumerGroup); return consumerGroup; } else if (group.type() == CONSUMER) { @@ -990,7 +990,7 @@ public class GroupMetadataManager { // offsets if no group existed. Simple classic groups are not backed by any records // in the __consumer_offsets topic hence we can safely replace it here. Without this, // replaying consumer group records after offset commit records would not work. - ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, groupId, metrics); + ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, groupId); groups.put(groupId, consumerGroup); return consumerGroup; } else { @@ -1364,7 +1364,6 @@ public class GroupMetadataManager { try { consumerGroup = ConsumerGroup.fromClassicGroup( snapshotRegistry, - metrics, classicGroup, topicHashCache, metadataImage diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java index 19b103c71a7..2db94cb0535 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java @@ -38,7 +38,6 @@ import org.apache.kafka.coordinator.group.Utils; import org.apache.kafka.coordinator.group.api.assignor.SubscriptionType; import org.apache.kafka.coordinator.group.classic.ClassicGroup; import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue; -import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard; import org.apache.kafka.coordinator.group.modern.Assignment; import org.apache.kafka.coordinator.group.modern.MemberState; import org.apache.kafka.coordinator.group.modern.ModernGroup; @@ -119,11 +118,6 @@ public class ConsumerGroup extends ModernGroup { */ private final TimelineHashMap serverAssignors; - /** - * The coordinator metrics. - */ - private final GroupCoordinatorMetricsShard metrics; - /** * The number of members that use the classic protocol. */ @@ -155,14 +149,12 @@ public class ConsumerGroup extends ModernGroup { public ConsumerGroup( SnapshotRegistry snapshotRegistry, - String groupId, - GroupCoordinatorMetricsShard metrics + String groupId ) { super(snapshotRegistry, groupId); this.state = new TimelineObject<>(snapshotRegistry, EMPTY); this.staticMembers = new TimelineHashMap<>(snapshotRegistry, 0); this.serverAssignors = new TimelineHashMap<>(snapshotRegistry, 0); - this.metrics = Objects.requireNonNull(metrics); this.numClassicProtocolMembers = new TimelineInteger(snapshotRegistry); this.classicProtocolMembersSupportedProtocols = new TimelineHashMap<>(snapshotRegistry, 0); this.currentPartitionEpoch = new TimelineHashMap<>(snapshotRegistry, 0); @@ -1130,7 +1122,6 @@ public class ConsumerGroup extends ModernGroup { * Create a new consumer group according to the given classic group. * * @param snapshotRegistry The SnapshotRegistry. - * @param metrics The GroupCoordinatorMetricsShard. * @param classicGroup The converted classic group. * @param topicHashCache The cache for topic hashes. * @param metadataImage The current metadata image for the Kafka cluster. @@ -1141,13 +1132,12 @@ public class ConsumerGroup extends ModernGroup { */ public static ConsumerGroup fromClassicGroup( SnapshotRegistry snapshotRegistry, - GroupCoordinatorMetricsShard metrics, ClassicGroup classicGroup, Map topicHashCache, CoordinatorMetadataImage metadataImage ) { String groupId = classicGroup.groupId(); - ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, groupId, metrics); + ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, groupId); consumerGroup.setGroupEpoch(classicGroup.generationId()); consumerGroup.setTargetAssignmentEpoch(classicGroup.generationId()); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorShardTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorShardTest.java index 9a7d3820126..145b7613f1f 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorShardTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorShardTest.java @@ -84,7 +84,6 @@ import org.apache.kafka.coordinator.group.generated.StreamsGroupTargetAssignment import org.apache.kafka.coordinator.group.generated.StreamsGroupTargetAssignmentMetadataValue; import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyKey; import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue; -import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard; import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup; import org.apache.kafka.coordinator.group.modern.share.ShareGroup; import org.apache.kafka.coordinator.group.streams.StreamsGroupHeartbeatResult; @@ -1384,10 +1383,9 @@ public class GroupCoordinatorShardTest { ArgumentCaptor> recordsCapture = ArgumentCaptor.forClass(List.class); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); - GroupCoordinatorMetricsShard metricsShard = mock(GroupCoordinatorMetricsShard.class); - ConsumerGroup group1 = new ConsumerGroup(snapshotRegistry, "group-id", metricsShard); - ConsumerGroup group2 = new ConsumerGroup(snapshotRegistry, "other-group-id", metricsShard); + ConsumerGroup group1 = new ConsumerGroup(snapshotRegistry, "group-id"); + ConsumerGroup group2 = new ConsumerGroup(snapshotRegistry, "other-group-id"); when(groupMetadataManager.groupIds()).thenReturn(Set.of("group-id", "other-group-id")); when(groupMetadataManager.group("group-id")).thenReturn(group1); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupTest.java index 2e6cf0012f7..24212d10982 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupTest.java @@ -45,7 +45,6 @@ import org.apache.kafka.coordinator.group.OffsetAndMetadata; import org.apache.kafka.coordinator.group.OffsetExpirationCondition; import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl; import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue; -import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard; import org.apache.kafka.coordinator.group.modern.Assignment; import org.apache.kafka.coordinator.group.modern.MemberState; import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup; @@ -81,7 +80,6 @@ import static org.junit.jupiter.api.Assertions.assertFalse; 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.mock; public class ClassicGroupTest { private final String protocolType = "consumer"; @@ -1383,8 +1381,7 @@ public class ClassicGroupTest { ConsumerGroup consumerGroup = new ConsumerGroup( new SnapshotRegistry(logContext), - groupId, - mock(GroupCoordinatorMetricsShard.class) + groupId ); consumerGroup.setGroupEpoch(10); consumerGroup.setTargetAssignmentEpoch(10); @@ -1536,8 +1533,7 @@ public class ClassicGroupTest { ConsumerGroup consumerGroup = new ConsumerGroup( new SnapshotRegistry(logContext), - groupId, - mock(GroupCoordinatorMetricsShard.class) + groupId ); consumerGroup.setGroupEpoch(10); consumerGroup.setTargetAssignmentEpoch(10); 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 202b91a5e41..5956e51b161 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 @@ -25,7 +25,6 @@ import org.apache.kafka.common.errors.IllegalGenerationException; import org.apache.kafka.common.errors.StaleMemberEpochException; import org.apache.kafka.common.errors.UnknownMemberIdException; import org.apache.kafka.common.errors.UnsupportedVersionException; -import org.apache.kafka.common.internals.Topic; import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData; import org.apache.kafka.common.message.JoinGroupRequestData; import org.apache.kafka.common.protocol.ApiKeys; @@ -45,7 +44,6 @@ import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl; import org.apache.kafka.coordinator.group.classic.ClassicGroup; import org.apache.kafka.coordinator.group.classic.ClassicGroupMember; import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue; -import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard; import org.apache.kafka.coordinator.group.modern.Assignment; import org.apache.kafka.coordinator.group.modern.MemberState; import org.apache.kafka.coordinator.group.modern.ModernGroup; @@ -83,7 +81,6 @@ import static org.junit.jupiter.api.Assertions.assertFalse; 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.mock; public class ConsumerGroupTest { @@ -91,8 +88,7 @@ public class ConsumerGroupTest { SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); return new ConsumerGroup( snapshotRegistry, - groupId, - mock(GroupCoordinatorMetricsShard.class) + groupId ); } @@ -700,8 +696,7 @@ public class ConsumerGroupTest { @Test public void testUpdateInvertedAssignment() { SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); - GroupCoordinatorMetricsShard metricsShard = mock(GroupCoordinatorMetricsShard.class); - ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, "test-group", metricsShard); + ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, "test-group"); Uuid topicId = Uuid.randomUuid(); String memberId1 = "member1"; String memberId2 = "member2"; @@ -916,12 +911,7 @@ public class ConsumerGroupTest { @Test public void testAsListedGroup() { SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); - GroupCoordinatorMetricsShard metricsShard = new GroupCoordinatorMetricsShard( - snapshotRegistry, - Map.of(), - new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0) - ); - ConsumerGroup group = new ConsumerGroup(snapshotRegistry, "group-foo", metricsShard); + ConsumerGroup group = new ConsumerGroup(snapshotRegistry, "group-foo"); snapshotRegistry.idempotentCreateSnapshot(0); assertEquals(ConsumerGroup.ConsumerGroupState.EMPTY.toString(), group.stateAsString(0)); group.updateMember(new ConsumerGroupMember.Builder("member1") @@ -937,8 +927,7 @@ public class ConsumerGroupTest { SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); ConsumerGroup group = new ConsumerGroup( snapshotRegistry, - "group-foo", - mock(GroupCoordinatorMetricsShard.class) + "group-foo" ); // Simulate a call from the admin client without member id and member epoch. @@ -997,7 +986,7 @@ public class ConsumerGroupTest { long commitTimestamp = 20000L; long offsetsRetentionMs = 10000L; OffsetAndMetadata offsetAndMetadata = new OffsetAndMetadata(15000L, OptionalInt.empty(), "", commitTimestamp, OptionalLong.empty(), Uuid.ZERO_UUID); - ConsumerGroup group = new ConsumerGroup(new SnapshotRegistry(new LogContext()), "group-id", mock(GroupCoordinatorMetricsShard.class)); + ConsumerGroup group = new ConsumerGroup(new SnapshotRegistry(new LogContext()), "group-id"); Optional offsetExpirationCondition = group.offsetExpirationCondition(); assertTrue(offsetExpirationCondition.isPresent()); @@ -1034,7 +1023,7 @@ public class ConsumerGroupTest { @Test public void testAsDescribedGroup() { SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); - ConsumerGroup group = new ConsumerGroup(snapshotRegistry, "group-id-1", mock(GroupCoordinatorMetricsShard.class)); + ConsumerGroup group = new ConsumerGroup(snapshotRegistry, "group-id-1"); snapshotRegistry.idempotentCreateSnapshot(0); assertEquals(ConsumerGroup.ConsumerGroupState.EMPTY.toString(), group.stateAsString(0)); @@ -1071,12 +1060,7 @@ public class ConsumerGroupTest { @Test public void testIsInStatesCaseInsensitive() { SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); - GroupCoordinatorMetricsShard metricsShard = new GroupCoordinatorMetricsShard( - snapshotRegistry, - Map.of(), - new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0) - ); - ConsumerGroup group = new ConsumerGroup(snapshotRegistry, "group-foo", metricsShard); + ConsumerGroup group = new ConsumerGroup(snapshotRegistry, "group-foo"); snapshotRegistry.idempotentCreateSnapshot(0); assertTrue(group.isInStates(Set.of("empty"), 0)); assertFalse(group.isInStates(Set.of("Empty"), 0)); @@ -1307,7 +1291,6 @@ public class ConsumerGroupTest { ConsumerGroup consumerGroup = ConsumerGroup.fromClassicGroup( new SnapshotRegistry(logContext), - mock(GroupCoordinatorMetricsShard.class), classicGroup, new HashMap<>(), metadataImage @@ -1315,8 +1298,7 @@ public class ConsumerGroupTest { ConsumerGroup expectedConsumerGroup = new ConsumerGroup( new SnapshotRegistry(logContext), - groupId, - mock(GroupCoordinatorMetricsShard.class) + groupId ); expectedConsumerGroup.setGroupEpoch(10); expectedConsumerGroup.setTargetAssignmentEpoch(10); From dbd2b527d0fa5eda4ec603ee449a47c69ac41852 Mon Sep 17 00:00:00 2001 From: Sean Quah Date: Thu, 18 Sep 2025 14:52:03 +0100 Subject: [PATCH 048/100] MINOR: Fix format in CoordinatorLoaderImplTest (#20548) Fix indentation in `CoordinatorLoaderImplTest` to be consistent with the rest of the code in the package. Reviewers: TengYao Chi , David Jacot --- .../runtime/CoordinatorLoaderImplTest.java | 316 +++++++++--------- 1 file changed, 158 insertions(+), 158 deletions(-) diff --git a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImplTest.java b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImplTest.java index 8760e9347a1..9f8ab68c66f 100644 --- a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImplTest.java +++ b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImplTest.java @@ -72,8 +72,8 @@ class CoordinatorLoaderImplTest { @Override public Map.Entry deserialize(ByteBuffer key, ByteBuffer value) throws RuntimeException { return Map.entry( - StandardCharsets.UTF_8.decode(key).toString(), - StandardCharsets.UTF_8.decode(value).toString() + StandardCharsets.UTF_8.decode(key).toString(), + StandardCharsets.UTF_8.decode(value).toString() ); } } @@ -87,11 +87,11 @@ class CoordinatorLoaderImplTest { CoordinatorPlayback> coordinator = mock(CoordinatorPlayback.class); try (CoordinatorLoaderImpl> loader = new CoordinatorLoaderImpl<>( - Time.SYSTEM, - partitionLogSupplier, - partitionLogEndOffsetSupplier, - serde, - 1000 + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 )) { assertFutureThrows(NotLeaderOrFollowerException.class, loader.load(tp, coordinator)); } @@ -106,11 +106,11 @@ class CoordinatorLoaderImplTest { CoordinatorPlayback> coordinator = mock(CoordinatorPlayback.class); try (CoordinatorLoaderImpl> loader = new CoordinatorLoaderImpl<>( - Time.SYSTEM, - partitionLogSupplier, - partitionLogEndOffsetSupplier, - serde, - 1000 + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 )) { loader.close(); assertFutureThrows(RuntimeException.class, loader.load(tp, coordinator)); @@ -127,59 +127,59 @@ class CoordinatorLoaderImplTest { CoordinatorPlayback> coordinator = mock(CoordinatorPlayback.class); try (CoordinatorLoaderImpl> loader = new CoordinatorLoaderImpl<>( - Time.SYSTEM, - partitionLogSupplier, - partitionLogEndOffsetSupplier, - serde, - 1000 + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 )) { when(log.logStartOffset()).thenReturn(0L); when(log.highWatermark()).thenReturn(0L); FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( - new SimpleRecord("k1".getBytes(), "v1".getBytes()), - new SimpleRecord("k2".getBytes(), "v2".getBytes()) + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) )); when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) - .thenReturn(readResult1); + .thenReturn(readResult1); FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( - new SimpleRecord("k3".getBytes(), "v3".getBytes()), - new SimpleRecord("k4".getBytes(), "v4".getBytes()), - new SimpleRecord("k5".getBytes(), "v5".getBytes()) + new SimpleRecord("k3".getBytes(), "v3".getBytes()), + new SimpleRecord("k4".getBytes(), "v4".getBytes()), + new SimpleRecord("k5".getBytes(), "v5".getBytes()) )); when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) - .thenReturn(readResult2); + .thenReturn(readResult2); FetchDataInfo readResult3 = logReadResult(5, 100L, (short) 5, Arrays.asList( - new SimpleRecord("k6".getBytes(), "v6".getBytes()), - new SimpleRecord("k7".getBytes(), "v7".getBytes()) + new SimpleRecord("k6".getBytes(), "v6".getBytes()), + new SimpleRecord("k7".getBytes(), "v7".getBytes()) )); when(log.read(5L, 1000, FetchIsolation.LOG_END, true)) - .thenReturn(readResult3); + .thenReturn(readResult3); FetchDataInfo readResult4 = logReadResult( - 7, - 100L, - (short) 5, - ControlRecordType.COMMIT + 7, + 100L, + (short) 5, + ControlRecordType.COMMIT ); when(log.read(7L, 1000, FetchIsolation.LOG_END, true)) - .thenReturn(readResult4); + .thenReturn(readResult4); FetchDataInfo readResult5 = logReadResult( - 8, - 500L, - (short) 10, - ControlRecordType.ABORT + 8, + 500L, + (short) 10, + ControlRecordType.ABORT ); when(log.read(8L, 1000, FetchIsolation.LOG_END, true)) - .thenReturn(readResult5); + .thenReturn(readResult5); CoordinatorLoader.LoadSummary summary = loader.load(tp, coordinator).get(10, TimeUnit.SECONDS); assertNotNull(summary); @@ -213,25 +213,25 @@ class CoordinatorLoaderImplTest { CoordinatorPlayback> coordinator = mock(CoordinatorPlayback.class); try (CoordinatorLoaderImpl> loader = new CoordinatorLoaderImpl<>( - Time.SYSTEM, - partitionLogSupplier, - partitionLogEndOffsetSupplier, - serde, - 1000 + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 )) { when(log.logStartOffset()).thenReturn(0L); FetchDataInfo readResult = logReadResult(0, Arrays.asList( - new SimpleRecord("k1".getBytes(), "v1".getBytes()), - new SimpleRecord("k2".getBytes(), "v2".getBytes()) + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) )); CountDownLatch latch = new CountDownLatch(1); when(log.read( - anyLong(), - eq(1000), - eq(FetchIsolation.LOG_END), - eq(true) + anyLong(), + eq(1000), + eq(FetchIsolation.LOG_END), + eq(true) )).thenAnswer((InvocationOnMock invocation) -> { latch.countDown(); return readResult; @@ -258,25 +258,25 @@ class CoordinatorLoaderImplTest { CoordinatorPlayback> coordinator = mock(CoordinatorPlayback.class); try (CoordinatorLoaderImpl> loader = new CoordinatorLoaderImpl<>( - Time.SYSTEM, - partitionLogSupplier, - partitionLogEndOffsetSupplier, - serde, - 1000 + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 )) { when(log.logStartOffset()).thenReturn(0L); FetchDataInfo readResult = logReadResult(0, Arrays.asList( - new SimpleRecord("k1".getBytes(), "v1".getBytes()), - new SimpleRecord("k2".getBytes(), "v2".getBytes()) + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) )); when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) - .thenReturn(readResult); + .thenReturn(readResult); when(serde.deserialize(any(ByteBuffer.class), any(ByteBuffer.class))) - .thenThrow(new Deserializer.UnknownRecordTypeException((short) 1)) - .thenReturn(Map.entry("k2", "v2")); + .thenThrow(new Deserializer.UnknownRecordTypeException((short) 1)) + .thenReturn(Map.entry("k2", "v2")); loader.load(tp, coordinator).get(10, TimeUnit.SECONDS); @@ -294,24 +294,24 @@ class CoordinatorLoaderImplTest { CoordinatorPlayback> coordinator = mock(CoordinatorPlayback.class); try (CoordinatorLoaderImpl> loader = new CoordinatorLoaderImpl<>( - Time.SYSTEM, - partitionLogSupplier, - partitionLogEndOffsetSupplier, - serde, - 1000 + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 )) { when(log.logStartOffset()).thenReturn(0L); FetchDataInfo readResult = logReadResult(0, Arrays.asList( - new SimpleRecord("k1".getBytes(), "v1".getBytes()), - new SimpleRecord("k2".getBytes(), "v2".getBytes()) + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) )); when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) - .thenReturn(readResult); + .thenReturn(readResult); when(serde.deserialize(any(ByteBuffer.class), any(ByteBuffer.class))) - .thenThrow(new RuntimeException("Error!")); + .thenThrow(new RuntimeException("Error!")); RuntimeException ex = assertFutureThrows(RuntimeException.class, loader.load(tp, coordinator)); @@ -333,18 +333,18 @@ class CoordinatorLoaderImplTest { CoordinatorPlayback> coordinator = mock(CoordinatorPlayback.class); try (CoordinatorLoaderImpl> loader = new CoordinatorLoaderImpl<>( - Time.SYSTEM, - partitionLogSupplier, - partitionLogEndOffsetSupplier, - serde, - 1000 + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 )) { when(log.logStartOffset()).thenReturn(0L); FetchDataInfo readResult = logReadResult(0, List.of()); when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) - .thenReturn(readResult); + .thenReturn(readResult); assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS)); } @@ -361,34 +361,34 @@ class CoordinatorLoaderImplTest { MockTime time = new MockTime(); try (CoordinatorLoaderImpl> loader = new CoordinatorLoaderImpl<>( - time, - partitionLogSupplier, - partitionLogEndOffsetSupplier, - serde, - 1000 + time, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 )) { long startTimeMs = time.milliseconds(); when(log.logStartOffset()).thenReturn(0L); FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( - new SimpleRecord("k1".getBytes(), "v1".getBytes()), - new SimpleRecord("k2".getBytes(), "v2".getBytes()) + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) )); when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) - .thenAnswer((InvocationOnMock invocation) -> { - time.sleep(1000); - return readResult1; - }); + .thenAnswer((InvocationOnMock invocation) -> { + time.sleep(1000); + return readResult1; + }); FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( - new SimpleRecord("k3".getBytes(), "v3".getBytes()), - new SimpleRecord("k4".getBytes(), "v4".getBytes()), - new SimpleRecord("k5".getBytes(), "v5".getBytes()) + new SimpleRecord("k3".getBytes(), "v3".getBytes()), + new SimpleRecord("k4".getBytes(), "v4".getBytes()), + new SimpleRecord("k5".getBytes(), "v5".getBytes()) )); when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) - .thenReturn(readResult2); + .thenReturn(readResult2); CoordinatorLoader.LoadSummary summary = loader.load(tp, coordinator).get(10, TimeUnit.SECONDS); assertEquals(startTimeMs, summary.startTimeMs()); @@ -408,39 +408,39 @@ class CoordinatorLoaderImplTest { CoordinatorPlayback> coordinator = mock(CoordinatorPlayback.class); try (CoordinatorLoaderImpl> loader = new CoordinatorLoaderImpl<>( - Time.SYSTEM, - partitionLogSupplier, - partitionLogEndOffsetSupplier, - serde, - 1000 + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 )) { when(log.logStartOffset()).thenReturn(0L); when(log.highWatermark()).thenReturn(0L, 0L, 2L); FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( - new SimpleRecord("k1".getBytes(), "v1".getBytes()), - new SimpleRecord("k2".getBytes(), "v2".getBytes()) + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) )); when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) .thenReturn(readResult1); FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( - new SimpleRecord("k3".getBytes(), "v3".getBytes()), - new SimpleRecord("k4".getBytes(), "v4".getBytes()), - new SimpleRecord("k5".getBytes(), "v5".getBytes()) + new SimpleRecord("k3".getBytes(), "v3".getBytes()), + new SimpleRecord("k4".getBytes(), "v4".getBytes()), + new SimpleRecord("k5".getBytes(), "v5".getBytes()) )); when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) - .thenReturn(readResult2); + .thenReturn(readResult2); FetchDataInfo readResult3 = logReadResult(5, Arrays.asList( - new SimpleRecord("k6".getBytes(), "v6".getBytes()), - new SimpleRecord("k7".getBytes(), "v7".getBytes()) + new SimpleRecord("k6".getBytes(), "v6".getBytes()), + new SimpleRecord("k7".getBytes(), "v7".getBytes()) )); when(log.read(5L, 1000, FetchIsolation.LOG_END, true)) - .thenReturn(readResult3); + .thenReturn(readResult3); assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS)); @@ -471,11 +471,11 @@ class CoordinatorLoaderImplTest { CoordinatorPlayback> coordinator = mock(CoordinatorPlayback.class); try (CoordinatorLoaderImpl> loader = new CoordinatorLoaderImpl<>( - Time.SYSTEM, - partitionLogSupplier, - partitionLogEndOffsetSupplier, - serde, - 1000 + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 )) { when(log.logStartOffset()).thenReturn(0L); when(log.highWatermark()).thenReturn(0L); @@ -497,39 +497,39 @@ class CoordinatorLoaderImplTest { CoordinatorPlayback> coordinator = mock(CoordinatorPlayback.class); try (CoordinatorLoaderImpl> loader = new CoordinatorLoaderImpl<>( - Time.SYSTEM, - partitionLogSupplier, - partitionLogEndOffsetSupplier, - serde, - 1000 + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 )) { when(log.logStartOffset()).thenReturn(0L); when(log.highWatermark()).thenReturn(5L, 7L, 7L); FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( - new SimpleRecord("k1".getBytes(), "v1".getBytes()), - new SimpleRecord("k2".getBytes(), "v2".getBytes()) + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) )); when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) - .thenReturn(readResult1); + .thenReturn(readResult1); FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( - new SimpleRecord("k3".getBytes(), "v3".getBytes()), - new SimpleRecord("k4".getBytes(), "v4".getBytes()), - new SimpleRecord("k5".getBytes(), "v5".getBytes()) + new SimpleRecord("k3".getBytes(), "v3".getBytes()), + new SimpleRecord("k4".getBytes(), "v4".getBytes()), + new SimpleRecord("k5".getBytes(), "v5".getBytes()) )); when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) - .thenReturn(readResult2); + .thenReturn(readResult2); FetchDataInfo readResult3 = logReadResult(5, Arrays.asList( - new SimpleRecord("k6".getBytes(), "v6".getBytes()), - new SimpleRecord("k7".getBytes(), "v7".getBytes()) + new SimpleRecord("k6".getBytes(), "v6".getBytes()), + new SimpleRecord("k7".getBytes(), "v7".getBytes()) )); when(log.read(5L, 1000, FetchIsolation.LOG_END, true)) - .thenReturn(readResult3); + .thenReturn(readResult3); assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS)); @@ -561,32 +561,32 @@ class CoordinatorLoaderImplTest { CoordinatorPlayback> coordinator = mock(CoordinatorPlayback.class); try (CoordinatorLoaderImpl> loader = new CoordinatorLoaderImpl<>( - Time.SYSTEM, - partitionLogSupplier, - partitionLogEndOffsetSupplier, - serde, - 1000 + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 )) { when(log.logStartOffset()).thenReturn(0L); when(log.highWatermark()).thenReturn(0L); when(partitionLogEndOffsetSupplier.apply(tp)).thenReturn(Optional.of(5L)).thenReturn(Optional.of(-1L)); FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( - new SimpleRecord("k1".getBytes(), "v1".getBytes()), - new SimpleRecord("k2".getBytes(), "v2".getBytes()) + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) )); when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) - .thenReturn(readResult1); + .thenReturn(readResult1); FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( - new SimpleRecord("k3".getBytes(), "v3".getBytes()), - new SimpleRecord("k4".getBytes(), "v4".getBytes()), - new SimpleRecord("k5".getBytes(), "v5".getBytes()) + new SimpleRecord("k3".getBytes(), "v3".getBytes()), + new SimpleRecord("k4".getBytes(), "v4".getBytes()), + new SimpleRecord("k5".getBytes(), "v5".getBytes()) )); when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) - .thenReturn(readResult2); + .thenReturn(readResult2); assertFutureThrows(NotLeaderOrFollowerException.class, loader.load(tp, coordinator)); } @@ -597,28 +597,28 @@ class CoordinatorLoaderImplTest { } private FetchDataInfo logReadResult( - long startOffset, - long producerId, - short producerEpoch, - List records + long startOffset, + long producerId, + short producerEpoch, + List records ) throws IOException { FileRecords fileRecords = mock(FileRecords.class); MemoryRecords memoryRecords; if (producerId == RecordBatch.NO_PRODUCER_ID) { memoryRecords = MemoryRecords.withRecords( - startOffset, - Compression.NONE, - records.toArray(new SimpleRecord[0]) + startOffset, + Compression.NONE, + records.toArray(new SimpleRecord[0]) ); } else { memoryRecords = MemoryRecords.withTransactionalRecords( - startOffset, - Compression.NONE, - producerId, - producerEpoch, - 0, - RecordBatch.NO_PARTITION_LEADER_EPOCH, - records.toArray(new SimpleRecord[0]) + startOffset, + Compression.NONE, + producerId, + producerEpoch, + 0, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + records.toArray(new SimpleRecord[0]) ); } @@ -635,19 +635,19 @@ class CoordinatorLoaderImplTest { } private FetchDataInfo logReadResult( - long startOffset, - long producerId, - short producerEpoch, - ControlRecordType controlRecordType + long startOffset, + long producerId, + short producerEpoch, + ControlRecordType controlRecordType ) throws IOException { FileRecords fileRecords = mock(FileRecords.class); MemoryRecords memoryRecords = MemoryRecords.withEndTransactionMarker( - startOffset, - 0L, - RecordBatch.NO_PARTITION_LEADER_EPOCH, - producerId, - producerEpoch, - new EndTransactionMarker(controlRecordType, 0) + startOffset, + 0L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + producerId, + producerEpoch, + new EndTransactionMarker(controlRecordType, 0) ); when(fileRecords.sizeInBytes()).thenReturn(memoryRecords.sizeInBytes()); From 5ed4a48829d47fb2567abad08fad09546a531b5e Mon Sep 17 00:00:00 2001 From: Andrew Schofield Date: Fri, 19 Sep 2025 03:10:12 +0100 Subject: [PATCH 049/100] MINOR: Tighten up argument descriptions for console CLI tools (#20554) Small improvements to the argument descriptions in the usage messages for the console producer/consumer tools. Reviewers: Chia-Ping Tsai --- .../org/apache/kafka/tools/ConsoleProducer.java | 4 ++-- .../tools/consumer/ConsoleConsumerOptions.java | 14 +++++++------- .../consumer/ConsoleShareConsumerOptions.java | 8 ++++---- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/tools/src/main/java/org/apache/kafka/tools/ConsoleProducer.java b/tools/src/main/java/org/apache/kafka/tools/ConsoleProducer.java index 3e9e34f4210..333daaf0fd5 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ConsoleProducer.java +++ b/tools/src/main/java/org/apache/kafka/tools/ConsoleProducer.java @@ -254,12 +254,12 @@ public class ConsoleProducer { .withRequiredArg() .describedAs("config file") .ofType(String.class); - producerPropertyOpt = parser.accepts("producer-property", "(DEPRECATED) A mechanism to pass user-defined properties in the form key=value to the producer." + + producerPropertyOpt = parser.accepts("producer-property", "(DEPRECATED) Producer config properties in the form key=value. " + "This option will be removed in a future version. Use --command-property instead.") .withRequiredArg() .describedAs("producer_prop") .ofType(String.class); - commandPropertyOpt = parser.accepts("command-property", "A mechanism to pass user-defined properties in the form key=value to the producer.") + commandPropertyOpt = parser.accepts("command-property", "Producer config properties in the form key=value.") .withRequiredArg() .describedAs("producer_prop") .ofType(String.class); diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java index abe6322fd97..ade48b5b670 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java @@ -71,7 +71,7 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions { @SuppressWarnings("deprecation") public ConsoleConsumerOptions(String[] args) throws IOException { super(args); - topicOpt = parser.accepts("topic", "The topic to consume on.") + topicOpt = parser.accepts("topic", "The topic to consume from.") .withRequiredArg() .describedAs("topic") .ofType(String.class); @@ -91,12 +91,12 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions { .ofType(String.class) .defaultsTo("latest"); @Deprecated(since = "4.2", forRemoval = true) - OptionSpec consumerPropertyOpt = parser.accepts("consumer-property", "(DEPRECATED) A mechanism to pass user-defined properties in the form key=value to the consumer. " + + OptionSpec consumerPropertyOpt = parser.accepts("consumer-property", "(DEPRECATED) Consumer config properties in the form key=value. " + "This option will be removed in a future version. Use --command-property instead.") .withRequiredArg() .describedAs("consumer_prop") .ofType(String.class); - OptionSpec commandPropertyOpt = parser.accepts("command-property", "A mechanism to pass user-defined properties in the form key=value to the consumer.") + OptionSpec commandPropertyOpt = parser.accepts("command-property", "Consumer config properties in the form key=value.") .withRequiredArg() .describedAs("consumer_prop") .ofType(String.class); @@ -126,7 +126,7 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions { " print.value=true|false\n" + " key.separator=\n" + " line.separator=\n" + - " headers.separator=\n" + + " headers.separator=\n" + " null.literal=\n" + " key.deserializer=\n" + " value.deserializer=\n" + @@ -176,11 +176,11 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions { .withRequiredArg() .describedAs("server to connect to") .ofType(String.class); - keyDeserializerOpt = parser.accepts("key-deserializer") + keyDeserializerOpt = parser.accepts("key-deserializer", "The name of the class to use for deserializing keys.") .withRequiredArg() - .describedAs("deserializer for key") + .describedAs("deserializer for keys") .ofType(String.class); - valueDeserializerOpt = parser.accepts("value-deserializer") + valueDeserializerOpt = parser.accepts("value-deserializer", "The name of the class to use for deserializing values.") .withRequiredArg() .describedAs("deserializer for values") .ofType(String.class); diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptions.java b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptions.java index 3472d07afa5..2c9d9600c14 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptions.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptions.java @@ -62,12 +62,12 @@ public final class ConsoleShareConsumerOptions extends CommandDefaultOptions { .describedAs("topic") .ofType(String.class); @Deprecated(since = "4.2", forRemoval = true) - OptionSpec consumerPropertyOpt = parser.accepts("consumer-property", "(DEPRECATED) A mechanism to pass user-defined properties in the form key=value to the consumer. " + + OptionSpec consumerPropertyOpt = parser.accepts("consumer-property", "(DEPRECATED) Consumer config properties in the form key=value. " + "This option will be removed in a future version. Use --command-property instead.") .withRequiredArg() .describedAs("consumer_prop") .ofType(String.class); - OptionSpec commandPropertyOpt = parser.accepts("command-property", "A mechanism to pass user-defined properties in the form key=value to the consumer.") + OptionSpec commandPropertyOpt = parser.accepts("command-property", "Consumer config properties in the form key=value.") .withRequiredArg() .describedAs("consumer_prop") .ofType(String.class); @@ -98,7 +98,7 @@ public final class ConsoleShareConsumerOptions extends CommandDefaultOptions { " print.value=true|false\n" + " key.separator=\n" + " line.separator=\n" + - " headers.separator=\n" + + " headers.separator=\n" + " null.literal=\n" + " key.deserializer=\n" + " value.deserializer=\n" + @@ -151,7 +151,7 @@ public final class ConsoleShareConsumerOptions extends CommandDefaultOptions { .ofType(String.class); keyDeserializerOpt = parser.accepts("key-deserializer", "The name of the class to use for deserializing keys.") .withRequiredArg() - .describedAs("deserializer for key") + .describedAs("deserializer for keys") .ofType(String.class); valueDeserializerOpt = parser.accepts("value-deserializer", "The name of the class to use for deserializing values.") .withRequiredArg() From b72db2b2c7885937878139bcf7ccb06a1268effd Mon Sep 17 00:00:00 2001 From: Ryan Dielhenn Date: Thu, 18 Sep 2025 19:22:05 -0700 Subject: [PATCH 050/100] MINOR: Delete temporary directories after using them in RaftManagerTest Updated (#20550) Follow-up to [#11193](https://github.com/apache/kafka/pull/11193). This change adds cleanup of the temporary log and metadata directories created by RaftManagerTest so they are removed after each test run. Without this cleanup, the directories remain until the entire test suite completes, leaving extra files in the system temporary directory. Testing: - Ran `./gradlew core:test --tests kafka.raft.RaftManagerTest` and confirmed all tests pass. Reviewers: TengYao Chi , Chia-Ping Tsai --- .../unit/kafka/raft/RaftManagerTest.scala | 83 +++++++++++-------- 1 file changed, 49 insertions(+), 34 deletions(-) diff --git a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala index 38187b11eb8..117c2b63978 100644 --- a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala +++ b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala @@ -30,6 +30,7 @@ import org.apache.kafka.common.Uuid import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.utils.Time +import org.apache.kafka.common.utils.Utils import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.raft.{Endpoints, MetadataLogConfig, QuorumConfig} import org.apache.kafka.server.ProcessRole @@ -126,17 +127,21 @@ class RaftManagerTest { val logDir = TestUtils.tempDir() val nodeId = 1 - val raftManager = createRaftManager( - new TopicPartition("__raft_id_test", 0), - createConfig( - processRolesSet, - nodeId, - Seq(logDir.toPath), - None + try { + val raftManager = createRaftManager( + new TopicPartition("__raft_id_test", 0), + createConfig( + processRolesSet, + nodeId, + Seq(logDir.toPath), + None + ) ) - ) - assertEquals(nodeId, raftManager.client.nodeId.getAsInt) - raftManager.shutdown() + assertEquals(nodeId, raftManager.client.nodeId.getAsInt) + raftManager.shutdown() + } finally { + Utils.delete(logDir) + } } @ParameterizedTest @@ -155,22 +160,27 @@ class RaftManagerTest { } val nodeId = 1 - val raftManager = createRaftManager( - new TopicPartition("__raft_id_test", 0), - createConfig( - Set(ProcessRole.ControllerRole), - nodeId, - logDir, - metadataDir + try { + val raftManager = createRaftManager( + new TopicPartition("__raft_id_test", 0), + createConfig( + Set(ProcessRole.ControllerRole), + nodeId, + logDir, + metadataDir + ) ) - ) - val lockPath = metadataDir.getOrElse(logDir.head).resolve(LogManager.LOCK_FILE_NAME) - assertTrue(fileLocked(lockPath)) + val lockPath = metadataDir.getOrElse(logDir.head).resolve(LogManager.LOCK_FILE_NAME) + assertTrue(fileLocked(lockPath)) - raftManager.shutdown() + raftManager.shutdown() - assertFalse(fileLocked(lockPath)) + assertFalse(fileLocked(lockPath)) + } finally { + logDir.foreach(p => Utils.delete(p.toFile)) + metadataDir.foreach(p => Utils.delete(p.toFile)) + } } @Test @@ -179,22 +189,27 @@ class RaftManagerTest { val metadataDir = Some(TestUtils.tempDir().toPath) val nodeId = 1 - val raftManager = createRaftManager( - new TopicPartition("__raft_id_test", 0), - createConfig( - Set(ProcessRole.BrokerRole), - nodeId, - logDir, - metadataDir + try { + val raftManager = createRaftManager( + new TopicPartition("__raft_id_test", 0), + createConfig( + Set(ProcessRole.BrokerRole), + nodeId, + logDir, + metadataDir + ) ) - ) - val lockPath = metadataDir.getOrElse(logDir.head).resolve(LogManager.LOCK_FILE_NAME) - assertTrue(fileLocked(lockPath)) + val lockPath = metadataDir.getOrElse(logDir.head).resolve(LogManager.LOCK_FILE_NAME) + assertTrue(fileLocked(lockPath)) - raftManager.shutdown() + raftManager.shutdown() - assertFalse(fileLocked(lockPath)) + assertFalse(fileLocked(lockPath)) + } finally { + logDir.foreach(p => Utils.delete(p.toFile)) + metadataDir.foreach(p => Utils.delete(p.toFile)) + } } def createMetadataLog(config: KafkaConfig): Unit = { From d067c6c04089a3d24e1f72e6cb1b10b0d85f76da Mon Sep 17 00:00:00 2001 From: Sean Quah Date: Fri, 19 Sep 2025 08:44:07 +0100 Subject: [PATCH 051/100] KAFKA-19716: Clear out coordinator snapshots periodically while loading (#20547) When nested Timeline collections are created and discarded while loading a coordinator partition, references to them accumulate in the current snapshot. Allow the GC to reclaim them by starting a new snapshot and discarding previous snapshots every 16,384 records. Small intervals degrade loading times for non-transactional offset commit workloads while large intervals degrade loading times for transactional workloads. A default of 16,384 was chosen as a compromise. Also add a benchmark for group coordinator loading. Reviewers: David Jacot --- build.gradle | 1 + checkstyle/import-control-jmh-benchmarks.xml | 4 +- .../common/runtime/CoordinatorLoaderImpl.java | 39 +- .../runtime/SnapshottableCoordinator.java | 2 +- .../runtime/CoordinatorLoaderImplTest.java | 109 +++++- .../scala/kafka/server/BrokerServer.scala | 6 +- ...GroupCoordinatorShardLoadingBenchmark.java | 355 ++++++++++++++++++ .../apache/kafka/jmh/coordinator/MockLog.java | 74 ++++ 8 files changed, 563 insertions(+), 27 deletions(-) create mode 100644 jmh-benchmarks/src/main/java/org/apache/kafka/jmh/coordinator/GroupCoordinatorShardLoadingBenchmark.java create mode 100644 jmh-benchmarks/src/main/java/org/apache/kafka/jmh/coordinator/MockLog.java diff --git a/build.gradle b/build.gradle index 26ca93d0738..753a86cfc35 100644 --- a/build.gradle +++ b/build.gradle @@ -3350,6 +3350,7 @@ project(':jmh-benchmarks') { implementation project(':raft') implementation project(':clients') implementation project(':coordinator-common') + implementation project(':coordinator-common').sourceSets.test.output implementation project(':group-coordinator') implementation project(':group-coordinator:group-coordinator-api') implementation project(':metadata') diff --git a/checkstyle/import-control-jmh-benchmarks.xml b/checkstyle/import-control-jmh-benchmarks.xml index d2f87a3577f..4c11bc3acb4 100644 --- a/checkstyle/import-control-jmh-benchmarks.xml +++ b/checkstyle/import-control-jmh-benchmarks.xml @@ -52,9 +52,7 @@ - - - + diff --git a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImpl.java b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImpl.java index 6613ce25fc8..3a8b7434326 100644 --- a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImpl.java +++ b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImpl.java @@ -50,6 +50,20 @@ import java.util.function.Function; */ public class CoordinatorLoaderImpl implements CoordinatorLoader { + /** + * The interval between updating the last committed offset during loading, in offsets. Smaller + * values commit more often at the expense of loading times when the workload is simple and does + * not create collections that need to participate in {@link CoordinatorPlayback} snapshotting. + * Larger values commit less often and allow more temporary data to accumulate before the next + * commit when the workload creates many temporary collections that need to be snapshotted. + * + * The value of 16,384 was chosen as a trade-off between the performance of these two workloads. + * + * When changing this value, please run the GroupCoordinatorShardLoadingBenchmark to evaluate + * the relative change in performance. + */ + public static final long DEFAULT_COMMIT_INTERVAL_OFFSETS = 16384; + private static final Logger LOG = LoggerFactory.getLogger(CoordinatorLoaderImpl.class); private final Time time; @@ -57,6 +71,7 @@ public class CoordinatorLoaderImpl implements CoordinatorLoader { private final Function> partitionLogEndOffsetSupplier; private final Deserializer deserializer; private final int loadBufferSize; + private final long commitIntervalOffsets; private final AtomicBoolean isRunning = new AtomicBoolean(true); private final KafkaScheduler scheduler = new KafkaScheduler(1); @@ -66,13 +81,15 @@ public class CoordinatorLoaderImpl implements CoordinatorLoader { Function> partitionLogSupplier, Function> partitionLogEndOffsetSupplier, Deserializer deserializer, - int loadBufferSize + int loadBufferSize, + long commitIntervalOffsets ) { this.time = time; this.partitionLogSupplier = partitionLogSupplier; this.partitionLogEndOffsetSupplier = partitionLogEndOffsetSupplier; this.deserializer = deserializer; this.loadBufferSize = loadBufferSize; + this.commitIntervalOffsets = commitIntervalOffsets; this.scheduler.startup(); } @@ -121,7 +138,7 @@ public class CoordinatorLoaderImpl implements CoordinatorLoader { long currentOffset = log.logStartOffset(); LoadStats stats = new LoadStats(); - long previousHighWatermark = -1L; + long lastCommittedOffset = -1L; while (shouldFetchNextBatch(currentOffset, logEndOffset(tp), stats.readAtLeastOneRecord)) { FetchDataInfo fetchDataInfo = log.read(currentOffset, loadBufferSize, FetchIsolation.LOG_END, true); @@ -133,9 +150,9 @@ public class CoordinatorLoaderImpl implements CoordinatorLoader { buffer = memoryRecords.buffer(); } - ReplayResult replayResult = processMemoryRecords(tp, log, memoryRecords, coordinator, stats, currentOffset, previousHighWatermark); + ReplayResult replayResult = processMemoryRecords(tp, log, memoryRecords, coordinator, stats, currentOffset, lastCommittedOffset); currentOffset = replayResult.nextOffset; - previousHighWatermark = replayResult.highWatermark; + lastCommittedOffset = replayResult.lastCommittedOffset; } long endTimeMs = time.milliseconds(); @@ -207,7 +224,7 @@ public class CoordinatorLoaderImpl implements CoordinatorLoader { CoordinatorPlayback coordinator, LoadStats loadStats, long currentOffset, - long previousHighWatermark + long lastCommittedOffset ) { for (MutableRecordBatch batch : memoryRecords.batches()) { if (batch.isControlBatch()) { @@ -286,14 +303,18 @@ public class CoordinatorLoaderImpl implements CoordinatorLoader { if (currentOffset >= currentHighWatermark) { coordinator.updateLastWrittenOffset(currentOffset); - if (currentHighWatermark > previousHighWatermark) { + if (currentHighWatermark > lastCommittedOffset) { coordinator.updateLastCommittedOffset(currentHighWatermark); - previousHighWatermark = currentHighWatermark; + lastCommittedOffset = currentHighWatermark; } + } else if (currentOffset - lastCommittedOffset >= commitIntervalOffsets) { + coordinator.updateLastWrittenOffset(currentOffset); + coordinator.updateLastCommittedOffset(currentOffset); + lastCommittedOffset = currentOffset; } } loadStats.numBytes += memoryRecords.sizeInBytes(); - return new ReplayResult(currentOffset, previousHighWatermark); + return new ReplayResult(currentOffset, lastCommittedOffset); } /** @@ -326,5 +347,5 @@ public class CoordinatorLoaderImpl implements CoordinatorLoader { } } - private record ReplayResult(long nextOffset, long highWatermark) { } + private record ReplayResult(long nextOffset, long lastCommittedOffset) { } } diff --git a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/SnapshottableCoordinator.java b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/SnapshottableCoordinator.java index 1550e444bf4..278373e6842 100644 --- a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/SnapshottableCoordinator.java +++ b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/SnapshottableCoordinator.java @@ -61,7 +61,7 @@ public class SnapshottableCoordinator, U> implemen */ private long lastCommittedOffset; - SnapshottableCoordinator( + public SnapshottableCoordinator( LogContext logContext, SnapshotRegistry snapshotRegistry, S coordinator, diff --git a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImplTest.java b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImplTest.java index 9f8ab68c66f..11cdab83cac 100644 --- a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImplTest.java +++ b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImplTest.java @@ -91,7 +91,8 @@ class CoordinatorLoaderImplTest { partitionLogSupplier, partitionLogEndOffsetSupplier, serde, - 1000 + 1000, + CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS )) { assertFutureThrows(NotLeaderOrFollowerException.class, loader.load(tp, coordinator)); } @@ -110,7 +111,8 @@ class CoordinatorLoaderImplTest { partitionLogSupplier, partitionLogEndOffsetSupplier, serde, - 1000 + 1000, + CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS )) { loader.close(); assertFutureThrows(RuntimeException.class, loader.load(tp, coordinator)); @@ -131,7 +133,8 @@ class CoordinatorLoaderImplTest { partitionLogSupplier, partitionLogEndOffsetSupplier, serde, - 1000 + 1000, + CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS )) { when(log.logStartOffset()).thenReturn(0L); when(log.highWatermark()).thenReturn(0L); @@ -217,7 +220,8 @@ class CoordinatorLoaderImplTest { partitionLogSupplier, partitionLogEndOffsetSupplier, serde, - 1000 + 1000, + CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS )) { when(log.logStartOffset()).thenReturn(0L); @@ -262,7 +266,8 @@ class CoordinatorLoaderImplTest { partitionLogSupplier, partitionLogEndOffsetSupplier, serde, - 1000 + 1000, + CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS )) { when(log.logStartOffset()).thenReturn(0L); @@ -298,7 +303,8 @@ class CoordinatorLoaderImplTest { partitionLogSupplier, partitionLogEndOffsetSupplier, serde, - 1000 + 1000, + CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS )) { when(log.logStartOffset()).thenReturn(0L); @@ -337,7 +343,8 @@ class CoordinatorLoaderImplTest { partitionLogSupplier, partitionLogEndOffsetSupplier, serde, - 1000 + 1000, + CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS )) { when(log.logStartOffset()).thenReturn(0L); @@ -365,7 +372,8 @@ class CoordinatorLoaderImplTest { partitionLogSupplier, partitionLogEndOffsetSupplier, serde, - 1000 + 1000, + CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS )) { long startTimeMs = time.milliseconds(); when(log.logStartOffset()).thenReturn(0L); @@ -412,7 +420,8 @@ class CoordinatorLoaderImplTest { partitionLogSupplier, partitionLogEndOffsetSupplier, serde, - 1000 + 1000, + CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS )) { when(log.logStartOffset()).thenReturn(0L); when(log.highWatermark()).thenReturn(0L, 0L, 2L); @@ -475,7 +484,8 @@ class CoordinatorLoaderImplTest { partitionLogSupplier, partitionLogEndOffsetSupplier, serde, - 1000 + 1000, + CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS )) { when(log.logStartOffset()).thenReturn(0L); when(log.highWatermark()).thenReturn(0L); @@ -501,7 +511,8 @@ class CoordinatorLoaderImplTest { partitionLogSupplier, partitionLogEndOffsetSupplier, serde, - 1000 + 1000, + CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS )) { when(log.logStartOffset()).thenReturn(0L); when(log.highWatermark()).thenReturn(5L, 7L, 7L); @@ -551,6 +562,79 @@ class CoordinatorLoaderImplTest { } } + @Test + void testUpdateLastWrittenOffsetCommitInterval() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + UnifiedLog log = mock(UnifiedLog.class); + Function> partitionLogSupplier = partition -> Optional.of(log); + Function> partitionLogEndOffsetSupplier = partition -> Optional.of(7L); + StringKeyValueDeserializer serde = new StringKeyValueDeserializer(); + CoordinatorPlayback> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000, + 2L + )) { + when(log.logStartOffset()).thenReturn(0L); + when(log.highWatermark()).thenReturn(7L); + + FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) + )); + + when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult1); + + FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( + new SimpleRecord("k3".getBytes(), "v3".getBytes()), + new SimpleRecord("k4".getBytes(), "v4".getBytes()), + new SimpleRecord("k5".getBytes(), "v5".getBytes()) + )); + + when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult2); + + FetchDataInfo readResult3 = logReadResult(5, Arrays.asList( + new SimpleRecord("k6".getBytes(), "v6".getBytes()) + )); + + when(log.read(5L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult3); + + FetchDataInfo readResult4 = logReadResult(6, Arrays.asList( + new SimpleRecord("k7".getBytes(), "v7".getBytes()) + )); + + when(log.read(6L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult4); + + assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS)); + + verify(coordinator).replay(0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, Map.entry("k1", "v1")); + verify(coordinator).replay(1L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, Map.entry("k2", "v2")); + verify(coordinator).replay(2L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, Map.entry("k3", "v3")); + verify(coordinator).replay(3L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, Map.entry("k4", "v4")); + verify(coordinator).replay(4L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, Map.entry("k5", "v5")); + verify(coordinator).replay(5L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, Map.entry("k6", "v6")); + verify(coordinator).replay(6L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, Map.entry("k7", "v7")); + verify(coordinator, times(0)).updateLastWrittenOffset(0L); + verify(coordinator, times(1)).updateLastWrittenOffset(2L); + verify(coordinator, times(1)).updateLastWrittenOffset(5L); + verify(coordinator, times(0)).updateLastWrittenOffset(6L); + verify(coordinator, times(1)).updateLastWrittenOffset(7L); + verify(coordinator, times(0)).updateLastCommittedOffset(0L); + verify(coordinator, times(1)).updateLastCommittedOffset(2L); + verify(coordinator, times(1)).updateLastCommittedOffset(5L); + verify(coordinator, times(0)).updateLastCommittedOffset(6L); + verify(coordinator, times(1)).updateLastCommittedOffset(7L); + } + } + @Test void testPartitionGoesOfflineDuringLoad() throws Exception { TopicPartition tp = new TopicPartition("foo", 0); @@ -565,7 +649,8 @@ class CoordinatorLoaderImplTest { partitionLogSupplier, partitionLogEndOffsetSupplier, serde, - 1000 + 1000, + CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS )) { when(log.logStartOffset()).thenReturn(0L); when(log.highWatermark()).thenReturn(0L); diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 3ded033020b..47085169979 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -613,7 +613,8 @@ class BrokerServer( tp => replicaManager.getLog(tp).toJava, tp => replicaManager.getLogEndOffset(tp).map(Long.box).toJava, serde, - config.groupCoordinatorConfig.offsetsLoadBufferSize + config.groupCoordinatorConfig.offsetsLoadBufferSize, + CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS ) val writer = new CoordinatorPartitionWriter( replicaManager @@ -644,7 +645,8 @@ class BrokerServer( tp => replicaManager.getLog(tp).toJava, tp => replicaManager.getLogEndOffset(tp).map(Long.box).toJava, serde, - config.shareCoordinatorConfig.shareCoordinatorLoadBufferSize() + config.shareCoordinatorConfig.shareCoordinatorLoadBufferSize(), + CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS ) val writer = new CoordinatorPartitionWriter( replicaManager diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/coordinator/GroupCoordinatorShardLoadingBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/coordinator/GroupCoordinatorShardLoadingBenchmark.java new file mode 100644 index 00000000000..0a6612ca8fd --- /dev/null +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/coordinator/GroupCoordinatorShardLoadingBenchmark.java @@ -0,0 +1,355 @@ +/* + * 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.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.compress.Compression; +import org.apache.kafka.common.internals.Topic; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.record.ControlRecordType; +import org.apache.kafka.common.record.EndTransactionMarker; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.SimpleRecord; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.coordinator.common.runtime.CoordinatorLoader; +import org.apache.kafka.coordinator.common.runtime.CoordinatorLoaderImpl; +import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord; +import org.apache.kafka.coordinator.common.runtime.MockCoordinatorExecutor; +import org.apache.kafka.coordinator.common.runtime.MockCoordinatorTimer; +import org.apache.kafka.coordinator.common.runtime.SnapshottableCoordinator; +import org.apache.kafka.coordinator.group.GroupConfigManager; +import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; +import org.apache.kafka.coordinator.group.GroupCoordinatorRecordHelpers; +import org.apache.kafka.coordinator.group.GroupCoordinatorRecordSerde; +import org.apache.kafka.coordinator.group.GroupCoordinatorShard; +import org.apache.kafka.coordinator.group.OffsetAndMetadata; +import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics; +import org.apache.kafka.server.storage.log.FetchIsolation; +import org.apache.kafka.storage.internals.log.FetchDataInfo; +import org.apache.kafka.storage.internals.log.LogOffsetMetadata; +import org.apache.kafka.storage.internals.log.UnifiedLog; +import org.apache.kafka.timeline.SnapshotRegistry; + +import com.yammer.metrics.core.MetricsRegistry; + +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 org.openjdk.jmh.infra.BenchmarkParams; +import org.openjdk.jmh.infra.IterationParams; +import org.openjdk.jmh.runner.IterationType; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.OptionalLong; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 1) +@Measurement(iterations = 1) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.MILLISECONDS) +public class GroupCoordinatorShardLoadingBenchmark { + + private static final String GROUP_ID = "test-group"; + + @Param({"1", "4", "16", "64", "256", "1024", "4096", "16384", "65536", "262144", "1048576"}) + private long commitInterval; + + @Param({"8192"}) + private int batchCount; + + @Param({"2048"}) + private int batchSize; + + private TopicPartition topicPartition; + private MockTime time; + private GroupCoordinatorConfig config; + private GroupCoordinatorRecordSerde serde; + private GroupCoordinatorShard coordinatorShard; + private SnapshottableCoordinator snapshottableCoordinator; + private UnifiedLog offsetCommitLog; + private UnifiedLog transactionalOffsetCommitLog; + + static class OffsetCommitLog extends MockLog { + private final int batchCount; + private final SimpleRecord[] batch; + + public OffsetCommitLog(TopicPartition tp, int batchSize, int batchCount) throws IOException { + super(tp); + + this.batchCount = batchCount; + + List batchRecords = new ArrayList<>(); + for (int i = 0; i < batchSize; i++) { + String topic = "topic-" + i; + int partition = 0; + + OffsetAndMetadata offsetAndMetadata = new OffsetAndMetadata( + 0L, + OptionalInt.of(0), + OffsetAndMetadata.NO_METADATA, + 0L, + OptionalLong.empty(), + Uuid.randomUuid() + ); + + CoordinatorRecord coordinatorRecord = GroupCoordinatorRecordHelpers.newOffsetCommitRecord( + GROUP_ID, topic, partition, offsetAndMetadata + ); + + byte[] keyBytes = new GroupCoordinatorRecordSerde().serializeKey(coordinatorRecord); + byte[] valueBytes = new GroupCoordinatorRecordSerde().serializeValue(coordinatorRecord); + SimpleRecord simpleRecord = new SimpleRecord(keyBytes, valueBytes); + batchRecords.add(simpleRecord); + } + + this.batch = batchRecords.toArray(new SimpleRecord[0]); + } + + @Override + public long logStartOffset() { + return 0L; + } + + @Override + public long logEndOffset() { + if (batch == null) { + return 0L; + } + + return (long) batchCount * (long) batch.length; + } + + @Override + public FetchDataInfo read(long startOffset, int maxLength, FetchIsolation isolation, boolean minOneMessage) { + if (startOffset < 0 || startOffset >= logEndOffset()) { + return new FetchDataInfo(new LogOffsetMetadata(startOffset), MemoryRecords.EMPTY); + } + + MemoryRecords records = MemoryRecords.withRecords( + startOffset, + Compression.NONE, + batch + ); + return new FetchDataInfo(new LogOffsetMetadata(startOffset), records); + } + } + + static class TransactionalOffsetCommitLog extends MockLog { + private final int batchCount; + private final SimpleRecord[] batch; + private final long producerId; + private final short producerEpoch; + private final int coordinatorEpoch; + + public TransactionalOffsetCommitLog(TopicPartition tp, int batchSize, int batchCount) throws IOException { + super(tp); + + this.batchCount = batchCount; + this.producerId = 1000L; + this.producerEpoch = 0; + this.coordinatorEpoch = 100; + + List batchRecords = new ArrayList<>(); + for (int i = 0; i < batchSize - 1; i++) { + String topic = "topic-" + i; + int partition = 0; + + OffsetAndMetadata offsetAndMetadata = new OffsetAndMetadata( + 0L, + OptionalInt.of(0), + OffsetAndMetadata.NO_METADATA, + 0L, + OptionalLong.empty(), + Uuid.randomUuid() + ); + + CoordinatorRecord coordinatorRecord = GroupCoordinatorRecordHelpers.newOffsetCommitRecord( + GROUP_ID, topic, partition, offsetAndMetadata + ); + + byte[] keyBytes = new GroupCoordinatorRecordSerde().serializeKey(coordinatorRecord); + byte[] valueBytes = new GroupCoordinatorRecordSerde().serializeValue(coordinatorRecord); + SimpleRecord simpleRecord = new SimpleRecord(keyBytes, valueBytes); + batchRecords.add(simpleRecord); + } + + this.batch = batchRecords.toArray(new SimpleRecord[0]); + } + + @Override + public long logStartOffset() { + return 0L; + } + + @Override + public long logEndOffset() { + if (batch == null) { + return 0L; + } + + return (long) (batch.length + 1) * (long) batchCount; + } + + @Override + public FetchDataInfo read(long startOffset, int maxLength, FetchIsolation isolation, boolean minOneMessage) { + if (startOffset < 0 || startOffset >= logEndOffset()) { + return new FetchDataInfo(new LogOffsetMetadata(startOffset), MemoryRecords.EMPTY); + } + + // Repeat the batch followed by a commit marker. + long patternLength = batch.length + 1; + if (startOffset % patternLength < batch.length) { + MemoryRecords records = MemoryRecords.withTransactionalRecords( + startOffset, + Compression.NONE, + producerId, + producerEpoch, + 0, + 0, + batch + ); + return new FetchDataInfo(new LogOffsetMetadata(startOffset), records); + } else { + MemoryRecords records = MemoryRecords.withEndTransactionMarker( + startOffset, + 0L, + 0, + producerId, + producerEpoch, + new EndTransactionMarker(ControlRecordType.COMMIT, coordinatorEpoch) + ); + return new FetchDataInfo(new LogOffsetMetadata(startOffset), records); + } + } + } + + @Setup(Level.Trial) + public void setup() throws Exception { + topicPartition = new TopicPartition("__consumer_offsets", 0); + time = new MockTime(); + Map props = new HashMap<>(); + config = GroupCoordinatorConfig.fromProps(props); + serde = new GroupCoordinatorRecordSerde(); + } + + @Setup(Level.Iteration) + public void setupIteration(BenchmarkParams benchmarkParams, IterationParams iterationParams) throws IOException { + // Reduce the data size for warmup iterations, since transactional offset commit loading + // takes longer than 20 seconds. + int iterationBatchCount = batchCount; + if (iterationParams.getType() == IterationType.WARMUP) { + iterationBatchCount = Math.min(iterationBatchCount, 1024); + } + + offsetCommitLog = new OffsetCommitLog(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0), batchSize, iterationBatchCount); + transactionalOffsetCommitLog = new TransactionalOffsetCommitLog(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0), batchSize, iterationBatchCount); + } + + @Setup(Level.Invocation) + public void setupInvocation() { + GroupConfigManager configManager = new GroupConfigManager(new HashMap<>()); + LogContext logContext = new LogContext(); + SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext); + + MetricsRegistry metricsRegistry = new MetricsRegistry(); + Metrics metrics = new Metrics(); + GroupCoordinatorMetrics coordinatorMetrics = new GroupCoordinatorMetrics(metricsRegistry, metrics); + + coordinatorShard = new GroupCoordinatorShard.Builder(config, configManager) + .withAuthorizerPlugin(Optional.empty()) + .withLogContext(logContext) + .withSnapshotRegistry(snapshotRegistry) + .withTime(time) + .withTimer(new MockCoordinatorTimer<>(time)) + .withExecutor(new MockCoordinatorExecutor<>()) + .withCoordinatorMetrics(coordinatorMetrics) + .withTopicPartition(topicPartition) + .build(); + + snapshottableCoordinator = new SnapshottableCoordinator<>( + logContext, + snapshotRegistry, + coordinatorShard, + topicPartition + ); + } + + private CoordinatorLoader.LoadSummary loadRecords(UnifiedLog log) throws ExecutionException, InterruptedException { + Function> partitionLogSupplier = tp -> Optional.of(log); + Function> partitionLogEndOffsetSupplier = tp -> Optional.of(log.logEndOffset()); + + CoordinatorLoaderImpl loader = new CoordinatorLoaderImpl<>( + time, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + config.offsetsLoadBufferSize(), + commitInterval + ); + + return loader.load(topicPartition, snapshottableCoordinator).get(); + } + + @Benchmark + @Threads(1) + @OutputTimeUnit(TimeUnit.MILLISECONDS) + public CoordinatorLoader.LoadSummary loadOffsetCommitRecords() throws ExecutionException, InterruptedException { + return loadRecords(offsetCommitLog); + } + + @Benchmark + @Threads(1) + @OutputTimeUnit(TimeUnit.MILLISECONDS) + public CoordinatorLoader.LoadSummary loadTransactionalOffsetCommitRecords() throws ExecutionException, InterruptedException { + return loadRecords(transactionalOffsetCommitLog); + } + + public static void main(String[] args) throws RunnerException { + Options opt = new OptionsBuilder() + .include(GroupCoordinatorShardLoadingBenchmark.class.getSimpleName()) + .forks(1) + .build(); + + new Runner(opt).run(); + } +} diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/coordinator/MockLog.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/coordinator/MockLog.java new file mode 100644 index 00000000000..a7de83d5bcb --- /dev/null +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/coordinator/MockLog.java @@ -0,0 +1,74 @@ +/* + * 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.TopicPartition; +import org.apache.kafka.server.storage.log.FetchIsolation; +import org.apache.kafka.server.util.Scheduler; +import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache; +import org.apache.kafka.storage.internals.log.FetchDataInfo; +import org.apache.kafka.storage.internals.log.LocalLog; +import org.apache.kafka.storage.internals.log.LogOffsetsListener; +import org.apache.kafka.storage.internals.log.LogSegments; +import org.apache.kafka.storage.internals.log.ProducerStateManager; +import org.apache.kafka.storage.internals.log.UnifiedLog; +import org.apache.kafka.storage.log.metrics.BrokerTopicStats; + +import java.io.IOException; +import java.util.Optional; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public abstract class MockLog extends UnifiedLog { + + public MockLog(TopicPartition tp) throws IOException { + super( + 0, + createMockLocalLog(tp), + mock(BrokerTopicStats.class), + Integer.MAX_VALUE, + mock(LeaderEpochFileCache.class), + mock(ProducerStateManager.class), + Optional.empty(), + false, + LogOffsetsListener.NO_OP_OFFSETS_LISTENER + ); + } + + @Override + public abstract long logStartOffset(); + + @Override + public abstract long logEndOffset(); + + @Override + public long highWatermark() { + return logEndOffset(); + } + + @Override + public abstract FetchDataInfo read(long startOffset, int maxLength, FetchIsolation isolation, boolean minOneMessage); + + private static LocalLog createMockLocalLog(TopicPartition tp) { + LocalLog localLog = mock(LocalLog.class); + when(localLog.scheduler()).thenReturn(mock(Scheduler.class)); + when(localLog.segments()).thenReturn(mock(LogSegments.class)); + when(localLog.topicPartition()).thenReturn(tp); + return localLog; + } +} From cfa0b416efbd354f5209492c121d47daa8436e53 Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Fri, 19 Sep 2025 16:32:41 +0800 Subject: [PATCH 052/100] MINOR: Remove metrics attribute from StreamsGroup (#20559) The `metrics` attribute in `StreamsGroup` is not used anymore. This patch removes it. Reviewers: Ken Huang , Lucas Brutschy , Chia-Ping Tsai --- .../group/GroupMetadataManager.java | 6 ++-- .../group/streams/StreamsGroup.java | 10 +----- .../group/streams/StreamsGroupTest.java | 34 ++++++------------- 3 files changed, 14 insertions(+), 36 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 477e0499565..4f156d75daa 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 @@ -851,10 +851,10 @@ public class GroupMetadataManager { Group group = groups.get(groupId); if (group == null) { - return new StreamsGroup(logContext, snapshotRegistry, groupId, metrics); + return new StreamsGroup(logContext, snapshotRegistry, groupId); } else if (maybeDeleteEmptyClassicGroup(group, records)) { log.info("[GroupId {}] Converted the empty classic group to a streams group.", groupId); - return new StreamsGroup(logContext, snapshotRegistry, groupId, metrics); + return new StreamsGroup(logContext, snapshotRegistry, groupId); } else { return castToStreamsGroup(group); } @@ -1023,7 +1023,7 @@ public class GroupMetadataManager { } if (group == null) { - StreamsGroup streamsGroup = new StreamsGroup(logContext, snapshotRegistry, groupId, metrics); + StreamsGroup streamsGroup = new StreamsGroup(logContext, snapshotRegistry, groupId); groups.put(groupId, streamsGroup); return streamsGroup; } else if (group.type() == STREAMS) { diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java index b75f3926d08..72d4386321e 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java @@ -31,7 +31,6 @@ import org.apache.kafka.coordinator.group.Group; import org.apache.kafka.coordinator.group.OffsetExpirationCondition; import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl; import org.apache.kafka.coordinator.group.Utils; -import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard; import org.apache.kafka.coordinator.group.streams.topics.ConfiguredSubtopology; import org.apache.kafka.coordinator.group.streams.topics.ConfiguredTopology; import org.apache.kafka.timeline.SnapshotRegistry; @@ -179,11 +178,6 @@ public class StreamsGroup implements Group { private final TimelineHashMap>> currentStandbyTaskToProcessIds; private final TimelineHashMap>> currentWarmupTaskToProcessIds; - /** - * The coordinator metrics. - */ - private final GroupCoordinatorMetricsShard metrics; - /** * The Streams topology. */ @@ -220,8 +214,7 @@ public class StreamsGroup implements Group { public StreamsGroup( LogContext logContext, SnapshotRegistry snapshotRegistry, - String groupId, - GroupCoordinatorMetricsShard metrics + String groupId ) { this.log = logContext.logger(StreamsGroup.class); this.logContext = logContext; @@ -238,7 +231,6 @@ public class StreamsGroup implements Group { this.currentActiveTaskToProcessId = new TimelineHashMap<>(snapshotRegistry, 0); this.currentStandbyTaskToProcessIds = new TimelineHashMap<>(snapshotRegistry, 0); this.currentWarmupTaskToProcessIds = new TimelineHashMap<>(snapshotRegistry, 0); - this.metrics = Objects.requireNonNull(metrics); this.topology = new TimelineObject<>(snapshotRegistry, Optional.empty()); this.configuredTopology = new TimelineObject<>(snapshotRegistry, Optional.empty()); } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java index a76e282be6e..71feb2a1e90 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.coordinator.group.streams; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.GroupNotEmptyException; import org.apache.kafka.common.errors.StaleMemberEpochException; @@ -43,7 +42,6 @@ import org.apache.kafka.coordinator.group.generated.StreamsGroupTargetAssignment import org.apache.kafka.coordinator.group.generated.StreamsGroupTargetAssignmentMetadataKey; import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyKey; import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue; -import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard; import org.apache.kafka.coordinator.group.streams.StreamsGroup.StreamsGroupState; import org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.TaskRole; import org.apache.kafka.coordinator.group.streams.topics.ConfiguredTopology; @@ -90,8 +88,7 @@ public class StreamsGroupTest { return new StreamsGroup( LOG_CONTEXT, snapshotRegistry, - groupId, - mock(GroupCoordinatorMetricsShard.class) + groupId ); } @@ -693,8 +690,7 @@ public class StreamsGroupTest { StreamsGroup group = new StreamsGroup( LOG_CONTEXT, snapshotRegistry, - "group-foo", - mock(GroupCoordinatorMetricsShard.class) + "group-foo" ); group.setGroupEpoch(1); group.setTopology(new StreamsTopology(1, Map.of())); @@ -719,8 +715,7 @@ public class StreamsGroupTest { StreamsGroup group = new StreamsGroup( LOG_CONTEXT, snapshotRegistry, - "group-foo", - mock(GroupCoordinatorMetricsShard.class) + "group-foo" ); // Simulate a call from the admin client without member ID and member epoch. @@ -790,7 +785,7 @@ public class StreamsGroupTest { long commitTimestamp = 20000L; long offsetsRetentionMs = 10000L; OffsetAndMetadata offsetAndMetadata = new OffsetAndMetadata(15000L, OptionalInt.empty(), "", commitTimestamp, OptionalLong.empty(), Uuid.ZERO_UUID); - StreamsGroup group = new StreamsGroup(LOG_CONTEXT, new SnapshotRegistry(LOG_CONTEXT), "group-id", mock(GroupCoordinatorMetricsShard.class)); + StreamsGroup group = new StreamsGroup(LOG_CONTEXT, new SnapshotRegistry(LOG_CONTEXT), "group-id"); Optional offsetExpirationCondition = group.offsetExpirationCondition(); assertTrue(offsetExpirationCondition.isPresent()); @@ -803,7 +798,7 @@ public class StreamsGroupTest { @Test public void testAsDescribedGroup() { SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); - StreamsGroup group = new StreamsGroup(LOG_CONTEXT, snapshotRegistry, "group-id-1", mock(GroupCoordinatorMetricsShard.class)); + StreamsGroup group = new StreamsGroup(LOG_CONTEXT, snapshotRegistry, "group-id-1"); snapshotRegistry.idempotentCreateSnapshot(0); assertEquals(StreamsGroup.StreamsGroupState.EMPTY.toString(), group.stateAsString(0)); @@ -887,12 +882,7 @@ public class StreamsGroupTest { @Test public void testIsInStatesCaseInsensitiveAndUnderscored() { SnapshotRegistry snapshotRegistry = new SnapshotRegistry(LOG_CONTEXT); - GroupCoordinatorMetricsShard metricsShard = new GroupCoordinatorMetricsShard( - snapshotRegistry, - Map.of(), - new TopicPartition("__consumer_offsets", 0) - ); - StreamsGroup group = new StreamsGroup(LOG_CONTEXT, snapshotRegistry, "group-foo", metricsShard); + StreamsGroup group = new StreamsGroup(LOG_CONTEXT, snapshotRegistry, "group-foo"); snapshotRegistry.idempotentCreateSnapshot(0); assertTrue(group.isInStates(Set.of("empty"), 0)); assertFalse(group.isInStates(Set.of("Empty"), 0)); @@ -911,8 +901,7 @@ public class StreamsGroupTest { StreamsGroup streamsGroup = new StreamsGroup( LOG_CONTEXT, snapshotRegistry, - "group-foo", - mock(GroupCoordinatorMetricsShard.class) + "group-foo" ); MetadataImage metadataImage = new MetadataImageBuilder() @@ -933,8 +922,7 @@ public class StreamsGroupTest { StreamsGroup streamsGroup = new StreamsGroup( LOG_CONTEXT, snapshotRegistry, - "test-group", - mock(GroupCoordinatorMetricsShard.class) + "test-group" ); streamsGroup.updateMember(new StreamsGroupMember.Builder("member1") .setMemberEpoch(1) @@ -961,8 +949,7 @@ public class StreamsGroupTest { public void testIsSubscribedToTopic() { LogContext logContext = new LogContext(); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext); - GroupCoordinatorMetricsShard metricsShard = mock(GroupCoordinatorMetricsShard.class); - StreamsGroup streamsGroup = new StreamsGroup(logContext, snapshotRegistry, "test-group", metricsShard); + StreamsGroup streamsGroup = new StreamsGroup(logContext, snapshotRegistry, "test-group"); assertFalse(streamsGroup.isSubscribedToTopic("test-topic1")); assertFalse(streamsGroup.isSubscribedToTopic("test-topic2")); @@ -1008,8 +995,7 @@ public class StreamsGroupTest { String memberId2 = "test-member-id2"; LogContext logContext = new LogContext(); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext); - GroupCoordinatorMetricsShard metricsShard = mock(GroupCoordinatorMetricsShard.class); - StreamsGroup streamsGroup = new StreamsGroup(logContext, snapshotRegistry, "test-group", metricsShard); + StreamsGroup streamsGroup = new StreamsGroup(logContext, snapshotRegistry, "test-group"); streamsGroup.updateMember(streamsGroup.getOrCreateDefaultMember(memberId1)); streamsGroup.updateMember(streamsGroup.getOrCreateDefaultMember(memberId2)); From 848e3d0092ba2f1ef54f45d210abffe5b94e232a Mon Sep 17 00:00:00 2001 From: Lianet Magrans <98415067+lianetm@users.noreply.github.com> Date: Fri, 19 Sep 2025 12:42:43 -0400 Subject: [PATCH 053/100] KAFKA-19722: Adding missing metric assigned-partitions for new consumer (#20557) Adding the missing metric to track the number of partitions assigned. This metric should be registered whenever the consumer is using a groupId, and should track the number of partitions from the subscription state, regardless of the subscription type (manual or automatic). This PR registers the missing metric as part of the ConsumerRebalanceMetricsManager setup. This manager is created if there is a group ID, and reused by the consumer membershipMgr and the streamsMemberhipMgr, so we ensure that the metric is registered for the new consumer and streams. Reviewers: Lucas Brutschy , TengYao Chi --- .../internals/ConsumerMembershipManager.java | 2 +- .../internals/StreamsMembershipManager.java | 2 +- .../consumer/internals/SubscriptionState.java | 2 +- .../ConsumerRebalanceMetricsManager.java | 23 ++++++- .../clients/consumer/KafkaConsumerTest.java | 31 +++++++++ .../ConsumerMembershipManagerTest.java | 13 +++- .../StreamsMembershipManagerTest.java | 10 +++ .../ConsumerRebalanceMetricsManagerTest.java | 63 +++++++++++++++++++ 8 files changed, 141 insertions(+), 5 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/metrics/ConsumerRebalanceMetricsManagerTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java index e07424a6393..25e523c3a0d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManager.java @@ -163,7 +163,7 @@ public class ConsumerMembershipManager extends AbstractMembershipManager lastRebalanceEndMs; } + + /** + * Register metric to track the number of assigned partitions. + * It will consider partitions assigned to the consumer + * regardless of whether they were assigned via {@link KafkaConsumer#subscribe(Pattern)} or + * {@link KafkaConsumer#assign(Collection)} + */ + private void registerAssignedPartitionCount(SubscriptionState subscriptions) { + Measurable numParts = (config, now) -> subscriptions.numAssignedPartitions(); + metrics.addMetric(assignedPartitionsCount, numParts); + } } \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 60063e52268..78ff15cee5f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -155,6 +155,8 @@ import javax.management.ObjectName; import static java.util.Collections.singletonList; import static org.apache.kafka.clients.consumer.internals.ClassicKafkaConsumer.DEFAULT_REASON; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.COORDINATOR_METRICS_SUFFIX; import static org.apache.kafka.common.requests.FetchMetadata.INVALID_SESSION_ID; import static org.apache.kafka.common.utils.Utils.propsToMap; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; @@ -271,6 +273,35 @@ public class KafkaConsumerTest { } } + @ParameterizedTest + @EnumSource(GroupProtocol.class) + public void testAssignedPartitionsMetrics(GroupProtocol groupProtocol) throws InterruptedException { + consumer = newConsumer(groupProtocol, time, mock(KafkaClient.class), subscription, + mock(ConsumerMetadata.class), assignor, false, groupInstanceId); + Metrics metrics = consumer.metricsRegistry(); + + // This metric is added in the background thread for the AsyncConsumer, so waiting on it to avoid flakiness. + TestUtils.waitForCondition(() -> getMetric(metrics, "assigned-partitions") != null, + "Consumer should register the assigned-partitions metric"); + assertNotNull(getMetric(metrics, "assigned-partitions")); + assertEquals(0.0d, getMetric(metrics, "assigned-partitions").metricValue()); + + subscription.assignFromUser(Set.of(tp0)); + assertEquals(1.0d, getMetric(metrics, "assigned-partitions").metricValue()); + + subscription.assignFromUser(Set.of(tp0, tp1)); + assertEquals(2.0d, getMetric(metrics, "assigned-partitions").metricValue()); + + subscription.unsubscribe(); + subscription.subscribe(Set.of(topic), Optional.empty()); + subscription.assignFromSubscribed(Set.of(tp0)); + assertEquals(1.0d, getMetric(metrics, "assigned-partitions").metricValue()); + } + + private KafkaMetric getMetric(Metrics metrics, String name) { + return metrics.metrics().get(metrics.metricName(name, CONSUMER_METRIC_GROUP_PREFIX + COORDINATOR_METRICS_SUFFIX)); + } + @ParameterizedTest @EnumSource(GroupProtocol.class) public void testUnsubscribingCustomMetricsWithSameNameDoesntAffectConsumerMetrics(GroupProtocol groupProtocol) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java index aa8c7bdb1dc..9edf1781828 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerMembershipManagerTest.java @@ -71,6 +71,8 @@ import java.util.stream.Stream; import static org.apache.kafka.clients.consumer.internals.AbstractMembershipManager.TOPIC_PARTITION_COMPARATOR; import static org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer.invokeRebalanceCallbacks; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.COORDINATOR_METRICS_SUFFIX; import static org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; @@ -125,7 +127,7 @@ public class ConsumerMembershipManagerTest { time = new MockTime(0); backgroundEventHandler = new BackgroundEventHandler(backgroundEventQueue, time, mock(AsyncConsumerMetrics.class)); metrics = new Metrics(time); - rebalanceMetricsManager = new ConsumerRebalanceMetricsManager(metrics); + rebalanceMetricsManager = new ConsumerRebalanceMetricsManager(metrics, subscriptionState); when(commitRequestManager.maybeAutoCommitSyncBeforeRebalance(anyLong())).thenReturn(CompletableFuture.completedFuture(null)); } @@ -181,6 +183,15 @@ public class ConsumerMembershipManagerTest { assertEquals(Optional.of("rack1"), membershipManager.rackId()); } + @Test + public void testAssignedPartitionCountMetricRegistered() { + MetricName metricName = metrics.metricName( + "assigned-partitions", + CONSUMER_METRIC_GROUP_PREFIX + COORDINATOR_METRICS_SUFFIX + ); + assertNotNull(metrics.metric(metricName), "Metric assigned-partitions should have been registered"); + } + @Test public void testMembershipManagerInitSupportsEmptyGroupInstanceId() { createMembershipManagerJoiningGroup(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsMembershipManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsMembershipManagerTest.java index 993393bb157..c0255a4d29a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsMembershipManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsMembershipManagerTest.java @@ -67,6 +67,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotEquals; +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.mockito.ArgumentMatchers.any; @@ -131,6 +132,15 @@ public class StreamsMembershipManagerTest { verifyInStateUnsubscribed(membershipManager); } + @Test + public void testAssignedPartitionCountMetricRegistered() { + MetricName metricName = metrics.metricName( + "assigned-partitions", + CONSUMER_METRIC_GROUP_PREFIX + COORDINATOR_METRICS_SUFFIX + ); + assertNotNull(metrics.metric(metricName), "Metric assigned-partitions should have been registered"); + } + @Test public void testUnexpectedErrorInHeartbeatResponse() { final String errorMessage = "Nobody expects the Spanish Inquisition!"; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/metrics/ConsumerRebalanceMetricsManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/metrics/ConsumerRebalanceMetricsManagerTest.java new file mode 100644 index 00000000000..a7d91227767 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/metrics/ConsumerRebalanceMetricsManagerTest.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals.metrics; + +import org.apache.kafka.clients.consumer.internals.AutoOffsetResetStrategy; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; + +import org.junit.jupiter.api.Test; + +import java.util.Optional; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.mockito.Mockito.mock; + + +class ConsumerRebalanceMetricsManagerTest { + + private final Time time = new MockTime(); + private final Metrics metrics = new Metrics(time); + + @Test + public void testAssignedPartitionCountMetric() { + SubscriptionState subscriptionState = new SubscriptionState(mock(LogContext.class), AutoOffsetResetStrategy.EARLIEST); + ConsumerRebalanceMetricsManager consumerRebalanceMetricsManager = new ConsumerRebalanceMetricsManager(metrics, subscriptionState); + + assertNotNull(metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount), "Metric assigned-partitions has not been registered as expected"); + + // Check for manually assigned partitions + subscriptionState.assignFromUser(Set.of(new TopicPartition("topic", 0), new TopicPartition("topic", 1))); + assertEquals(2.0d, metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount).metricValue()); + subscriptionState.assignFromUser(Set.of()); + assertEquals(0.0d, metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount).metricValue()); + + subscriptionState.unsubscribe(); + assertEquals(0.0d, metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount).metricValue()); + + // Check for automatically assigned partitions + subscriptionState.subscribe(Set.of("topic"), Optional.empty()); + subscriptionState.assignFromSubscribed(Set.of(new TopicPartition("topic", 0))); + assertEquals(1.0d, metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount).metricValue()); + } +} From 57e9f98e156186d62e43a9fec19e0a8d49690102 Mon Sep 17 00:00:00 2001 From: Jhen-Yung Hsu Date: Sat, 20 Sep 2025 11:46:19 +0800 Subject: [PATCH 054/100] KAFKA-19644 Enhance the documentation for producer headers and integration tests (#20524) - Improve the docs for Record Headers. - Add integration tests to verify that the order of headers in a record is preserved when producing and consuming. - Add unit tests for RecordHeaders.java. Reviewers: Ken Huang , Hong-Yi Chen , Chia-Ping Tsai --- .../consumer/PlaintextConsumerTest.java | 10 ++++ .../clients/consumer/ShareConsumerTest.java | 16 +++++-- .../apache/kafka/common/header/Header.java | 17 ++++++- .../apache/kafka/common/header/Headers.java | 24 ++++++---- .../header/internals/RecordHeadersTest.java | 47 +++++++++++++++++++ docs/implementation.html | 1 + 6 files changed, 100 insertions(+), 15 deletions(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java index 13e681cfdda..bd92f0c5685 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerTest.java @@ -203,7 +203,10 @@ public class PlaintextConsumerTest { ) { var record = new ProducerRecord<>(TP.topic(), TP.partition(), null, "key".getBytes(), "value".getBytes()); record.headers().add("headerKey", "headerValue".getBytes()); + record.headers().add("headerKey2", "headerValue2".getBytes()); + record.headers().add("headerKey3", "headerValue3".getBytes()); producer.send(record); + producer.flush(); assertEquals(0, consumer.assignment().size()); consumer.assign(List.of(TP)); @@ -212,8 +215,15 @@ public class PlaintextConsumerTest { consumer.seek(TP, 0); var records = consumeRecords(consumer, numRecords); assertEquals(numRecords, records.size()); + var header = records.get(0).headers().lastHeader("headerKey"); assertEquals("headerValue", header == null ? null : new String(header.value())); + + // Test the order of headers in a record is preserved when producing and consuming + Header[] headers = records.get(0).headers().toArray(); + assertEquals("headerKey", headers[0].key()); + assertEquals("headerKey2", headers[1].key()); + assertEquals("headerKey3", headers[2].key()); } } diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java index 2335c223b07..998ac2c585d 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java @@ -467,6 +467,8 @@ public class ShareConsumerTest { int numRecords = 1; ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); record.headers().add("headerKey", "headerValue".getBytes()); + record.headers().add("headerKey2", "headerValue2".getBytes()); + record.headers().add("headerKey3", "headerValue3".getBytes()); producer.send(record); producer.flush(); @@ -475,11 +477,15 @@ public class ShareConsumerTest { List> records = consumeRecords(shareConsumer, numRecords); assertEquals(numRecords, records.size()); - for (ConsumerRecord consumerRecord : records) { - Header header = consumerRecord.headers().lastHeader("headerKey"); - if (header != null) - assertEquals("headerValue", new String(header.value())); - } + Header header = records.get(0).headers().lastHeader("headerKey"); + assertEquals("headerValue", new String(header.value())); + + // Test the order of headers in a record is preserved when producing and consuming + Header[] headers = records.get(0).headers().toArray(); + assertEquals("headerKey", headers[0].key()); + assertEquals("headerKey2", headers[1].key()); + assertEquals("headerKey3", headers[2].key()); + verifyShareGroupStateTopicRecordsProduced(); } } diff --git a/clients/src/main/java/org/apache/kafka/common/header/Header.java b/clients/src/main/java/org/apache/kafka/common/header/Header.java index 58869b41fb7..e1d0aa00a44 100644 --- a/clients/src/main/java/org/apache/kafka/common/header/Header.java +++ b/clients/src/main/java/org/apache/kafka/common/header/Header.java @@ -16,10 +16,23 @@ */ package org.apache.kafka.common.header; +/** + * A header is a key-value pair. + */ public interface Header { - + + /** + * Returns the key of the header. + * + * @return the header's key; must not be null. + */ String key(); + /** + * Returns the value of the header. + * + * @return the header's value; may be null. + */ byte[] value(); - + } diff --git a/clients/src/main/java/org/apache/kafka/common/header/Headers.java b/clients/src/main/java/org/apache/kafka/common/header/Headers.java index b736cbcabcc..9cce54a5c5c 100644 --- a/clients/src/main/java/org/apache/kafka/common/header/Headers.java +++ b/clients/src/main/java/org/apache/kafka/common/header/Headers.java @@ -16,12 +16,18 @@ */ package org.apache.kafka.common.header; + +/** + * A mutable ordered collection of {@link Header} objects. Note that multiple headers may have the same {@link Header#key() key}. + *

    + * The order of headers is preserved in the order they were added. + */ public interface Headers extends Iterable

    { /** * Adds a header (key inside), to the end, returning if the operation succeeded. * - * @param header the Header to be added + * @param header the Header to be added. * @return this instance of the Headers, once the header is added. * @throws IllegalStateException is thrown if headers are in a read-only state. */ @@ -30,17 +36,18 @@ public interface Headers extends Iterable
    { /** * Creates and adds a header, to the end, returning if the operation succeeded. * - * @param key of the header to be added. - * @param value of the header to be added. + * @param key of the header to be added; must not be null. + * @param value of the header to be added; may be null. * @return this instance of the Headers, once the header is added. * @throws IllegalStateException is thrown if headers are in a read-only state. */ Headers add(String key, byte[] value) throws IllegalStateException; /** - * Removes all headers for the given key returning if the operation succeeded. + * Removes all headers for the given key returning if the operation succeeded, + * while preserving the insertion order of the remaining headers. * - * @param key to remove all headers for. + * @param key to remove all headers for; must not be null. * @return this instance of the Headers, once the header is removed. * @throws IllegalStateException is thrown if headers are in a read-only state. */ @@ -49,16 +56,17 @@ public interface Headers extends Iterable
    { /** * Returns just one (the very last) header for the given key, if present. * - * @param key to get the last header for. + * @param key to get the last header for; must not be null. * @return this last header matching the given key, returns null if not present. */ Header lastHeader(String key); /** * Returns all headers for the given key, in the order they were added in, if present. + * The iterator does not support {@link java.util.Iterator#remove()}. * - * @param key to return the headers for. - * @return all headers for the given key, in the order they were added in, if NO headers are present an empty iterable is returned. + * @param key to return the headers for; must not be null. + * @return all headers for the given key, in the order they were added in, if NO headers are present an empty iterable is returned. */ Iterable
    headers(String key); diff --git a/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java b/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java index 3d431f202ec..41104194991 100644 --- a/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java +++ b/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java @@ -47,6 +47,21 @@ public class RecordHeadersTest { assertEquals(2, getCount(headers)); } + @Test + public void testAddHeadersPreserveOrder() { + Headers headers = new RecordHeaders(); + headers.add(new RecordHeader("key", "value".getBytes())); + headers.add(new RecordHeader("key2", "value2".getBytes())); + headers.add(new RecordHeader("key3", "value3".getBytes())); + + Header[] headersArr = headers.toArray(); + assertHeader("key", "value", headersArr[0]); + assertHeader("key2", "value2", headersArr[1]); + assertHeader("key3", "value3", headersArr[2]); + + assertEquals(3, getCount(headers)); + } + @Test public void testRemove() { Headers headers = new RecordHeaders(); @@ -59,6 +74,27 @@ public class RecordHeadersTest { assertFalse(headers.iterator().hasNext()); } + @Test + public void testPreserveOrderAfterRemove() { + Headers headers = new RecordHeaders(); + headers.add(new RecordHeader("key", "value".getBytes())); + headers.add(new RecordHeader("key2", "value2".getBytes())); + headers.add(new RecordHeader("key3", "value3".getBytes())); + + headers.remove("key"); + Header[] headersArr = headers.toArray(); + assertHeader("key2", "value2", headersArr[0]); + assertHeader("key3", "value3", headersArr[1]); + assertEquals(2, getCount(headers)); + + headers.add(new RecordHeader("key4", "value4".getBytes())); + headers.remove("key3"); + headersArr = headers.toArray(); + assertHeader("key2", "value2", headersArr[0]); + assertHeader("key4", "value4", headersArr[1]); + assertEquals(2, getCount(headers)); + } + @Test public void testAddRemoveInterleaved() { Headers headers = new RecordHeaders(); @@ -127,6 +163,17 @@ public class RecordHeadersTest { } + @Test + public void testHeadersIteratorRemove() { + Headers headers = new RecordHeaders(); + headers.add(new RecordHeader("key", "value".getBytes())); + + Iterator
    headersIterator = headers.headers("key").iterator(); + headersIterator.next(); + assertThrows(UnsupportedOperationException.class, + headersIterator::remove); + } + @Test public void testReadOnly() { RecordHeaders headers = new RecordHeaders(); diff --git a/docs/implementation.html b/docs/implementation.html index 1573dd3d60e..3be539e0ba8 100644 --- a/docs/implementation.html +++ b/docs/implementation.html @@ -101,6 +101,7 @@ Headers => [Header] headerKey: String headerValueLength: varint Value: byte[] +

    The key of a record header is guaranteed to be non-null, while the value of a record header may be null. The order of headers in a record is preserved when producing and consuming.

    We use the same varint encoding as Protobuf. More information on the latter can be found here. The count of headers in a record is also encoded as a varint.

    From c49ab6b4ae69e3c4ad0ea31acb99223f19897ce7 Mon Sep 17 00:00:00 2001 From: Now <156046839+Kguswo@users.noreply.github.com> Date: Sat, 20 Sep 2025 12:52:29 +0900 Subject: [PATCH 055/100] MINOR: Optimize map lookup efficiency with getOrDefault (#20229) Optimized `getRemainingRecords()` method by replacing inefficient `containsKey() + get()` pattern with `getOrDefault()` to reduce map lookups from 2 to 1 per partition. Reviewers: Chia-Ping Tsai --- .../java/kafka/examples/ExactlyOnceMessageProcessor.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/examples/src/main/java/kafka/examples/ExactlyOnceMessageProcessor.java b/examples/src/main/java/kafka/examples/ExactlyOnceMessageProcessor.java index 175385b8ea8..d4249526f3b 100644 --- a/examples/src/main/java/kafka/examples/ExactlyOnceMessageProcessor.java +++ b/examples/src/main/java/kafka/examples/ExactlyOnceMessageProcessor.java @@ -216,11 +216,7 @@ public class ExactlyOnceMessageProcessor extends Thread implements ConsumerRebal } return consumer.assignment().stream().mapToLong(partition -> { long currentPosition = consumer.position(partition); - if (fullEndOffsets.containsKey(partition)) { - return fullEndOffsets.get(partition) - currentPosition; - } else { - return 0; - } + return fullEndOffsets.getOrDefault(partition, currentPosition) - currentPosition; }).sum(); } From 5919762009a0bbc4948bf881a6fbd885e30527b1 Mon Sep 17 00:00:00 2001 From: Chang-Chi Hsu Date: Sat, 20 Sep 2025 12:04:10 +0200 Subject: [PATCH 056/100] MINOR: Remove exitMessage.set() call in TopicBasedRemoteLogMetadataManagerTest (#20563) - **Reasons:** In this case, the `exit(int statusCode)` method invokes `exit(statusCode, null)`, which means the `message` argument is always `null` in this code path. As a result, assigning `exitMessage` has no effect and can be safely removed. - **Changes:** Remove a redundant field assignment. Reviewers: Chia-Ping Tsai --- .../storage/TopicBasedRemoteLogMetadataManagerTest.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java index c3a5beebad9..723e756036c 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java @@ -46,7 +46,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -344,13 +343,11 @@ public class TopicBasedRemoteLogMetadataManagerTest { // Set up a custom exit procedure for testing final AtomicBoolean exitCalled = new AtomicBoolean(false); final AtomicInteger exitCode = new AtomicInteger(-1); - final AtomicReference exitMessage = new AtomicReference<>(); // Set custom exit procedure that won't actually exit the process Exit.setExitProcedure((statusCode, message) -> { exitCalled.set(true); exitCode.set(statusCode); - exitMessage.set(message); }); try (TopicBasedRemoteLogMetadataManager rlmm = new TopicBasedRemoteLogMetadataManager()) { From 07b786e5bfe9685316d8ec51d58f67c8f1536a49 Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Sun, 21 Sep 2025 12:35:46 +0800 Subject: [PATCH 057/100] KAFKA-19681 Improve MetadataShell tool by skipping missing children and removing zkMigrationState (#20504) The current `metadata-shell` find command throws an exception due to child node `zkMigrationState`. This interrupts the output and makes the CLI less usable. Additionally, `zkMigrationState` is no longer used in Kafka 4.x, but it is still defined under image/features, which causes misleading error messages. Reviewers: Chia-Ping Tsai --- .../java/org/apache/kafka/image/node/FeaturesImageNode.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/metadata/src/main/java/org/apache/kafka/image/node/FeaturesImageNode.java b/metadata/src/main/java/org/apache/kafka/image/node/FeaturesImageNode.java index 286e31dba0a..09244a28db7 100644 --- a/metadata/src/main/java/org/apache/kafka/image/node/FeaturesImageNode.java +++ b/metadata/src/main/java/org/apache/kafka/image/node/FeaturesImageNode.java @@ -34,11 +34,6 @@ public class FeaturesImageNode implements MetadataNode { */ public static final String METADATA_VERSION = "metadataVersion"; - /** - * The name of the zk migration state child node. - */ - public static final String ZK_MIGRATION_STATE = "zkMigrationState"; - /** * The prefix to put before finalized feature children. */ @@ -57,7 +52,6 @@ public class FeaturesImageNode implements MetadataNode { public Collection childNames() { ArrayList childNames = new ArrayList<>(); childNames.add(METADATA_VERSION); - childNames.add(ZK_MIGRATION_STATE); for (String featureName : image.finalizedVersions().keySet()) { childNames.add(FINALIZED_PREFIX + featureName); } From 01fccd35138b407d87828ceb557d33c4b5450cac Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Sun, 21 Sep 2025 16:28:03 +0800 Subject: [PATCH 058/100] KAFKA-15186 AppInfo metrics don't contain the client-id (#20493) All Kafka component register AppInfo metrics to track the application start time or commit-id etc. These metrics are useful for monitoring and debugging. However, the AppInfo doesn't provide client-id, which is an important information for custom metrics reporter. The AppInfoParser class registers a JMX MBean with the provided client-id, but when it adds metrics to the Metrics registry, the client-id is not included. This KIP aims to add the client-id as a tag. Reviewers: Chia-Ping Tsai --- .../kafka/common/utils/AppInfoParser.java | 42 +++++++---- .../kafka/common/utils/AppInfoParserTest.java | 75 +++++++++++++++---- docs/upgrade.html | 27 +++++-- 3 files changed, 110 insertions(+), 34 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/utils/AppInfoParser.java b/clients/src/main/java/org/apache/kafka/common/utils/AppInfoParser.java index e23bda36d5f..cc2d7b75f1e 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/AppInfoParser.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/AppInfoParser.java @@ -25,6 +25,7 @@ import org.slf4j.LoggerFactory; import java.io.InputStream; import java.lang.management.ManagementFactory; +import java.util.Map; import java.util.Properties; import javax.management.JMException; @@ -68,7 +69,7 @@ public class AppInfoParser { AppInfo mBean = new AppInfo(nowMs); server.registerMBean(mBean, name); - registerMetrics(metrics, mBean); // prefix will be added later by JmxReporter + registerMetrics(metrics, mBean, id); // prefix will be added later by JmxReporter } catch (JMException e) { log.warn("Error registering AppInfo mbean", e); } @@ -81,7 +82,7 @@ public class AppInfoParser { if (server.isRegistered(name)) server.unregisterMBean(name); - unregisterMetrics(metrics); + unregisterMetrics(metrics, id); } catch (JMException e) { log.warn("Error unregistering AppInfo mbean", e); } finally { @@ -89,23 +90,36 @@ public class AppInfoParser { } } - private static MetricName metricName(Metrics metrics, String name) { - return metrics.metricName(name, "app-info", "Metric indicating " + name); + private static MetricName metricName(Metrics metrics, String name, Map tags) { + return metrics.metricName(name, "app-info", "Metric indicating " + name, tags); } - private static void registerMetrics(Metrics metrics, AppInfo appInfo) { - if (metrics != null) { - metrics.addMetric(metricName(metrics, "version"), (Gauge) (config, now) -> appInfo.getVersion()); - metrics.addMetric(metricName(metrics, "commit-id"), (Gauge) (config, now) -> appInfo.getCommitId()); - metrics.addMetric(metricName(metrics, "start-time-ms"), (Gauge) (config, now) -> appInfo.getStartTimeMs()); + private static void registerMetrics(Metrics metrics, AppInfo appInfo, String clientId) { + if (metrics == null) return; + // Most Kafka clients (producer/consumer/admin) set the client-id tag in the metrics config. + // Although we don’t explicitly parse client-id here, these metrics are automatically tagged with client-id. + metrics.addMetric(metricName(metrics, "version", Map.of()), (Gauge) (config, now) -> appInfo.getVersion()); + metrics.addMetric(metricName(metrics, "commit-id", Map.of()), (Gauge) (config, now) -> appInfo.getCommitId()); + metrics.addMetric(metricName(metrics, "start-time-ms", Map.of()), (Gauge) (config, now) -> appInfo.getStartTimeMs()); + // MirrorMaker/Worker doesn't set client-id tag into the metrics config, so we need to set it here. + if (!metrics.config().tags().containsKey("client-id") && clientId != null) { + metrics.addMetric(metricName(metrics, "version", Map.of("client-id", clientId)), (Gauge) (config, now) -> appInfo.getVersion()); + metrics.addMetric(metricName(metrics, "commit-id", Map.of("client-id", clientId)), (Gauge) (config, now) -> appInfo.getCommitId()); + metrics.addMetric(metricName(metrics, "start-time-ms", Map.of("client-id", clientId)), (Gauge) (config, now) -> appInfo.getStartTimeMs()); } } - private static void unregisterMetrics(Metrics metrics) { - if (metrics != null) { - metrics.removeMetric(metricName(metrics, "version")); - metrics.removeMetric(metricName(metrics, "commit-id")); - metrics.removeMetric(metricName(metrics, "start-time-ms")); + private static void unregisterMetrics(Metrics metrics, String clientId) { + if (metrics == null) return; + + metrics.removeMetric(metricName(metrics, "version", Map.of())); + metrics.removeMetric(metricName(metrics, "commit-id", Map.of())); + metrics.removeMetric(metricName(metrics, "start-time-ms", Map.of())); + + if (!metrics.config().tags().containsKey("client-id") && clientId != null) { + metrics.removeMetric(metricName(metrics, "version", Map.of("client-id", clientId))); + metrics.removeMetric(metricName(metrics, "commit-id", Map.of("client-id", clientId))); + metrics.removeMetric(metricName(metrics, "start-time-ms", Map.of("client-id", clientId))); } } diff --git a/clients/src/test/java/org/apache/kafka/common/utils/AppInfoParserTest.java b/clients/src/test/java/org/apache/kafka/common/utils/AppInfoParserTest.java index aac13f299fe..7e153be5862 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/AppInfoParserTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/AppInfoParserTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.common.utils; +import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; import org.junit.jupiter.api.AfterEach; @@ -23,6 +24,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import java.lang.management.ManagementFactory; +import java.util.Map; import javax.management.JMException; import javax.management.MBeanServer; @@ -41,38 +43,49 @@ public class AppInfoParserTest { private static final String METRICS_PREFIX = "app-info-test"; private static final String METRICS_ID = "test"; - private Metrics metrics; private MBeanServer mBeanServer; @BeforeEach public void setUp() { - metrics = new Metrics(new MockTime(1)); mBeanServer = ManagementFactory.getPlatformMBeanServer(); } @AfterEach - public void tearDown() { - metrics.close(); + public void tearDown() throws JMException { + if (mBeanServer.isRegistered(expectedAppObjectName())) { + mBeanServer.unregisterMBean(expectedAppObjectName()); + } } @Test public void testRegisterAppInfoRegistersMetrics() throws JMException { - registerAppInfo(); - registerAppInfoMultipleTimes(); + try (Metrics metrics = new Metrics(new MockTime(1))) { + registerAppInfo(metrics); + registerAppInfoMultipleTimes(metrics); + AppInfoParser.unregisterAppInfo(METRICS_PREFIX, METRICS_ID, metrics); + } } @Test public void testUnregisterAppInfoUnregistersMetrics() throws JMException { - registerAppInfo(); - AppInfoParser.unregisterAppInfo(METRICS_PREFIX, METRICS_ID, metrics); + try (Metrics metrics = new Metrics(new MockTime(1))) { + registerAppInfo(metrics); + AppInfoParser.unregisterAppInfo(METRICS_PREFIX, METRICS_ID, metrics); - assertFalse(mBeanServer.isRegistered(expectedAppObjectName())); - assertNull(metrics.metric(metrics.metricName("commit-id", "app-info"))); - assertNull(metrics.metric(metrics.metricName("version", "app-info"))); - assertNull(metrics.metric(metrics.metricName("start-time-ms", "app-info"))); + assertFalse(mBeanServer.isRegistered(expectedAppObjectName())); + assertNull(metrics.metric(metrics.metricName("commit-id", "app-info"))); + assertNull(metrics.metric(metrics.metricName("version", "app-info"))); + assertNull(metrics.metric(metrics.metricName("start-time-ms", "app-info"))); + + Map idTag = Map.of("client-id", METRICS_ID); + assertNull(metrics.metric(metrics.metricName("commit-id", "app-info", idTag))); + assertNull(metrics.metric(metrics.metricName("version", "app-info", idTag))); + assertNull(metrics.metric(metrics.metricName("start-time-ms", "app-info", idTag))); + AppInfoParser.unregisterAppInfo(METRICS_PREFIX, METRICS_ID, metrics); + } } - private void registerAppInfo() throws JMException { + private void registerAppInfo(Metrics metrics) throws JMException { assertEquals(EXPECTED_COMMIT_VERSION, AppInfoParser.getCommitId()); assertEquals(EXPECTED_VERSION, AppInfoParser.getVersion()); @@ -82,9 +95,15 @@ public class AppInfoParserTest { assertEquals(EXPECTED_COMMIT_VERSION, metrics.metric(metrics.metricName("commit-id", "app-info")).metricValue()); assertEquals(EXPECTED_VERSION, metrics.metric(metrics.metricName("version", "app-info")).metricValue()); assertEquals(EXPECTED_START_MS, metrics.metric(metrics.metricName("start-time-ms", "app-info")).metricValue()); + + Map idTag = Map.of("client-id", METRICS_ID); + assertTrue(mBeanServer.isRegistered(expectedAppObjectName())); + assertEquals(EXPECTED_COMMIT_VERSION, metrics.metric(metrics.metricName("commit-id", "app-info", idTag)).metricValue()); + assertEquals(EXPECTED_VERSION, metrics.metric(metrics.metricName("version", "app-info", idTag)).metricValue()); + assertEquals(EXPECTED_START_MS, metrics.metric(metrics.metricName("start-time-ms", "app-info", idTag)).metricValue()); } - private void registerAppInfoMultipleTimes() throws JMException { + private void registerAppInfoMultipleTimes(Metrics metrics) throws JMException { assertEquals(EXPECTED_COMMIT_VERSION, AppInfoParser.getCommitId()); assertEquals(EXPECTED_VERSION, AppInfoParser.getVersion()); @@ -95,9 +114,37 @@ public class AppInfoParserTest { assertEquals(EXPECTED_COMMIT_VERSION, metrics.metric(metrics.metricName("commit-id", "app-info")).metricValue()); assertEquals(EXPECTED_VERSION, metrics.metric(metrics.metricName("version", "app-info")).metricValue()); assertEquals(EXPECTED_START_MS, metrics.metric(metrics.metricName("start-time-ms", "app-info")).metricValue()); + + Map idTag = Map.of("client-id", METRICS_ID); + assertEquals(EXPECTED_COMMIT_VERSION, metrics.metric(metrics.metricName("commit-id", "app-info", idTag)).metricValue()); + assertEquals(EXPECTED_VERSION, metrics.metric(metrics.metricName("version", "app-info", idTag)).metricValue()); + assertEquals(EXPECTED_START_MS, metrics.metric(metrics.metricName("start-time-ms", "app-info", idTag)).metricValue()); } private ObjectName expectedAppObjectName() throws MalformedObjectNameException { return new ObjectName(METRICS_PREFIX + ":type=app-info,id=" + METRICS_ID); } + + @Test + public void testClientIdWontAddRepeatedly() throws JMException { + Map tags = Map.of( + "client-id", METRICS_ID, + "other-tag", "tag-value", + "another-tag", "another-value" + ); + Metrics metrics = new Metrics(new MetricConfig().tags(tags), new MockTime(1)); + AppInfoParser.registerAppInfo(METRICS_PREFIX, METRICS_ID, metrics, EXPECTED_START_MS); + + assertTrue(mBeanServer.isRegistered(expectedAppObjectName())); + assertEquals(EXPECTED_COMMIT_VERSION, metrics.metric(metrics.metricName("commit-id", "app-info", tags)).metricValue()); + assertEquals(EXPECTED_VERSION, metrics.metric(metrics.metricName("version", "app-info", tags)).metricValue()); + assertEquals(EXPECTED_START_MS, metrics.metric(metrics.metricName("start-time-ms", "app-info", tags)).metricValue()); + + Map idTag = Map.of("client-id", METRICS_ID); + assertEquals(EXPECTED_COMMIT_VERSION, metrics.metric(metrics.metricName("commit-id", "app-info", idTag)).metricValue()); + assertEquals(EXPECTED_VERSION, metrics.metric(metrics.metricName("version", "app-info", idTag)).metricValue()); + assertEquals(EXPECTED_START_MS, metrics.metric(metrics.metricName("start-time-ms", "app-info", idTag)).metricValue()); + metrics.close(); + AppInfoParser.unregisterAppInfo(METRICS_PREFIX, METRICS_ID, metrics); + } } diff --git a/docs/upgrade.html b/docs/upgrade.html index ec417e88c6b..66e05d90a5d 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -147,6 +147,21 @@ and defaults to false. For further details, please refer to KIP-853. +
  • + The AppInfo metrics will deprecate the following metric names, which will be removed in Kafka 5.0: +
      +
    • [name=start-time-ms, group=app-info, description=Metric indicating start-time-ms, tags={}]
    • +
    • [name=commit-id, group=app-info, description=Metric indicating commit-id, tags={}]
    • +
    • [name=version, group=app-info, description=Metric indicating version, tags={}]
    • +
    + In addition, the client-id will be added to the tags of these metrics. The new metric names will be: +
      +
    • [name=start-time-ms, group=app-info, description=Metric indicating start-time-ms, tags={client-id=...}]
    • +
    • [name=commit-id, group=app-info, description=Metric indicating commit-id, tags={client-id=...}]
    • +
    • [name=version, group=app-info, description=Metric indicating version, tags={client-id=...}]
    • +
    + For further details, please refer to KIP-1120. +
  • Upgrading to 4.1.0

    @@ -423,12 +438,12 @@
  • The --whitelist option was removed from the kafka-console-consumer command line tool. Please use --include instead.
  • -
  • Redirections from the old tools packages have been removed: - kafka.admin.FeatureCommand, - kafka.tools.ClusterTool, +
  • Redirections from the old tools packages have been removed: + kafka.admin.FeatureCommand, + kafka.tools.ClusterTool, kafka.tools.EndToEndLatency, - kafka.tools.StateChangeLogMerger, - kafka.tools.StreamsResetter, + kafka.tools.StateChangeLogMerger, + kafka.tools.StreamsResetter, kafka.tools.JmxTool.
  • The --authorizer, --authorizer-properties, and --zk-tls-config-file options were removed from the kafka-acls command line tool. @@ -492,7 +507,7 @@
  • The deprecated sendOffsetsToTransaction(Map<TopicPartition, OffsetAndMetadata>, String) method has been removed from the Producer API.
  • -
  • The default linger.ms changed from 0 to 5 in Apache Kafka 4.0 as the efficiency gains from larger batches typically result in +
  • The default linger.ms changed from 0 to 5 in Apache Kafka 4.0 as the efficiency gains from larger batches typically result in similar or lower producer latency despite the increased linger.
  • From a0640f95177cc06259ddf0721bf59abc785f2439 Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Mon, 22 Sep 2025 22:57:30 +0800 Subject: [PATCH 059/100] KAFKA-18351: Remove top-level version field from docker-compose.yml files (#18322) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Reviewers: Mickael Maison , Sylwester Lachiewicz , Dávid Szigecsán --- .../cluster/combined/plaintext/docker-compose.yml | 1 - .../docker-compose-files/cluster/combined/ssl/docker-compose.yml | 1 - .../cluster/isolated/plaintext/docker-compose.yml | 1 - .../docker-compose-files/cluster/isolated/ssl/docker-compose.yml | 1 - .../single-node/file-input/docker-compose.yml | 1 - .../single-node/plaintext/docker-compose.yml | 1 - .../docker-compose-files/single-node/ssl/docker-compose.yml | 1 - docker/test/fixtures/mode/combined/docker-compose.yml | 1 - docker/test/fixtures/mode/isolated/docker-compose.yml | 1 - 9 files changed, 9 deletions(-) diff --git a/docker/examples/docker-compose-files/cluster/combined/plaintext/docker-compose.yml b/docker/examples/docker-compose-files/cluster/combined/plaintext/docker-compose.yml index e27b8ebba5a..6ca7081bb76 100644 --- a/docker/examples/docker-compose-files/cluster/combined/plaintext/docker-compose.yml +++ b/docker/examples/docker-compose-files/cluster/combined/plaintext/docker-compose.yml @@ -14,7 +14,6 @@ # limitations under the License. --- -version: '2' services: kafka-1: image: ${IMAGE} diff --git a/docker/examples/docker-compose-files/cluster/combined/ssl/docker-compose.yml b/docker/examples/docker-compose-files/cluster/combined/ssl/docker-compose.yml index dba7fe87d3c..0b80670a2ec 100644 --- a/docker/examples/docker-compose-files/cluster/combined/ssl/docker-compose.yml +++ b/docker/examples/docker-compose-files/cluster/combined/ssl/docker-compose.yml @@ -14,7 +14,6 @@ # limitations under the License. --- -version: '2' services: kafka-1: image: ${IMAGE} diff --git a/docker/examples/docker-compose-files/cluster/isolated/plaintext/docker-compose.yml b/docker/examples/docker-compose-files/cluster/isolated/plaintext/docker-compose.yml index 946cb5e3293..e4994ce9318 100644 --- a/docker/examples/docker-compose-files/cluster/isolated/plaintext/docker-compose.yml +++ b/docker/examples/docker-compose-files/cluster/isolated/plaintext/docker-compose.yml @@ -14,7 +14,6 @@ # limitations under the License. --- -version: '2' services: controller-1: image: ${IMAGE} diff --git a/docker/examples/docker-compose-files/cluster/isolated/ssl/docker-compose.yml b/docker/examples/docker-compose-files/cluster/isolated/ssl/docker-compose.yml index 5a99526b609..aefb84b024c 100644 --- a/docker/examples/docker-compose-files/cluster/isolated/ssl/docker-compose.yml +++ b/docker/examples/docker-compose-files/cluster/isolated/ssl/docker-compose.yml @@ -14,7 +14,6 @@ # limitations under the License. --- -version: '2' services: controller-1: image: ${IMAGE} diff --git a/docker/examples/docker-compose-files/single-node/file-input/docker-compose.yml b/docker/examples/docker-compose-files/single-node/file-input/docker-compose.yml index 34a7264d3e2..8a4fd85b9ba 100644 --- a/docker/examples/docker-compose-files/single-node/file-input/docker-compose.yml +++ b/docker/examples/docker-compose-files/single-node/file-input/docker-compose.yml @@ -14,7 +14,6 @@ # limitations under the License. --- -version: '2' services: broker: image: ${IMAGE} diff --git a/docker/examples/docker-compose-files/single-node/plaintext/docker-compose.yml b/docker/examples/docker-compose-files/single-node/plaintext/docker-compose.yml index c7ae3705042..c33c947f073 100644 --- a/docker/examples/docker-compose-files/single-node/plaintext/docker-compose.yml +++ b/docker/examples/docker-compose-files/single-node/plaintext/docker-compose.yml @@ -14,7 +14,6 @@ # limitations under the License. --- -version: '2' services: broker: image: ${IMAGE} diff --git a/docker/examples/docker-compose-files/single-node/ssl/docker-compose.yml b/docker/examples/docker-compose-files/single-node/ssl/docker-compose.yml index 91df50ecf95..cc98efcefcd 100644 --- a/docker/examples/docker-compose-files/single-node/ssl/docker-compose.yml +++ b/docker/examples/docker-compose-files/single-node/ssl/docker-compose.yml @@ -14,7 +14,6 @@ # limitations under the License. --- -version: '2' services: broker: image: ${IMAGE} diff --git a/docker/test/fixtures/mode/combined/docker-compose.yml b/docker/test/fixtures/mode/combined/docker-compose.yml index 9c6d81f83bd..8691019d024 100644 --- a/docker/test/fixtures/mode/combined/docker-compose.yml +++ b/docker/test/fixtures/mode/combined/docker-compose.yml @@ -14,7 +14,6 @@ # limitations under the License. --- -version: '2' services: broker1: image: {$IMAGE} diff --git a/docker/test/fixtures/mode/isolated/docker-compose.yml b/docker/test/fixtures/mode/isolated/docker-compose.yml index f0216a520ac..2c77fc123f1 100644 --- a/docker/test/fixtures/mode/isolated/docker-compose.yml +++ b/docker/test/fixtures/mode/isolated/docker-compose.yml @@ -14,7 +14,6 @@ # limitations under the License. --- -version: '2' services: controller1: image: {$IMAGE} From da6a562f6d5e4fba49207084b8431c2506931fd2 Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Mon, 22 Sep 2025 23:04:42 +0800 Subject: [PATCH 060/100] KAFKA-17834: Improvements to Dockerfile (#17554) Reviewers: Mickael Maison , Chia-Ping Tsai --- docker/jvm/Dockerfile | 2 +- docker/native/Dockerfile | 2 +- tests/docker/Dockerfile | 6 +++--- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docker/jvm/Dockerfile b/docker/jvm/Dockerfile index 3d2f06820d4..1c7c95d4869 100644 --- a/docker/jvm/Dockerfile +++ b/docker/jvm/Dockerfile @@ -68,7 +68,7 @@ LABEL org.label-schema.name="kafka" \ org.label-schema.description="Apache Kafka" \ org.label-schema.build-date="${build_date}" \ org.label-schema.vcs-url="https://github.com/apache/kafka" \ - maintainer="Apache Kafka" + org.opencontainers.image.authors="Apache Kafka" RUN mkdir opt/kafka; \ set -eux ; \ diff --git a/docker/native/Dockerfile b/docker/native/Dockerfile index 010edbcd51c..d458dab72ce 100644 --- a/docker/native/Dockerfile +++ b/docker/native/Dockerfile @@ -55,7 +55,7 @@ LABEL org.label-schema.name="kafka" \ org.label-schema.description="Apache Kafka" \ org.label-schema.build-date="${build_date}" \ org.label-schema.vcs-url="https://github.com/apache/kafka" \ - maintainer="Apache Kafka" + org.opencontainers.image.authors="Apache Kafka" RUN apk update ; \ apk add --no-cache gcompat ; \ diff --git a/tests/docker/Dockerfile b/tests/docker/Dockerfile index bf69bc0e371..f1cbd987352 100644 --- a/tests/docker/Dockerfile +++ b/tests/docker/Dockerfile @@ -44,14 +44,14 @@ RUN if [ "$KAFKA_MODE" = "native" ]; then \ FROM $jdk_version -MAINTAINER Apache Kafka dev@kafka.apache.org +LABEL org.opencontainers.image.authors="Apache Kafka dev@kafka.apache.org" VOLUME ["/opt/kafka-dev"] # Set the timezone. ENV TZ="/usr/share/zoneinfo/America/Los_Angeles" # Do not ask for confirmations when running apt-get, etc. -ENV DEBIAN_FRONTEND noninteractive +ENV DEBIAN_FRONTEND=noninteractive # Set the ducker.creator label so that we know that this is a ducker image. This will make it # visible to 'ducker purge'. The ducker.creator label also lets us know what UNIX user built this @@ -155,4 +155,4 @@ RUN useradd -u $UID -ms /bin/bash ducker \ USER ducker -CMD sudo service ssh start && tail -f /dev/null +CMD ["sudo", "service", "ssh", "start", "-D"] From f16d1f3c9dd10ef7214da4d699936f49d2a09527 Mon Sep 17 00:00:00 2001 From: Uladzislau Blok <123193120+UladzislauBlok@users.noreply.github.com> Date: Mon, 22 Sep 2025 17:20:14 +0200 Subject: [PATCH 061/100] KAFKA-19299: Fix race condition in RemoteIndexCacheTest (#19927) This MR should be couple of race conditions in RemoteIndexCacheTest. 1. There was a race condition between cache-cleanup-thread and test thread, which wants to check that cache is gone. This was fixed with TestUtils#waitForCondition 2. After each test we check that there is not thread leak. This check wasn't working properly, because live of thread status is set by JVM level, we can only set interrupted status (using private native void interrupt0(); method under the hood), but we don't really know when JVM will change the live status of thread. To fix this I've refactored TestUtils#assertNoLeakedThreadsWithNameAndDaemonStatus method to use TestUtils#waitForCondition. This fix should also affect few other tests, which were flaky because of this check. See gradle run on [develocity](https://develocity.apache.org/scans/tests?search.rootProjectNames=kafka&search.timeZoneId=Europe%2FLondon&tests.container=org.apache.kafka.storage.internals.log.RemoteIndexCacheTest&tests.sortField=FLAKY) After fix test were run 10000 times with repeated test annotation: `./gradlew clean storage:test --tests org.apache.kafka.storage.internals.log.RemoteIndexCacheTest.testCacheEntryIsDeletedOnRemoval` ... `Gradle Test Run :storage:test > Gradle Test Executor 20 > RemoteIndexCacheTest > testCacheEntryIsDeletedOnRemoval() > repetition 9998 of 10000 PASSED` `Gradle Test Run :storage:test > Gradle Test Executor 20 > RemoteIndexCacheTest > testCacheEntryIsDeletedOnRemoval() > repetition 9999 of 10000 PASSED` `Gradle Test Run :storage:test > Gradle Test Executor 20 > RemoteIndexCacheTest > testCacheEntryIsDeletedOnRemoval() > repetition 10000 of 10000 PASSED` `BUILD SUCCESSFUL in 20m 9s` `148 actionable tasks: 148 executed` Reviewers: Lianet Magrans , Chia-Ping Tsai --- .../clients/producer/KafkaProducerTest.java | 2 +- .../java/org/apache/kafka/test/TestUtils.java | 16 +++++++--------- .../internals/log/RemoteIndexCacheTest.java | 10 +++++----- 3 files changed, 13 insertions(+), 15 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index bf27fbaa11f..9d8aa35f8fe 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -219,7 +219,7 @@ public class KafkaProducerTest { } @AfterEach - public void detectLeaks() { + public void detectLeaks() throws InterruptedException { // Assert no thread leakage of Kafka producer. TestUtils.assertNoLeakedThreadsWithNameAndDaemonStatus(NETWORK_THREAD_PREFIX, Boolean.TRUE); } diff --git a/clients/src/test/java/org/apache/kafka/test/TestUtils.java b/clients/src/test/java/org/apache/kafka/test/TestUtils.java index 7748bbe15f0..078d006e37a 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestUtils.java @@ -74,7 +74,6 @@ import java.util.function.Consumer; import java.util.function.Supplier; import java.util.regex.Matcher; import java.util.regex.Pattern; -import java.util.stream.Collectors; import static java.util.Arrays.asList; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -161,7 +160,9 @@ public class TestUtils { * Asserts that there are no leaked threads with a specified name prefix and daemon status. * This method checks all threads in the JVM, filters them by the provided thread name prefix * and daemon status, and verifies that no matching threads are alive. - * If any matching threads are found, the test will fail. + * Use the {@link #waitForCondition(TestCondition, String) waitForCondition} to retry the check at a regular interval + * until either no matching threads are found or the timeout is exceeded. + * If any matching, alive threads are found after the timeout has elapsed, the assertion will fail. * * @param threadName The prefix of the thread names to check. Only threads whose names * start with this prefix will be considered. @@ -169,14 +170,11 @@ public class TestUtils { * daemon status (either true for daemon threads or false for non-daemon threads) * will be considered. * - * @throws AssertionError If any thread with the specified name prefix and daemon status is found and is alive. + * @throws AssertionError If any thread with the specified name prefix and daemon status are found after the timeout. */ - public static void assertNoLeakedThreadsWithNameAndDaemonStatus(String threadName, boolean isDaemon) { - List threads = Thread.getAllStackTraces().keySet().stream() - .filter(t -> t.isDaemon() == isDaemon && t.isAlive() && t.getName().startsWith(threadName)) - .collect(Collectors.toList()); - int threadCount = threads.size(); - assertEquals(0, threadCount); + public static void assertNoLeakedThreadsWithNameAndDaemonStatus(String threadName, boolean isDaemon) throws InterruptedException { + waitForCondition(() -> Thread.getAllStackTraces().keySet().stream() + .noneMatch(t -> t.isDaemon() == isDaemon && t.isAlive() && t.getName().startsWith(threadName)), String.format("Thread leak detected: %s", threadName)); } /** diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/RemoteIndexCacheTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/RemoteIndexCacheTest.java index 42002b69da6..25c8bce5c3a 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/RemoteIndexCacheTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/RemoteIndexCacheTest.java @@ -127,7 +127,7 @@ public class RemoteIndexCacheTest { } @AfterEach - public void cleanup() { + public void cleanup() throws InterruptedException { reset(rsm); // the files created for the test will be deleted automatically on thread exit since we use temp dir Utils.closeQuietly(cache, "RemoteIndexCache created for unit test"); @@ -344,13 +344,13 @@ public class RemoteIndexCacheTest { }, "Failed to delete index file"); // verify no index files on disk - assertFalse(getIndexFileFromRemoteCacheDir(cache, LogFileUtils.INDEX_FILE_SUFFIX).isPresent(), + TestUtils.waitForCondition(() -> getIndexFileFromRemoteCacheDir(cache, LogFileUtils.INDEX_FILE_SUFFIX).isEmpty(), "Offset index file should not be present on disk at " + tpDir.toPath()); - assertFalse(getIndexFileFromRemoteCacheDir(cache, LogFileUtils.TXN_INDEX_FILE_SUFFIX).isPresent(), + TestUtils.waitForCondition(() -> getIndexFileFromRemoteCacheDir(cache, LogFileUtils.TXN_INDEX_FILE_SUFFIX).isEmpty(), "Txn index file should not be present on disk at " + tpDir.toPath()); - assertFalse(getIndexFileFromRemoteCacheDir(cache, LogFileUtils.TIME_INDEX_FILE_SUFFIX).isPresent(), + TestUtils.waitForCondition(() -> getIndexFileFromRemoteCacheDir(cache, LogFileUtils.TIME_INDEX_FILE_SUFFIX).isEmpty(), "Time index file should not be present on disk at " + tpDir.toPath()); - assertFalse(getIndexFileFromRemoteCacheDir(cache, LogFileUtils.DELETED_FILE_SUFFIX).isPresent(), + TestUtils.waitForCondition(() -> getIndexFileFromRemoteCacheDir(cache, LogFileUtils.DELETED_FILE_SUFFIX).isEmpty(), "Index file marked for deletion should not be present on disk at " + tpDir.toPath()); } From 71efb892900387cf4cd8c65cd949609c712c19cc Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Mon, 22 Sep 2025 09:54:50 -0700 Subject: [PATCH 062/100] MINOR: fix incorrect offset reset logging (#20558) We need to only pass in the reset strategy, as the `logMessage` parameter was removed. Reviewers: Chia-Ping Tsai , Lucas Brutschy --- .../streams/processor/internals/StreamThread.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) 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 d8932206217..649a1ec666c 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 @@ -1532,6 +1532,7 @@ public class StreamThread extends Thread implements ProcessingThread { try { records = mainConsumer.poll(pollTime); } catch (final InvalidOffsetException e) { + log.info("Found no valid offset for {} partitions, resetting.", e.partitions().size()); resetOffsets(e.partitions(), e); } @@ -1647,14 +1648,14 @@ public class StreamThread extends Thread implements ProcessingThread { addToResetList( partition, seekToBeginning, - "Setting topic '{}' to consume from earliest offset", + "Setting topic '{}' to consume from 'earliest' offset", loggedTopics ); } else if (resetPolicy == AutoOffsetResetStrategy.LATEST) { addToResetList( partition, seekToEnd, - "Setting topic '{}' to consume from latest offset", + "Setting topic '{}' to consume from 'latest' offset", loggedTopics ); } else if (resetPolicy.type() == AutoOffsetResetStrategy.StrategyType.BY_DURATION) { @@ -1662,7 +1663,7 @@ public class StreamThread extends Thread implements ProcessingThread { partition, seekByDuration, resetPolicy.duration().get(), - "Setting topic '{}' to consume from by_duration:{}", + "Setting topic '{}' to consume from 'by_duration:{}'", resetPolicy.duration().get().toString(), loggedTopics ); @@ -1778,12 +1779,12 @@ public class StreamThread extends Thread implements ProcessingThread { private void addToResetList( final TopicPartition partition, final Set partitions, - final String resetPolicy, + final String logMessage, final Set loggedTopics ) { final String topic = partition.topic(); if (loggedTopics.add(topic)) { - log.info("Setting topic '{}' to consume from {} offset", topic, resetPolicy); + log.info(logMessage, topic); } partitions.add(partition); } From dbe9d34e47452059b78a73f782c4911ec7e366f7 Mon Sep 17 00:00:00 2001 From: ally heev Date: Tue, 23 Sep 2025 14:31:40 +0530 Subject: [PATCH 063/100] KAFKA-19624: Improve consistency of command-line arguments for consumer performance tests (#20385) resolves https://issues.apache.org/jira/browse/KAFKA-19624 Reviewers: @brandboat, @AndrewJSchofield, @m1a2st --- .../performance/consumer_performance.py | 15 +- .../performance/share_consumer_performance.py | 17 +- .../kafka/tools/ConsumerPerformance.java | 133 ++++++++++------ .../kafka/tools/ShareConsumerPerformance.java | 145 ++++++++++++------ .../kafka/tools/ConsumerPerformanceTest.java | 138 +++++++++++++++-- .../tools/ShareConsumerPerformanceTest.java | 132 +++++++++++++++- 6 files changed, 459 insertions(+), 121 deletions(-) diff --git a/tests/kafkatest/services/performance/consumer_performance.py b/tests/kafkatest/services/performance/consumer_performance.py index 28086e82818..aa414e5a89f 100644 --- a/tests/kafkatest/services/performance/consumer_performance.py +++ b/tests/kafkatest/services/performance/consumer_performance.py @@ -40,7 +40,7 @@ class ConsumerPerformanceService(PerformanceService): "socket-buffer-size", "The size of the tcp RECV size." "new-consumer", "Use the new consumer implementation." - "consumer.config", "Consumer config properties file." + "command-config", "Config properties file." """ # Root directory for persistent output @@ -83,10 +83,14 @@ class ConsumerPerformanceService(PerformanceService): def args(self, version): """Dictionary of arguments used to start the Consumer Performance script.""" args = { - 'topic': self.topic, - 'messages': self.messages + 'topic': self.topic } + if version.supports_command_config(): + args['num-records'] = self.messages + else: + args['messages'] = self.messages + if version < V_2_5_0: args['broker-list'] = self.kafka.bootstrap_servers(self.security_config.security_protocol) else: @@ -115,7 +119,10 @@ class ConsumerPerformanceService(PerformanceService): for key, value in self.args(node.version).items(): cmd += " --%s %s" % (key, value) - cmd += " --consumer.config %s" % ConsumerPerformanceService.CONFIG_FILE + if node.version.supports_command_config(): + cmd += " --command-config %s" % ConsumerPerformanceService.CONFIG_FILE + else: + cmd += " --consumer.config %s" % ConsumerPerformanceService.CONFIG_FILE for key, value in self.settings.items(): cmd += " %s=%s" % (str(key), str(value)) diff --git a/tests/kafkatest/services/performance/share_consumer_performance.py b/tests/kafkatest/services/performance/share_consumer_performance.py index ccb09524580..432d1e1da81 100644 --- a/tests/kafkatest/services/performance/share_consumer_performance.py +++ b/tests/kafkatest/services/performance/share_consumer_performance.py @@ -33,7 +33,7 @@ class ShareConsumerPerformanceService(PerformanceService): "socket-buffer-size", "The size of the tcp RECV size." - "consumer.config", "Consumer config properties file." + "command-config", "Config properties file." """ # Root directory for persistent output @@ -73,16 +73,20 @@ class ShareConsumerPerformanceService(PerformanceService): for node in self.nodes: node.version = version - def args(self): + def args(self, version): """Dictionary of arguments used to start the Share Consumer Performance script.""" args = { 'topic': self.topic, - 'messages': self.messages, 'bootstrap-server': self.kafka.bootstrap_servers(self.security_config.security_protocol), 'group': self.group, 'timeout': self.timeout } + if version.supports_command_config(): + args['num-records'] = self.messages + else: + args['messages'] = self.messages + if self.fetch_size is not None: args['fetch-size'] = self.fetch_size @@ -97,10 +101,13 @@ class ShareConsumerPerformanceService(PerformanceService): cmd += " export KAFKA_OPTS=%s;" % self.security_config.kafka_opts cmd += " export KAFKA_LOG4J_OPTS=\"%s%s\";" % (get_log4j_config_param(node), get_log4j_config_for_tools(node)) cmd += " %s" % self.path.script("kafka-share-consumer-perf-test.sh", node) - for key, value in self.args().items(): + for key, value in self.args(node.version).items(): cmd += " --%s %s" % (key, value) - cmd += " --consumer.config %s" % ShareConsumerPerformanceService.CONFIG_FILE + if node.version.supports_command_config(): + cmd += " --command-config %s" % ShareConsumerPerformanceService.CONFIG_FILE + else: + cmd += " --consumer.config %s" % ShareConsumerPerformanceService.CONFIG_FILE for key, value in self.settings.items(): cmd += " %s=%s" % (str(key), str(value)) diff --git a/tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java b/tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java index 62def15d324..0334af83aa1 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java +++ b/tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java @@ -48,6 +48,7 @@ import joptsimple.OptionException; import joptsimple.OptionSpec; import static joptsimple.util.RegexMatcher.regex; +import static org.apache.kafka.server.util.CommandLineUtils.parseKeyValueArgs; public class ConsumerPerformance { private static final Logger LOG = LoggerFactory.getLogger(ConsumerPerformance.class); @@ -61,7 +62,7 @@ public class ConsumerPerformance { try { LOG.info("Starting consumer..."); ConsumerPerfOptions options = new ConsumerPerfOptions(args); - AtomicLong totalMessagesRead = new AtomicLong(0); + AtomicLong totalRecordsRead = new AtomicLong(0); AtomicLong totalBytesRead = new AtomicLong(0); AtomicLong joinTimeMs = new AtomicLong(0); AtomicLong joinTimeMsInSingleRound = new AtomicLong(0); @@ -71,14 +72,14 @@ public class ConsumerPerformance { try (Consumer consumer = consumerCreator.apply(options.props())) { long bytesRead = 0L; - long messagesRead = 0L; + long recordsRead = 0L; long lastBytesRead = 0L; - long lastMessagesRead = 0L; + long lastRecordsRead = 0L; long currentTimeMs = System.currentTimeMillis(); long joinStartMs = currentTimeMs; long startMs = currentTimeMs; - consume(consumer, options, totalMessagesRead, totalBytesRead, joinTimeMs, - bytesRead, messagesRead, lastBytesRead, lastMessagesRead, + consume(consumer, options, totalRecordsRead, totalBytesRead, joinTimeMs, + bytesRead, recordsRead, lastBytesRead, lastRecordsRead, joinStartMs, joinTimeMsInSingleRound); long endMs = System.currentTimeMillis(); @@ -92,12 +93,12 @@ public class ConsumerPerformance { options.dateFormat().format(endMs), totalMbRead, totalMbRead / elapsedSec, - totalMessagesRead.get(), - totalMessagesRead.get() / elapsedSec, + totalRecordsRead.get(), + totalRecordsRead.get() / elapsedSec, joinTimeMs.get(), fetchTimeInMs, totalMbRead / (fetchTimeInMs / 1000.0), - totalMessagesRead.get() / (fetchTimeInMs / 1000.0) + totalRecordsRead.get() / (fetchTimeInMs / 1000.0) ); } @@ -122,16 +123,16 @@ public class ConsumerPerformance { private static void consume(Consumer consumer, ConsumerPerfOptions options, - AtomicLong totalMessagesRead, + AtomicLong totalRecordsRead, AtomicLong totalBytesRead, AtomicLong joinTimeMs, long bytesRead, - long messagesRead, + long recordsRead, long lastBytesRead, - long lastMessagesRead, + long lastRecordsRead, long joinStartMs, AtomicLong joinTimeMsInSingleRound) { - long numMessages = options.numMessages(); + long numRecords = options.numRecords(); long recordFetchTimeoutMs = options.recordFetchTimeoutMs(); long reportingIntervalMs = options.reportingIntervalMs(); boolean showDetailedStats = options.showDetailedStats(); @@ -149,55 +150,55 @@ public class ConsumerPerformance { long lastReportTimeMs = currentTimeMs; long lastConsumedTimeMs = currentTimeMs; - while (messagesRead < numMessages && currentTimeMs - lastConsumedTimeMs <= recordFetchTimeoutMs) { + while (recordsRead < numRecords && currentTimeMs - lastConsumedTimeMs <= recordFetchTimeoutMs) { ConsumerRecords records = consumer.poll(Duration.ofMillis(100)); currentTimeMs = System.currentTimeMillis(); if (!records.isEmpty()) lastConsumedTimeMs = currentTimeMs; for (ConsumerRecord record : records) { - messagesRead += 1; + recordsRead += 1; if (record.key() != null) bytesRead += record.key().length; if (record.value() != null) bytesRead += record.value().length; if (currentTimeMs - lastReportTimeMs >= reportingIntervalMs) { if (showDetailedStats) - printConsumerProgress(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, + printConsumerProgress(0, bytesRead, lastBytesRead, recordsRead, lastRecordsRead, lastReportTimeMs, currentTimeMs, dateFormat, joinTimeMsInSingleRound.get()); joinTimeMsInSingleRound = new AtomicLong(0); lastReportTimeMs = currentTimeMs; - lastMessagesRead = messagesRead; + lastRecordsRead = recordsRead; lastBytesRead = bytesRead; } } } - if (messagesRead < numMessages) - System.out.printf("WARNING: Exiting before consuming the expected number of messages: timeout (%d ms) exceeded. " + + if (recordsRead < numRecords) + System.out.printf("WARNING: Exiting before consuming the expected number of records: timeout (%d ms) exceeded. " + "You can use the --timeout option to increase the timeout.%n", recordFetchTimeoutMs); - totalMessagesRead.set(messagesRead); + totalRecordsRead.set(recordsRead); totalBytesRead.set(bytesRead); } protected static void printConsumerProgress(int id, long bytesRead, long lastBytesRead, - long messagesRead, - long lastMessagesRead, + long recordsRead, + long lastRecordsRead, long startMs, long endMs, SimpleDateFormat dateFormat, long joinTimeMsInSingleRound) { - printBasicProgress(id, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, dateFormat); - printExtendedProgress(bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, joinTimeMsInSingleRound); + printBasicProgress(id, bytesRead, lastBytesRead, recordsRead, lastRecordsRead, startMs, endMs, dateFormat); + printExtendedProgress(bytesRead, lastBytesRead, recordsRead, lastRecordsRead, startMs, endMs, joinTimeMsInSingleRound); System.out.println(); } private static void printBasicProgress(int id, long bytesRead, long lastBytesRead, - long messagesRead, - long lastMessagesRead, + long recordsRead, + long lastRecordsRead, long startMs, long endMs, SimpleDateFormat dateFormat) { @@ -205,25 +206,25 @@ public class ConsumerPerformance { double totalMbRead = (bytesRead * 1.0) / (1024 * 1024); double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024); double intervalMbPerSec = 1000.0 * intervalMbRead / elapsedMs; - double intervalMessagesPerSec = ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0; + double intervalRecordsPerSec = ((recordsRead - lastRecordsRead) / elapsedMs) * 1000.0; System.out.printf("%s, %d, %.4f, %.4f, %d, %.4f", dateFormat.format(endMs), id, - totalMbRead, intervalMbPerSec, messagesRead, intervalMessagesPerSec); + totalMbRead, intervalMbPerSec, recordsRead, intervalRecordsPerSec); } private static void printExtendedProgress(long bytesRead, long lastBytesRead, - long messagesRead, - long lastMessagesRead, + long recordsRead, + long lastRecordsRead, long startMs, long endMs, long joinTimeMsInSingleRound) { long fetchTimeMs = endMs - startMs - joinTimeMsInSingleRound; double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024); - long intervalMessagesRead = messagesRead - lastMessagesRead; + long intervalRecordsRead = recordsRead - lastRecordsRead; double intervalMbPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMbRead / fetchTimeMs; - double intervalMessagesPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMessagesRead / fetchTimeMs; + double intervalRecordsPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalRecordsRead / fetchTimeMs; System.out.printf(", %d, %d, %.4f, %.4f", joinTimeMsInSingleRound, - fetchTimeMs, intervalMbPerSec, intervalMessagesPerSec); + fetchTimeMs, intervalMbPerSec, intervalRecordsPerSec); } public static class ConsumerPerfRebListener implements ConsumerRebalanceListener { @@ -256,13 +257,18 @@ public class ConsumerPerformance { private final OptionSpec includeOpt; private final OptionSpec groupIdOpt; private final OptionSpec fetchSizeOpt; + private final OptionSpec commandPropertiesOpt; private final OptionSpec resetBeginningOffsetOpt; private final OptionSpec socketBufferSizeOpt; + @Deprecated(since = "4.2", forRemoval = true) private final OptionSpec consumerConfigOpt; + private final OptionSpec commandConfigOpt; private final OptionSpec printMetricsOpt; private final OptionSpec showDetailedStatsOpt; private final OptionSpec recordFetchTimeoutOpt; + @Deprecated(since = "4.2", forRemoval = true) private final OptionSpec numMessagesOpt; + private final OptionSpec numRecordsOpt; private final OptionSpec reportingIntervalOpt; private final OptionSpec dateFormatOpt; private final OptionSpec hideHeaderOpt; @@ -291,26 +297,41 @@ public class ConsumerPerformance { .describedAs("size") .ofType(Integer.class) .defaultsTo(1024 * 1024); + commandPropertiesOpt = parser.accepts("command-property", "Kafka consumer related configuration properties like client.id. " + + "These configs take precedence over those passed via --command-config or --consumer.config.") + .withRequiredArg() + .describedAs("prop1=val1") + .ofType(String.class); resetBeginningOffsetOpt = parser.accepts("from-latest", "If the consumer does not already have an established " + - "offset to consume from, start with the latest message present in the log rather than the earliest message."); + "offset to consume from, start with the latest record present in the log rather than the earliest record."); socketBufferSizeOpt = parser.accepts("socket-buffer-size", "The size of the tcp RECV size.") .withRequiredArg() .describedAs("size") .ofType(Integer.class) .defaultsTo(2 * 1024 * 1024); - consumerConfigOpt = parser.accepts("consumer.config", "Consumer config properties file.") + consumerConfigOpt = parser.accepts("consumer.config", "(DEPRECATED) Consumer config properties file. " + + "This option will be removed in a future version. Use --command-config instead.") + .withRequiredArg() + .describedAs("config file") + .ofType(String.class); + commandConfigOpt = parser.accepts("command-config", "Config properties file.") .withRequiredArg() .describedAs("config file") .ofType(String.class); printMetricsOpt = parser.accepts("print-metrics", "Print out the metrics."); showDetailedStatsOpt = parser.accepts("show-detailed-stats", "If set, stats are reported for each reporting " + - "interval as configured by reporting-interval"); + "interval as configured by reporting-interval."); recordFetchTimeoutOpt = parser.accepts("timeout", "The maximum allowed time in milliseconds between returned records.") .withOptionalArg() .describedAs("milliseconds") .ofType(Long.class) .defaultsTo(10_000L); - numMessagesOpt = parser.accepts("messages", "REQUIRED: The number of messages to consume.") + numMessagesOpt = parser.accepts("messages", "(DEPRECATED) The number of records to consume. " + + "This option will be removed in a future version. Use --num-records instead.") + .withRequiredArg() + .describedAs("count") + .ofType(Long.class); + numRecordsOpt = parser.accepts("num-records", "REQUIRED: The number of records to consume.") .withRequiredArg() .describedAs("count") .ofType(Long.class); @@ -326,7 +347,7 @@ public class ConsumerPerformance { .describedAs("date format") .ofType(String.class) .defaultsTo("yyyy-MM-dd HH:mm:ss:SSS"); - hideHeaderOpt = parser.accepts("hide-header", "If set, skips printing the header for the stats"); + hideHeaderOpt = parser.accepts("hide-header", "If set, skips printing the header for the stats."); try { options = parser.parse(args); } catch (OptionException e) { @@ -335,8 +356,19 @@ public class ConsumerPerformance { } if (options != null) { CommandLineUtils.maybePrintHelpOrVersion(this, "This tool is used to verify the consumer performance."); - CommandLineUtils.checkRequiredArgs(parser, options, numMessagesOpt); + CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt); CommandLineUtils.checkOneOfArgs(parser, options, topicOpt, includeOpt); + + CommandLineUtils.checkOneOfArgs(parser, options, numMessagesOpt, numRecordsOpt); + CommandLineUtils.checkInvalidArgs(parser, options, consumerConfigOpt, commandConfigOpt); + + if (options.has(numMessagesOpt)) { + System.out.println("Warning: --messages is deprecated. Use --num-records instead."); + } + + if (options.has(consumerConfigOpt)) { + System.out.println("Warning: --consumer.config is deprecated. Use --command-config instead."); + } } } @@ -348,10 +380,23 @@ public class ConsumerPerformance { return options.valueOf(bootstrapServerOpt); } + private Properties readProps(List commandProperties, String commandConfigFile) throws IOException { + Properties props = commandConfigFile != null + ? Utils.loadProps(commandConfigFile) + : new Properties(); + props.putAll(parseKeyValueArgs(commandProperties)); + return props; + } + public Properties props() throws IOException { - Properties props = (options.has(consumerConfigOpt)) - ? Utils.loadProps(options.valueOf(consumerConfigOpt)) - : new Properties(); + List commandProperties = options.valuesOf(commandPropertiesOpt); + String commandConfigFile; + if (options.has(consumerConfigOpt)) { + commandConfigFile = options.valueOf(consumerConfigOpt); + } else { + commandConfigFile = options.valueOf(commandConfigOpt); + } + Properties props = readProps(commandProperties, commandConfigFile); props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerHostsAndPorts()); props.put(ConsumerConfig.GROUP_ID_CONFIG, options.valueOf(groupIdOpt)); props.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, options.valueOf(socketBufferSizeOpt).toString()); @@ -378,8 +423,10 @@ public class ConsumerPerformance { : Optional.empty(); } - public long numMessages() { - return options.valueOf(numMessagesOpt); + public long numRecords() { + return options.has(numMessagesOpt) + ? options.valueOf(numMessagesOpt) + : options.valueOf(numRecordsOpt); } public long reportingIntervalMs() { diff --git a/tools/src/main/java/org/apache/kafka/tools/ShareConsumerPerformance.java b/tools/src/main/java/org/apache/kafka/tools/ShareConsumerPerformance.java index 5d6179efaeb..51c66704668 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ShareConsumerPerformance.java +++ b/tools/src/main/java/org/apache/kafka/tools/ShareConsumerPerformance.java @@ -55,6 +55,7 @@ import joptsimple.OptionException; import joptsimple.OptionSpec; import static joptsimple.util.RegexMatcher.regex; +import static org.apache.kafka.server.util.CommandLineUtils.parseKeyValueArgs; public class ShareConsumerPerformance { private static final Logger LOG = LoggerFactory.getLogger(ShareConsumerPerformance.class); @@ -67,7 +68,7 @@ public class ShareConsumerPerformance { try { LOG.info("Starting share consumer/consumers..."); ShareConsumerPerfOptions options = new ShareConsumerPerfOptions(args); - AtomicLong totalMessagesRead = new AtomicLong(0); + AtomicLong totalRecordsRead = new AtomicLong(0); AtomicLong totalBytesRead = new AtomicLong(0); if (!options.hideHeader()) @@ -78,7 +79,7 @@ public class ShareConsumerPerformance { shareConsumers.add(shareConsumerCreator.apply(options.props())); } long startMs = System.currentTimeMillis(); - consume(shareConsumers, options, totalMessagesRead, totalBytesRead, startMs); + consume(shareConsumers, options, totalRecordsRead, totalBytesRead, startMs); long endMs = System.currentTimeMillis(); List> shareConsumersMetrics = new ArrayList<>(); @@ -93,7 +94,7 @@ public class ShareConsumerPerformance { // Print final stats for share group. double elapsedSec = (endMs - startMs) / 1_000.0; long fetchTimeInMs = endMs - startMs; - printStats(totalBytesRead.get(), totalMessagesRead.get(), elapsedSec, fetchTimeInMs, startMs, endMs, + printStats(totalBytesRead.get(), totalRecordsRead.get(), elapsedSec, fetchTimeInMs, startMs, endMs, options.dateFormat(), -1); shareConsumersMetrics.forEach(ToolsUtils::printMetrics); @@ -113,15 +114,15 @@ public class ShareConsumerPerformance { private static void consume(List> shareConsumers, ShareConsumerPerfOptions options, - AtomicLong totalMessagesRead, + AtomicLong totalRecordsRead, AtomicLong totalBytesRead, long startMs) throws ExecutionException, InterruptedException { - long numMessages = options.numMessages(); + long numRecords = options.numRecords(); long recordFetchTimeoutMs = options.recordFetchTimeoutMs(); shareConsumers.forEach(shareConsumer -> shareConsumer.subscribe(options.topic())); // Now start the benchmark. - AtomicLong messagesRead = new AtomicLong(0); + AtomicLong recordsRead = new AtomicLong(0); AtomicLong bytesRead = new AtomicLong(0); List shareConsumersConsumptionDetails = new ArrayList<>(); @@ -133,7 +134,7 @@ public class ShareConsumerPerformance { ShareConsumerConsumption shareConsumerConsumption = new ShareConsumerConsumption(0, 0); futures.add(executorService.submit(() -> { try { - consumeMessagesForSingleShareConsumer(shareConsumers.get(index), messagesRead, bytesRead, options, + consumeRecordsForSingleShareConsumer(shareConsumers.get(index), recordsRead, bytesRead, options, shareConsumerConsumption, index + 1); } catch (InterruptedException e) { throw new RuntimeException(e); @@ -171,22 +172,22 @@ public class ShareConsumerPerformance { // Print stats for share consumer. double elapsedSec = (endMs - startMs) / 1_000.0; long fetchTimeInMs = endMs - startMs; - long messagesReadByConsumer = shareConsumersConsumptionDetails.get(index).messagesConsumed(); + long recordsReadByConsumer = shareConsumersConsumptionDetails.get(index).recordsConsumed(); long bytesReadByConsumer = shareConsumersConsumptionDetails.get(index).bytesConsumed(); - printStats(bytesReadByConsumer, messagesReadByConsumer, elapsedSec, fetchTimeInMs, startMs, endMs, options.dateFormat(), index + 1); + printStats(bytesReadByConsumer, recordsReadByConsumer, elapsedSec, fetchTimeInMs, startMs, endMs, options.dateFormat(), index + 1); } } - if (messagesRead.get() < numMessages) { - System.out.printf("WARNING: Exiting before consuming the expected number of messages: timeout (%d ms) exceeded. " + + if (recordsRead.get() < numRecords) { + System.out.printf("WARNING: Exiting before consuming the expected number of records: timeout (%d ms) exceeded. " + "You can use the --timeout option to increase the timeout.%n", recordFetchTimeoutMs); } - totalMessagesRead.set(messagesRead.get()); + totalRecordsRead.set(recordsRead.get()); totalBytesRead.set(bytesRead.get()); } - private static void consumeMessagesForSingleShareConsumer(ShareConsumer shareConsumer, - AtomicLong totalMessagesRead, + private static void consumeRecordsForSingleShareConsumer(ShareConsumer shareConsumer, + AtomicLong totalRecordsRead, AtomicLong totalBytesRead, ShareConsumerPerfOptions options, ShareConsumerConsumption shareConsumerConsumption, @@ -197,17 +198,17 @@ public class ShareConsumerPerformance { long lastReportTimeMs = currentTimeMs; long lastBytesRead = 0L; - long lastMessagesRead = 0L; - long messagesReadByConsumer = 0L; + long lastRecordsRead = 0L; + long recordsReadByConsumer = 0L; long bytesReadByConsumer = 0L; - while (totalMessagesRead.get() < options.numMessages() && currentTimeMs - lastConsumedTimeMs <= options.recordFetchTimeoutMs()) { + while (totalRecordsRead.get() < options.numRecords() && currentTimeMs - lastConsumedTimeMs <= options.recordFetchTimeoutMs()) { ConsumerRecords records = shareConsumer.poll(Duration.ofMillis(100)); currentTimeMs = System.currentTimeMillis(); if (!records.isEmpty()) lastConsumedTimeMs = currentTimeMs; for (ConsumerRecord record : records) { - messagesReadByConsumer += 1; - totalMessagesRead.addAndGet(1); + recordsReadByConsumer += 1; + totalRecordsRead.addAndGet(1); if (record.key() != null) { bytesReadByConsumer += record.key().length; totalBytesRead.addAndGet(record.key().length); @@ -218,13 +219,13 @@ public class ShareConsumerPerformance { } if (currentTimeMs - lastReportTimeMs >= options.reportingIntervalMs()) { if (options.showDetailedStats()) - printShareConsumerProgress(bytesReadByConsumer, lastBytesRead, messagesReadByConsumer, lastMessagesRead, + printShareConsumerProgress(bytesReadByConsumer, lastBytesRead, recordsReadByConsumer, lastRecordsRead, lastReportTimeMs, currentTimeMs, dateFormat, index); lastReportTimeMs = currentTimeMs; - lastMessagesRead = messagesReadByConsumer; + lastRecordsRead = recordsReadByConsumer; lastBytesRead = bytesReadByConsumer; } - shareConsumerConsumption.updateMessagesConsumed(messagesReadByConsumer); + shareConsumerConsumption.updateRecordsConsumed(recordsReadByConsumer); shareConsumerConsumption.updateBytesConsumed(bytesReadByConsumer); } } @@ -232,8 +233,8 @@ public class ShareConsumerPerformance { protected static void printShareConsumerProgress(long bytesRead, long lastBytesRead, - long messagesRead, - long lastMessagesRead, + long recordsRead, + long lastRecordsRead, long startMs, long endMs, SimpleDateFormat dateFormat, @@ -242,18 +243,18 @@ public class ShareConsumerPerformance { double totalMbRead = (bytesRead * 1.0) / (1024 * 1024); double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024); double intervalMbPerSec = 1000.0 * intervalMbRead / elapsedMs; - double intervalMessagesPerSec = ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0; + double intervalRecordsPerSec = ((recordsRead - lastRecordsRead) / elapsedMs) * 1000.0; long fetchTimeMs = endMs - startMs; System.out.printf("%s, %s, %.4f, %.4f, %.4f, %d, %d for share consumer %d", dateFormat.format(startMs), dateFormat.format(endMs), - totalMbRead, intervalMbPerSec, intervalMessagesPerSec, messagesRead, fetchTimeMs, index); + totalMbRead, intervalMbPerSec, intervalRecordsPerSec, recordsRead, fetchTimeMs, index); System.out.println(); } // Prints stats for both share consumer and share group. For share group, index is -1. For share consumer, // index is >= 1. private static void printStats(long bytesRead, - long messagesRead, + long recordsRead, double elapsedSec, long fetchTimeInMs, long startMs, @@ -268,8 +269,8 @@ public class ShareConsumerPerformance { dateFormat.format(endMs), totalMbRead, totalMbRead / elapsedSec, - messagesRead / elapsedSec, - messagesRead, + recordsRead / elapsedSec, + recordsRead, fetchTimeInMs ); return; @@ -279,8 +280,8 @@ public class ShareConsumerPerformance { dateFormat.format(endMs), totalMbRead, totalMbRead / elapsedSec, - messagesRead / elapsedSec, - messagesRead, + recordsRead / elapsedSec, + recordsRead, fetchTimeInMs ); } @@ -290,12 +291,17 @@ public class ShareConsumerPerformance { private final OptionSpec topicOpt; private final OptionSpec groupIdOpt; private final OptionSpec fetchSizeOpt; + private final OptionSpec commandPropertiesOpt; private final OptionSpec socketBufferSizeOpt; + @Deprecated(since = "4.2", forRemoval = true) private final OptionSpec consumerConfigOpt; + private final OptionSpec commandConfigOpt; private final OptionSpec printMetricsOpt; private final OptionSpec showDetailedStatsOpt; private final OptionSpec recordFetchTimeoutOpt; + @Deprecated(since = "4.2", forRemoval = true) private final OptionSpec numMessagesOpt; + private final OptionSpec numRecordsOpt; private final OptionSpec reportingIntervalOpt; private final OptionSpec dateFormatOpt; private final OptionSpec hideHeaderOpt; @@ -322,24 +328,39 @@ public class ShareConsumerPerformance { .describedAs("size") .ofType(Integer.class) .defaultsTo(1024 * 1024); + commandPropertiesOpt = parser.accepts("command-property", "Kafka share consumer related configuration properties like client.id. " + + "These configs take precedence over those passed via --command-config or --consumer.config.") + .withRequiredArg() + .describedAs("prop1=val1") + .ofType(String.class); socketBufferSizeOpt = parser.accepts("socket-buffer-size", "The size of the tcp RECV size.") .withRequiredArg() .describedAs("size") .ofType(Integer.class) .defaultsTo(2 * 1024 * 1024); - consumerConfigOpt = parser.accepts("consumer.config", "Share consumer config properties file.") + consumerConfigOpt = parser.accepts("consumer.config", "(DEPRECATED) Share consumer config properties file. " + + "This option will be removed in a future version. Use --command-config instead.") + .withRequiredArg() + .describedAs("config file") + .ofType(String.class); + commandConfigOpt = parser.accepts("command-config", "Config properties file.") .withRequiredArg() .describedAs("config file") .ofType(String.class); printMetricsOpt = parser.accepts("print-metrics", "Print out the metrics."); showDetailedStatsOpt = parser.accepts("show-detailed-stats", "If set, stats are reported for each reporting " + - "interval as configured by reporting-interval"); + "interval as configured by reporting-interval."); recordFetchTimeoutOpt = parser.accepts("timeout", "The maximum allowed time in milliseconds between returned records.") .withOptionalArg() .describedAs("milliseconds") .ofType(Long.class) .defaultsTo(10_000L); - numMessagesOpt = parser.accepts("messages", "REQUIRED: The number of messages to consume.") + numMessagesOpt = parser.accepts("messages", "(DEPRECATED) The number of records to consume. " + + "This option will be removed in a future version. Use --num-records instead.") + .withRequiredArg() + .describedAs("count") + .ofType(Long.class); + numRecordsOpt = parser.accepts("num-records", "REQUIRED: The number of records to consume.") .withRequiredArg() .describedAs("count") .ofType(Long.class); @@ -355,7 +376,7 @@ public class ShareConsumerPerformance { .describedAs("date format") .ofType(String.class) .defaultsTo("yyyy-MM-dd HH:mm:ss:SSS"); - hideHeaderOpt = parser.accepts("hide-header", "If set, skips printing the header for the stats"); + hideHeaderOpt = parser.accepts("hide-header", "If set, skips printing the header for the stats."); numThreadsOpt = parser.accepts("threads", "The number of share consumers to use for sharing the load.") .withRequiredArg() .describedAs("count") @@ -371,7 +392,18 @@ public class ShareConsumerPerformance { } if (options != null) { CommandLineUtils.maybePrintHelpOrVersion(this, "This tool is used to verify the share consumer performance."); - CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, numMessagesOpt); + CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, bootstrapServerOpt); + + CommandLineUtils.checkOneOfArgs(parser, options, numMessagesOpt, numRecordsOpt); + CommandLineUtils.checkInvalidArgs(parser, options, consumerConfigOpt, commandConfigOpt); + + if (options.has(numMessagesOpt)) { + System.out.println("Warning: --messages is deprecated. Use --num-records instead."); + } + + if (options.has(consumerConfigOpt)) { + System.out.println("Warning: --consumer.config is deprecated. Use --command-config instead."); + } } } @@ -383,10 +415,23 @@ public class ShareConsumerPerformance { return options.valueOf(bootstrapServerOpt); } - public Properties props() throws IOException { - Properties props = (options.has(consumerConfigOpt)) - ? Utils.loadProps(options.valueOf(consumerConfigOpt)) + private Properties readProps(List commandProperties, String commandConfigFile) throws IOException { + Properties props = commandConfigFile != null + ? Utils.loadProps(commandConfigFile) : new Properties(); + props.putAll(parseKeyValueArgs(commandProperties)); + return props; + } + + public Properties props() throws IOException { + List commandProperties = options.valuesOf(commandPropertiesOpt); + String commandConfigFile; + if (options.has(consumerConfigOpt)) { + commandConfigFile = options.valueOf(consumerConfigOpt); + } else { + commandConfigFile = options.valueOf(commandConfigOpt); + } + Properties props = readProps(commandProperties, commandConfigFile); props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerHostsAndPorts()); props.put(ConsumerConfig.GROUP_ID_CONFIG, options.valueOf(groupIdOpt)); props.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, options.valueOf(socketBufferSizeOpt).toString()); @@ -403,8 +448,10 @@ public class ShareConsumerPerformance { return Set.of(options.valueOf(topicOpt)); } - public long numMessages() { - return options.valueOf(numMessagesOpt); + public long numRecords() { + return options.has(numMessagesOpt) + ? options.valueOf(numMessagesOpt) + : options.valueOf(numRecordsOpt); } public int threads() { @@ -439,26 +486,26 @@ public class ShareConsumerPerformance { } } - // Helper class to know the final messages and bytes consumer by share consumer at the end of consumption. + // Helper class to know the final records and bytes consumed by share consumer at the end of consumption. private static class ShareConsumerConsumption { - private long messagesConsumed; + private long recordsConsumed; private long bytesConsumed; - public ShareConsumerConsumption(long messagesConsumed, long bytesConsumed) { - this.messagesConsumed = messagesConsumed; + public ShareConsumerConsumption(long recordsConsumed, long bytesConsumed) { + this.recordsConsumed = recordsConsumed; this.bytesConsumed = bytesConsumed; } - public long messagesConsumed() { - return messagesConsumed; + public long recordsConsumed() { + return recordsConsumed; } public long bytesConsumed() { return bytesConsumed; } - public void updateMessagesConsumed(long messagesConsumed) { - this.messagesConsumed = messagesConsumed; + public void updateRecordsConsumed(long recordsConsumed) { + this.recordsConsumed = recordsConsumed; } public void updateBytesConsumed(long bytesConsumed) { diff --git a/tools/src/test/java/org/apache/kafka/tools/ConsumerPerformanceTest.java b/tools/src/test/java/org/apache/kafka/tools/ConsumerPerformanceTest.java index df6c3a93966..497deb7808d 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ConsumerPerformanceTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ConsumerPerformanceTest.java @@ -74,7 +74,7 @@ public class ConsumerPerformanceTest { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", - "--messages", "10", + "--num-records", "10", "--print-metrics" }; @@ -82,7 +82,56 @@ public class ConsumerPerformanceTest { assertEquals("localhost:9092", config.brokerHostsAndPorts()); assertTrue(config.topic().get().contains("test")); - assertEquals(10, config.numMessages()); + assertEquals(10, config.numRecords()); + } + + @Test + public void testBootstrapServerNotPresent() { + String[] args = new String[]{ + "--topic", "test" + }; + + String err = ToolsTestUtils.captureStandardErr(() -> + new ConsumerPerformance.ConsumerPerfOptions(args)); + assertTrue(err.contains("Missing required argument \"[bootstrap-server]\"")); + } + + @Test + public void testNumOfRecordsNotPresent() { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test" + }; + + String err = ToolsTestUtils.captureStandardErr(() -> + new ConsumerPerformance.ConsumerPerfOptions(args)); + assertTrue(err.contains("Exactly one of the following arguments is required:")); + } + + @Test + public void testMessagesDeprecated() { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--messages", "10" + }; + + ConsumerPerformance.ConsumerPerfOptions config = new ConsumerPerformance.ConsumerPerfOptions(args); + assertEquals(10, config.numRecords()); + } + + @Test + public void testNumOfRecordsWithMessagesPresent() { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--messages", "10", + "--num-records", "20" + }; + + String err = ToolsTestUtils.captureStandardErr(() -> + new ConsumerPerformance.ConsumerPerfOptions(args)); + assertTrue(err.contains("Exactly one of the following arguments is required")); } @Test @@ -90,7 +139,7 @@ public class ConsumerPerformanceTest { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", - "--messages", "10", + "--num-records", "10", "--new-consumer" }; @@ -104,14 +153,14 @@ public class ConsumerPerformanceTest { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--include", "test.*", - "--messages", "10" + "--num-records", "10" }; ConsumerPerformance.ConsumerPerfOptions config = new ConsumerPerformance.ConsumerPerfOptions(args); assertEquals("localhost:9092", config.brokerHostsAndPorts()); assertTrue(config.include().get().toString().contains("test.*")); - assertEquals(10, config.numMessages()); + assertEquals(10, config.numRecords()); } @Test @@ -120,7 +169,7 @@ public class ConsumerPerformanceTest { "--bootstrap-server", "localhost:9092", "--topic", "test", "--include", "test.*", - "--messages", "10" + "--num-records", "10" }; String err = ToolsTestUtils.captureStandardErr(() -> new ConsumerPerformance.ConsumerPerfOptions(args)); @@ -132,7 +181,7 @@ public class ConsumerPerformanceTest { public void testConfigWithoutTopicAndInclude() { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", - "--messages", "10" + "--num-records", "10" }; String err = ToolsTestUtils.captureStandardErr(() -> new ConsumerPerformance.ConsumerPerfOptions(args)); @@ -141,8 +190,10 @@ public class ConsumerPerformanceTest { } @Test - public void testClientIdOverride() throws IOException { - File tempFile = Files.createFile(tempDir.resolve("test_consumer_config.conf")).toFile(); + public void testCommandProperty() throws IOException { + Path configPath = tempDir.resolve("test_command_property_consumer_perf.conf"); + Files.deleteIfExists(configPath); + File tempFile = Files.createFile(configPath).toFile(); try (PrintWriter output = new PrintWriter(Files.newOutputStream(tempFile.toPath()))) { output.println("client.id=consumer-1"); output.flush(); @@ -151,7 +202,54 @@ public class ConsumerPerformanceTest { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", - "--messages", "10", + "--num-records", "10", + "--command-property", "client.id=consumer-2", + "--command-config", tempFile.getAbsolutePath(), + "--command-property", "prop=val" + }; + + ConsumerPerformance.ConsumerPerfOptions config = new ConsumerPerformance.ConsumerPerfOptions(args); + + assertEquals("consumer-2", config.props().getProperty(ConsumerConfig.CLIENT_ID_CONFIG)); + assertEquals("val", config.props().getProperty("prop")); + } + + @Test + public void testClientIdOverride() throws IOException { + Path configPath = tempDir.resolve("test_client_id_override_consumer_perf.conf"); + Files.deleteIfExists(configPath); + File tempFile = Files.createFile(configPath).toFile(); + try (PrintWriter output = new PrintWriter(Files.newOutputStream(tempFile.toPath()))) { + output.println("client.id=consumer-1"); + output.flush(); + } + + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--num-records", "10", + "--command-config", tempFile.getAbsolutePath() + }; + + ConsumerPerformance.ConsumerPerfOptions config = new ConsumerPerformance.ConsumerPerfOptions(args); + + assertEquals("consumer-1", config.props().getProperty(ConsumerConfig.CLIENT_ID_CONFIG)); + } + + @Test + public void testConsumerConfigDeprecated() throws IOException { + Path configPath = tempDir.resolve("test_consumer_config_deprecated_consumer_perf.conf"); + Files.deleteIfExists(configPath); + File tempFile = Files.createFile(configPath).toFile(); + try (PrintWriter output = new PrintWriter(Files.newOutputStream(tempFile.toPath()))) { + output.println("client.id=consumer-1"); + output.flush(); + } + + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--num-records", "10", "--consumer.config", tempFile.getAbsolutePath() }; @@ -160,12 +258,28 @@ public class ConsumerPerformanceTest { assertEquals("consumer-1", config.props().getProperty(ConsumerConfig.CLIENT_ID_CONFIG)); } + @Test + public void testCommandConfigWithConsumerConfigPresent() { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--num-records", "10", + "--consumer.config", "some-path", + "--command-config", "some-path" + }; + + String err = ToolsTestUtils.captureStandardErr(() -> + new ConsumerPerformance.ConsumerPerfOptions(args)); + assertTrue(err.contains(String.format("Option \"%s\" can't be used with option \"%s\"", + "[consumer.config]", "[command-config]"))); + } + @Test public void testDefaultClientId() throws IOException { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", - "--messages", "10" + "--num-records", "10" }; ConsumerPerformance.ConsumerPerfOptions config = new ConsumerPerformance.ConsumerPerfOptions(args); @@ -178,7 +292,7 @@ public class ConsumerPerformanceTest { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", - "--messages", "0", + "--num-records", "0", "--print-metrics" }; diff --git a/tools/src/test/java/org/apache/kafka/tools/ShareConsumerPerformanceTest.java b/tools/src/test/java/org/apache/kafka/tools/ShareConsumerPerformanceTest.java index a22a97f8211..6cffde627bb 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ShareConsumerPerformanceTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ShareConsumerPerformanceTest.java @@ -67,7 +67,7 @@ public class ShareConsumerPerformanceTest { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", - "--messages", "10", + "--num-records", "10", "--print-metrics" }; @@ -75,7 +75,57 @@ public class ShareConsumerPerformanceTest { assertEquals("localhost:9092", config.brokerHostsAndPorts()); assertTrue(config.topic().contains("test")); - assertEquals(10, config.numMessages()); + assertEquals(10, config.numRecords()); + } + + @Test + public void testBootstrapServerNotPresent() { + String[] args = new String[]{ + "--topic", "test" + }; + + String err = ToolsTestUtils.captureStandardErr(() -> + new ShareConsumerPerformance.ShareConsumerPerfOptions(args)); + assertTrue(err.contains("Missing required argument \"[bootstrap-server]\"")); + } + + @Test + public void testNumOfRecordsNotPresent() { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test" + }; + + String err = ToolsTestUtils.captureStandardErr(() -> + new ShareConsumerPerformance.ShareConsumerPerfOptions(args)); + assertTrue(err.contains("Exactly one of the following arguments is required:")); + } + + @Test + public void testMessagesDeprecated() { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--messages", "10" + }; + + ShareConsumerPerformance.ShareConsumerPerfOptions config = + new ShareConsumerPerformance.ShareConsumerPerfOptions(args); + assertEquals(10, config.numRecords()); + } + + @Test + public void testNumOfRecordsWithMessagesPresent() { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--messages", "10", + "--num-records", "20" + }; + + String err = ToolsTestUtils.captureStandardErr(() -> + new ShareConsumerPerformance.ShareConsumerPerfOptions(args)); + assertTrue(err.contains("Exactly one of the following arguments is required")); } @Test @@ -83,7 +133,7 @@ public class ShareConsumerPerformanceTest { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", - "--messages", "10", + "--num-records", "10", "--new-share-consumer" }; @@ -92,9 +142,36 @@ public class ShareConsumerPerformanceTest { assertTrue(err.contains("new-share-consumer is not a recognized option")); } + @Test + public void testCommandProperty() throws IOException { + Path configPath = tempDir.resolve("test_command_property_share_consumer_perf.conf"); + Files.deleteIfExists(configPath); + File tempFile = Files.createFile(configPath).toFile(); + try (PrintWriter output = new PrintWriter(Files.newOutputStream(tempFile.toPath()))) { + output.println("client.id=consumer-1"); + output.flush(); + } + + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--num-records", "10", + "--command-property", "client.id=consumer-2", + "--command-config", tempFile.getAbsolutePath(), + "--command-property", "prop=val" + }; + + ShareConsumerPerformance.ShareConsumerPerfOptions config = new ShareConsumerPerformance.ShareConsumerPerfOptions(args); + + assertEquals("consumer-2", config.props().getProperty(ConsumerConfig.CLIENT_ID_CONFIG)); + assertEquals("val", config.props().getProperty("prop")); + } + @Test public void testClientIdOverride() throws IOException { - File tempFile = Files.createFile(tempDir.resolve("test_share_consumer_config.conf")).toFile(); + Path configPath = tempDir.resolve("test_client_id_override_share_consumer_perf.conf"); + Files.deleteIfExists(configPath); + File tempFile = Files.createFile(configPath).toFile(); try (PrintWriter output = new PrintWriter(Files.newOutputStream(tempFile.toPath()))) { output.println("client.id=share-consumer-1"); output.flush(); @@ -103,8 +180,8 @@ public class ShareConsumerPerformanceTest { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", - "--messages", "10", - "--consumer.config", tempFile.getAbsolutePath() + "--num-records", "10", + "--command-config", tempFile.getAbsolutePath() }; ShareConsumerPerformance.ShareConsumerPerfOptions config = new ShareConsumerPerformance.ShareConsumerPerfOptions(args); @@ -112,12 +189,51 @@ public class ShareConsumerPerformanceTest { assertEquals("share-consumer-1", config.props().getProperty(ConsumerConfig.CLIENT_ID_CONFIG)); } + @Test + public void testConsumerConfigDeprecated() throws IOException { + Path configPath = tempDir.resolve("test_consumer_config_deprecated_share_consumer_perf.conf"); + Files.deleteIfExists(configPath); + File tempFile = Files.createFile(configPath).toFile(); + try (PrintWriter output = new PrintWriter(Files.newOutputStream(tempFile.toPath()))) { + output.println("client.id=share-consumer-1"); + output.flush(); + } + + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--num-records", "10", + "--consumer.config", tempFile.getAbsolutePath() + }; + + ShareConsumerPerformance.ShareConsumerPerfOptions config = + new ShareConsumerPerformance.ShareConsumerPerfOptions(args); + + assertEquals("share-consumer-1", config.props().getProperty(ConsumerConfig.CLIENT_ID_CONFIG)); + } + + @Test + public void testCommandConfigWithConsumerConfigPresent() { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--num-records", "10", + "--consumer.config", "some-path", + "--command-config", "some-path" + }; + + String err = ToolsTestUtils.captureStandardErr(() -> + new ShareConsumerPerformance.ShareConsumerPerfOptions(args)); + assertTrue(err.contains(String.format("Option \"%s\" can't be used with option \"%s\"", + "[consumer.config]", "[command-config]"))); + } + @Test public void testDefaultClientId() throws IOException { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", - "--messages", "10" + "--num-records", "10" }; ShareConsumerPerformance.ShareConsumerPerfOptions config = new ShareConsumerPerformance.ShareConsumerPerfOptions(args); @@ -130,7 +246,7 @@ public class ShareConsumerPerformanceTest { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", - "--messages", "0", + "--num-records", "0", "--print-metrics" }; From 0a483618b9cc169a0f923478812141630baf2a4c Mon Sep 17 00:00:00 2001 From: Ritika Reddy <98577846+rreddy-22@users.noreply.github.com> Date: Tue, 23 Sep 2025 13:45:42 -0700 Subject: [PATCH 064/100] KAFKA-19690-Add epoch check before verification guard check to prevent unexpected fatal error (#20534) We are seeing cases where a Kafka Streams (KS) thread stalls for ~20 seconds. During this stall, the broker correctly aborts the open transaction (triggered by the 10-second transaction timeout). However, when the KS thread resumes, instead of receiving the expected InvalidProducerEpochException (which we already handle gracefully as part of transaction abort), the client is instead hit with an InvalidTxnStateException. KS currently treats this as a fatal error, causing the application to fail. To fix this, we've added an epoch check before the verification check to send the recoverable InvalidProducerEpochException instead of the fatal InvalidTxnStateException. This helps safeguard both tv1 and tv2 clients Reviewers: Justine Olshan --- .../scala/unit/kafka/log/UnifiedLogTest.scala | 90 +++++++++++++++++++ .../storage/internals/log/UnifiedLog.java | 17 +++- 2 files changed, 103 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index c4cb7a9f7b4..e6fdf09331b 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -5065,6 +5065,96 @@ class UnifiedLogTest { } case class TimestampAndEpoch(timestamp: Long, leaderEpoch: Int) + + @Test + def testStaleProducerEpochReturnsRecoverableErrorForTV1Clients(): Unit = { + // Producer epoch gets incremented (coordinator fail over, completed transaction, etc.) + // and client has stale cached epoch. Fix prevents fatal InvalidTxnStateException. + + val producerStateManagerConfig = new ProducerStateManagerConfig(86400000, true) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5) + val log = createLog(logDir, logConfig, producerStateManagerConfig = producerStateManagerConfig) + + val producerId = 123L + val oldEpoch = 5.toShort + val newEpoch = 6.toShort + + // Step 1: Simulate a scenario where producer epoch was incremented to fence the producer + val previousRecords = MemoryRecords.withTransactionalRecords( + Compression.NONE, producerId, newEpoch, 0, + new SimpleRecord("previous-key".getBytes, "previous-value".getBytes) + ) + val previousGuard = log.maybeStartTransactionVerification(producerId, 0, newEpoch, false) // TV1 = supportsEpochBump = false + log.appendAsLeader(previousRecords, 0, AppendOrigin.CLIENT, RequestLocal.noCaching, previousGuard) + + // Complete the transaction normally (commits do update producer state with current epoch) + val commitMarker = MemoryRecords.withEndTransactionMarker( + producerId, newEpoch, new EndTransactionMarker(ControlRecordType.COMMIT, 0) + ) + log.appendAsLeader(commitMarker, 0, AppendOrigin.COORDINATOR, RequestLocal.noCaching, VerificationGuard.SENTINEL) + + // Step 2: TV1 client tries to write with stale cached epoch (before learning about epoch increment) + val staleEpochRecords = MemoryRecords.withTransactionalRecords( + Compression.NONE, producerId, oldEpoch, 0, + new SimpleRecord("stale-epoch-key".getBytes, "stale-epoch-value".getBytes) + ) + + // Step 3: Verify our fix - should get InvalidProducerEpochException (recoverable), not InvalidTxnStateException (fatal) + val exception = assertThrows(classOf[InvalidProducerEpochException], () => { + val staleGuard = log.maybeStartTransactionVerification(producerId, 0, oldEpoch, false) + log.appendAsLeader(staleEpochRecords, 0, AppendOrigin.CLIENT, RequestLocal.noCaching, staleGuard) + }) + + // Verify the error message indicates epoch mismatch + assertTrue(exception.getMessage.contains("smaller than the last seen epoch")) + assertTrue(exception.getMessage.contains(s"$oldEpoch")) + assertTrue(exception.getMessage.contains(s"$newEpoch")) + } + + @Test + def testStaleProducerEpochReturnsRecoverableErrorForTV2Clients(): Unit = { + // Check producer epoch FIRST - if stale, return recoverable error before verification checks. + + val producerStateManagerConfig = new ProducerStateManagerConfig(86400000, true) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5) + val log = createLog(logDir, logConfig, producerStateManagerConfig = producerStateManagerConfig) + + val producerId = 456L + val originalEpoch = 3.toShort + val bumpedEpoch = 4.toShort + + // Step 1: Start transaction with epoch 3 (before timeout) + val initialRecords = MemoryRecords.withTransactionalRecords( + Compression.NONE, producerId, originalEpoch, 0, + new SimpleRecord("ks-initial-key".getBytes, "ks-initial-value".getBytes) + ) + val initialGuard = log.maybeStartTransactionVerification(producerId, 0, originalEpoch, true) // TV2 = supportsEpochBump = true + log.appendAsLeader(initialRecords, 0, AppendOrigin.CLIENT, RequestLocal.noCaching, initialGuard) + + // Step 2: Coordinator times out and aborts transaction + // TV2 (KIP-890): Coordinator bumps epoch from 3 → 4 and sends abort marker with epoch 4 + val abortMarker = MemoryRecords.withEndTransactionMarker( + producerId, bumpedEpoch, new EndTransactionMarker(ControlRecordType.ABORT, 0) + ) + log.appendAsLeader(abortMarker, 0, AppendOrigin.COORDINATOR, RequestLocal.noCaching, VerificationGuard.SENTINEL) + + // Step 3: TV2 transactional producer tries to append with stale epoch (timeout recovery scenario) + val staleEpochRecords = MemoryRecords.withTransactionalRecords( + Compression.NONE, producerId, originalEpoch, 0, + new SimpleRecord("ks-resume-key".getBytes, "ks-resume-value".getBytes) + ) + + // Step 4: Verify our fix works for TV2 - should get InvalidProducerEpochException (recoverable), not InvalidTxnStateException (fatal) + val exception = assertThrows(classOf[InvalidProducerEpochException], () => { + val staleGuard = log.maybeStartTransactionVerification(producerId, 0, originalEpoch, true) // TV2 = supportsEpochBump = true + log.appendAsLeader(staleEpochRecords, 0, AppendOrigin.CLIENT, RequestLocal.noCaching, staleGuard) + }) + + // Verify the error message indicates epoch mismatch (3 < 4) + assertTrue(exception.getMessage.contains("smaller than the last seen epoch")) + assertTrue(exception.getMessage.contains(s"$originalEpoch")) + assertTrue(exception.getMessage.contains(s"$bumpedEpoch")) + } } object UnifiedLogTest { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java index 61b4b9d0edb..d190c861998 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java @@ -1385,10 +1385,19 @@ public class UnifiedLog implements AutoCloseable { // transaction is completed or aborted. We can guarantee the transaction coordinator knows about the transaction given step 1 and that the transaction is still // ongoing. If the transaction is expected to be ongoing, we will not set a VerificationGuard. If the transaction is aborted, hasOngoingTransaction is false and // requestVerificationGuard is the sentinel, so we will throw an error. A subsequent produce request (retry) should create verification state and return to phase 1. - if (batch.isTransactional() - && !hasOngoingTransaction(batch.producerId(), batch.producerEpoch()) - && batchMissingRequiredVerification(batch, requestVerificationGuard)) { - throw new InvalidTxnStateException("Record was not part of an ongoing transaction"); + if (batch.isTransactional() && !hasOngoingTransaction(batch.producerId(), batch.producerEpoch())) { + // Check epoch first: if producer epoch is stale, throw recoverable InvalidProducerEpochException. + ProducerStateEntry entry = producerStateManager.activeProducers().get(batch.producerId()); + if (entry != null && batch.producerEpoch() < entry.producerEpoch()) { + String message = "Epoch of producer " + batch.producerId() + " is " + batch.producerEpoch() + + ", which is smaller than the last seen epoch " + entry.producerEpoch(); + throw new InvalidProducerEpochException(message); + } + + // Only check verification if epoch is current + if (batchMissingRequiredVerification(batch, requestVerificationGuard)) { + throw new InvalidTxnStateException("Record was not part of an ongoing transaction"); + } } } From 1f7631c8c656cff27d36e6a4fa9118888effbd02 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Wed, 24 Sep 2025 09:03:07 +0200 Subject: [PATCH 065/100] MINOR: Fix StreamsRebalanceListenerInvoker (#20575) StreamsRebalanceListenerInvoker was implemented to match the behavior of ConsumerRebalanceListenerInvoker, however StreamsRebalanceListener has a subtly different interface than ConsumerRebalanceListener - it does not throw exceptions, but returns it as an Optional. In the interest of consistency, this change fixes this mismatch by changing the StreamsRebalanceListener interface to behave more like the ConsumerRebalanceListener - throwing exceptions directly. In another minor fix, the StreamsRebalanceListenerInvoker is changed to simply skip callback execution instead of throwing an IllegalStateException when no streamRebalanceListener is defined. This can happen when the consumer is closed before Consumer.subscribe is called. Reviewers: Lianet Magrans , Matthias J. Sax --- .../internals/StreamsRebalanceListener.java | 13 ++-- .../StreamsRebalanceListenerInvoker.java | 8 +-- .../internals/AsyncKafkaConsumerTest.java | 2 - .../StreamsRebalanceListenerInvokerTest.java | 33 +++------ .../kafka/api/AuthorizerIntegrationTest.scala | 11 +-- .../kafka/api/IntegrationTestHarness.scala | 10 +-- .../DefaultStreamsRebalanceListener.java | 72 ++++++++----------- .../DefaultStreamsRebalanceListenerTest.java | 34 +++++---- 8 files changed, 66 insertions(+), 117 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListener.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListener.java index 55de41d77a5..2c8a7449a72 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListener.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListener.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals; -import java.util.Optional; import java.util.Set; /** @@ -28,22 +27,18 @@ public interface StreamsRebalanceListener { * Called when tasks are revoked from a stream thread. * * @param tasks The tasks to be revoked. - * @return The exception thrown during the callback, if any. */ - Optional onTasksRevoked(final Set tasks); + void onTasksRevoked(final Set tasks); /** * Called when tasks are assigned from a stream thread. * * @param assignment The tasks assigned. - * @return The exception thrown during the callback, if any. */ - Optional onTasksAssigned(final StreamsRebalanceData.Assignment assignment); + void onTasksAssigned(final StreamsRebalanceData.Assignment assignment); /** - * Called when a stream thread loses all assigned tasks. - * - * @return The exception thrown during the callback, if any. + * Called when a stream thread loses all assigned tasks */ - Optional onAllTasksLost(); + void onAllTasksLost(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvoker.java index f4c5aa4addc..fc8c78c13dc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvoker.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvoker.java @@ -51,14 +51,14 @@ public class StreamsRebalanceListenerInvoker { public Exception invokeAllTasksRevoked() { if (listener.isEmpty()) { - throw new IllegalStateException("StreamsRebalanceListener is not defined"); + return null; } return invokeTasksRevoked(streamsRebalanceData.reconciledAssignment().activeTasks()); } public Exception invokeTasksAssigned(final StreamsRebalanceData.Assignment assignment) { if (listener.isEmpty()) { - throw new IllegalStateException("StreamsRebalanceListener is not defined"); + return null; } log.info("Invoking tasks assigned callback for new assignment: {}", assignment); try { @@ -78,7 +78,7 @@ public class StreamsRebalanceListenerInvoker { public Exception invokeTasksRevoked(final Set tasks) { if (listener.isEmpty()) { - throw new IllegalStateException("StreamsRebalanceListener is not defined"); + return null; } log.info("Invoking task revoked callback for revoked active tasks {}", tasks); try { @@ -98,7 +98,7 @@ public class StreamsRebalanceListenerInvoker { public Exception invokeAllTasksLost() { if (listener.isEmpty()) { - throw new IllegalStateException("StreamsRebalanceListener is not defined"); + return null; } log.info("Invoking tasks lost callback for all tasks"); try { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 9aef4cf7f52..8e44b3fcc25 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -2218,7 +2218,6 @@ public class AsyncKafkaConsumerTest { try (final MockedStatic requestManagers = mockStatic(RequestManagers.class)) { consumer = newConsumerWithStreamRebalanceData(requiredConsumerConfigAndGroupId(groupId), streamsRebalanceData); StreamsRebalanceListener mockStreamsListener = mock(StreamsRebalanceListener.class); - when(mockStreamsListener.onTasksRevoked(any())).thenReturn(Optional.empty()); consumer.subscribe(singletonList("topic"), mockStreamsListener); final MemberStateListener groupMetadataUpdateListener = captureGroupMetadataUpdateListener(requestManagers); final int memberEpoch = 42; @@ -2239,7 +2238,6 @@ public class AsyncKafkaConsumerTest { try (final MockedStatic requestManagers = mockStatic(RequestManagers.class)) { consumer = newConsumerWithStreamRebalanceData(requiredConsumerConfigAndGroupId(groupId), streamsRebalanceData); StreamsRebalanceListener mockStreamsListener = mock(StreamsRebalanceListener.class); - when(mockStreamsListener.onAllTasksLost()).thenReturn(Optional.empty()); consumer.subscribe(singletonList("topic"), mockStreamsListener); final MemberStateListener groupMetadataUpdateListener = captureGroupMetadataUpdateListener(requestManagers); final int memberEpoch = 0; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvokerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvokerTest.java index 2f3e5ab0523..749a4594ab8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvokerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvokerTest.java @@ -28,7 +28,6 @@ import org.mockito.junit.jupiter.MockitoExtension; import org.mockito.junit.jupiter.MockitoSettings; import org.mockito.quality.Strictness; -import java.util.Optional; import java.util.Set; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -73,7 +72,6 @@ public class StreamsRebalanceListenerInvokerTest { StreamsRebalanceData.Assignment mockAssignment = createMockAssignment(); when(streamsRebalanceData.reconciledAssignment()).thenReturn(mockAssignment); - when(secondListener.onTasksRevoked(any())).thenReturn(Optional.empty()); // Set first listener invoker.setRebalanceListener(firstListener); @@ -89,21 +87,10 @@ public class StreamsRebalanceListenerInvokerTest { @Test public void testInvokeMethodsWithNoListener() { - IllegalStateException exception1 = assertThrows(IllegalStateException.class, - () -> invoker.invokeAllTasksRevoked()); - assertEquals("StreamsRebalanceListener is not defined", exception1.getMessage()); - - IllegalStateException exception2 = assertThrows(IllegalStateException.class, - () -> invoker.invokeTasksAssigned(createMockAssignment())); - assertEquals("StreamsRebalanceListener is not defined", exception2.getMessage()); - - IllegalStateException exception3 = assertThrows(IllegalStateException.class, - () -> invoker.invokeTasksRevoked(createMockTasks())); - assertEquals("StreamsRebalanceListener is not defined", exception3.getMessage()); - - IllegalStateException exception4 = assertThrows(IllegalStateException.class, - () -> invoker.invokeAllTasksLost()); - assertEquals("StreamsRebalanceListener is not defined", exception4.getMessage()); + assertNull(invoker.invokeAllTasksRevoked()); + assertNull(invoker.invokeTasksAssigned(createMockAssignment())); + assertNull(invoker.invokeTasksRevoked(createMockTasks())); + assertNull(invoker.invokeAllTasksLost()); } @Test @@ -112,8 +99,7 @@ public class StreamsRebalanceListenerInvokerTest { StreamsRebalanceData.Assignment mockAssignment = createMockAssignment(); when(streamsRebalanceData.reconciledAssignment()).thenReturn(mockAssignment); - when(mockListener.onTasksRevoked(any())).thenReturn(Optional.empty()); - + Exception result = invoker.invokeAllTasksRevoked(); assertNull(result); @@ -124,8 +110,7 @@ public class StreamsRebalanceListenerInvokerTest { public void testInvokeTasksAssignedWithListener() { invoker.setRebalanceListener(mockListener); StreamsRebalanceData.Assignment assignment = createMockAssignment(); - when(mockListener.onTasksAssigned(assignment)).thenReturn(Optional.empty()); - + Exception result = invoker.invokeTasksAssigned(assignment); assertNull(result); @@ -177,8 +162,7 @@ public class StreamsRebalanceListenerInvokerTest { public void testInvokeTasksRevokedWithListener() { invoker.setRebalanceListener(mockListener); Set tasks = createMockTasks(); - when(mockListener.onTasksRevoked(tasks)).thenReturn(Optional.empty()); - + Exception result = invoker.invokeTasksRevoked(tasks); assertNull(result); @@ -229,8 +213,7 @@ public class StreamsRebalanceListenerInvokerTest { @Test public void testInvokeAllTasksLostWithListener() { invoker.setRebalanceListener(mockListener); - when(mockListener.onAllTasksLost()).thenReturn(Optional.empty()); - + Exception result = invoker.invokeAllTasksLost(); assertNull(result); diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index f950362354c..bfcc0bb0d4f 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -3912,14 +3912,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { consumer.subscribe( if (topicAsSourceTopic || topicAsRepartitionSourceTopic) util.Set.of(sourceTopic, topic) else util.Set.of(sourceTopic), new StreamsRebalanceListener { - override def onTasksRevoked(tasks: util.Set[StreamsRebalanceData.TaskId]): Optional[Exception] = - Optional.empty() - - override def onTasksAssigned(assignment: StreamsRebalanceData.Assignment): Optional[Exception] = - Optional.empty() - - override def onAllTasksLost(): Optional[Exception] = - Optional.empty() + override def onTasksRevoked(tasks: util.Set[StreamsRebalanceData.TaskId]): Unit = () + override def onTasksAssigned(assignment: StreamsRebalanceData.Assignment): Unit = () + override def onAllTasksLost(): Unit = () } ) consumer.poll(Duration.ofMillis(500L)) diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index fe24e45f16b..303e989e9b4 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -272,13 +272,9 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness { ) consumer.subscribe(util.Set.of(inputTopic), new StreamsRebalanceListener { - override def onTasksRevoked(tasks: util.Set[StreamsRebalanceData.TaskId]): Optional[Exception] = - Optional.empty() - override def onTasksAssigned(assignment: StreamsRebalanceData.Assignment): Optional[Exception] = { - Optional.empty() - } - override def onAllTasksLost(): Optional[Exception] = - Optional.empty() + override def onTasksRevoked(tasks: util.Set[StreamsRebalanceData.TaskId]): Unit = () + override def onTasksAssigned(assignment: StreamsRebalanceData.Assignment): Unit = () + override def onAllTasksLost(): Unit = () }) consumer } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListener.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListener.java index a95fcef5a6c..de74b05ceb5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListener.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListener.java @@ -26,7 +26,6 @@ import org.slf4j.Logger; import java.util.Collection; import java.util.Map; -import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -52,59 +51,44 @@ public class DefaultStreamsRebalanceListener implements StreamsRebalanceListener } @Override - public Optional onTasksRevoked(final Set tasks) { - try { - final Map> activeTasksToRevokeWithPartitions = - pairWithTopicPartitions(tasks.stream()); - final Set partitionsToRevoke = activeTasksToRevokeWithPartitions.values().stream() - .flatMap(Collection::stream) - .collect(Collectors.toSet()); + public void onTasksRevoked(final Set tasks) { + final Map> activeTasksToRevokeWithPartitions = + pairWithTopicPartitions(tasks.stream()); + final Set partitionsToRevoke = activeTasksToRevokeWithPartitions.values().stream() + .flatMap(Collection::stream) + .collect(Collectors.toSet()); - final long start = time.milliseconds(); - try { - log.info("Revoking active tasks {}.", tasks); - taskManager.handleRevocation(partitionsToRevoke); - } finally { - log.info("partition revocation took {} ms.", time.milliseconds() - start); - } - if (streamThread.state() != StreamThread.State.PENDING_SHUTDOWN) { - streamThread.setState(StreamThread.State.PARTITIONS_REVOKED); - } - } catch (final Exception exception) { - return Optional.of(exception); + final long start = time.milliseconds(); + try { + log.info("Revoking active tasks {}.", tasks); + taskManager.handleRevocation(partitionsToRevoke); + } finally { + log.info("partition revocation took {} ms.", time.milliseconds() - start); + } + if (streamThread.state() != StreamThread.State.PENDING_SHUTDOWN) { + streamThread.setState(StreamThread.State.PARTITIONS_REVOKED); } - return Optional.empty(); } @Override - public Optional onTasksAssigned(final StreamsRebalanceData.Assignment assignment) { - try { - final Map> activeTasksWithPartitions = - pairWithTopicPartitions(assignment.activeTasks().stream()); - final Map> standbyTasksWithPartitions = - pairWithTopicPartitions(Stream.concat(assignment.standbyTasks().stream(), assignment.warmupTasks().stream())); + public void onTasksAssigned(final StreamsRebalanceData.Assignment assignment) { + final Map> activeTasksWithPartitions = + pairWithTopicPartitions(assignment.activeTasks().stream()); + final Map> standbyTasksWithPartitions = + pairWithTopicPartitions(Stream.concat(assignment.standbyTasks().stream(), assignment.warmupTasks().stream())); - log.info("Processing new assignment {} from Streams Rebalance Protocol", assignment); + log.info("Processing new assignment {} from Streams Rebalance Protocol", assignment); - taskManager.handleAssignment(activeTasksWithPartitions, standbyTasksWithPartitions); - streamThread.setState(StreamThread.State.PARTITIONS_ASSIGNED); - taskManager.handleRebalanceComplete(); - streamsRebalanceData.setReconciledAssignment(assignment); - } catch (final Exception exception) { - return Optional.of(exception); - } - return Optional.empty(); + taskManager.handleAssignment(activeTasksWithPartitions, standbyTasksWithPartitions); + streamThread.setState(StreamThread.State.PARTITIONS_ASSIGNED); + taskManager.handleRebalanceComplete(); + streamsRebalanceData.setReconciledAssignment(assignment); } @Override - public Optional onAllTasksLost() { - try { - taskManager.handleLostAll(); - streamsRebalanceData.setReconciledAssignment(StreamsRebalanceData.Assignment.EMPTY); - } catch (final Exception exception) { - return Optional.of(exception); - } - return Optional.empty(); + public void onAllTasksLost() { + taskManager.handleLostAll(); + streamsRebalanceData.setReconciledAssignment(StreamsRebalanceData.Assignment.EMPTY); } private Map> pairWithTopicPartitions(final Stream taskIdStream) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListenerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListenerTest.java index 1297df7b1ee..736fa17e4a4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListenerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListenerTest.java @@ -32,8 +32,9 @@ import java.util.Optional; import java.util.Set; import java.util.UUID; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.inOrder; @@ -84,11 +85,9 @@ public class DefaultStreamsRebalanceListenerTest { )); when(streamThread.state()).thenReturn(state); - final Optional result = defaultStreamsRebalanceListener.onTasksRevoked( + assertDoesNotThrow(() -> defaultStreamsRebalanceListener.onTasksRevoked( Set.of(new StreamsRebalanceData.TaskId("1", 0)) - ); - - assertTrue(result.isEmpty()); + )); final InOrder inOrder = inOrder(taskManager, streamThread); inOrder.verify(taskManager).handleRevocation( @@ -109,9 +108,9 @@ public class DefaultStreamsRebalanceListenerTest { createRebalanceListenerWithRebalanceData(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())); - final Optional result = defaultStreamsRebalanceListener.onTasksRevoked(Set.of()); + final Exception actualException = assertThrows(RuntimeException.class, () -> defaultStreamsRebalanceListener.onTasksRevoked(Set.of())); - assertTrue(result.isPresent()); + assertEquals(actualException, exception); verify(taskManager).handleRevocation(any()); verify(streamThread, never()).setState(any()); } @@ -153,9 +152,7 @@ public class DefaultStreamsRebalanceListenerTest { Set.of(new StreamsRebalanceData.TaskId("3", 0)) ); - final Optional result = defaultStreamsRebalanceListener.onTasksAssigned(assignment); - - assertTrue(result.isEmpty()); + assertDoesNotThrow(() -> defaultStreamsRebalanceListener.onTasksAssigned(assignment)); final InOrder inOrder = inOrder(taskManager, streamThread, streamsRebalanceData); inOrder.verify(taskManager).handleAssignment( @@ -179,11 +176,11 @@ public class DefaultStreamsRebalanceListenerTest { when(streamsRebalanceData.subtopologies()).thenReturn(Map.of()); createRebalanceListenerWithRebalanceData(streamsRebalanceData); - final Optional result = defaultStreamsRebalanceListener.onTasksAssigned( + final Exception actualException = assertThrows(RuntimeException.class, () -> defaultStreamsRebalanceListener.onTasksAssigned( new StreamsRebalanceData.Assignment(Set.of(), Set.of(), Set.of()) - ); - assertTrue(result.isPresent()); - assertEquals(exception, result.get()); + )); + + assertEquals(exception, actualException); verify(taskManager).handleAssignment(any(), any()); verify(streamThread, never()).setState(StreamThread.State.PARTITIONS_ASSIGNED); verify(taskManager, never()).handleRebalanceComplete(); @@ -196,7 +193,7 @@ public class DefaultStreamsRebalanceListenerTest { when(streamsRebalanceData.subtopologies()).thenReturn(Map.of()); createRebalanceListenerWithRebalanceData(streamsRebalanceData); - assertTrue(defaultStreamsRebalanceListener.onAllTasksLost().isEmpty()); + assertDoesNotThrow(() -> defaultStreamsRebalanceListener.onAllTasksLost()); final InOrder inOrder = inOrder(taskManager, streamsRebalanceData); inOrder.verify(taskManager).handleLostAll(); @@ -211,9 +208,10 @@ public class DefaultStreamsRebalanceListenerTest { final StreamsRebalanceData streamsRebalanceData = mock(StreamsRebalanceData.class); when(streamsRebalanceData.subtopologies()).thenReturn(Map.of()); createRebalanceListenerWithRebalanceData(streamsRebalanceData); - final Optional result = defaultStreamsRebalanceListener.onAllTasksLost(); - assertTrue(result.isPresent()); - assertEquals(exception, result.get()); + + final Exception actualException = assertThrows(RuntimeException.class, () -> defaultStreamsRebalanceListener.onAllTasksLost()); + + assertEquals(exception, actualException); verify(taskManager).handleLostAll(); verify(streamsRebalanceData, never()).setReconciledAssignment(any()); } From 8036e49a6e8ffe5bb1ab4fad88482853f615efce Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Wed, 24 Sep 2025 18:42:25 +0800 Subject: [PATCH 066/100] KAFKA-17554 Flaky testFutureCompletionOutsidePoll in ConsumerNetworkClientTest (#18298) Jira: https://issues.apache.org/jira/browse/KAFKA-17554 In the previous workflow, the test passes under two conditions: 1. The `t1` thread is waiting for the main thread's `client.wakeup()`. If successful, `t1` will wake up `t2`, allowing `t2` to complete the future. 2. If `t1` fails to receive the `client.wakeup()` from the main thread, `t2` will be woken up by the main thread. In the previous implementation, we used a `CountDownLatch` to control the execution of three threads, but it often led to race conditions. Currently, we have modified it to use two threads to test this scenario. I run `I=0; while ./gradlew :clients:test --tests ConsumerNetworkClientTest.testFutureCompletionOutsidePoll --rerun --fail-fast; do (( I=$I+1 )); echo "Completed run: $I"; sleep 1; done` and pass 3000+ times. ![image](https://github.com/user-attachments/assets/3b8d804e-fbe0-4030-8686-4960fc717d07) Reviewers: Chia-Ping Tsai --- .../internals/ConsumerNetworkClientTest.java | 36 +++++++------------ 1 file changed, 12 insertions(+), 24 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java index b5ab39e62c7..1f5551e7df1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java @@ -41,7 +41,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.test.TestUtils; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import java.time.Duration; @@ -266,45 +265,34 @@ public class ConsumerNetworkClientTest { assertEquals(metadataException, exc); } - @Disabled("KAFKA-17554") @Test public void testFutureCompletionOutsidePoll() throws Exception { // Tests the scenario in which the request that is being awaited in one thread // is received and completed in another thread. - - final CountDownLatch t1TheardCountDownLatch = new CountDownLatch(1); - final CountDownLatch t2ThreadCountDownLatch = new CountDownLatch(2); - final RequestFuture future = consumerClient.send(node, heartbeat()); consumerClient.pollNoWakeup(); // dequeue and send the request + CountDownLatch bothThreadsReady = new CountDownLatch(2); + client.enableBlockingUntilWakeup(2); - Thread t1 = new Thread(() -> { - t1TheardCountDownLatch.countDown(); + + Thread t1 = new Thread(() -> { + bothThreadsReady.countDown(); consumerClient.pollNoWakeup(); - t2ThreadCountDownLatch.countDown(); }); - - t1.start(); Thread t2 = new Thread(() -> { - try { - t2ThreadCountDownLatch.await(); - consumerClient.poll(future); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } + bothThreadsReady.countDown(); + consumerClient.poll(future); }); + + t1.start(); t2.start(); - - // Simulate a network response and return from the poll in t1 + + // Wait until both threads are blocked in poll + bothThreadsReady.await(); client.respond(heartbeatResponse(Errors.NONE)); - // Wait for t1 to block in poll - t1TheardCountDownLatch.await(); - client.wakeup(); - // while t1 is blocked in poll, t2 should be able to complete the future - t2ThreadCountDownLatch.countDown(); // Both threads should complete since t1 should wakeup t2 t1.join(); From 486b991f221dd75637402e2ff0a10d0a482e6547 Mon Sep 17 00:00:00 2001 From: Maros Orsak Date: Wed, 24 Sep 2025 14:19:08 +0200 Subject: [PATCH 067/100] KAFKA-18711 Move DelegationTokenPublisher to metadata module (#20475) Basically, one of the refactor tasks. In this PR, I have moved `DelegationTokenPublisher` to the metadata module. Similar to the `ScramPublisher` migration (commit feee50f476), I have moved `DelegationTokenManager` to the server-common module, as it would otherwise create a circular dependency. Moreover, I have made multiple changes throughout the codebase to reference `DelegationTokenManager` from server-common instead of the server module. Reviewers: Mickael Maison , Chia-Ping Tsai --- checkstyle/import-control-server-common.xml | 4 + .../server/builders/KafkaApisBuilder.java | 2 +- .../scala/kafka/server/BrokerServer.scala | 8 +- .../scala/kafka/server/ControllerApis.scala | 3 +- .../scala/kafka/server/ControllerServer.scala | 13 +-- .../main/scala/kafka/server/KafkaApis.scala | 3 +- .../metadata/BrokerMetadataPublisher.scala | 4 +- .../metadata/DelegationTokenPublisher.scala | 83 ------------------- .../BrokerMetadataPublisherTest.scala | 2 +- .../publisher/DelegationTokenPublisher.java | 73 ++++++++++++++++ .../security}/DelegationTokenManager.java | 2 +- .../config/DelegationTokenManagerConfigs.java | 0 12 files changed, 97 insertions(+), 100 deletions(-) delete mode 100644 core/src/main/scala/kafka/server/metadata/DelegationTokenPublisher.scala create mode 100644 metadata/src/main/java/org/apache/kafka/metadata/publisher/DelegationTokenPublisher.java rename {server/src/main/java/org/apache/kafka/server => server-common/src/main/java/org/apache/kafka/security}/DelegationTokenManager.java (99%) rename {server => server-common}/src/main/java/org/apache/kafka/server/config/DelegationTokenManagerConfigs.java (100%) diff --git a/checkstyle/import-control-server-common.xml b/checkstyle/import-control-server-common.xml index 21b13ed91d2..95a014b87e4 100644 --- a/checkstyle/import-control-server-common.xml +++ b/checkstyle/import-control-server-common.xml @@ -33,6 +33,7 @@ + @@ -49,6 +50,9 @@ + + + diff --git a/core/src/main/java/kafka/server/builders/KafkaApisBuilder.java b/core/src/main/java/kafka/server/builders/KafkaApisBuilder.java index ecbb6c8b154..e03ab35e90e 100644 --- a/core/src/main/java/kafka/server/builders/KafkaApisBuilder.java +++ b/core/src/main/java/kafka/server/builders/KafkaApisBuilder.java @@ -36,9 +36,9 @@ import org.apache.kafka.coordinator.group.GroupCoordinator; import org.apache.kafka.coordinator.share.ShareCoordinator; import org.apache.kafka.metadata.ConfigRepository; import org.apache.kafka.metadata.MetadataCache; +import org.apache.kafka.security.DelegationTokenManager; import org.apache.kafka.server.ApiVersionManager; import org.apache.kafka.server.ClientMetricsManager; -import org.apache.kafka.server.DelegationTokenManager; import org.apache.kafka.server.authorizer.Authorizer; import org.apache.kafka.storage.log.metrics.BrokerTopicStats; diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 47085169979..689c62b8687 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -42,8 +42,8 @@ import org.apache.kafka.coordinator.share.{ShareCoordinator, ShareCoordinatorRec import org.apache.kafka.coordinator.transaction.ProducerIdManager import org.apache.kafka.image.publisher.{BrokerRegistrationTracker, MetadataPublisher} import org.apache.kafka.metadata.{BrokerState, ListenerInfo} -import org.apache.kafka.metadata.publisher.{AclPublisher, ScramPublisher} -import org.apache.kafka.security.CredentialProvider +import org.apache.kafka.metadata.publisher.{AclPublisher, DelegationTokenPublisher, ScramPublisher} +import org.apache.kafka.security.{CredentialProvider, DelegationTokenManager} import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.common.{ApiMessageAndVersion, DirectoryEventHandler, NodeToControllerChannelManager, TopicIdPartition} import org.apache.kafka.server.config.{ConfigType, DelegationTokenManagerConfigs} @@ -54,7 +54,7 @@ import org.apache.kafka.server.share.persister.{DefaultStatePersister, NoOpState import org.apache.kafka.server.share.session.ShareSessionCache import org.apache.kafka.server.util.timer.{SystemTimer, SystemTimerReaper} import org.apache.kafka.server.util.{Deadline, FutureUtils, KafkaScheduler} -import org.apache.kafka.server.{AssignmentsManager, BrokerFeatures, ClientMetricsManager, DefaultApiVersionManager, DelayedActionQueue, DelegationTokenManager, ProcessRole} +import org.apache.kafka.server.{AssignmentsManager, BrokerFeatures, ClientMetricsManager, DefaultApiVersionManager, DelayedActionQueue, ProcessRole} import org.apache.kafka.server.transaction.AddPartitionsToTxnManager import org.apache.kafka.storage.internals.log.LogDirFailureChannel import org.apache.kafka.storage.log.metrics.BrokerTopicStats @@ -502,7 +502,7 @@ class BrokerServer( "broker", credentialProvider), new DelegationTokenPublisher( - config, + config.nodeId, sharedServer.metadataPublishingFaultHandler, "broker", tokenManager), diff --git a/core/src/main/scala/kafka/server/ControllerApis.scala b/core/src/main/scala/kafka/server/ControllerApis.scala index ac9a2d9eff1..f10b769d9c1 100644 --- a/core/src/main/scala/kafka/server/ControllerApis.scala +++ b/core/src/main/scala/kafka/server/ControllerApis.scala @@ -55,7 +55,8 @@ import org.apache.kafka.metadata.{BrokerHeartbeatReply, BrokerRegistrationReply} import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.raft.RaftManager -import org.apache.kafka.server.{ApiVersionManager, DelegationTokenManager, ProcessRole} +import org.apache.kafka.security.DelegationTokenManager +import org.apache.kafka.server.{ApiVersionManager, ProcessRole} import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.common.{ApiMessageAndVersion, RequestLocal} import org.apache.kafka.server.quota.ControllerMutationQuota diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index badcb9b2d8a..e41705ed3ba 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -22,7 +22,7 @@ import kafka.raft.KafkaRaftManager import kafka.server.QuotaFactory.QuotaManagers import scala.collection.immutable -import kafka.server.metadata.{ClientQuotaMetadataManager, DelegationTokenPublisher, DynamicClientQuotaPublisher, DynamicConfigPublisher, DynamicTopicClusterQuotaPublisher, KRaftMetadataCache, KRaftMetadataCachePublisher} +import kafka.server.metadata.{ClientQuotaMetadataManager, DynamicClientQuotaPublisher, DynamicConfigPublisher, DynamicTopicClusterQuotaPublisher, KRaftMetadataCache, KRaftMetadataCachePublisher} import kafka.utils.{CoreUtils, Logging} import org.apache.kafka.common.internals.Plugin import org.apache.kafka.common.message.ApiMessageType.ListenerType @@ -38,14 +38,15 @@ import org.apache.kafka.image.publisher.{ControllerRegistrationsPublisher, Metad import org.apache.kafka.metadata.{KafkaConfigSchema, ListenerInfo} import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer import org.apache.kafka.metadata.bootstrap.BootstrapMetadata -import org.apache.kafka.metadata.publisher.{AclPublisher, FeaturesPublisher, ScramPublisher} +import org.apache.kafka.metadata.publisher.{AclPublisher, DelegationTokenPublisher, FeaturesPublisher, ScramPublisher} import org.apache.kafka.raft.QuorumConfig -import org.apache.kafka.security.CredentialProvider -import org.apache.kafka.server.{DelegationTokenManager, ProcessRole, SimpleApiVersionManager} +import org.apache.kafka.security.{CredentialProvider, DelegationTokenManager} +import org.apache.kafka.server.{ProcessRole, SimpleApiVersionManager} import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.config.ServerLogConfigs.{ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG} import org.apache.kafka.server.common.{ApiMessageAndVersion, KRaftVersion, NodeToControllerChannelManager} -import org.apache.kafka.server.config.{ConfigType, DelegationTokenManagerConfigs} +import org.apache.kafka.server.config.ConfigType +import org.apache.kafka.server.config.DelegationTokenManagerConfigs import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics, LinuxIoMetricsCollector} import org.apache.kafka.server.network.{EndpointReadyFutures, KafkaAuthorizerServerInfo} import org.apache.kafka.server.policy.{AlterConfigPolicy, CreateTopicPolicy} @@ -360,7 +361,7 @@ class ControllerServer( // We need a tokenManager for the Publisher // The tokenCache in the tokenManager is the same used in DelegationTokenControlManager metadataPublishers.add(new DelegationTokenPublisher( - config, + config.nodeId, sharedServer.metadataPublishingFaultHandler, "controller", new DelegationTokenManager(delegationTokenManagerConfigs, tokenCache) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index d3935c8e507..4cbef3fa648 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -60,7 +60,8 @@ import org.apache.kafka.common.{Node, TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.coordinator.group.{Group, GroupConfig, GroupConfigManager, GroupCoordinator} import org.apache.kafka.coordinator.share.ShareCoordinator import org.apache.kafka.metadata.{ConfigRepository, MetadataCache} -import org.apache.kafka.server.{ApiVersionManager, ClientMetricsManager, DelegationTokenManager, ProcessRole} +import org.apache.kafka.security.DelegationTokenManager +import org.apache.kafka.server.{ApiVersionManager, ClientMetricsManager, ProcessRole} import org.apache.kafka.server.authorizer._ import org.apache.kafka.server.common.{GroupVersion, RequestLocal, ShareVersion, StreamsVersion, TransactionVersion} import org.apache.kafka.server.config.DelegationTokenManagerConfigs diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala index 30ea835b5be..8df8a275580 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala @@ -33,7 +33,7 @@ import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.image.loader.LoaderManifest import org.apache.kafka.image.publisher.MetadataPublisher import org.apache.kafka.image.{MetadataDelta, MetadataImage, TopicDelta} -import org.apache.kafka.metadata.publisher.{AclPublisher, ScramPublisher} +import org.apache.kafka.metadata.publisher.{AclPublisher, DelegationTokenPublisher, ScramPublisher} import org.apache.kafka.server.common.MetadataVersion.MINIMUM_VERSION import org.apache.kafka.server.common.{FinalizedFeatures, RequestLocal, ShareVersion} import org.apache.kafka.server.fault.FaultHandler @@ -227,7 +227,7 @@ class BrokerMetadataPublisher( scramPublisher.onMetadataUpdate(delta, newImage, manifest) // Apply DelegationToken delta. - delegationTokenPublisher.onMetadataUpdate(delta, newImage) + delegationTokenPublisher.onMetadataUpdate(delta, newImage, manifest) // Apply ACL delta. aclPublisher.onMetadataUpdate(delta, newImage, manifest) diff --git a/core/src/main/scala/kafka/server/metadata/DelegationTokenPublisher.scala b/core/src/main/scala/kafka/server/metadata/DelegationTokenPublisher.scala deleted file mode 100644 index 0e12c34b3c5..00000000000 --- a/core/src/main/scala/kafka/server/metadata/DelegationTokenPublisher.scala +++ /dev/null @@ -1,83 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.server.metadata - -import kafka.server.KafkaConfig -import kafka.utils.Logging -import org.apache.kafka.image.loader.LoaderManifest -import org.apache.kafka.image.{MetadataDelta, MetadataImage} -import org.apache.kafka.server.DelegationTokenManager -import org.apache.kafka.server.fault.FaultHandler - - -class DelegationTokenPublisher( - conf: KafkaConfig, - faultHandler: FaultHandler, - nodeType: String, - tokenManager: DelegationTokenManager, -) extends Logging with org.apache.kafka.image.publisher.MetadataPublisher { - logIdent = s"[${name()}] " - - var _firstPublish = true - - override def name(): String = s"DelegationTokenPublisher $nodeType id=${conf.nodeId}" - - override def onMetadataUpdate( - delta: MetadataDelta, - newImage: MetadataImage, - manifest: LoaderManifest - ): Unit = { - onMetadataUpdate(delta, newImage) - } - - def onMetadataUpdate( - delta: MetadataDelta, - newImage: MetadataImage, - ): Unit = { - val deltaName = if (_firstPublish) { - s"initial MetadataDelta up to ${newImage.highestOffsetAndEpoch().offset}" - } else { - s"update MetadataDelta up to ${newImage.highestOffsetAndEpoch().offset}" - } - try { - if (_firstPublish) { - // Initialize the tokenCache with the Image - Option(newImage.delegationTokens()).foreach { delegationTokenImage => - delegationTokenImage.tokens().forEach { (_, delegationTokenData) => - tokenManager.updateToken(tokenManager.getDelegationToken(delegationTokenData.tokenInformation())) - } - } - _firstPublish = false - } - // Apply changes to DelegationTokens. - Option(delta.delegationTokenDelta()).foreach { delegationTokenDelta => - delegationTokenDelta.changes().forEach { - case (tokenId, delegationTokenData) => - if (delegationTokenData.isPresent) { - tokenManager.updateToken(tokenManager.getDelegationToken(delegationTokenData.get().tokenInformation())) - } else { - tokenManager.removeToken(tokenId) - } - } - } - } catch { - case t: Throwable => faultHandler.handleFault("Uncaught exception while " + - s"publishing DelegationToken changes from $deltaName", t) - } - } -} 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 828ca0d7ad4..32727a4c3cc 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala @@ -40,7 +40,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinator import org.apache.kafka.coordinator.share.ShareCoordinator import org.apache.kafka.image.{AclsImage, ClientQuotasImage, ClusterImageTest, ConfigurationsImage, DelegationTokenImage, FeaturesImage, MetadataDelta, MetadataImage, MetadataImageTest, MetadataProvenance, ProducerIdsImage, ScramImage, TopicsImage} import org.apache.kafka.image.loader.LogDeltaManifest -import org.apache.kafka.metadata.publisher.{AclPublisher, ScramPublisher} +import org.apache.kafka.metadata.publisher.{AclPublisher, DelegationTokenPublisher, ScramPublisher} import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.raft.LeaderAndEpoch import org.apache.kafka.server.common.{KRaftVersion, MetadataVersion, ShareVersion} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/publisher/DelegationTokenPublisher.java b/metadata/src/main/java/org/apache/kafka/metadata/publisher/DelegationTokenPublisher.java new file mode 100644 index 00000000000..347b0d7f531 --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/metadata/publisher/DelegationTokenPublisher.java @@ -0,0 +1,73 @@ +/* + * 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.metadata.publisher; + +import org.apache.kafka.image.DelegationTokenImage; +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.image.loader.LoaderManifest; +import org.apache.kafka.image.publisher.MetadataPublisher; +import org.apache.kafka.security.DelegationTokenManager; +import org.apache.kafka.server.fault.FaultHandler; + +public class DelegationTokenPublisher implements MetadataPublisher { + private final int nodeId; + private final FaultHandler faultHandler; + private final String nodeType; + private final DelegationTokenManager tokenManager; + private boolean firstPublish = true; + + public DelegationTokenPublisher(int nodeId, FaultHandler faultHandler, String nodeType, DelegationTokenManager tokenManager) { + this.nodeId = nodeId; + this.faultHandler = faultHandler; + this.nodeType = nodeType; + this.tokenManager = tokenManager; + } + + @Override + public final String name() { + return "DelegationTokenPublisher " + nodeType + " id=" + nodeId; + } + + @Override + public void onMetadataUpdate(MetadataDelta delta, MetadataImage newImage, LoaderManifest manifest) { + var first = firstPublish; + try { + if (firstPublish) { + // Initialize the tokenCache with the Image + DelegationTokenImage delegationTokenImage = newImage.delegationTokens(); + for (var token : delegationTokenImage.tokens().entrySet()) { + tokenManager.updateToken(tokenManager.getDelegationToken(token.getValue().tokenInformation())); + } + firstPublish = false; + } + // Apply changes to DelegationTokens. + for (var token : delta.getOrCreateDelegationTokenDelta().changes().entrySet()) { + var tokenId = token.getKey(); + var delegationTokenData = token.getValue(); + if (delegationTokenData.isPresent()) + tokenManager.updateToken(tokenManager.getDelegationToken(delegationTokenData.get().tokenInformation())); + else + tokenManager.removeToken(tokenId); + } + } catch (Throwable t) { + var msg = String.format("Uncaught exception while publishing DelegationToken changes from %s MetadataDelta up to %s", + first ? "initial" : "update", newImage.highestOffsetAndEpoch().offset()); + faultHandler.handleFault(msg, t); + } + } +} \ No newline at end of file diff --git a/server/src/main/java/org/apache/kafka/server/DelegationTokenManager.java b/server-common/src/main/java/org/apache/kafka/security/DelegationTokenManager.java similarity index 99% rename from server/src/main/java/org/apache/kafka/server/DelegationTokenManager.java rename to server-common/src/main/java/org/apache/kafka/security/DelegationTokenManager.java index 54832fbd502..ef82a0702c8 100644 --- a/server/src/main/java/org/apache/kafka/server/DelegationTokenManager.java +++ b/server-common/src/main/java/org/apache/kafka/security/DelegationTokenManager.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.server; +package org.apache.kafka.security; import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.apache.kafka.common.security.scram.ScramCredential; diff --git a/server/src/main/java/org/apache/kafka/server/config/DelegationTokenManagerConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/DelegationTokenManagerConfigs.java similarity index 100% rename from server/src/main/java/org/apache/kafka/server/config/DelegationTokenManagerConfigs.java rename to server-common/src/main/java/org/apache/kafka/server/config/DelegationTokenManagerConfigs.java From 1e4b8a1a6b9ca8afd9635f90ad3116f68ee3da94 Mon Sep 17 00:00:00 2001 From: Abhi Tiwari <113602461+abhi-ksolves@users.noreply.github.com> Date: Wed, 24 Sep 2025 18:22:27 +0530 Subject: [PATCH 068/100] KAFKA-6333: java.awt.headless should not be on commandline (#20044) Reviewers: Mickael Maison --- bin/kafka-run-class.sh | 2 +- bin/windows/kafka-run-class.bat | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 90b6e4e4c28..0a5ecfae04e 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -282,7 +282,7 @@ fi # JVM performance options # MaxInlineLevel=15 is the default since JDK 14 and can be removed once older JDKs are no longer supported if [ -z "$KAFKA_JVM_PERFORMANCE_OPTS" ]; then - KAFKA_JVM_PERFORMANCE_OPTS="-server -XX:+UseG1GC -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:+ExplicitGCInvokesConcurrent -XX:MaxInlineLevel=15 -Djava.awt.headless=true" + KAFKA_JVM_PERFORMANCE_OPTS="-server -XX:+UseG1GC -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:+ExplicitGCInvokesConcurrent -XX:MaxInlineLevel=15" fi while [ $# -gt 0 ]; do diff --git a/bin/windows/kafka-run-class.bat b/bin/windows/kafka-run-class.bat index 9994d501700..a73ae2b26f2 100755 --- a/bin/windows/kafka-run-class.bat +++ b/bin/windows/kafka-run-class.bat @@ -177,7 +177,7 @@ IF ["%KAFKA_HEAP_OPTS%"] EQU [""] ( rem JVM performance options IF ["%KAFKA_JVM_PERFORMANCE_OPTS%"] EQU [""] ( - set KAFKA_JVM_PERFORMANCE_OPTS=-server -XX:+UseG1GC -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:+ExplicitGCInvokesConcurrent -Djava.awt.headless=true + set KAFKA_JVM_PERFORMANCE_OPTS=-server -XX:+UseG1GC -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:+ExplicitGCInvokesConcurrent ) IF not defined CLASSPATH ( From 55020f909d2707fd2ff22e436a56daf094c6c9ed Mon Sep 17 00:00:00 2001 From: majialong Date: Wed, 24 Sep 2025 20:56:13 +0800 Subject: [PATCH 069/100] MINOR: Improve the documentation content for DistributedConfig (#20576) 1. Fix doc of `inter.worker.signature.algorithm` config in `DistributedConfig`. 2. Improve the style of the `inter.worker.verification.algorithms` and `worker.unsync.backoff.ms` config. 3. `INTER_WORKER_KEY_TTL_MS_MS_DOC` -> `INTER_WORKER_KEY_TTL_MS_DOC`. Reviewers: Mickael Maison --- .../runtime/distributed/DistributedConfig.java | 16 ++++++++-------- .../distributed/DistributedHerderTest.java | 4 ++-- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java index 6b209da331a..ccf33926bf9 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java @@ -119,7 +119,7 @@ public final class DistributedConfig extends WorkerConfig { */ public static final String WORKER_UNSYNC_BACKOFF_MS_CONFIG = "worker.unsync.backoff.ms"; private static final String WORKER_UNSYNC_BACKOFF_MS_DOC = "When the worker is out of sync with other workers and " + - " fails to catch up within worker.sync.timeout.ms, leave the Connect cluster for this long before rejoining."; + " fails to catch up within the worker.sync.timeout.ms, leave the Connect cluster for this long before rejoining."; public static final int WORKER_UNSYNC_BACKOFF_MS_DEFAULT = 5 * 60 * 1000; public static final String CONFIG_STORAGE_PREFIX = "config.storage."; @@ -206,20 +206,20 @@ public final class DistributedConfig extends WorkerConfig { public static final Long INTER_WORKER_KEY_SIZE_DEFAULT = null; public static final String INTER_WORKER_KEY_TTL_MS_CONFIG = "inter.worker.key.ttl.ms"; - public static final String INTER_WORKER_KEY_TTL_MS_MS_DOC = "The TTL of generated session keys used for " + public static final String INTER_WORKER_KEY_TTL_MS_DOC = "The TTL of generated session keys used for " + "internal request validation (in milliseconds)"; - public static final int INTER_WORKER_KEY_TTL_MS_MS_DEFAULT = Math.toIntExact(TimeUnit.HOURS.toMillis(1)); + public static final int INTER_WORKER_KEY_TTL_MS_DEFAULT = Math.toIntExact(TimeUnit.HOURS.toMillis(1)); public static final String INTER_WORKER_SIGNATURE_ALGORITHM_CONFIG = "inter.worker.signature.algorithm"; public static final String INTER_WORKER_SIGNATURE_ALGORITHM_DEFAULT = "HmacSHA256"; - public static final String INTER_WORKER_SIGNATURE_ALGORITHM_DOC = "The algorithm used to sign internal requests" - + "The algorithm '" + INTER_WORKER_SIGNATURE_ALGORITHM_CONFIG + "' will be used as a default on JVMs that support it; " + public static final String INTER_WORKER_SIGNATURE_ALGORITHM_DOC = "The algorithm used to sign internal requests. " + + "The algorithm '" + INTER_WORKER_SIGNATURE_ALGORITHM_DEFAULT + "' will be used as a default on JVMs that support it; " + "on other JVMs, no default is used and a value for this property must be manually specified in the worker config."; public static final String INTER_WORKER_VERIFICATION_ALGORITHMS_CONFIG = "inter.worker.verification.algorithms"; public static final List INTER_WORKER_VERIFICATION_ALGORITHMS_DEFAULT = List.of(INTER_WORKER_SIGNATURE_ALGORITHM_DEFAULT); public static final String INTER_WORKER_VERIFICATION_ALGORITHMS_DOC = "A list of permitted algorithms for verifying internal requests, " - + "which must include the algorithm used for the " + INTER_WORKER_SIGNATURE_ALGORITHM_CONFIG + " property. " + + "which must include the algorithm used for the " + INTER_WORKER_SIGNATURE_ALGORITHM_CONFIG + " property. " + "The algorithm(s) '" + INTER_WORKER_VERIFICATION_ALGORITHMS_DEFAULT + "' will be used as a default on JVMs that provide them; " + "on other JVMs, no default is used and a value for this property must be manually specified in the worker config."; private final Crypto crypto; @@ -490,10 +490,10 @@ public final class DistributedConfig extends WorkerConfig { SCHEDULED_REBALANCE_MAX_DELAY_MS_DOC) .define(INTER_WORKER_KEY_TTL_MS_CONFIG, ConfigDef.Type.INT, - INTER_WORKER_KEY_TTL_MS_MS_DEFAULT, + INTER_WORKER_KEY_TTL_MS_DEFAULT, between(0, Integer.MAX_VALUE), ConfigDef.Importance.LOW, - INTER_WORKER_KEY_TTL_MS_MS_DOC) + INTER_WORKER_KEY_TTL_MS_DOC) .define(INTER_WORKER_KEY_GENERATION_ALGORITHM_CONFIG, ConfigDef.Type.STRING, defaultKeyGenerationAlgorithm(crypto), diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java index fcee7cb4db4..820de522f12 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java @@ -2563,7 +2563,7 @@ public class DistributedHerderTest { @Test public void testKeyRotationWhenWorkerBecomesLeader() { - long rotationTtlDelay = DistributedConfig.INTER_WORKER_KEY_TTL_MS_MS_DEFAULT; + long rotationTtlDelay = DistributedConfig.INTER_WORKER_KEY_TTL_MS_DEFAULT; when(member.memberId()).thenReturn("member"); when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V2); @@ -2616,7 +2616,7 @@ public class DistributedHerderTest { @Test public void testKeyRotationDisabledWhenWorkerBecomesFollower() { - long rotationTtlDelay = DistributedConfig.INTER_WORKER_KEY_TTL_MS_MS_DEFAULT; + long rotationTtlDelay = DistributedConfig.INTER_WORKER_KEY_TTL_MS_DEFAULT; when(member.memberId()).thenReturn("member"); when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V2); From cbea4f69bd82cbe6dcdcf643198e0ec265ae7d74 Mon Sep 17 00:00:00 2001 From: Dongnuo Lyu <139248811+dongnuo123@users.noreply.github.com> Date: Wed, 24 Sep 2025 09:40:45 -0400 Subject: [PATCH 070/100] KAFKA-19546: Rebalance should be triggered by subscription change during group protocol downgrade (#20417) During online downgrade, when a static member using the consumer protocol which is also the last member using the consumer protocol is replaced by another static member using the classic protocol with the same instance id, the latter will take the assignment of the former and an online downgrade will be triggered. In the current implementation, if the replacing static member has a different subscription, no rebalance will be triggered when the downgrade happens. The patch checks whether the static member has changed subscription and triggers a rebalance when it does. Reviewers: Sean Quah , David Jacot --- .../group/GroupMetadataManager.java | 114 +++++++++------- .../group/GroupMetadataManagerTest.java | 129 +++++++++++------- 2 files changed, 149 insertions(+), 94 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 4f156d75daa..cea68e09ffb 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 @@ -1257,16 +1257,19 @@ public class GroupMetadataManager { /** * Creates a ClassicGroup corresponding to the given ConsumerGroup. * - * @param consumerGroup The converted ConsumerGroup. - * @param leavingMembers The leaving member(s) that triggered the downgrade validation. - * @param joiningMember The newly joined member if the downgrade is triggered by static member replacement. - * When not null, must have an instanceId that matches an existing member. - * @param records The record list to which the conversion records are added. + * @param consumerGroup The converted ConsumerGroup. + * @param leavingMembers The leaving member(s) that triggered the downgrade validation. + * @param joiningMember The newly joined member if the downgrade is triggered by static member replacement. + * When not null, must have an instanceId that matches the replaced member. + * @param hasSubscriptionChanged The boolean indicating whether the joining member has a different subscription + * from the replaced member. Only used when joiningMember is set. + * @param records The record list to which the conversion records are added. */ private void convertToClassicGroup( ConsumerGroup consumerGroup, Set leavingMembers, ConsumerGroupMember joiningMember, + boolean hasSubscriptionChanged, List records ) { if (joiningMember == null) { @@ -1307,9 +1310,12 @@ public class GroupMetadataManager { classicGroup.allMembers().forEach(member -> rescheduleClassicGroupMemberHeartbeat(classicGroup, member)); - // If the downgrade is triggered by a member leaving the group, a rebalance should be triggered. + // If the downgrade is triggered by a member leaving the group or a static + // member replacement with a different subscription, a rebalance should be triggered. if (joiningMember == null) { - prepareRebalance(classicGroup, String.format("Downgrade group %s from consumer to classic.", classicGroup.groupId())); + prepareRebalance(classicGroup, String.format("Downgrade group %s from consumer to classic for member leaving.", classicGroup.groupId())); + } else if (hasSubscriptionChanged) { + prepareRebalance(classicGroup, String.format("Downgrade group %s from consumer to classic for static member replacement with different subscription.", classicGroup.groupId())); } log.info("[GroupId {}] Converted the consumer group to a classic group.", consumerGroup.groupId()); @@ -2401,6 +2407,10 @@ public class GroupMetadataManager { ); } + ConsumerGroupMember existingStaticMemberOrNull = group.staticMember(request.groupInstanceId()); + boolean downgrade = existingStaticMemberOrNull != null && + validateOnlineDowngradeWithReplacedMember(group, existingStaticMemberOrNull); + int groupEpoch = group.groupEpoch(); SubscriptionType subscriptionType = group.subscriptionType(); final ConsumerProtocolSubscription subscription = deserializeSubscription(protocols); @@ -2447,49 +2457,61 @@ public class GroupMetadataManager { subscriptionType = result.subscriptionType; } - // 2. Update the target assignment if the group epoch is larger than the target assignment epoch. The delta between - // the existing and the new target assignment is persisted to the partition. - final int targetAssignmentEpoch; - final Assignment targetAssignment; - - if (groupEpoch > group.assignmentEpoch()) { - targetAssignment = updateTargetAssignment( - group, - groupEpoch, - member, - updatedMember, - subscriptionType, - records - ); - targetAssignmentEpoch = groupEpoch; - } else { - targetAssignmentEpoch = group.assignmentEpoch(); - targetAssignment = group.targetAssignment(updatedMember.memberId(), updatedMember.instanceId()); - - } - - // 3. Reconcile the member's assignment with the target assignment if the member is not - // fully reconciled yet. - updatedMember = maybeReconcile( - groupId, - updatedMember, - group::currentPartitionEpoch, - targetAssignmentEpoch, - targetAssignment, - toTopicPartitions(subscription.ownedPartitions(), metadataImage), - records - ); - - // 4. Maybe downgrade the consumer group if the last static member using the - // consumer protocol is replaced by the joining static member. - ConsumerGroupMember existingStaticMemberOrNull = group.staticMember(request.groupInstanceId()); - boolean downgrade = existingStaticMemberOrNull != null && - validateOnlineDowngradeWithReplacedMember(group, existingStaticMemberOrNull); if (downgrade) { + // 2. If the static member subscription hasn't changed, reconcile the member's assignment with the existing + // assignment if the member is not fully reconciled yet. If the static member subscription has changed, a + // rebalance will be triggered during downgrade anyway so we can skip the reconciliation. + if (!bumpGroupEpoch) { + updatedMember = maybeReconcile( + groupId, + updatedMember, + group::currentPartitionEpoch, + group.assignmentEpoch(), + group.targetAssignment(updatedMember.memberId(), updatedMember.instanceId()), + toTopicPartitions(subscription.ownedPartitions(), metadataImage), + records + ); + } + + // 3. Downgrade the consumer group. convertToClassicGroup( group, Set.of(), updatedMember, + bumpGroupEpoch, + records + ); + } else { + // If no downgrade is triggered. + + // 2. Update the target assignment if the group epoch is larger than the target assignment epoch. + // The delta between the existing and the new target assignment is persisted to the partition. + final int targetAssignmentEpoch; + final Assignment targetAssignment; + + if (groupEpoch > group.assignmentEpoch()) { + targetAssignment = updateTargetAssignment( + group, + groupEpoch, + member, + updatedMember, + subscriptionType, + records + ); + targetAssignmentEpoch = groupEpoch; + } else { + targetAssignmentEpoch = group.assignmentEpoch(); + targetAssignment = group.targetAssignment(updatedMember.memberId(), updatedMember.instanceId()); + } + + // 3. Reconcile the member's assignment with the target assignment if the member is not fully reconciled yet. + updatedMember = maybeReconcile( + groupId, + updatedMember, + group::currentPartitionEpoch, + targetAssignmentEpoch, + targetAssignment, + toTopicPartitions(subscription.ownedPartitions(), metadataImage), records ); } @@ -4084,7 +4106,7 @@ public class GroupMetadataManager { List records = new ArrayList<>(); if (validateOnlineDowngradeWithFencedMembers(group, members)) { - convertToClassicGroup(group, members, null, records); + convertToClassicGroup(group, members, null, false, records); return new CoordinatorResult<>(records, response, null, false); } else { for (ConsumerGroupMember member : members) { 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 efe2ad96435..8d4ae4fbe07 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 @@ -12457,8 +12457,11 @@ public class GroupMetadataManagerTest { assertTrue(classicGroup.isInState(PREPARING_REBALANCE)); } - @Test - public void testLastStaticConsumerProtocolMemberReplacedByClassicProtocolMember() throws ExecutionException, InterruptedException { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testLastStaticConsumerProtocolMemberReplacedByClassicProtocolMember( + boolean isSubscriptionChanged + ) throws ExecutionException, InterruptedException { String groupId = "group-id"; String memberId1 = Uuid.randomUuid().toString(); String oldMemberId2 = Uuid.randomUuid().toString(); @@ -12469,11 +12472,9 @@ public class GroupMetadataManagerTest { Uuid barTopicId = Uuid.randomUuid(); String barTopicName = "bar"; - MockPartitionAssignor assignor = new MockPartitionAssignor("range"); - List protocols1 = List.of( new ConsumerGroupMemberMetadataValue.ClassicProtocol() - .setName("range") + .setName(NoOpPartitionAssignor.NAME) .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( List.of(fooTopicName, barTopicName), null, @@ -12493,8 +12494,8 @@ public class GroupMetadataManagerTest { .setPreviousMemberEpoch(9) .setClientId(DEFAULT_CLIENT_ID) .setClientHost(DEFAULT_CLIENT_ADDRESS.toString()) - .setSubscribedTopicNames(List.of("foo", "bar")) - .setServerAssignorName("range") + .setSubscribedTopicNames(List.of(fooTopicName, barTopicName)) + .setServerAssignorName(NoOpPartitionAssignor.NAME) .setRebalanceTimeoutMs(45000) .setClassicMemberMetadata( new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() @@ -12512,8 +12513,8 @@ public class GroupMetadataManagerTest { .setPreviousMemberEpoch(9) .setClientId(DEFAULT_CLIENT_ID) .setClientHost(DEFAULT_CLIENT_ADDRESS.toString()) - .setSubscribedTopicNames(List.of("foo")) - .setServerAssignorName("range") + .setSubscribedTopicNames(List.of(fooTopicName)) + .setServerAssignorName(NoOpPartitionAssignor.NAME) .setRebalanceTimeoutMs(45000) .setAssignedPartitions(mkAssignment( mkTopicAssignment(fooTopicId, 3, 4, 5))) @@ -12524,12 +12525,14 @@ public class GroupMetadataManagerTest { .addTopic(barTopicId, barTopicName, 2) .addRacks() .buildCoordinatorMetadataImage(); + long fooTopicHash = computeTopicHash(fooTopicName, metadataImage); + long barTopicHash = computeTopicHash(barTopicName, metadataImage); // Consumer group with two members. // Member 1 uses the classic protocol and static member 2 uses the consumer protocol. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_CONFIG, ConsumerGroupMigrationPolicy.DOWNGRADE.toString()) - .withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor)) + .withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(new NoOpPartitionAssignor())) .withMetadataImage(metadataImage) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(member1) @@ -12549,12 +12552,19 @@ public class GroupMetadataManagerTest { context.groupMetadataManager.consumerGroup(groupId).setMetadataRefreshDeadline(Long.MAX_VALUE, 10); // A new member using classic protocol with the same instance id joins, scheduling the downgrade. + byte[] protocolsMetadata2 = Utils.toArray(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( + isSubscriptionChanged ? List.of(fooTopicName, barTopicName) : List.of(fooTopicName)))); + JoinGroupRequestData.JoinGroupRequestProtocolCollection protocols2 = + new JoinGroupRequestData.JoinGroupRequestProtocolCollection(1); + protocols2.add(new JoinGroupRequestProtocol() + .setName(NoOpPartitionAssignor.NAME) + .setMetadata(protocolsMetadata2)); JoinGroupRequestData joinRequest = new GroupMetadataManagerTestContext.JoinGroupRequestBuilder() .withGroupId(groupId) .withMemberId(UNKNOWN_MEMBER_ID) .withGroupInstanceId(instanceId) .withProtocolType(ConsumerProtocol.PROTOCOL_TYPE) - .withDefaultProtocolTypeAndProtocols() + .withProtocols(protocols2) .build(); GroupMetadataManagerTestContext.JoinResult result = context.sendClassicGroupJoin(joinRequest); result.appendFuture.complete(null); @@ -12566,14 +12576,15 @@ public class GroupMetadataManagerTest { .build(); ConsumerGroupMember expectedNewClassicMember2 = new ConsumerGroupMember.Builder(oldMember2, newMemberId2) .setPreviousMemberEpoch(0) + .setMemberEpoch(isSubscriptionChanged ? 11 : 10) + .setSubscribedTopicNames(isSubscriptionChanged ? List.of(fooTopicName, barTopicName) : List.of(fooTopicName)) .setRebalanceTimeoutMs(joinRequest.rebalanceTimeoutMs()) .setClassicMemberMetadata( new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() .setSessionTimeoutMs(joinRequest.sessionTimeoutMs()) .setSupportedProtocols(List.of(new ConsumerGroupMemberMetadataValue.ClassicProtocol() - .setName("range") - .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - List.of(fooTopicName))))))) + .setName(NoOpPartitionAssignor.NAME) + .setMetadata(protocolsMetadata2))) ).build(); byte[] assignment1 = Utils.toArray(ConsumerProtocol.serializeAssignment(new ConsumerPartitionAssignor.Assignment(List.of( @@ -12600,7 +12611,7 @@ public class GroupMetadataManagerTest { context.time, 10, Optional.of(ConsumerProtocol.PROTOCOL_TYPE), - Optional.of("range"), + Optional.of(NoOpPartitionAssignor.NAME), Optional.of(memberId1), Optional.of(context.time.milliseconds()) ); @@ -12636,42 +12647,60 @@ public class GroupMetadataManagerTest { assertTrue(Set.of(memberId1, newMemberId2).contains(leader)); expectedClassicGroup.setLeaderId(Optional.of(leader)); - assertUnorderedRecordsEquals( - List.of( - // Remove the existing member 2 that uses the consumer protocol. - List.of(GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentTombstoneRecord(groupId, oldMemberId2)), - List.of(GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentTombstoneRecord(groupId, oldMemberId2)), - List.of(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId, oldMemberId2)), + List> replacingRecords = List.of( + // Remove the existing member 2 that uses the consumer protocol. + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentTombstoneRecord(groupId, oldMemberId2)), + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentTombstoneRecord(groupId, oldMemberId2)), + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId, oldMemberId2)), - // Create the new member 2 that uses the consumer protocol. - List.of(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedNewConsumerMember2)), - List.of(GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, newMemberId2, expectedNewConsumerMember2.assignedPartitions())), - List.of(GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedNewConsumerMember2)), + // Create the new member 2 that uses the consumer protocol. + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedNewConsumerMember2)), + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, newMemberId2, expectedNewConsumerMember2.assignedPartitions())), + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedNewConsumerMember2)) + ); - // Update the new member 2 to the member that uses classic protocol. + List> memberUpdateRecords; + if (isSubscriptionChanged) { + memberUpdateRecords = List.of( List.of(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedNewClassicMember2)), - List.of(GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedNewClassicMember2)), + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, computeGroupHash(Map.of( + fooTopicName, fooTopicHash, + barTopicName, barTopicHash + )))) + ); + } else { + memberUpdateRecords = List.of( + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedNewClassicMember2)), + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedNewClassicMember2)) + ); + } - // Remove member 1, member 2 and the consumer group. - List.of( - GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentTombstoneRecord(groupId, memberId1), - GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentTombstoneRecord(groupId, newMemberId2) - ), - List.of( - GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentTombstoneRecord(groupId, memberId1), - GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentTombstoneRecord(groupId, newMemberId2) - ), - List.of(GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochTombstoneRecord(groupId)), - List.of( - GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId, memberId1), - GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId, newMemberId2) - ), - List.of(GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataTombstoneRecord(groupId)), - List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochTombstoneRecord(groupId)), - - // Create the classic group. - List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(expectedClassicGroup, assignments)) + List> downgradeRecords = List.of( + // Remove member 1, member 2 and the consumer group. + List.of( + GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentTombstoneRecord(groupId, memberId1), + GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentTombstoneRecord(groupId, newMemberId2) ), + List.of( + GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentTombstoneRecord(groupId, memberId1), + GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentTombstoneRecord(groupId, newMemberId2) + ), + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochTombstoneRecord(groupId)), + List.of( + GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId, memberId1), + GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId, newMemberId2) + ), + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataTombstoneRecord(groupId)), + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochTombstoneRecord(groupId)), + + // Create the classic group. + List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(expectedClassicGroup, assignments)) + ); + + assertUnorderedRecordsEquals( + Stream.of(replacingRecords, memberUpdateRecords, downgradeRecords) + .flatMap(List::stream) + .collect(Collectors.toList()), result.records ); @@ -12681,9 +12710,13 @@ public class GroupMetadataManagerTest { ); assertNotNull(heartbeatTimeout); - // No rebalance is triggered. + // If the subscription is changed, a rebalance is triggered. ClassicGroup classicGroup = context.groupMetadataManager.getOrMaybeCreateClassicGroup(groupId, false); - assertTrue(classicGroup.isInState(STABLE)); + if (isSubscriptionChanged) { + assertTrue(classicGroup.isInState(PREPARING_REBALANCE)); + } else { + assertTrue(classicGroup.isInState(STABLE)); + } } @Test From 348e64c57d80188334b441d1a06c4181c00a8afa Mon Sep 17 00:00:00 2001 From: Shivsundar R Date: Wed, 24 Sep 2025 12:15:05 -0400 Subject: [PATCH 071/100] MINOR: Add unit tests for verifying --formatter-property in console tools. (#20560) *What* In the implementation of KIP-1147 for console tools - https://github.com/apache/kafka/pull/20479/files#diff-85b87c675a4b933e8e0e05c654d35d60e9cfd36cebe3331af825191b2cc688ee, we missed adding unit tests for verifying the new "`--formatter-property`" option. Thanks to @Yunyung for pointing this out. PR adds unit tests to both `ConsoleConsumerOptionsTest` and `ConsoleShareConsumerOptionsTest` to verify the same. Reviewers: Jhen-Yung Hsu , Chia-Ping Tsai --- .../consumer/ConsoleConsumerOptionsTest.java | 79 ++++++++++++++++++- .../ConsoleShareConsumerOptionsTest.java | 56 ++++++++++++- 2 files changed, 128 insertions(+), 7 deletions(-) diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java index 4639ff63a85..843d9785cf8 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java @@ -151,7 +151,7 @@ public class ConsoleConsumerOptionsTest { } @Test - public void shouldParseValidSimpleConsumerValidConfigWithStringOffset() throws Exception { + public void shouldParseValidSimpleConsumerValidConfigWithStringOffsetDeprecated() throws Exception { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", @@ -171,6 +171,27 @@ public class ConsoleConsumerOptionsTest { assertFalse(((DefaultMessageFormatter) config.formatter()).printValue()); } + @Test + public void shouldParseValidSimpleConsumerValidConfigWithStringOffset() throws Exception { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--partition", "0", + "--offset", "LatEst", + "--formatter-property", "print.value=false" + }; + + ConsoleConsumerOptions config = new ConsoleConsumerOptions(args); + + assertEquals("localhost:9092", config.bootstrapServer()); + assertEquals("test", config.topicArg().orElse("")); + assertTrue(config.partitionArg().isPresent()); + assertEquals(0, config.partitionArg().getAsInt()); + assertEquals(-1, config.offsetArg()); + assertFalse(config.fromBeginning()); + assertFalse(((DefaultMessageFormatter) config.formatter()).printValue()); + } + @Test public void shouldParseValidConsumerConfigWithAutoOffsetResetLatestDeprecated() throws IOException { String[] args = new String[]{ @@ -355,7 +376,7 @@ public class ConsoleConsumerOptionsTest { } @Test - public void testCustomPropertyShouldBePassedToConfigureMethod() throws Exception { + public void testCustomPropertyShouldBePassedToConfigureMethodDeprecated() throws Exception { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", @@ -377,6 +398,56 @@ public class ConsoleConsumerOptionsTest { assertTrue(keyDeserializer.isKey); } + @Test + public void testCustomPropertyShouldBePassedToConfigureMethod() throws Exception { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--formatter-property", "print.key=true", + "--formatter-property", "key.deserializer=org.apache.kafka.test.MockDeserializer", + "--formatter-property", "key.deserializer.my-props=abc" + }; + + ConsoleConsumerOptions config = new ConsoleConsumerOptions(args); + + assertInstanceOf(DefaultMessageFormatter.class, config.formatter()); + assertTrue(config.formatterArgs().containsKey("key.deserializer.my-props")); + DefaultMessageFormatter formatter = (DefaultMessageFormatter) config.formatter(); + assertTrue(formatter.keyDeserializer().isPresent()); + assertInstanceOf(MockDeserializer.class, formatter.keyDeserializer().get()); + MockDeserializer keyDeserializer = (MockDeserializer) formatter.keyDeserializer().get(); + assertEquals(1, keyDeserializer.configs.size()); + assertEquals("abc", keyDeserializer.configs.get("my-props")); + assertTrue(keyDeserializer.isKey); + } + + @Test + public void testCustomConfigShouldBePassedToConfigureMethodDeprecated() throws Exception { + Map configs = new HashMap<>(); + configs.put("key.deserializer.my-props", "abc"); + configs.put("print.key", "false"); + File propsFile = ToolsTestUtils.tempPropertiesFile(configs); + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--property", "print.key=true", + "--property", "key.deserializer=org.apache.kafka.test.MockDeserializer", + "--formatter-config", propsFile.getAbsolutePath() + }; + + ConsoleConsumerOptions config = new ConsoleConsumerOptions(args); + + assertInstanceOf(DefaultMessageFormatter.class, config.formatter()); + assertTrue(config.formatterArgs().containsKey("key.deserializer.my-props")); + DefaultMessageFormatter formatter = (DefaultMessageFormatter) config.formatter(); + assertTrue(formatter.keyDeserializer().isPresent()); + assertInstanceOf(MockDeserializer.class, formatter.keyDeserializer().get()); + MockDeserializer keyDeserializer = (MockDeserializer) formatter.keyDeserializer().get(); + assertEquals(1, keyDeserializer.configs.size()); + assertEquals("abc", keyDeserializer.configs.get("my-props")); + assertTrue(keyDeserializer.isKey); + } + @Test public void testCustomConfigShouldBePassedToConfigureMethod() throws Exception { Map configs = new HashMap<>(); @@ -386,8 +457,8 @@ public class ConsoleConsumerOptionsTest { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", - "--property", "print.key=true", - "--property", "key.deserializer=org.apache.kafka.test.MockDeserializer", + "--formatter-property", "print.key=true", + "--formatter-property", "key.deserializer=org.apache.kafka.test.MockDeserializer", "--formatter-config", propsFile.getAbsolutePath() }; diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptionsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptionsTest.java index fecf53dbbec..a097a9bf536 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptionsTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptionsTest.java @@ -222,7 +222,7 @@ public class ConsoleShareConsumerOptionsTest { } @Test - public void testCustomPropertyShouldBePassedToConfigureMethod() throws Exception { + public void testCustomPropertyShouldBePassedToConfigureMethodDeprecated() throws Exception { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", @@ -244,6 +244,56 @@ public class ConsoleShareConsumerOptionsTest { assertTrue(keyDeserializer.isKey); } + @Test + public void testCustomPropertyShouldBePassedToConfigureMethod() throws Exception { + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--formatter-property", "print.key=true", + "--formatter-property", "key.deserializer=org.apache.kafka.test.MockDeserializer", + "--formatter-property", "key.deserializer.my-props=abc" + }; + + ConsoleShareConsumerOptions config = new ConsoleShareConsumerOptions(args); + + assertInstanceOf(DefaultMessageFormatter.class, config.formatter()); + assertTrue(config.formatterArgs().containsKey("key.deserializer.my-props")); + DefaultMessageFormatter formatter = (DefaultMessageFormatter) config.formatter(); + assertTrue(formatter.keyDeserializer().isPresent()); + assertInstanceOf(MockDeserializer.class, formatter.keyDeserializer().get()); + MockDeserializer keyDeserializer = (MockDeserializer) formatter.keyDeserializer().get(); + assertEquals(1, keyDeserializer.configs.size()); + assertEquals("abc", keyDeserializer.configs.get("my-props")); + assertTrue(keyDeserializer.isKey); + } + + @Test + public void testCustomConfigShouldBePassedToConfigureMethodDeprecated() throws Exception { + Map configs = new HashMap<>(); + configs.put("key.deserializer.my-props", "abc"); + configs.put("print.key", "false"); + File propsFile = ToolsTestUtils.tempPropertiesFile(configs); + String[] args = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--property", "print.key=true", + "--property", "key.deserializer=org.apache.kafka.test.MockDeserializer", + "--formatter-config", propsFile.getAbsolutePath() + }; + + ConsoleShareConsumerOptions config = new ConsoleShareConsumerOptions(args); + + assertInstanceOf(DefaultMessageFormatter.class, config.formatter()); + assertTrue(config.formatterArgs().containsKey("key.deserializer.my-props")); + DefaultMessageFormatter formatter = (DefaultMessageFormatter) config.formatter(); + assertTrue(formatter.keyDeserializer().isPresent()); + assertInstanceOf(MockDeserializer.class, formatter.keyDeserializer().get()); + MockDeserializer keyDeserializer = (MockDeserializer) formatter.keyDeserializer().get(); + assertEquals(1, keyDeserializer.configs.size()); + assertEquals("abc", keyDeserializer.configs.get("my-props")); + assertTrue(keyDeserializer.isKey); + } + @Test public void testCustomConfigShouldBePassedToConfigureMethod() throws Exception { Map configs = new HashMap<>(); @@ -253,8 +303,8 @@ public class ConsoleShareConsumerOptionsTest { String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", - "--property", "print.key=true", - "--property", "key.deserializer=org.apache.kafka.test.MockDeserializer", + "--formatter-property", "print.key=true", + "--formatter-property", "key.deserializer=org.apache.kafka.test.MockDeserializer", "--formatter-config", propsFile.getAbsolutePath() }; From ac63ce9789a7d7d283958ed6f01af996fcd85159 Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Thu, 25 Sep 2025 01:06:54 +0800 Subject: [PATCH 072/100] KAFKA-19544 Improve `MetadataVersion.fromVersionString()` to take an enableUnstableFeature flag (#20248) Improve `MetadataVersion.fromVersionString()` to take an `enableUnstableFeature` flag, and enable `FeatureCommand` and `StorageTool` to leverage the exception message from `fromVersionString`. Reviewers: Chia-Ping Tsai --- .../main/scala/kafka/tools/StorageTool.scala | 22 ++---- .../kafka/server/KRaftClusterTest.scala | 2 +- .../unit/kafka/tools/StorageToolTest.scala | 14 ++-- .../PartitionChangeBuilderTest.java | 10 +-- .../kafka/server/common/MetadataVersion.java | 27 +++++-- .../server/common/MetadataVersionTest.java | 79 +++++++++++-------- .../apache/kafka/tools/FeatureCommand.java | 26 ++---- .../kafka/tools/FeatureCommandTest.java | 20 ++--- 8 files changed, 101 insertions(+), 99 deletions(-) diff --git a/core/src/main/scala/kafka/tools/StorageTool.scala b/core/src/main/scala/kafka/tools/StorageTool.scala index 63993ed5ea9..c342ddfe071 100644 --- a/core/src/main/scala/kafka/tools/StorageTool.scala +++ b/core/src/main/scala/kafka/tools/StorageTool.scala @@ -129,17 +129,12 @@ object StorageTool extends Logging { setControllerListenerName(config.controllerListenerNames.get(0)). setMetadataLogDirectory(config.metadataLogDir) - def metadataVersionsToString(first: MetadataVersion, last: MetadataVersion): String = { - val versions = MetadataVersion.VERSIONS.slice(first.ordinal, last.ordinal + 1) - versions.map(_.toString).mkString(", ") - } Option(namespace.getString("release_version")).foreach(releaseVersion => { try { - formatter.setReleaseVersion(MetadataVersion.fromVersionString(releaseVersion)) + formatter.setReleaseVersion(MetadataVersion.fromVersionString(releaseVersion, config.unstableFeatureVersionsEnabled)) } catch { - case _: Throwable => - throw new TerseFailure(s"Unknown metadata.version $releaseVersion. Supported metadata.version are " + - s"${metadataVersionsToString(MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestProduction())}") + case e: Throwable => + throw new TerseFailure(e.getMessage) } }) @@ -184,9 +179,9 @@ object StorageTool extends Logging { * Maps the given release version to the corresponding metadata version * and prints the corresponding features. * - * @param namespace Arguments containing the release version. - * @param printStream The print stream to output the version mapping. - * @param validFeatures List of features to be considered in the output + * @param namespace Arguments containing the release version. + * @param printStream The print stream to output the version mapping. + * @param validFeatures List of features to be considered in the output. */ def runVersionMappingCommand( namespace: Namespace, @@ -195,7 +190,7 @@ object StorageTool extends Logging { ): Unit = { val releaseVersion = Option(namespace.getString("release_version")).getOrElse(MetadataVersion.LATEST_PRODUCTION.toString) try { - val metadataVersion = MetadataVersion.fromVersionString(releaseVersion) + val metadataVersion = MetadataVersion.fromVersionString(releaseVersion, true) val metadataVersionLevel = metadataVersion.featureLevel() printStream.print(f"metadata.version=$metadataVersionLevel%d ($releaseVersion%s)%n") @@ -206,8 +201,7 @@ object StorageTool extends Logging { } } catch { case e: IllegalArgumentException => - throw new TerseFailure(s"Unknown release version '$releaseVersion'. Supported versions are: " + - s"${MetadataVersion.MINIMUM_VERSION.version} to ${MetadataVersion.latestTesting().version()}") + throw new TerseFailure(e.getMessage) } } diff --git a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala index dfc999d7c64..27b06daed21 100644 --- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala +++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala @@ -807,7 +807,7 @@ class KRaftClusterTest { val cluster = new KafkaClusterTestKit.Builder( new TestKitNodes.Builder(). setNumBrokerNodes(4). - setBootstrapMetadataVersion(MetadataVersion.fromVersionString(metadataVersionString)). + setBootstrapMetadataVersion(MetadataVersion.fromVersionString(metadataVersionString, true)). setNumControllerNodes(3).build()). build() try { diff --git a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala index 344df08f56b..1e938ea9cd8 100644 --- a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala +++ b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala @@ -315,7 +315,7 @@ Found problem: val stream = new ByteArrayOutputStream() val failure = assertThrows(classOf[TerseFailure], () => runFormatCommand(stream, properties, Seq("--release-version", "3.3-IV1"))).getMessage - assertTrue(failure.contains("Unknown metadata.version 3.3-IV1")) + assertTrue(failure.contains("Unknown metadata.version '3.3-IV1'")) assertTrue(failure.contains(MetadataVersion.MINIMUM_VERSION.version)) assertTrue(failure.contains(MetadataVersion.latestProduction().version)) } @@ -735,18 +735,18 @@ Found problem: runVersionMappingCommand(stream, "2.9-IV2") }) - assertEquals("Unknown release version '2.9-IV2'." + - " Supported versions are: " + MetadataVersion.MINIMUM_VERSION.version + - " to " + MetadataVersion.latestTesting().version, exception.getMessage + assertEquals("Unknown metadata.version '2.9-IV2'. Supported metadata.version are: " + + MetadataVersion.metadataVersionsToString(MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestTesting()), + exception.getMessage ) val exception2 = assertThrows(classOf[TerseFailure], () => { runVersionMappingCommand(stream, "invalid") }) - assertEquals("Unknown release version 'invalid'." + - " Supported versions are: " + MetadataVersion.MINIMUM_VERSION.version + - " to " + MetadataVersion.latestTesting().version, exception2.getMessage + assertEquals("Unknown metadata.version 'invalid'. Supported metadata.version are: " + + MetadataVersion.metadataVersionsToString(MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestTesting()), + exception2.getMessage ) } diff --git a/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java b/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java index bb5f5b9216d..312a207f8d7 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java @@ -310,7 +310,7 @@ public class PartitionChangeBuilderTest { @ParameterizedTest @ValueSource(strings = {"3.6-IV0", "3.7-IV2", "4.0-IV0"}) public void testNoLeaderEpochBumpOnIsrShrink(String metadataVersionString) { - MetadataVersion metadataVersion = MetadataVersion.fromVersionString(metadataVersionString); + MetadataVersion metadataVersion = MetadataVersion.fromVersionString(metadataVersionString, true); testTriggerLeaderEpochBumpIfNeeded( createFooBuilder(metadataVersion).setTargetIsrWithBrokerStates( AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(List.of(2, 1))), @@ -325,7 +325,7 @@ public class PartitionChangeBuilderTest { @ParameterizedTest @ValueSource(strings = {"3.4-IV0", "3.5-IV2"}) public void testLeaderEpochBumpOnIsrShrink(String metadataVersionString) { - MetadataVersion metadataVersion = MetadataVersion.fromVersionString(metadataVersionString); + MetadataVersion metadataVersion = MetadataVersion.fromVersionString(metadataVersionString, true); testTriggerLeaderEpochBumpIfNeeded( createFooBuilder(metadataVersion).setTargetIsrWithBrokerStates( AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(List.of(2, 1))), @@ -339,7 +339,7 @@ public class PartitionChangeBuilderTest { @ParameterizedTest @ValueSource(strings = {"3.4-IV0", "3.5-IV2", "3.6-IV0", "3.7-IV2", "4.0-IV0"}) public void testNoLeaderEpochBumpOnIsrExpansion(String metadataVersionString) { - MetadataVersion metadataVersion = MetadataVersion.fromVersionString(metadataVersionString); + MetadataVersion metadataVersion = MetadataVersion.fromVersionString(metadataVersionString, true); testTriggerLeaderEpochBumpIfNeeded( createFooBuilder(metadataVersion).setTargetIsrWithBrokerStates( AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(List.of(2, 1, 3, 4))), @@ -354,7 +354,7 @@ public class PartitionChangeBuilderTest { @ParameterizedTest @ValueSource(strings = {"3.4-IV0", "3.5-IV2", "3.6-IV0", "3.7-IV2", "4.0-IV0"}) public void testLeaderEpochBumpOnNewReplicaSetDisjoint(String metadataVersionString) { - MetadataVersion metadataVersion = MetadataVersion.fromVersionString(metadataVersionString); + MetadataVersion metadataVersion = MetadataVersion.fromVersionString(metadataVersionString, true); testTriggerLeaderEpochBumpIfNeeded( createFooBuilder(metadataVersion).setTargetReplicas(List.of(2, 1, 4)), new PartitionChangeRecord(), @@ -368,7 +368,7 @@ public class PartitionChangeBuilderTest { @ParameterizedTest @ValueSource(strings = {"3.4-IV0", "3.5-IV2", "3.6-IV0", "3.7-IV2"}) public void testNoLeaderEpochBumpOnEmptyTargetIsr(String metadataVersionString) { - MetadataVersion metadataVersion = MetadataVersion.fromVersionString(metadataVersionString); + MetadataVersion metadataVersion = MetadataVersion.fromVersionString(metadataVersionString, true); PartitionRegistration partition = new PartitionRegistration.Builder(). setReplicas(new int[] {2}). setDirectories(new Uuid[]{ diff --git a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java index ceca9a6a7de..940f58e26c6 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java @@ -19,9 +19,11 @@ package org.apache.kafka.server.common; import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.regex.Pattern; +import java.util.stream.Collectors; /** * This class contains the different Kafka versions. @@ -340,11 +342,12 @@ public enum MetadataVersion { /** * Return an `MetadataVersion` instance for `versionString`, which can be in a variety of formats (e.g. "3.8", "3.8.x", - * "3.8.0", "3.8-IV0"). `IllegalArgumentException` is thrown if `versionString` cannot be mapped to an `MetadataVersion`. + * "3.8.0", "3.8-IV0"). The `unstableFeatureVersionsEnabled` parameter determines whether unstable versions are permitted. + * `IllegalArgumentException` is thrown if `versionString` cannot be mapped to an `MetadataVersion`. * Note that 'misconfigured' values such as "3.8.1" will be parsed to `IBP_3_8_IV0` as we ignore anything after the first * two segments. */ - public static MetadataVersion fromVersionString(String versionString) { + public static MetadataVersion fromVersionString(String versionString, boolean unstableFeatureVersionsEnabled) { String[] versionSegments = versionString.split(Pattern.quote(".")); int numSegments = 2; String key; @@ -353,10 +356,22 @@ public enum MetadataVersion { } else { key = String.join(".", Arrays.copyOfRange(versionSegments, 0, numSegments)); } - return Optional.ofNullable(IBP_VERSIONS.get(key)).orElseThrow(() -> - new IllegalArgumentException("Version " + versionString + " is not a valid version. The minimum version is " + MINIMUM_VERSION - + " and the maximum version is " + latestTesting()) - ); + + MetadataVersion metadataVersion = IBP_VERSIONS.get(key); + if (metadataVersion == null || (!unstableFeatureVersionsEnabled && !metadataVersion.isProduction())) { + String errorMsg = "Unknown metadata.version '" + versionString + "'. Supported metadata.version are: " + + metadataVersionsToString(MetadataVersion.MINIMUM_VERSION, + unstableFeatureVersionsEnabled ? MetadataVersion.latestTesting() : MetadataVersion.latestProduction()); + throw new IllegalArgumentException(errorMsg); + } + return metadataVersion; + } + + public static String metadataVersionsToString(MetadataVersion first, MetadataVersion last) { + List versions = List.of(MetadataVersion.VERSIONS).subList(first.ordinal(), last.ordinal() + 1); + return versions.stream() + .map(String::valueOf) + .collect(Collectors.joining(", ")); } public static MetadataVersion fromFeatureLevel(short version) { diff --git a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java index 49a200f6225..136fdeaa4ec 100644 --- a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java @@ -26,6 +26,7 @@ import org.junit.jupiter.params.provider.EnumSource; import static org.apache.kafka.server.common.MetadataVersion.*; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; class MetadataVersionTest { @@ -42,55 +43,69 @@ class MetadataVersionTest { @SuppressWarnings("checkstyle:JavaNCSS") public void testFromVersionString() { // 3.3-IV3 is the latest production version in the 3.3 line - assertEquals(IBP_3_3_IV3, MetadataVersion.fromVersionString("3.3")); - assertEquals(IBP_3_3_IV3, MetadataVersion.fromVersionString("3.3-IV3")); + assertEquals(IBP_3_3_IV3, MetadataVersion.fromVersionString("3.3", true)); + assertEquals(IBP_3_3_IV3, MetadataVersion.fromVersionString("3.3-IV3", true)); // 3.4-IV0 is the latest production version in the 3.4 line - assertEquals(IBP_3_4_IV0, MetadataVersion.fromVersionString("3.4")); - assertEquals(IBP_3_4_IV0, MetadataVersion.fromVersionString("3.4-IV0")); + assertEquals(IBP_3_4_IV0, MetadataVersion.fromVersionString("3.4", true)); + assertEquals(IBP_3_4_IV0, MetadataVersion.fromVersionString("3.4-IV0", true)); // 3.5-IV2 is the latest production version in the 3.5 line - assertEquals(IBP_3_5_IV2, MetadataVersion.fromVersionString("3.5")); - assertEquals(IBP_3_5_IV0, MetadataVersion.fromVersionString("3.5-IV0")); - assertEquals(IBP_3_5_IV1, MetadataVersion.fromVersionString("3.5-IV1")); - assertEquals(IBP_3_5_IV2, MetadataVersion.fromVersionString("3.5-IV2")); + assertEquals(IBP_3_5_IV2, MetadataVersion.fromVersionString("3.5", true)); + assertEquals(IBP_3_5_IV0, MetadataVersion.fromVersionString("3.5-IV0", true)); + assertEquals(IBP_3_5_IV1, MetadataVersion.fromVersionString("3.5-IV1", true)); + assertEquals(IBP_3_5_IV2, MetadataVersion.fromVersionString("3.5-IV2", true)); // 3.6-IV2 is the latest production version in the 3.6 line - assertEquals(IBP_3_6_IV2, MetadataVersion.fromVersionString("3.6")); - assertEquals(IBP_3_6_IV0, MetadataVersion.fromVersionString("3.6-IV0")); - assertEquals(IBP_3_6_IV1, MetadataVersion.fromVersionString("3.6-IV1")); - assertEquals(IBP_3_6_IV2, MetadataVersion.fromVersionString("3.6-IV2")); + assertEquals(IBP_3_6_IV2, MetadataVersion.fromVersionString("3.6", true)); + assertEquals(IBP_3_6_IV0, MetadataVersion.fromVersionString("3.6-IV0", true)); + assertEquals(IBP_3_6_IV1, MetadataVersion.fromVersionString("3.6-IV1", true)); + assertEquals(IBP_3_6_IV2, MetadataVersion.fromVersionString("3.6-IV2", true)); // 3.7-IV4 is the latest production version in the 3.7 line - assertEquals(IBP_3_7_IV4, MetadataVersion.fromVersionString("3.7")); - assertEquals(IBP_3_7_IV0, MetadataVersion.fromVersionString("3.7-IV0")); - assertEquals(IBP_3_7_IV1, MetadataVersion.fromVersionString("3.7-IV1")); - assertEquals(IBP_3_7_IV2, MetadataVersion.fromVersionString("3.7-IV2")); - assertEquals(IBP_3_7_IV3, MetadataVersion.fromVersionString("3.7-IV3")); - assertEquals(IBP_3_7_IV4, MetadataVersion.fromVersionString("3.7-IV4")); + assertEquals(IBP_3_7_IV4, MetadataVersion.fromVersionString("3.7", true)); + assertEquals(IBP_3_7_IV0, MetadataVersion.fromVersionString("3.7-IV0", true)); + assertEquals(IBP_3_7_IV1, MetadataVersion.fromVersionString("3.7-IV1", true)); + assertEquals(IBP_3_7_IV2, MetadataVersion.fromVersionString("3.7-IV2", true)); + assertEquals(IBP_3_7_IV3, MetadataVersion.fromVersionString("3.7-IV3", true)); + assertEquals(IBP_3_7_IV4, MetadataVersion.fromVersionString("3.7-IV4", true)); // 3.8-IV0 is the latest production version in the 3.8 line - assertEquals(IBP_3_8_IV0, MetadataVersion.fromVersionString("3.8")); - assertEquals(IBP_3_8_IV0, MetadataVersion.fromVersionString("3.8-IV0")); + assertEquals(IBP_3_8_IV0, MetadataVersion.fromVersionString("3.8", true)); + assertEquals(IBP_3_8_IV0, MetadataVersion.fromVersionString("3.8-IV0", true)); // 3.9-IV0 is the latest production version in the 3.9 line - assertEquals(IBP_3_9_IV0, MetadataVersion.fromVersionString("3.9")); - assertEquals(IBP_3_9_IV0, MetadataVersion.fromVersionString("3.9-IV0")); + assertEquals(IBP_3_9_IV0, MetadataVersion.fromVersionString("3.9", true)); + assertEquals(IBP_3_9_IV0, MetadataVersion.fromVersionString("3.9-IV0", true)); // 4.0-IV3 is the latest production version in the 4.0 line - assertEquals(IBP_4_0_IV3, MetadataVersion.fromVersionString("4.0")); - assertEquals(IBP_4_0_IV0, MetadataVersion.fromVersionString("4.0-IV0")); - assertEquals(IBP_4_0_IV1, MetadataVersion.fromVersionString("4.0-IV1")); - assertEquals(IBP_4_0_IV2, MetadataVersion.fromVersionString("4.0-IV2")); - assertEquals(IBP_4_0_IV3, MetadataVersion.fromVersionString("4.0-IV3")); + assertEquals(IBP_4_0_IV3, MetadataVersion.fromVersionString("4.0", true)); + assertEquals(IBP_4_0_IV0, MetadataVersion.fromVersionString("4.0-IV0", true)); + assertEquals(IBP_4_0_IV1, MetadataVersion.fromVersionString("4.0-IV1", true)); + assertEquals(IBP_4_0_IV2, MetadataVersion.fromVersionString("4.0-IV2", true)); + assertEquals(IBP_4_0_IV3, MetadataVersion.fromVersionString("4.0-IV3", true)); // 4.1-IV1 is the latest production version in the 4.1 line - assertEquals(IBP_4_1_IV1, MetadataVersion.fromVersionString("4.1")); - assertEquals(IBP_4_1_IV0, MetadataVersion.fromVersionString("4.1-IV0")); - assertEquals(IBP_4_1_IV1, MetadataVersion.fromVersionString("4.1-IV1")); + assertEquals(IBP_4_1_IV1, MetadataVersion.fromVersionString("4.1", true)); + assertEquals(IBP_4_1_IV0, MetadataVersion.fromVersionString("4.1-IV0", true)); + assertEquals(IBP_4_1_IV1, MetadataVersion.fromVersionString("4.1-IV1", true)); - assertEquals(IBP_4_2_IV0, MetadataVersion.fromVersionString("4.2-IV0")); - assertEquals(IBP_4_2_IV1, MetadataVersion.fromVersionString("4.2-IV1")); + assertEquals(IBP_4_2_IV0, MetadataVersion.fromVersionString("4.2-IV0", true)); + assertEquals(IBP_4_2_IV1, MetadataVersion.fromVersionString("4.2-IV1", true)); + + // Throws exception when unstableFeatureVersionsEnabled is false + assertEquals("Unknown metadata.version '4.2-IV0'. Supported metadata.version are: 3.3-IV3, 3.4-IV0, 3.5-IV0, 3.5-IV1, 3.5-IV2, " + + "3.6-IV0, 3.6-IV1, 3.6-IV2, 3.7-IV0, 3.7-IV1, 3.7-IV2, 3.7-IV3, 3.7-IV4, 3.8-IV0, 3.9-IV0, 4.0-IV0, 4.0-IV1, 4.0-IV2, 4.0-IV3, 4.1-IV0, 4.1-IV1", + assertThrows(IllegalArgumentException.class, () -> fromVersionString("4.2-IV0", false)).getMessage()); + assertEquals("Unknown metadata.version '4.2-IV1'. Supported metadata.version are: 3.3-IV3, 3.4-IV0, 3.5-IV0, 3.5-IV1, 3.5-IV2, " + + "3.6-IV0, 3.6-IV1, 3.6-IV2, 3.7-IV0, 3.7-IV1, 3.7-IV2, 3.7-IV3, 3.7-IV4, 3.8-IV0, 3.9-IV0, 4.0-IV0, 4.0-IV1, 4.0-IV2, 4.0-IV3, 4.1-IV0, 4.1-IV1", + assertThrows(IllegalArgumentException.class, () -> fromVersionString("4.2-IV1", false)).getMessage()); + } + + @Test + public void testMetadataVersionsToString() { + assertEquals("3.5-IV0, 3.5-IV1, 3.5-IV2, 3.6-IV0", + MetadataVersion.metadataVersionsToString(MetadataVersion.IBP_3_5_IV0, MetadataVersion.IBP_3_6_IV0)); } @Test diff --git a/tools/src/main/java/org/apache/kafka/tools/FeatureCommand.java b/tools/src/main/java/org/apache/kafka/tools/FeatureCommand.java index 103821cf21d..87e4c228baf 100644 --- a/tools/src/main/java/org/apache/kafka/tools/FeatureCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/FeatureCommand.java @@ -46,7 +46,6 @@ import java.util.Optional; import java.util.Properties; import java.util.TreeMap; import java.util.concurrent.ExecutionException; -import java.util.stream.Collectors; import static net.sourceforge.argparse4j.impl.Arguments.append; import static net.sourceforge.argparse4j.impl.Arguments.store; @@ -238,13 +237,6 @@ public class FeatureCommand { }); } - static String metadataVersionsToString(MetadataVersion first, MetadataVersion last) { - List versions = List.of(MetadataVersion.VERSIONS).subList(first.ordinal(), last.ordinal() + 1); - return versions.stream() - .map(String::valueOf) - .collect(Collectors.joining(", ")); - } - static void handleUpgrade(Namespace namespace, Admin adminClient) throws TerseException { handleUpgradeOrDowngrade("upgrade", namespace, adminClient, FeatureUpdate.UpgradeType.UPGRADE); } @@ -292,12 +284,10 @@ public class FeatureCommand { if (releaseVersion != null) { try { - metadataVersion = MetadataVersion.fromVersionString(releaseVersion); + metadataVersion = MetadataVersion.fromVersionString(releaseVersion, true); updates.put(metadataVersion.featureName(), new FeatureUpdate(metadataVersion.featureLevel(), upgradeType)); } catch (Throwable e) { - throw new TerseException("Unknown metadata.version " + releaseVersion + - ". Supported metadata.version are " + metadataVersionsToString( - MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestProduction())); + throw new TerseException(e.getMessage()); } try { for (Feature feature : Feature.PRODUCTION_FEATURES) { @@ -315,11 +305,9 @@ public class FeatureCommand { if (metadata != null) { System.out.println(" `metadata` flag is deprecated and may be removed in a future release."); try { - metadataVersion = MetadataVersion.fromVersionString(metadata); + metadataVersion = MetadataVersion.fromVersionString(metadata, true); } catch (Throwable e) { - throw new TerseException("Unknown metadata.version " + metadata + - ". Supported metadata.version are " + metadataVersionsToString( - MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestProduction())); + throw new TerseException(e.getMessage()); } updates.put(MetadataVersion.FEATURE_NAME, new FeatureUpdate(metadataVersion.featureLevel(), upgradeType)); } @@ -361,7 +349,7 @@ public class FeatureCommand { .orElseGet(() -> MetadataVersion.latestProduction().version()); try { - MetadataVersion version = MetadataVersion.fromVersionString(releaseVersion); + MetadataVersion version = MetadataVersion.fromVersionString(releaseVersion, true); short metadataVersionLevel = version.featureLevel(); System.out.printf("metadata.version=%d (%s)%n", metadataVersionLevel, releaseVersion); @@ -371,9 +359,7 @@ public class FeatureCommand { System.out.printf("%s=%d%n", feature.featureName(), featureLevel); } } catch (IllegalArgumentException e) { - throw new TerseException("Unknown release version '" + releaseVersion + "'." + - " Supported versions are: " + MetadataVersion.MINIMUM_VERSION + - " to " + MetadataVersion.latestTesting().version()); + throw new TerseException(e.getMessage()); } } diff --git a/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java index 19cd8bf2a37..2caaf8a2918 100644 --- a/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java @@ -210,12 +210,6 @@ public class FeatureCommandTest { FeatureCommand.levelToString(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_9_IV0.featureLevel())); } - @Test - public void testMetadataVersionsToString() { - assertEquals("3.5-IV0, 3.5-IV1, 3.5-IV2, 3.6-IV0", - FeatureCommand.metadataVersionsToString(MetadataVersion.IBP_3_5_IV0, MetadataVersion.IBP_3_6_IV0)); - } - @Test public void testDowngradeType() { assertEquals(SAFE_DOWNGRADE, FeatureCommand.downgradeType( @@ -274,7 +268,7 @@ public class FeatureCommandTest { namespace.put("feature", List.of("foo.bar=6")); namespace.put("dry_run", false); Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleUpgrade(new Namespace(namespace), buildAdminClient())); - assertTrue(t.getMessage().contains("Unknown metadata.version 3.3-IV1")); + assertTrue(t.getMessage().contains("Unknown metadata.version '3.3-IV1'")); } @Test @@ -371,7 +365,7 @@ public class FeatureCommandTest { namespace.put("release_version", "foo"); ToolsTestUtils.captureStandardOut(() -> { Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleUpgrade(new Namespace(namespace), buildAdminClient())); - assertTrue(t.getMessage().contains("Unknown metadata.version foo.")); + assertTrue(t.getMessage().contains("Unknown metadata.version 'foo'.")); }); } @@ -452,9 +446,8 @@ public class FeatureCommandTest { FeatureCommand.handleVersionMapping(new Namespace(namespace), testingFeatures) ); - assertEquals("Unknown release version '2.9-IV2'." + - " Supported versions are: " + MetadataVersion.MINIMUM_VERSION + - " to " + MetadataVersion.latestTesting().version(), exception1.getMessage()); + assertEquals("Unknown metadata.version '2.9-IV2'. Supported metadata.version are: " + MetadataVersion.metadataVersionsToString( + MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestTesting()), exception1.getMessage()); namespace.put("release_version", "invalid"); @@ -462,9 +455,8 @@ public class FeatureCommandTest { FeatureCommand.handleVersionMapping(new Namespace(namespace), testingFeatures) ); - assertEquals("Unknown release version 'invalid'." + - " Supported versions are: " + MetadataVersion.MINIMUM_VERSION + - " to " + MetadataVersion.latestTesting().version(), exception2.getMessage()); + assertEquals("Unknown metadata.version 'invalid'. Supported metadata.version are: " + MetadataVersion.metadataVersionsToString( + MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestTesting()), exception2.getMessage()); } @Test From 563338c0e950209edebf064353a626249c5efc3c Mon Sep 17 00:00:00 2001 From: Maros Orsak Date: Wed, 24 Sep 2025 20:25:44 +0200 Subject: [PATCH 073/100] MINOR: Refactor on DelegationTokenManager follow up with KAFKA-18711 (#20579) Follow-up PR of KAFKA-18711. The motivation and reason for this change are outlined in [1]. [1] - https://github.com/apache/kafka/pull/20475#discussion_r2375608168 Reviewers: Chia-Ping Tsai --- core/src/main/scala/kafka/server/KafkaApis.scala | 3 +-- .../org/apache/kafka/security/DelegationTokenManager.java | 4 ++++ 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 4cbef3fa648..6ca9014f654 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -64,7 +64,6 @@ import org.apache.kafka.security.DelegationTokenManager import org.apache.kafka.server.{ApiVersionManager, ClientMetricsManager, ProcessRole} import org.apache.kafka.server.authorizer._ import org.apache.kafka.server.common.{GroupVersion, RequestLocal, ShareVersion, StreamsVersion, TransactionVersion} -import org.apache.kafka.server.config.DelegationTokenManagerConfigs import org.apache.kafka.server.share.context.ShareFetchContext import org.apache.kafka.server.share.{ErroneousAndValidPartitionData, SharePartitionKey} import org.apache.kafka.server.share.acknowledge.ShareAcknowledgementBatch @@ -2314,7 +2313,7 @@ class KafkaApis(val requestChannel: RequestChannel, if (!allowTokenRequests(request)) sendResponseCallback(Errors.DELEGATION_TOKEN_REQUEST_NOT_ALLOWED, Collections.emptyList) - else if (!new DelegationTokenManagerConfigs(config).tokenAuthEnabled) + else if (!tokenManager.isEnabled) sendResponseCallback(Errors.DELEGATION_TOKEN_AUTH_DISABLED, Collections.emptyList) else { val requestPrincipal = request.context.principal diff --git a/server-common/src/main/java/org/apache/kafka/security/DelegationTokenManager.java b/server-common/src/main/java/org/apache/kafka/security/DelegationTokenManager.java index ef82a0702c8..e078d9a2ea8 100644 --- a/server-common/src/main/java/org/apache/kafka/security/DelegationTokenManager.java +++ b/server-common/src/main/java/org/apache/kafka/security/DelegationTokenManager.java @@ -104,6 +104,10 @@ public class DelegationTokenManager { .toList(); } + public boolean isEnabled() { + return secretKey != null; + } + public static boolean filterToken( KafkaPrincipal requesterPrincipal, Optional> owners, From 444ceeb325056037111fa5f3e983c3301ece172c Mon Sep 17 00:00:00 2001 From: Mickael Maison Date: Thu, 25 Sep 2025 09:39:37 +0200 Subject: [PATCH 074/100] MINOR: Tidy up the Connect docs (#20531) Remove invalid mentions of default values for group.id, config.storage.topic, offset.storage.topic, status.storage.topic Reviewers: Luke Chen , Ken Huang --- docs/connect.html | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/connect.html b/docs/connect.html index aa3c2af3ea9..85571bf115c 100644 --- a/docs/connect.html +++ b/docs/connect.html @@ -47,7 +47,7 @@
  • bootstrap.servers - List of Kafka servers used to bootstrap connections to Kafka
  • key.converter - Converter class used to convert between Kafka Connect format and the serialized form that is written to Kafka. This controls the format of the keys in messages written to or read from Kafka, and since this is independent of connectors it allows any connector to work with any serialization format. Examples of common formats include JSON and Avro.
  • value.converter - Converter class used to convert between Kafka Connect format and the serialized form that is written to Kafka. This controls the format of the values in messages written to or read from Kafka, and since this is independent of connectors it allows any connector to work with any serialization format. Examples of common formats include JSON and Avro.
  • -
  • plugin.path (default empty) - a list of paths that contain Connect plugins (connectors, converters, transformations). Before running quick starts, users must add the absolute path that contains the example FileStreamSourceConnector and FileStreamSinkConnector packaged in connect-file-"version".jar, because these connectors are not included by default to the CLASSPATH or the plugin.path of the Connect worker (see plugin.path property for examples).
  • +
  • plugin.path (default null) - a list of paths that contain Connect plugins (connectors, converters, transformations). Before running quick starts, users must add the absolute path that contains the example FileStreamSourceConnector and FileStreamSinkConnector packaged in connect-file-{{fullDotVersion}}.jar, because these connectors are not included by default to the CLASSPATH or the plugin.path of the Connect worker (see plugin.path property for examples).
  • The important configuration options specific to standalone mode are:

    @@ -57,7 +57,7 @@

    The parameters that are configured here are intended for producers and consumers used by Kafka Connect to access the configuration, offset and status topics. For configuration of the producers used by Kafka source tasks and the consumers used by Kafka sink tasks, the same parameters can be used but need to be prefixed with producer. and consumer. respectively. The only Kafka client parameter that is inherited without a prefix from the worker configuration is bootstrap.servers, which in most cases will be sufficient, since the same cluster is often used for all purposes. A notable exception is a secured cluster, which requires extra parameters to allow connections. These parameters will need to be set up to three times in the worker configuration, once for management access, once for Kafka sources and once for Kafka sinks.

    -

    Starting with 2.3.0, client configuration overrides can be configured individually per connector by using the prefixes producer.override. and consumer.override. for Kafka sources or Kafka sinks respectively. These overrides are included with the rest of the connector's configuration properties.

    +

    Client configuration overrides can be configured individually per connector by using the prefixes producer.override. and consumer.override. for Kafka sources or Kafka sinks respectively. These overrides are included with the rest of the connector's configuration properties.

    The remaining parameters are connector configuration files. Each file may either be a Java Properties file or a JSON file containing an object with the same structure as the request body of either the POST /connectors endpoint or the PUT /connectors/{name}/config endpoint (see the OpenAPI documentation). You may include as many as you want, but all will execute within the same process (on different threads). You can also choose not to specify any connector configuration files on the command line, and instead use the REST API to create connectors at runtime after your standalone worker starts.

    @@ -69,10 +69,10 @@

    In particular, the following configuration parameters, in addition to the common settings mentioned above, are critical to set before starting your cluster:

      -
    • group.id (default connect-cluster) - unique name for the cluster, used in forming the Connect cluster group; note that this must not conflict with consumer group IDs
    • -
    • config.storage.topic (default connect-configs) - topic to use for storing connector and task configurations; note that this should be a single partition, highly replicated, compacted topic. You may need to manually create the topic to ensure the correct configuration as auto created topics may have multiple partitions or be automatically configured for deletion rather than compaction
    • -
    • offset.storage.topic (default connect-offsets) - topic to use for storing offsets; this topic should have many partitions, be replicated, and be configured for compaction
    • -
    • status.storage.topic (default connect-status) - topic to use for storing statuses; this topic can have multiple partitions, and should be replicated and configured for compaction
    • +
    • group.id - Unique name for the cluster, used in forming the Connect cluster group; note that this must not conflict with consumer group IDs
    • +
    • config.storage.topic - Name for the topic to use for storing connector and task configurations; this topic should have a single partition, be replicated, and be configured for compaction
    • +
    • offset.storage.topic - Name for the topic to use for storing offsets; this topic should have many partitions, be replicated, and be configured for compaction
    • +
    • status.storage.topic - Name for the topic to use for storing statuses; this topic can have multiple partitions, be replicated, and be configured for compaction

    Note that in distributed mode the connector configurations are not passed on the command line. Instead, use the REST API described below to create, modify, and destroy connectors.

    From 14917ae7273c77f3b17801652dff03340f76b274 Mon Sep 17 00:00:00 2001 From: Jinhe Zhang Date: Thu, 25 Sep 2025 05:06:22 -0400 Subject: [PATCH 075/100] MINOR: Handle envelope response in AutoTopicCreationManager (#20569) In the create topic request we send a CreateTopic request in an Envelope, so we need to also unpack the response correctly Reviewers: Lucas Brutschy --- .../server/AutoTopicCreationManager.scala | 154 ++++++--- .../server/AutoTopicCreationManagerTest.scala | 319 ++++++++++++++++++ 2 files changed, 427 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala b/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala index 5da400e29de..1398a8ad7c1 100644 --- a/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala +++ b/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.message.CreateTopicsRequestData import org.apache.kafka.common.message.CreateTopicsRequestData.{CreatableTopic, CreatableTopicConfig, CreatableTopicConfigCollection} import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic import org.apache.kafka.common.protocol.{ApiKeys, Errors} -import org.apache.kafka.common.requests.{CreateTopicsRequest, CreateTopicsResponse, RequestContext, RequestHeader} +import org.apache.kafka.common.requests.{AbstractResponse, CreateTopicsRequest, CreateTopicsResponse, EnvelopeResponse, RequestContext, RequestHeader} import org.apache.kafka.coordinator.group.GroupCoordinator import org.apache.kafka.coordinator.share.ShareCoordinator import org.apache.kafka.coordinator.transaction.TransactionLogConfig @@ -198,6 +198,22 @@ class DefaultAutoTopicCreationManager( .setTopics(topicsToCreate) ) + // Capture request header information for proper envelope response parsing + val requestHeaderForParsing = requestContext.map { context => + val requestVersion = + channelManager.controllerApiVersions.toScala match { + case None => + ApiKeys.CREATE_TOPICS.latestVersion() + case Some(nodeApiVersions) => + nodeApiVersions.latestUsableVersion(ApiKeys.CREATE_TOPICS) + } + + new RequestHeader(ApiKeys.CREATE_TOPICS, + requestVersion, + context.clientId, + context.correlationId) + } + val requestCompletionHandler = new ControllerRequestCompletionHandler { override def onTimeout(): Unit = { clearInflightRequests(creatableTopics) @@ -213,6 +229,33 @@ class DefaultAutoTopicCreationManager( } else { if (response.hasResponse) { response.responseBody() match { + case envelopeResponse: EnvelopeResponse => + // Unwrap the envelope response to get the actual CreateTopicsResponse + val envelopeError = envelopeResponse.error() + if (envelopeError != Errors.NONE) { + warn(s"Auto topic creation failed for ${creatableTopics.keys} with envelope error: ${envelopeError}") + } else { + requestHeaderForParsing match { + case Some(requestHeader) => + try { + // Use the captured request header for proper envelope response parsing + val createTopicsResponse = AbstractResponse.parseResponse( + envelopeResponse.responseData(), requestHeader).asInstanceOf[CreateTopicsResponse] + + createTopicsResponse.data().topics().forEach(topicResult => { + val error = Errors.forCode(topicResult.errorCode) + if (error != Errors.NONE) { + warn(s"Auto topic creation failed for ${topicResult.name} with error '${error.name}': ${topicResult.errorMessage}") + } + }) + } catch { + case e: Exception => + warn(s"Failed to parse envelope response for auto topic creation of ${creatableTopics.keys}", e) + } + case None => + warn(s"Cannot parse envelope response without original request header information") + } + } case createTopicsResponse: CreateTopicsResponse => createTopicsResponse.data().topics().forEach(topicResult => { val error = Errors.forCode(topicResult.errorCode) @@ -229,26 +272,13 @@ class DefaultAutoTopicCreationManager( } } - val request = requestContext.map { context => - val requestVersion = - channelManager.controllerApiVersions.toScala match { - case None => - // We will rely on the Metadata request to be retried in the case - // that the latest version is not usable by the controller. - ApiKeys.CREATE_TOPICS.latestVersion() - case Some(nodeApiVersions) => - nodeApiVersions.latestUsableVersion(ApiKeys.CREATE_TOPICS) - } - - // Borrow client information such as client id and correlation id from the original request, - // in order to correlate the create request with the original metadata request. - val requestHeader = new RequestHeader(ApiKeys.CREATE_TOPICS, - requestVersion, - context.clientId, - context.correlationId) - ForwardingManager.buildEnvelopeRequest(context, - createTopicsRequest.build(requestVersion).serializeWithHeader(requestHeader)) - }.getOrElse(createTopicsRequest) + val request = (requestContext, requestHeaderForParsing) match { + case (Some(context), Some(requestHeader)) => + ForwardingManager.buildEnvelopeRequest(context, + createTopicsRequest.build(requestHeader.apiVersion()).serializeWithHeader(requestHeader)) + case _ => + createTopicsRequest + } channelManager.sendRequest(request, requestCompletionHandler) @@ -364,6 +394,22 @@ class DefaultAutoTopicCreationManager( .setTopics(topicsToCreate) ) + // Capture request header information for proper envelope response parsing + val requestHeaderForParsing = requestContext.map { context => + val requestVersion = + channelManager.controllerApiVersions.toScala match { + case None => + ApiKeys.CREATE_TOPICS.latestVersion() + case Some(nodeApiVersions) => + nodeApiVersions.latestUsableVersion(ApiKeys.CREATE_TOPICS) + } + + new RequestHeader(ApiKeys.CREATE_TOPICS, + requestVersion, + context.clientId, + context.correlationId) + } + val requestCompletionHandler = new ControllerRequestCompletionHandler { override def onTimeout(): Unit = { clearInflightRequests(creatableTopics) @@ -382,36 +428,52 @@ class DefaultAutoTopicCreationManager( warn(s"Auto topic creation failed for ${creatableTopics.keys} with version mismatch exception: ${versionException.getMessage}") cacheTopicCreationErrors(creatableTopics.keys.toSet, versionException.getMessage, timeoutMs) } else { - response.responseBody() match { - case createTopicsResponse: CreateTopicsResponse => - cacheTopicCreationErrorsFromResponse(createTopicsResponse, timeoutMs) - case _ => - debug(s"Auto topic creation completed for ${creatableTopics.keys} with response ${response.responseBody}.") + if (response.hasResponse) { + response.responseBody() match { + case envelopeResponse: EnvelopeResponse => + // Unwrap the envelope response to get the actual CreateTopicsResponse + val envelopeError = envelopeResponse.error() + if (envelopeError != Errors.NONE) { + warn(s"Auto topic creation failed for ${creatableTopics.keys} with envelope error: ${envelopeError}") + cacheTopicCreationErrors(creatableTopics.keys.toSet, s"Envelope error: ${envelopeError}", timeoutMs) + } else { + requestHeaderForParsing match { + case Some(requestHeader) => + try { + // Use the captured request header for proper envelope response parsing + val createTopicsResponse = AbstractResponse.parseResponse( + envelopeResponse.responseData(), requestHeader).asInstanceOf[CreateTopicsResponse] + + cacheTopicCreationErrorsFromResponse(createTopicsResponse, timeoutMs) + } catch { + case e: Exception => + warn(s"Failed to parse envelope response for auto topic creation of ${creatableTopics.keys}", e) + cacheTopicCreationErrors(creatableTopics.keys.toSet, s"Response parsing error: ${e.getMessage}", timeoutMs) + } + case None => + warn(s"Cannot parse envelope response without original request header information") + cacheTopicCreationErrors(creatableTopics.keys.toSet, "Missing request header for envelope parsing", timeoutMs) + } + } + case createTopicsResponse: CreateTopicsResponse => + cacheTopicCreationErrorsFromResponse(createTopicsResponse, timeoutMs) + case unexpectedResponse => + warn(s"Auto topic creation request received unexpected response type: ${unexpectedResponse.getClass.getSimpleName}") + cacheTopicCreationErrors(creatableTopics.keys.toSet, s"Unexpected response type: ${unexpectedResponse.getClass.getSimpleName}", timeoutMs) + } + debug(s"Auto topic creation completed for ${creatableTopics.keys} with response ${response.responseBody}.") } } } } - val request = requestContext.map { context => - val requestVersion = - channelManager.controllerApiVersions.toScala match { - case None => - // We will rely on the Metadata request to be retried in the case - // that the latest version is not usable by the controller. - ApiKeys.CREATE_TOPICS.latestVersion() - case Some(nodeApiVersions) => - nodeApiVersions.latestUsableVersion(ApiKeys.CREATE_TOPICS) - } - - // Borrow client information such as client id and correlation id from the original request, - // in order to correlate the create request with the original metadata request. - val requestHeader = new RequestHeader(ApiKeys.CREATE_TOPICS, - requestVersion, - context.clientId, - context.correlationId) - ForwardingManager.buildEnvelopeRequest(context, - createTopicsRequest.build(requestVersion).serializeWithHeader(requestHeader)) - }.getOrElse(createTopicsRequest) + val request = (requestContext, requestHeaderForParsing) match { + case (Some(context), Some(requestHeader)) => + ForwardingManager.buildEnvelopeRequest(context, + createTopicsRequest.build(requestHeader.apiVersion()).serializeWithHeader(requestHeader)) + case _ => + createTopicsRequest + } channelManager.sendRequest(request, requestCompletionHandler) diff --git a/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala b/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala index cd17d7df2b3..b226b58c816 100644 --- a/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala @@ -33,6 +33,7 @@ import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopi import org.apache.kafka.common.network.{ClientInformation, ListenerName} import org.apache.kafka.common.protocol.{ApiKeys, ByteBufferAccessor, Errors} import org.apache.kafka.common.requests._ +import org.apache.kafka.common.requests.RequestUtils import org.apache.kafka.common.security.auth.{KafkaPrincipal, KafkaPrincipalSerde, SecurityProtocol} import org.apache.kafka.common.utils.{SecurityUtils, Utils} import org.apache.kafka.server.util.MockTime @@ -523,6 +524,324 @@ class AutoTopicCreationManagerTest { assertTrue(expiredErrors.isEmpty, "Expired errors should be proactively cleaned up") } + @Test + def testEnvelopeResponseSuccessfulParsing(): Unit = { + autoTopicCreationManager = new DefaultAutoTopicCreationManager( + config, + brokerToController, + groupCoordinator, + transactionCoordinator, + shareCoordinator, + mockTime, + topicErrorCacheCapacity = testCacheCapacity) + + val topics = Map( + "test-topic" -> new CreatableTopic().setName("test-topic").setNumPartitions(1).setReplicationFactor(1) + ) + val requestContext = initializeRequestContextWithUserPrincipal() + val timeoutMs = 5000L + + autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext, timeoutMs) + + val argumentCaptor = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) + Mockito.verify(brokerToController).sendRequest( + any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]), + argumentCaptor.capture()) + + // Create a successful CreateTopicsResponse + val createTopicsResponseData = new org.apache.kafka.common.message.CreateTopicsResponseData() + val topicResult = new org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult() + .setName("test-topic") + .setErrorCode(Errors.NONE.code()) + .setNumPartitions(1) + .setReplicationFactor(1.toShort) + createTopicsResponseData.topics().add(topicResult) + + val createTopicsResponse = new CreateTopicsResponse(createTopicsResponseData) + val requestVersion = ApiKeys.CREATE_TOPICS.latestVersion() + val correlationId = requestContext.correlationId // Use the actual correlation ID from request context + val clientId = requestContext.clientId + + // Serialize the CreateTopicsResponse with header as it would appear in an envelope + val responseHeader = new ResponseHeader(correlationId, ApiKeys.CREATE_TOPICS.responseHeaderVersion(requestVersion)) + val serializedResponse = RequestUtils.serialize(responseHeader.data(), responseHeader.headerVersion(), + createTopicsResponse.data(), requestVersion) + + // Create an EnvelopeResponse containing the serialized CreateTopicsResponse + val envelopeResponse = new EnvelopeResponse(serializedResponse, Errors.NONE) + val requestHeader = new RequestHeader(ApiKeys.ENVELOPE, 0, clientId, correlationId) + val clientResponse = new ClientResponse(requestHeader, null, null, + 0, 0, false, null, null, envelopeResponse) + + // Trigger the completion handler + argumentCaptor.getValue.onComplete(clientResponse) + + // Verify no errors were cached (successful response) + val cachedErrors = autoTopicCreationManager.getStreamsInternalTopicCreationErrors(Set("test-topic"), mockTime.milliseconds()) + assertTrue(cachedErrors.isEmpty, "No errors should be cached for successful response") + } + + @Test + def testEnvelopeResponseWithEnvelopeError(): Unit = { + autoTopicCreationManager = new DefaultAutoTopicCreationManager( + config, + brokerToController, + groupCoordinator, + transactionCoordinator, + shareCoordinator, + mockTime, + topicErrorCacheCapacity = testCacheCapacity) + + val topics = Map( + "test-topic" -> new CreatableTopic().setName("test-topic").setNumPartitions(1).setReplicationFactor(1) + ) + val requestContext = initializeRequestContextWithUserPrincipal() + val timeoutMs = 5000L + + autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext, timeoutMs) + + val argumentCaptor = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) + Mockito.verify(brokerToController).sendRequest( + any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]), + argumentCaptor.capture()) + + // Create an EnvelopeResponse with an envelope-level error + val envelopeResponse = new EnvelopeResponse(ByteBuffer.allocate(0), Errors.UNSUPPORTED_VERSION) + val requestHeader = new RequestHeader(ApiKeys.ENVELOPE, 0, requestContext.clientId, requestContext.correlationId) + val clientResponse = new ClientResponse(requestHeader, null, null, + 0, 0, false, null, null, envelopeResponse) + + // Trigger the completion handler + argumentCaptor.getValue.onComplete(clientResponse) + + // Verify the envelope error was cached + val cachedErrors = autoTopicCreationManager.getStreamsInternalTopicCreationErrors(Set("test-topic"), mockTime.milliseconds()) + assertEquals(1, cachedErrors.size) + assertTrue(cachedErrors("test-topic").contains("Envelope error: UNSUPPORTED_VERSION")) + } + + @Test + def testEnvelopeResponseParsingException(): Unit = { + autoTopicCreationManager = new DefaultAutoTopicCreationManager( + config, + brokerToController, + groupCoordinator, + transactionCoordinator, + shareCoordinator, + mockTime, + topicErrorCacheCapacity = testCacheCapacity) + + val topics = Map( + "test-topic" -> new CreatableTopic().setName("test-topic").setNumPartitions(1).setReplicationFactor(1) + ) + val requestContext = initializeRequestContextWithUserPrincipal() + val timeoutMs = 5000L + + autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext, timeoutMs) + + val argumentCaptor = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) + Mockito.verify(brokerToController).sendRequest( + any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]), + argumentCaptor.capture()) + + // Create an EnvelopeResponse with malformed response data that will cause parsing to fail + val malformedData = ByteBuffer.wrap("invalid response data".getBytes()) + val envelopeResponse = new EnvelopeResponse(malformedData, Errors.NONE) + val requestHeader = new RequestHeader(ApiKeys.ENVELOPE, 0, requestContext.clientId, requestContext.correlationId) + val clientResponse = new ClientResponse(requestHeader, null, null, + 0, 0, false, null, null, envelopeResponse) + + // Trigger the completion handler + argumentCaptor.getValue.onComplete(clientResponse) + + // Verify the parsing error was cached + val cachedErrors = autoTopicCreationManager.getStreamsInternalTopicCreationErrors(Set("test-topic"), mockTime.milliseconds()) + assertEquals(1, cachedErrors.size) + assertTrue(cachedErrors("test-topic").contains("Response parsing error:")) + } + + @Test + def testEnvelopeResponseCorrelationIdMismatch(): Unit = { + autoTopicCreationManager = new DefaultAutoTopicCreationManager( + config, + brokerToController, + groupCoordinator, + transactionCoordinator, + shareCoordinator, + mockTime, + topicErrorCacheCapacity = testCacheCapacity) + + val topics = Map( + "test-topic" -> new CreatableTopic().setName("test-topic").setNumPartitions(1).setReplicationFactor(1) + ) + val requestContext = initializeRequestContextWithUserPrincipal() + val timeoutMs = 5000L + + autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext, timeoutMs) + + val argumentCaptor = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) + Mockito.verify(brokerToController).sendRequest( + any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]), + argumentCaptor.capture()) + + // Create a CreateTopicsResponse with a different correlation ID than the request + val createTopicsResponseData = new org.apache.kafka.common.message.CreateTopicsResponseData() + val topicResult = new org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult() + .setName("test-topic") + .setErrorCode(Errors.NONE.code()) + createTopicsResponseData.topics().add(topicResult) + + val createTopicsResponse = new CreateTopicsResponse(createTopicsResponseData) + val requestVersion = ApiKeys.CREATE_TOPICS.latestVersion() + val requestCorrelationId = 123 + val responseCorrelationId = 456 // Different correlation ID + val clientId = "test-client" + + // Serialize the CreateTopicsResponse with mismatched correlation ID + val responseHeader = new ResponseHeader(responseCorrelationId, ApiKeys.CREATE_TOPICS.responseHeaderVersion(requestVersion)) + val serializedResponse = RequestUtils.serialize(responseHeader.data(), responseHeader.headerVersion(), + createTopicsResponse.data(), requestVersion) + + // Create an EnvelopeResponse containing the serialized CreateTopicsResponse + val envelopeResponse = new EnvelopeResponse(serializedResponse, Errors.NONE) + val requestHeader = new RequestHeader(ApiKeys.ENVELOPE, 0, clientId, requestCorrelationId) + val clientResponse = new ClientResponse(requestHeader, null, null, + 0, 0, false, null, null, envelopeResponse) + + // Trigger the completion handler + argumentCaptor.getValue.onComplete(clientResponse) + + // Verify the correlation ID mismatch error was cached + val cachedErrors = autoTopicCreationManager.getStreamsInternalTopicCreationErrors(Set("test-topic"), mockTime.milliseconds()) + assertEquals(1, cachedErrors.size) + assertTrue(cachedErrors("test-topic").contains("Response parsing error:")) + } + + @Test + def testEnvelopeResponseWithTopicErrors(): Unit = { + autoTopicCreationManager = new DefaultAutoTopicCreationManager( + config, + brokerToController, + groupCoordinator, + transactionCoordinator, + shareCoordinator, + mockTime, + topicErrorCacheCapacity = testCacheCapacity) + + val topics = Map( + "test-topic-1" -> new CreatableTopic().setName("test-topic-1").setNumPartitions(1).setReplicationFactor(1), + "test-topic-2" -> new CreatableTopic().setName("test-topic-2").setNumPartitions(1).setReplicationFactor(1) + ) + val requestContext = initializeRequestContextWithUserPrincipal() + val timeoutMs = 5000L + + autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext, timeoutMs) + + val argumentCaptor = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) + Mockito.verify(brokerToController).sendRequest( + any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]), + argumentCaptor.capture()) + + // Create a CreateTopicsResponse with mixed success and error results + val createTopicsResponseData = new org.apache.kafka.common.message.CreateTopicsResponseData() + + // Successful topic + val successResult = new org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult() + .setName("test-topic-1") + .setErrorCode(Errors.NONE.code()) + .setNumPartitions(1) + .setReplicationFactor(1.toShort) + createTopicsResponseData.topics().add(successResult) + + // Failed topic + val errorResult = new org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult() + .setName("test-topic-2") + .setErrorCode(Errors.TOPIC_ALREADY_EXISTS.code()) + .setErrorMessage("Topic already exists") + createTopicsResponseData.topics().add(errorResult) + + val createTopicsResponse = new CreateTopicsResponse(createTopicsResponseData) + val requestVersion = ApiKeys.CREATE_TOPICS.latestVersion() + val correlationId = requestContext.correlationId // Use the actual correlation ID from request context + val clientId = requestContext.clientId + + // Serialize the CreateTopicsResponse with header + val responseHeader = new ResponseHeader(correlationId, ApiKeys.CREATE_TOPICS.responseHeaderVersion(requestVersion)) + val serializedResponse = RequestUtils.serialize(responseHeader.data(), responseHeader.headerVersion(), + createTopicsResponse.data(), requestVersion) + + // Create an EnvelopeResponse containing the serialized CreateTopicsResponse + val envelopeResponse = new EnvelopeResponse(serializedResponse, Errors.NONE) + val requestHeader = new RequestHeader(ApiKeys.ENVELOPE, 0, clientId, correlationId) + val clientResponse = new ClientResponse(requestHeader, null, null, + 0, 0, false, null, null, envelopeResponse) + + // Trigger the completion handler + argumentCaptor.getValue.onComplete(clientResponse) + + // Verify only the failed topic was cached + val cachedErrors = autoTopicCreationManager.getStreamsInternalTopicCreationErrors( + Set("test-topic-1", "test-topic-2"), mockTime.milliseconds()) + + assertEquals(1, cachedErrors.size, s"Expected only 1 error but found: $cachedErrors") + assertTrue(cachedErrors.contains("test-topic-2")) + assertEquals("Topic already exists", cachedErrors("test-topic-2")) + } + + @Test + def testSendCreateTopicRequestEnvelopeHandling(): Unit = { + // Test the sendCreateTopicRequest method (without error caching) handles envelopes correctly + autoTopicCreationManager = new DefaultAutoTopicCreationManager( + config, + brokerToController, + groupCoordinator, + transactionCoordinator, + shareCoordinator, + mockTime, + topicErrorCacheCapacity = testCacheCapacity) + + val requestContext = initializeRequestContextWithUserPrincipal() + + // Call createTopics which uses sendCreateTopicRequest internally + autoTopicCreationManager.createTopics( + Set("test-topic"), ControllerMutationQuota.UNBOUNDED_CONTROLLER_MUTATION_QUOTA, Some(requestContext)) + + val argumentCaptor = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) + Mockito.verify(brokerToController).sendRequest( + any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]), + argumentCaptor.capture()) + + // Create a CreateTopicsResponse with an error + val createTopicsResponseData = new org.apache.kafka.common.message.CreateTopicsResponseData() + val topicResult = new org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult() + .setName("test-topic") + .setErrorCode(Errors.INVALID_TOPIC_EXCEPTION.code()) + .setErrorMessage("Invalid topic name") + createTopicsResponseData.topics().add(topicResult) + + val createTopicsResponse = new CreateTopicsResponse(createTopicsResponseData) + val requestVersion = ApiKeys.CREATE_TOPICS.latestVersion() + val correlationId = requestContext.correlationId // Use the actual correlation ID from request context + val clientId = requestContext.clientId + + // Serialize the CreateTopicsResponse with header + val responseHeader = new ResponseHeader(correlationId, ApiKeys.CREATE_TOPICS.responseHeaderVersion(requestVersion)) + val serializedResponse = RequestUtils.serialize(responseHeader.data(), responseHeader.headerVersion(), + createTopicsResponse.data(), requestVersion) + + // Create an EnvelopeResponse containing the serialized CreateTopicsResponse + val envelopeResponse = new EnvelopeResponse(serializedResponse, Errors.NONE) + val requestHeader = new RequestHeader(ApiKeys.ENVELOPE, 0, clientId, correlationId) + val clientResponse = new ClientResponse(requestHeader, null, null, + 0, 0, false, null, null, envelopeResponse) + + // Trigger the completion handler + argumentCaptor.getValue.onComplete(clientResponse) + + // For sendCreateTopicRequest, errors are not cached, but we can verify the handler completed without exception + // The test passes if no exception is thrown during envelope processing + } + @Test def testErrorCacheExpirationBasedEviction(): Unit = { // Create manager with small cache size for testing From 97c8c6b595bbff31b06e457aa08131965c845040 Mon Sep 17 00:00:00 2001 From: Sanskar Jhajharia Date: Thu, 25 Sep 2025 21:07:02 +0530 Subject: [PATCH 076/100] KAFKA-19733 Fix arguments to assertEquals() in clients module (#20586) The given PR mostly fixes the order of arguments in `assertEquals()` for the Clients module. Some minor cleanups were included with the same too. Reviewers: Chia-Ping Tsai --- .../producer/ProducerIdExpirationTest.java | 12 +-- .../ProducerSendWhileDeletionTest.java | 5 +- .../clients/ClusterConnectionStatesTest.java | 4 +- .../apache/kafka/clients/MetadataTest.java | 74 +++++++-------- .../kafka/clients/admin/ConfigTest.java | 2 +- .../CooperativeStickyAssignorTest.java | 4 +- .../clients/consumer/MockConsumerTest.java | 4 +- .../internals/AbstractStickyAssignorTest.java | 4 +- .../internals/ConsumerCoordinatorTest.java | 46 ++++----- .../internals/FetchRequestManagerTest.java | 16 ++-- .../consumer/internals/FetcherTest.java | 20 ++-- .../internals/KafkaConsumerMetricsTest.java | 4 +- .../consumer/internals/OffsetFetcherTest.java | 6 +- .../OffsetForLeaderEpochClientTest.java | 6 +- ...reamsGroupHeartbeatRequestManagerTest.java | 95 +++++-------------- .../internals/StreamsRebalanceDataTest.java | 2 +- .../metrics/AsyncConsumerMetricsTest.java | 16 ++-- .../clients/producer/MockProducerTest.java | 8 +- .../producer/internals/BufferPoolTest.java | 4 +- .../internals/KafkaProducerMetricsTest.java | 4 +- .../internals/RecordAccumulatorTest.java | 4 +- .../producer/internals/SenderTest.java | 6 +- .../internals/TransactionManagerTest.java | 26 ++--- .../feature/SupportedVersionRangeTest.java | 2 +- .../kafka/common/metrics/MetricsTest.java | 4 +- .../kafka/common/metrics/SensorTest.java | 12 +-- .../requests/UpdateFeaturesRequestTest.java | 12 +-- .../SaslServerAuthenticatorTest.java | 2 +- .../security/kerberos/KerberosRuleTest.java | 12 +-- ...arerUnsecuredLoginCallbackHandlerTest.java | 2 +- .../ClientTelemetryReporterTest.java | 2 +- .../server/policy/AlterConfigPolicyTest.java | 4 +- .../org/apache/kafka/test/TestSslUtils.java | 42 ++++---- 33 files changed, 215 insertions(+), 251 deletions(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/producer/ProducerIdExpirationTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/producer/ProducerIdExpirationTest.java index f79b3786253..a9489c88327 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/producer/ProducerIdExpirationTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/producer/ProducerIdExpirationTest.java @@ -204,10 +204,10 @@ public class ProducerIdExpirationTest { // Update the producer ID expiration ms to a very high value. admin.incrementalAlterConfigs(producerIdExpirationConfig("100000")); - cluster.brokers().values().forEach(broker -> { + cluster.brokers().values().forEach(broker -> TestUtils.waitUntilTrue(() -> broker.logManager().producerStateManagerConfig().producerIdExpirationMs() == 100000, - () -> "Configuration was not updated.", DEFAULT_MAX_WAIT_MS, 100); - }); + () -> "Configuration was not updated.", DEFAULT_MAX_WAIT_MS, 100) + ); // Send more records to send producer ID back to brokers. producer.send(new ProducerRecord<>(topic1, 0, null, "key".getBytes(), "value".getBytes())); producer.flush(); @@ -226,10 +226,10 @@ public class ProducerIdExpirationTest { kafkaBroker.awaitShutdown(); kafkaBroker.startup(); cluster.waitForReadyBrokers(); - cluster.brokers().values().forEach(broker -> { + cluster.brokers().values().forEach(broker -> TestUtils.waitUntilTrue(() -> broker.logManager().producerStateManagerConfig().producerIdExpirationMs() == 100, - () -> "Configuration was not updated.", DEFAULT_MAX_WAIT_MS, 100); - }); + () -> "Configuration was not updated.", DEFAULT_MAX_WAIT_MS, 100) + ); // Ensure producer ID expires quickly again. waitProducerIdExpire(admin); diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/producer/ProducerSendWhileDeletionTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/producer/ProducerSendWhileDeletionTest.java index 4301eecbb9c..aa93431cf63 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/producer/ProducerSendWhileDeletionTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/producer/ProducerSendWhileDeletionTest.java @@ -184,9 +184,8 @@ public class ProducerSendWhileDeletionTest { try (var producer = createProducer()) { for (int i = 1; i <= numRecords; i++) { producer.send(new ProducerRecord<>(topic, null, ("value" + i).getBytes()), - (metadata, exception) -> { - numAcks.incrementAndGet(); - }); + (metadata, exception) -> numAcks.incrementAndGet() + ); } producer.flush(); } diff --git a/clients/src/test/java/org/apache/kafka/clients/ClusterConnectionStatesTest.java b/clients/src/test/java/org/apache/kafka/clients/ClusterConnectionStatesTest.java index f647d95445f..9812f490ddd 100644 --- a/clients/src/test/java/org/apache/kafka/clients/ClusterConnectionStatesTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/ClusterConnectionStatesTest.java @@ -186,7 +186,7 @@ public class ClusterConnectionStatesTest { connectionStates.authenticationFailed(nodeId1, time.milliseconds(), new AuthenticationException("No path to CA for certificate!")); time.sleep(1000); - assertEquals(connectionStates.connectionState(nodeId1), ConnectionState.AUTHENTICATION_FAILED); + assertEquals(ConnectionState.AUTHENTICATION_FAILED, connectionStates.connectionState(nodeId1)); assertNotNull(connectionStates.authenticationException(nodeId1)); assertFalse(connectionStates.hasReadyNodes(time.milliseconds())); assertFalse(connectionStates.canConnect(nodeId1, time.milliseconds())); @@ -210,7 +210,7 @@ public class ClusterConnectionStatesTest { connectionStates.remove(nodeId1); assertTrue(connectionStates.canConnect(nodeId1, time.milliseconds())); assertFalse(connectionStates.isBlackedOut(nodeId1, time.milliseconds())); - assertEquals(connectionStates.connectionDelay(nodeId1, time.milliseconds()), 0L); + assertEquals(0L, connectionStates.connectionDelay(nodeId1, time.milliseconds())); } @Test diff --git a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java index 13c378d3983..9ac75191004 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java @@ -361,28 +361,28 @@ public class MetadataTest { // Metadata with newer epoch is handled metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), Collections.singletonMap("topic-1", 1), _tp -> 10); metadata.updateWithCurrentRequestVersion(metadataResponse, false, 1L); - assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(leaderAndEpoch.intValue(), 10)); + assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(10, leaderAndEpoch.intValue())); // Don't update to an older one assertFalse(metadata.updateLastSeenEpochIfNewer(tp, 1)); - assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(leaderAndEpoch.intValue(), 10)); + assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(10, leaderAndEpoch.intValue())); // Don't cause update if it's the same one assertFalse(metadata.updateLastSeenEpochIfNewer(tp, 10)); - assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(leaderAndEpoch.intValue(), 10)); + assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(10, leaderAndEpoch.intValue())); // Update if we see newer epoch assertTrue(metadata.updateLastSeenEpochIfNewer(tp, 12)); - assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(leaderAndEpoch.intValue(), 12)); + assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(12, leaderAndEpoch.intValue())); metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), Collections.singletonMap("topic-1", 1), _tp -> 12); metadata.updateWithCurrentRequestVersion(metadataResponse, false, 2L); - assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(leaderAndEpoch.intValue(), 12)); + assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(12, leaderAndEpoch.intValue())); // Don't overwrite metadata with older epoch metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), Collections.singletonMap("topic-1", 1), _tp -> 11); metadata.updateWithCurrentRequestVersion(metadataResponse, false, 3L); - assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(leaderAndEpoch.intValue(), 12)); + assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(12, leaderAndEpoch.intValue())); } @Test @@ -465,7 +465,7 @@ public class MetadataTest { metadata.updateWithCurrentRequestVersion(metadataResponse, false, 10L); assertNotNull(metadata.fetch().partition(tp)); assertTrue(metadata.lastSeenLeaderEpoch(tp).isPresent()); - assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 100); + assertEquals(100, metadata.lastSeenLeaderEpoch(tp).get().longValue()); } // Fake an empty ISR, but with an older epoch, should reject it @@ -475,8 +475,8 @@ public class MetadataTest { new MetadataResponse.PartitionMetadata(error, partition, leader, leaderEpoch, replicas, Collections.emptyList(), offlineReplicas), ApiKeys.METADATA.latestVersion(), Collections.emptyMap()); metadata.updateWithCurrentRequestVersion(metadataResponse, false, 20L); - assertEquals(metadata.fetch().partition(tp).inSyncReplicas().length, 1); - assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 100); + assertEquals(1, metadata.fetch().partition(tp).inSyncReplicas().length); + assertEquals(100, metadata.lastSeenLeaderEpoch(tp).get().longValue()); } // Fake an empty ISR, with same epoch, accept it @@ -486,8 +486,8 @@ public class MetadataTest { new MetadataResponse.PartitionMetadata(error, partition, leader, leaderEpoch, replicas, Collections.emptyList(), offlineReplicas), ApiKeys.METADATA.latestVersion(), Collections.emptyMap()); metadata.updateWithCurrentRequestVersion(metadataResponse, false, 20L); - assertEquals(metadata.fetch().partition(tp).inSyncReplicas().length, 0); - assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 100); + assertEquals(0, metadata.fetch().partition(tp).inSyncReplicas().length); + assertEquals(100, metadata.lastSeenLeaderEpoch(tp).get().longValue()); } // Empty metadata response, should not keep old partition but should keep the last-seen epoch @@ -495,7 +495,7 @@ public class MetadataTest { MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), Collections.emptyMap()); metadata.updateWithCurrentRequestVersion(metadataResponse, false, 20L); assertNull(metadata.fetch().partition(tp)); - assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 100); + assertEquals(100, metadata.lastSeenLeaderEpoch(tp).get().longValue()); } // Back in the metadata, with old epoch, should not get added @@ -503,7 +503,7 @@ public class MetadataTest { MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), partitionCounts, _tp -> 99); metadata.updateWithCurrentRequestVersion(metadataResponse, false, 10L); assertNull(metadata.fetch().partition(tp)); - assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 100); + assertEquals(100, metadata.lastSeenLeaderEpoch(tp).get().longValue()); } } @@ -522,31 +522,31 @@ public class MetadataTest { metadata.updateWithCurrentRequestVersion(metadataResponse, false, 10L); assertNotNull(metadata.fetch().partition(tp)); assertTrue(metadata.lastSeenLeaderEpoch(tp).isPresent()); - assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 100); + assertEquals(100, metadata.lastSeenLeaderEpoch(tp).get().longValue()); // Simulate a leader epoch from another response, like a fetch response or list offsets assertTrue(metadata.updateLastSeenEpochIfNewer(tp, 101)); // Cache of partition stays, but current partition info is not available since it's stale assertNotNull(metadata.fetch().partition(tp)); - assertEquals(Objects.requireNonNull(metadata.fetch().partitionCountForTopic("topic-1")).longValue(), 5); + assertEquals(5, Objects.requireNonNull(metadata.fetch().partitionCountForTopic("topic-1")).longValue()); assertFalse(metadata.partitionMetadataIfCurrent(tp).isPresent()); - assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 101); + assertEquals(101, metadata.lastSeenLeaderEpoch(tp).get().longValue()); // Metadata with older epoch is rejected, metadata state is unchanged metadata.updateWithCurrentRequestVersion(metadataResponse, false, 20L); assertNotNull(metadata.fetch().partition(tp)); - assertEquals(Objects.requireNonNull(metadata.fetch().partitionCountForTopic("topic-1")).longValue(), 5); + assertEquals(5, Objects.requireNonNull(metadata.fetch().partitionCountForTopic("topic-1")).longValue()); assertFalse(metadata.partitionMetadataIfCurrent(tp).isPresent()); - assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 101); + assertEquals(101, metadata.lastSeenLeaderEpoch(tp).get().longValue()); // Metadata with equal or newer epoch is accepted metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), partitionCounts, _tp -> 101); metadata.updateWithCurrentRequestVersion(metadataResponse, false, 30L); assertNotNull(metadata.fetch().partition(tp)); - assertEquals(Objects.requireNonNull(metadata.fetch().partitionCountForTopic("topic-1")).longValue(), 5); + assertEquals(5, Objects.requireNonNull(metadata.fetch().partitionCountForTopic("topic-1")).longValue()); assertTrue(metadata.partitionMetadataIfCurrent(tp).isPresent()); - assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 101); + assertEquals(101, metadata.lastSeenLeaderEpoch(tp).get().longValue()); } @Test @@ -585,18 +585,18 @@ public class MetadataTest { metadata.updateWithCurrentRequestVersion(metadataResponse, false, 0L); Cluster cluster = metadata.fetch(); - assertEquals(cluster.clusterResource().clusterId(), "dummy"); - assertEquals(cluster.nodes().size(), 4); + assertEquals("dummy", cluster.clusterResource().clusterId()); + assertEquals(4, cluster.nodes().size()); // topic counts assertEquals(cluster.invalidTopics(), Collections.singleton("topic3")); assertEquals(cluster.unauthorizedTopics(), Collections.singleton("topic4")); - assertEquals(cluster.topics().size(), 3); + assertEquals(3, cluster.topics().size()); assertEquals(cluster.internalTopics(), Collections.singleton(Topic.GROUP_METADATA_TOPIC_NAME)); // partition counts - assertEquals(cluster.partitionsForTopic("topic1").size(), 2); - assertEquals(cluster.partitionsForTopic("topic2").size(), 3); + assertEquals(2, cluster.partitionsForTopic("topic1").size()); + assertEquals(3, cluster.partitionsForTopic("topic2").size()); // Sentinel instances InetSocketAddress address = InetSocketAddress.createUnresolved("localhost", 0); @@ -798,10 +798,10 @@ public class MetadataTest { TopicPartition tp = new TopicPartition("topic-1", 0); - assertOptional(metadata.fetch().nodeIfOnline(tp, 0), node -> assertEquals(node.id(), 0)); + assertOptional(metadata.fetch().nodeIfOnline(tp, 0), node -> assertEquals(0, node.id())); assertFalse(metadata.fetch().nodeIfOnline(tp, 1).isPresent()); - assertEquals(metadata.fetch().nodeById(0).id(), 0); - assertEquals(metadata.fetch().nodeById(1).id(), 1); + assertEquals(0, metadata.fetch().nodeById(0).id()); + assertEquals(1, metadata.fetch().nodeById(1).id()); } @Test @@ -831,7 +831,7 @@ public class MetadataTest { TopicPartition tp = new TopicPartition("topic-1", 0); - assertEquals(metadata.fetch().nodeById(0).id(), 0); + assertEquals(0, metadata.fetch().nodeById(0).id()); assertNull(metadata.fetch().partition(tp)); assertEquals(metadata.fetch().nodeIfOnline(tp, 0), Optional.empty()); } @@ -955,13 +955,13 @@ public class MetadataTest { // Update the metadata to add a new topic variant, "new", which will be retained with "keep". Note this // means that all of the "old" topics should be dropped. Cluster cluster = metadata.fetch(); - assertEquals(cluster.clusterResource().clusterId(), oldClusterId); - assertEquals(cluster.nodes().size(), oldNodes); + assertEquals(oldClusterId, cluster.clusterResource().clusterId()); + assertEquals(oldNodes, cluster.nodes().size()); assertEquals(cluster.invalidTopics(), Set.of("oldInvalidTopic", "keepInvalidTopic")); assertEquals(cluster.unauthorizedTopics(), Set.of("oldUnauthorizedTopic", "keepUnauthorizedTopic")); assertEquals(cluster.topics(), Set.of("oldValidTopic", "keepValidTopic")); - assertEquals(cluster.partitionsForTopic("oldValidTopic").size(), 2); - assertEquals(cluster.partitionsForTopic("keepValidTopic").size(), 3); + assertEquals(2, cluster.partitionsForTopic("oldValidTopic").size()); + assertEquals(3, cluster.partitionsForTopic("keepValidTopic").size()); assertEquals(new HashSet<>(cluster.topicIds()), new HashSet<>(topicIds.values())); String newClusterId = "newClusterId"; @@ -990,13 +990,13 @@ public class MetadataTest { assertNull(metadataTopicIds2.get("oldValidTopic")); cluster = metadata.fetch(); - assertEquals(cluster.clusterResource().clusterId(), newClusterId); + assertEquals(newClusterId, cluster.clusterResource().clusterId()); assertEquals(cluster.nodes().size(), newNodes); assertEquals(cluster.invalidTopics(), Set.of("keepInvalidTopic", "newInvalidTopic")); assertEquals(cluster.unauthorizedTopics(), Set.of("keepUnauthorizedTopic", "newUnauthorizedTopic")); assertEquals(cluster.topics(), Set.of("keepValidTopic", "newValidTopic")); - assertEquals(cluster.partitionsForTopic("keepValidTopic").size(), 2); - assertEquals(cluster.partitionsForTopic("newValidTopic").size(), 4); + assertEquals(2, cluster.partitionsForTopic("keepValidTopic").size()); + assertEquals(4, cluster.partitionsForTopic("newValidTopic").size()); assertEquals(new HashSet<>(cluster.topicIds()), new HashSet<>(topicIds.values())); // Perform another metadata update, but this time all topic metadata should be cleared. @@ -1008,7 +1008,7 @@ public class MetadataTest { topicIds.forEach((topicName, topicId) -> assertNull(metadataTopicIds3.get(topicName))); cluster = metadata.fetch(); - assertEquals(cluster.clusterResource().clusterId(), newClusterId); + assertEquals(newClusterId, cluster.clusterResource().clusterId()); assertEquals(cluster.nodes().size(), newNodes); assertEquals(cluster.invalidTopics(), Collections.emptySet()); assertEquals(cluster.unauthorizedTopics(), Collections.emptySet()); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/ConfigTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/ConfigTest.java index d09cca7ad66..f3b1e73d72e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/ConfigTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/ConfigTest.java @@ -64,7 +64,7 @@ public class ConfigTest { assertEquals(config, config); assertEquals(config, new Config(config.entries())); assertNotEquals(new Config(Collections.singletonList(E1)), config); - assertNotEquals(config, "this"); + assertNotEquals("this", config); } @Test diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/CooperativeStickyAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/CooperativeStickyAssignorTest.java index b85d000e167..6a6aa919be1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/CooperativeStickyAssignorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/CooperativeStickyAssignorTest.java @@ -81,7 +81,7 @@ public class CooperativeStickyAssignorTest extends AbstractStickyAssignorTest { Optional encodedGeneration = ((CooperativeStickyAssignor) assignor).memberData(subscription).generation; assertTrue(encodedGeneration.isPresent()); - assertEquals(encodedGeneration.get(), DEFAULT_GENERATION); + assertEquals(DEFAULT_GENERATION, encodedGeneration.get()); int generation = 10; assignor.onAssignment(null, new ConsumerGroupMetadata("dummy-group-id", generation, "dummy-member-id", Optional.empty())); @@ -90,7 +90,7 @@ public class CooperativeStickyAssignorTest extends AbstractStickyAssignorTest { encodedGeneration = ((CooperativeStickyAssignor) assignor).memberData(subscription).generation; assertTrue(encodedGeneration.isPresent()); - assertEquals(encodedGeneration.get(), generation); + assertEquals(generation, encodedGeneration.get()); } @Test 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 647976b1d1d..6968b45a57b 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 @@ -209,9 +209,7 @@ public class MockConsumerTest { 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)); - }); + IntStream.range(0, 10).forEach(offset -> consumer.addRecord(new ConsumerRecord<>("test", 0, offset, null, null))); consumer.setMaxPollRecords(2L); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignorTest.java index fe6b4d100ff..4e9525264a0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignorTest.java @@ -1025,7 +1025,7 @@ public abstract class AbstractStickyAssignorTest { Map> assignment = assignor.assignPartitions(partitionsPerTopic, subscriptions); assertTrue(assignor.partitionsTransferringOwnership.isEmpty()); - assertEquals(assignment.values().stream().mapToInt(List::size).sum(), 1 + 100); + assertEquals(1 + 100, assignment.values().stream().mapToInt(List::size).sum()); assertEquals(Collections.singleton(consumerId), assignment.keySet()); assertTrue(isFullyBalanced(assignment)); } @@ -1043,7 +1043,7 @@ public abstract class AbstractStickyAssignorTest { assignment = assignor.assign(Collections.emptyMap(), subscriptions); assertTrue(assignor.partitionsTransferringOwnership.isEmpty()); - assertEquals(assignment.size(), 1); + assertEquals(1, assignment.size()); assertTrue(assignment.get(consumerId).isEmpty()); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index fcbbfeb8eeb..623fd765f39 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -568,13 +568,13 @@ public abstract class ConsumerCoordinatorTest { assertFalse(client.hasInFlightRequests()); // should try to find coordinator since we are commit async - coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), (offsets, exception) -> { - fail("Commit should not get responses, but got offsets:" + offsets + ", and exception:" + exception); - }); + coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), (offsets, exception) -> + fail("Commit should not get responses, but got offsets:" + offsets + ", and exception:" + exception) + ); coordinator.poll(time.timer(0)); assertTrue(coordinator.coordinatorUnknown()); assertTrue(client.hasInFlightRequests()); - assertEquals(coordinator.inFlightAsyncCommits.get(), 0); + assertEquals(0, coordinator.inFlightAsyncCommits.get()); client.respond(groupCoordinatorResponse(node, Errors.NONE)); coordinator.poll(time.timer(0)); @@ -582,7 +582,7 @@ public abstract class ConsumerCoordinatorTest { // after we've discovered the coordinator we should send // out the commit request immediately assertTrue(client.hasInFlightRequests()); - assertEquals(coordinator.inFlightAsyncCommits.get(), 1); + assertEquals(1, coordinator.inFlightAsyncCommits.get()); } @Test @@ -619,13 +619,13 @@ public abstract class ConsumerCoordinatorTest { assertFalse(coordinator.commitOffsetsSync(Collections.emptyMap(), time.timer(100L)), "expected sync commit to fail"); assertFalse(committed.get()); - assertEquals(coordinator.inFlightAsyncCommits.get(), 1); + assertEquals(1, coordinator.inFlightAsyncCommits.get()); prepareOffsetCommitRequest(singletonMap(tp, 123L), Errors.NONE); assertTrue(coordinator.commitOffsetsSync(Collections.emptyMap(), time.timer(Long.MAX_VALUE)), "expected sync commit to succeed"); assertTrue(committed.get(), "expected commit callback to be invoked"); - assertEquals(coordinator.inFlightAsyncCommits.get(), 0); + assertEquals(0, coordinator.inFlightAsyncCommits.get()); } @Test @@ -646,13 +646,13 @@ public abstract class ConsumerCoordinatorTest { "Unexpected exception cause type: " + (cause == null ? null : cause.getClass())); }); } - assertEquals(coordinator.inFlightAsyncCommits.get(), numRequests); + assertEquals(numRequests, coordinator.inFlightAsyncCommits.get()); coordinator.markCoordinatorUnknown("test cause"); consumerClient.pollNoWakeup(); coordinator.invokeCompletedOffsetCommitCallbacks(); assertEquals(numRequests, responses.get()); - assertEquals(coordinator.inFlightAsyncCommits.get(), 0); + assertEquals(0, coordinator.inFlightAsyncCommits.get()); } @Test @@ -697,7 +697,7 @@ public abstract class ConsumerCoordinatorTest { coordinator.markCoordinatorUnknown("test cause"); consumerClient.pollNoWakeup(); assertTrue(asyncCallbackInvoked.get()); - assertEquals(coordinator.inFlightAsyncCommits.get(), 0); + assertEquals(0, coordinator.inFlightAsyncCommits.get()); } @Test @@ -2350,7 +2350,7 @@ public abstract class ConsumerCoordinatorTest { MockCommitCallback secondCommitCallback = new MockCommitCallback(); coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), firstCommitCallback); coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), secondCommitCallback); - assertEquals(coordinator.inFlightAsyncCommits.get(), 2); + assertEquals(2, coordinator.inFlightAsyncCommits.get()); respondToOffsetCommitRequest(singletonMap(t1p, 100L), error); consumerClient.pollNoWakeup(); @@ -2360,7 +2360,7 @@ public abstract class ConsumerCoordinatorTest { assertTrue(coordinator.coordinatorUnknown()); assertInstanceOf(RetriableCommitFailedException.class, firstCommitCallback.exception); assertInstanceOf(RetriableCommitFailedException.class, secondCommitCallback.exception); - assertEquals(coordinator.inFlightAsyncCommits.get(), 0); + assertEquals(0, coordinator.inFlightAsyncCommits.get()); } @Test @@ -2549,7 +2549,7 @@ public abstract class ConsumerCoordinatorTest { coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.NONE); coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), mockOffsetCommitCallback); - assertEquals(coordinator.inFlightAsyncCommits.get(), 0); + assertEquals(0, coordinator.inFlightAsyncCommits.get()); coordinator.invokeCompletedOffsetCommitCallbacks(); assertEquals(invokedBeforeTest + 1, mockOffsetCommitCallback.invoked); assertNull(mockOffsetCommitCallback.exception); @@ -2580,7 +2580,7 @@ public abstract class ConsumerCoordinatorTest { coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), callback(success)); coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(success.get()); - assertEquals(coordinator.inFlightAsyncCommits.get(), 0); + assertEquals(0, coordinator.inFlightAsyncCommits.get()); } @Test @@ -2590,7 +2590,7 @@ public abstract class ConsumerCoordinatorTest { coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.COORDINATOR_NOT_AVAILABLE); coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), mockOffsetCommitCallback); - assertEquals(coordinator.inFlightAsyncCommits.get(), 0); + assertEquals(0, coordinator.inFlightAsyncCommits.get()); coordinator.invokeCompletedOffsetCommitCallbacks(); assertEquals(invokedBeforeTest + 1, mockOffsetCommitCallback.invoked); assertInstanceOf(RetriableCommitFailedException.class, mockOffsetCommitCallback.exception); @@ -2605,7 +2605,7 @@ public abstract class ConsumerCoordinatorTest { MockCommitCallback cb = new MockCommitCallback(); prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.COORDINATOR_NOT_AVAILABLE); coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), cb); - assertEquals(coordinator.inFlightAsyncCommits.get(), 0); + assertEquals(0, coordinator.inFlightAsyncCommits.get()); coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(coordinator.coordinatorUnknown()); @@ -2622,7 +2622,7 @@ public abstract class ConsumerCoordinatorTest { MockCommitCallback cb = new MockCommitCallback(); prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.NOT_COORDINATOR); coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), cb); - assertEquals(coordinator.inFlightAsyncCommits.get(), 0); + assertEquals(0, coordinator.inFlightAsyncCommits.get()); coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(coordinator.coordinatorUnknown()); @@ -2639,7 +2639,7 @@ public abstract class ConsumerCoordinatorTest { MockCommitCallback cb = new MockCommitCallback(); prepareOffsetCommitRequestDisconnect(singletonMap(t1p, 100L)); coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), cb); - assertEquals(coordinator.inFlightAsyncCommits.get(), 0); + assertEquals(0, coordinator.inFlightAsyncCommits.get()); coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(coordinator.coordinatorUnknown()); @@ -2703,7 +2703,7 @@ public abstract class ConsumerCoordinatorTest { } }; - assertEquals(coordinator.inFlightAsyncCommits.get(), 1); + assertEquals(1, coordinator.inFlightAsyncCommits.get()); thread.start(); client.waitForRequests(2, 5000); @@ -2711,7 +2711,7 @@ public abstract class ConsumerCoordinatorTest { respondToOffsetCommitRequest(singletonMap(t1p, secondOffset.offset()), Errors.NONE); thread.join(); - assertEquals(coordinator.inFlightAsyncCommits.get(), 0); + assertEquals(0, coordinator.inFlightAsyncCommits.get()); assertEquals(Arrays.asList(firstOffset, secondOffset), committedOffsets); } @@ -3100,7 +3100,7 @@ public abstract class ConsumerCoordinatorTest { assertEquals(Collections.emptySet(), subscriptions.initializingPartitions()); assertFalse(subscriptions.hasAllFetchPositions()); assertTrue(subscriptions.awaitingValidation(t1p)); - assertEquals(subscriptions.position(t1p).offset, 100L); + assertEquals(100L, subscriptions.position(t1p).offset); assertNull(subscriptions.validPosition(t1p)); } @@ -3470,7 +3470,7 @@ public abstract class ConsumerCoordinatorTest { assertThrows(FencedInstanceIdException.class, this::receiveFencedInstanceIdException); assertThrows(FencedInstanceIdException.class, () -> coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), new MockCommitCallback())); - assertEquals(coordinator.inFlightAsyncCommits.get(), 0); + assertEquals(0, coordinator.inFlightAsyncCommits.get()); assertThrows(FencedInstanceIdException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p, new OffsetAndMetadata(100L)), time.timer(Long.MAX_VALUE))); } @@ -3739,7 +3739,7 @@ public abstract class ConsumerCoordinatorTest { prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.FENCED_INSTANCE_ID); coordinator.commitOffsetsAsync(singletonMap(t1p, new OffsetAndMetadata(100L)), new MockCommitCallback()); - assertEquals(coordinator.inFlightAsyncCommits.get(), 0); + assertEquals(0, coordinator.inFlightAsyncCommits.get()); coordinator.invokeCompletedOffsetCommitCallbacks(); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java index f806ab65b6b..1378e4b53a1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java @@ -1815,7 +1815,7 @@ public class FetchRequestManagerTest { assertEquals(1, oorExceptions.size()); OffsetOutOfRangeException oor = oorExceptions.get(0); assertTrue(oor.offsetOutOfRangePartitions().containsKey(tp0)); - assertEquals(oor.offsetOutOfRangePartitions().size(), 1); + assertEquals(1, oor.offsetOutOfRangePartitions().size()); fetchRecordsInto(fetchedRecords); @@ -2359,7 +2359,7 @@ public class FetchRequestManagerTest { Map>> fetchedRecords = fetchRecords(); assertTrue(fetchedRecords.containsKey(tp0)); - assertEquals(fetchedRecords.get(tp0).size(), 2); + assertEquals(2, fetchedRecords.get(tp0).size()); } @Test @@ -2477,7 +2477,7 @@ public class FetchRequestManagerTest { Map>> fetchedRecords = fetchRecords(); assertTrue(fetchedRecords.containsKey(tp0)); - assertEquals(fetchedRecords.get(tp0).size(), 2); + assertEquals(2, fetchedRecords.get(tp0).size()); List> fetchedConsumerRecords = fetchedRecords.get(tp0); Set expectedCommittedKeys = Set.of("commit1-1", "commit1-2"); Set actuallyCommittedKeys = new HashSet<>(); @@ -2854,7 +2854,7 @@ public class FetchRequestManagerTest { Map>> fetchedRecords = fetchRecords(); assertTrue(fetchedRecords.containsKey(tp0)); - assertEquals(fetchedRecords.get(tp0).size(), 2); + assertEquals(2, fetchedRecords.get(tp0).size()); } private MemoryRecords buildRecords(long baseOffset, int count, long firstMessageId) { @@ -2939,8 +2939,8 @@ public class FetchRequestManagerTest { Map>> partitionRecords = fetchRecords(); assertTrue(partitionRecords.containsKey(tp0)); - assertEquals(subscriptions.position(tp0).offset, 3L); - assertOptional(subscriptions.position(tp0).offsetEpoch, value -> assertEquals(value.intValue(), 1)); + assertEquals(3L, subscriptions.position(tp0).offset); + assertOptional(subscriptions.position(tp0).offsetEpoch, value -> assertEquals(1, value.intValue())); } @Test @@ -3110,7 +3110,7 @@ public class FetchRequestManagerTest { fetchRecords(); Node selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); - assertEquals(selected.id(), 1); + assertEquals(1, selected.id()); assertEquals(1, sendFetches()); assertFalse(fetcher.hasCompletedFetches()); @@ -3124,7 +3124,7 @@ public class FetchRequestManagerTest { fetchRecords(); selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); - assertEquals(selected.id(), -1); + assertEquals(-1, selected.id()); } @Test diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index b85daebb8d8..ee051a42ca8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -1802,7 +1802,7 @@ public class FetcherTest { assertEquals(1, oorExceptions.size()); OffsetOutOfRangeException oor = oorExceptions.get(0); assertTrue(oor.offsetOutOfRangePartitions().containsKey(tp0)); - assertEquals(oor.offsetOutOfRangePartitions().size(), 1); + assertEquals(1, oor.offsetOutOfRangePartitions().size()); fetchRecordsInto(fetchedRecords); @@ -2346,7 +2346,7 @@ public class FetcherTest { Map>> fetchedRecords = fetchRecords(); assertTrue(fetchedRecords.containsKey(tp0)); - assertEquals(fetchedRecords.get(tp0).size(), 2); + assertEquals(2, fetchedRecords.get(tp0).size()); } @Test @@ -2464,7 +2464,7 @@ public class FetcherTest { Map>> fetchedRecords = fetchRecords(); assertTrue(fetchedRecords.containsKey(tp0)); - assertEquals(fetchedRecords.get(tp0).size(), 2); + assertEquals(2, fetchedRecords.get(tp0).size()); List> fetchedConsumerRecords = fetchedRecords.get(tp0); Set expectedCommittedKeys = Set.of("commit1-1", "commit1-2"); Set actuallyCommittedKeys = new HashSet<>(); @@ -3054,7 +3054,7 @@ public class FetcherTest { Map>> fetchedRecords = fetchRecords(); assertTrue(fetchedRecords.containsKey(tp0)); - assertEquals(fetchedRecords.get(tp0).size(), 2); + assertEquals(2, fetchedRecords.get(tp0).size()); } private MemoryRecords buildRecords(long baseOffset, int count, long firstMessageId) { @@ -3139,8 +3139,8 @@ public class FetcherTest { Map>> partitionRecords = fetchRecords(); assertTrue(partitionRecords.containsKey(tp0)); - assertEquals(subscriptions.position(tp0).offset, 3L); - assertOptional(subscriptions.position(tp0).offsetEpoch, value -> assertEquals(value.intValue(), 1)); + assertEquals(3L, subscriptions.position(tp0).offset); + assertOptional(subscriptions.position(tp0).offsetEpoch, value -> assertEquals(1, value.intValue())); } @Test @@ -3217,8 +3217,8 @@ public class FetcherTest { Map>> partitionRecords = fetchRecords(); assertTrue(partitionRecords.containsKey(tp0)); - assertEquals(subscriptions.position(tp0).offset, 3L); - assertOptional(subscriptions.position(tp0).offsetEpoch, value -> assertEquals(value.intValue(), 1)); + assertEquals(3L, subscriptions.position(tp0).offset); + assertOptional(subscriptions.position(tp0).offsetEpoch, value -> assertEquals(1, value.intValue())); } @Test @@ -3388,7 +3388,7 @@ public class FetcherTest { fetchRecords(); Node selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); - assertEquals(selected.id(), 1); + assertEquals(1, selected.id()); assertEquals(1, sendFetches()); assertFalse(fetcher.hasCompletedFetches()); @@ -3402,7 +3402,7 @@ public class FetcherTest { fetchRecords(); selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); - assertEquals(selected.id(), -1); + assertEquals(-1, selected.id()); } @Test diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/KafkaConsumerMetricsTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/KafkaConsumerMetricsTest.java index 22aa33098ee..7fa9f7e31f1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/KafkaConsumerMetricsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/KafkaConsumerMetricsTest.java @@ -99,8 +99,8 @@ class KafkaConsumerMetricsTest { private void assertMetricValue(final String name) { assertEquals( - metrics.metric(metrics.metricName(name, CONSUMER_METRIC_GROUP)).metricValue(), - (double) METRIC_VALUE + (double) METRIC_VALUE, + metrics.metric(metrics.metricName(name, CONSUMER_METRIC_GROUP)).metricValue() ); } } \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetFetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetFetcherTest.java index 3b7fe70ea4c..182900c0207 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetFetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetFetcherTest.java @@ -246,7 +246,7 @@ public class OffsetFetcherTest { assertTrue(subscriptions.hasValidPosition(tp0)); assertFalse(subscriptions.isOffsetResetNeeded(tp0)); assertTrue(subscriptions.isFetchable(tp0)); - assertEquals(subscriptions.position(tp0).offset, 5L); + assertEquals(5L, subscriptions.position(tp0).offset); } @Test @@ -395,7 +395,7 @@ public class OffsetFetcherTest { assertFalse(subscriptions.isOffsetResetNeeded(tp0)); assertTrue(metadata.updateRequested()); - assertOptional(metadata.lastSeenLeaderEpoch(tp0), epoch -> assertEquals((long) epoch, 2)); + assertOptional(metadata.lastSeenLeaderEpoch(tp0), epoch -> assertEquals(2, (long) epoch)); } @Test @@ -902,7 +902,7 @@ public class OffsetFetcherTest { ListOffsetsRequest offsetRequest = (ListOffsetsRequest) body; int epoch = offsetRequest.topics().get(0).partitions().get(0).currentLeaderEpoch(); assertTrue(epoch != ListOffsetsResponse.UNKNOWN_EPOCH, "Expected Fetcher to set leader epoch in request"); - assertEquals(epoch, 99, "Expected leader epoch to match epoch from metadata update"); + assertEquals(99, epoch, "Expected leader epoch to match epoch from metadata update"); return true; } else { fail("Should have seen ListOffsetRequest"); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetForLeaderEpochClientTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetForLeaderEpochClientTest.java index a48b32b43ef..8a3617d61c7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetForLeaderEpochClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetForLeaderEpochClientTest.java @@ -102,8 +102,8 @@ public class OffsetForLeaderEpochClientTest { assertTrue(result.partitionsToRetry().isEmpty()); assertTrue(result.endOffsets().containsKey(tp0)); assertEquals(result.endOffsets().get(tp0).errorCode(), Errors.NONE.code()); - assertEquals(result.endOffsets().get(tp0).leaderEpoch(), 1); - assertEquals(result.endOffsets().get(tp0).endOffset(), 10L); + assertEquals(1, result.endOffsets().get(tp0).leaderEpoch()); + assertEquals(10L, result.endOffsets().get(tp0).endOffset()); } @Test @@ -121,7 +121,7 @@ public class OffsetForLeaderEpochClientTest { consumerClient.pollNoWakeup(); assertTrue(future.failed()); - assertEquals(future.exception().getClass(), TopicAuthorizationException.class); + assertEquals(TopicAuthorizationException.class, future.exception().getClass()); assertTrue(((TopicAuthorizationException) future.exception()).unauthorizedTopics().contains(tp0.topic())); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManagerTest.java index 15e23875562..f4a2726b9e5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManagerTest.java @@ -439,14 +439,10 @@ class StreamsGroupHeartbeatRequestManagerTest { try ( final MockedConstruction heartbeatRequestStateMockedConstruction = mockConstruction( HeartbeatRequestState.class, - (mock, context) -> { - when(mock.heartbeatIntervalMs()).thenReturn(heartbeatIntervalMs); - }); + (mock, context) -> when(mock.heartbeatIntervalMs()).thenReturn(heartbeatIntervalMs)); final MockedConstruction pollTimerMockedConstruction = mockConstruction( Timer.class, - (mock, context) -> { - when(mock.isExpired()).thenReturn(true); - }); + (mock, context) -> when(mock.isExpired()).thenReturn(true)); final MockedConstruction heartbeatStateMockedConstruction = mockConstruction( StreamsGroupHeartbeatRequestManager.HeartbeatState.class) ) { @@ -473,14 +469,10 @@ class StreamsGroupHeartbeatRequestManagerTest { try ( final MockedConstruction heartbeatRequestStateMockedConstruction = mockConstruction( HeartbeatRequestState.class, - (mock, context) -> { - when(mock.timeToNextHeartbeatMs(time.milliseconds())).thenReturn(timeToNextHeartbeatMs); - }); + (mock, context) -> when(mock.timeToNextHeartbeatMs(time.milliseconds())).thenReturn(timeToNextHeartbeatMs)); final MockedConstruction pollTimerMockedConstruction = mockConstruction( Timer.class, - (mock, context) -> { - when(mock.isExpired()).thenReturn(true); - }); + (mock, context) -> when(mock.isExpired()).thenReturn(true)); final MockedConstruction heartbeatStateMockedConstruction = mockConstruction( StreamsGroupHeartbeatRequestManager.HeartbeatState.class) ) { @@ -508,14 +500,10 @@ class StreamsGroupHeartbeatRequestManagerTest { try ( final MockedConstruction heartbeatRequestStateMockedConstruction = mockConstruction( HeartbeatRequestState.class, - (mock, context) -> { - when(mock.canSendRequest(time.milliseconds())).thenReturn(true); - }); + (mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true)); final MockedConstruction pollTimerMockedConstruction = mockConstruction( Timer.class, - (mock, context) -> { - when(mock.isExpired()).thenReturn(true); - }) + (mock, context) -> when(mock.isExpired()).thenReturn(true)) ) { final StreamsGroupHeartbeatRequestManager heartbeatRequestManager = createStreamsGroupHeartbeatRequestManager(); final HeartbeatRequestState heartbeatRequestState = heartbeatRequestStateMockedConstruction.constructed().get(0); @@ -551,9 +539,7 @@ class StreamsGroupHeartbeatRequestManagerTest { try ( final MockedConstruction heartbeatRequestStateMockedConstruction = mockConstruction( HeartbeatRequestState.class, - (mock, context) -> { - when(mock.canSendRequest(time.milliseconds())).thenReturn(true); - }) + (mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true)) ) { final StreamsGroupHeartbeatRequestManager heartbeatRequestManager = createStreamsGroupHeartbeatRequestManager(); final HeartbeatRequestState heartbeatRequestState = heartbeatRequestStateMockedConstruction.constructed().get(0); @@ -1001,9 +987,7 @@ class StreamsGroupHeartbeatRequestManagerTest { try ( final MockedConstruction heartbeatRequestStateMockedConstruction = mockConstruction( HeartbeatRequestState.class, - (mock, context) -> { - when(mock.canSendRequest(time.milliseconds())).thenReturn(true); - }); + (mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true)); final MockedConstruction heartbeatStateMockedConstruction = mockConstruction( StreamsGroupHeartbeatRequestManager.HeartbeatState.class) ) { @@ -1032,9 +1016,7 @@ class StreamsGroupHeartbeatRequestManagerTest { try ( final MockedConstruction heartbeatRequestStateMockedConstruction = mockConstruction( HeartbeatRequestState.class, - (mock, context) -> { - when(mock.canSendRequest(time.milliseconds())).thenReturn(true); - }); + (mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true)); final MockedConstruction heartbeatStateMockedConstruction = mockConstruction( StreamsGroupHeartbeatRequestManager.HeartbeatState.class) ) { @@ -1073,9 +1055,7 @@ class StreamsGroupHeartbeatRequestManagerTest { try ( final MockedConstruction heartbeatRequestStateMockedConstruction = mockConstruction( HeartbeatRequestState.class, - (mock, context) -> { - when(mock.canSendRequest(time.milliseconds())).thenReturn(true); - }); + (mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true)); final MockedConstruction heartbeatStateMockedConstruction = mockConstruction( StreamsGroupHeartbeatRequestManager.HeartbeatState.class) ) { @@ -1111,9 +1091,7 @@ class StreamsGroupHeartbeatRequestManagerTest { try ( final MockedConstruction heartbeatRequestStateMockedConstruction = mockConstruction( HeartbeatRequestState.class, - (mock, context) -> { - when(mock.canSendRequest(time.milliseconds())).thenReturn(true); - }); + (mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true)); final MockedConstruction heartbeatStateMockedConstruction = mockConstruction( StreamsGroupHeartbeatRequestManager.HeartbeatState.class) ) { @@ -1145,9 +1123,7 @@ class StreamsGroupHeartbeatRequestManagerTest { try ( final MockedConstruction heartbeatRequestStateMockedConstruction = mockConstruction( HeartbeatRequestState.class, - (mock, context) -> { - when(mock.canSendRequest(time.milliseconds())).thenReturn(true); - }); + (mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true)); final MockedConstruction heartbeatStateMockedConstruction = mockConstruction( StreamsGroupHeartbeatRequestManager.HeartbeatState.class) ) { @@ -1173,9 +1149,7 @@ class StreamsGroupHeartbeatRequestManagerTest { try ( final MockedConstruction heartbeatRequestStateMockedConstruction = mockConstruction( HeartbeatRequestState.class, - (mock, context) -> { - when(mock.canSendRequest(time.milliseconds())).thenReturn(true); - }); + (mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true)); final MockedConstruction heartbeatStateMockedConstruction = mockConstruction( StreamsGroupHeartbeatRequestManager.HeartbeatState.class); final LogCaptureAppender logAppender = LogCaptureAppender.createAndRegister(StreamsGroupHeartbeatRequestManager.class) @@ -1212,9 +1186,7 @@ class StreamsGroupHeartbeatRequestManagerTest { try ( final MockedConstruction heartbeatRequestStateMockedConstruction = mockConstruction( HeartbeatRequestState.class, - (mock, context) -> { - when(mock.canSendRequest(time.milliseconds())).thenReturn(true); - }); + (mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true)); final MockedConstruction heartbeatStateMockedConstruction = mockConstruction( StreamsGroupHeartbeatRequestManager.HeartbeatState.class); final LogCaptureAppender logAppender = LogCaptureAppender.createAndRegister(StreamsGroupHeartbeatRequestManager.class) @@ -1261,9 +1233,7 @@ class StreamsGroupHeartbeatRequestManagerTest { try ( final MockedConstruction heartbeatRequestStateMockedConstruction = mockConstruction( HeartbeatRequestState.class, - (mock, context) -> { - when(mock.canSendRequest(time.milliseconds())).thenReturn(true); - }); + (mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true)); final MockedConstruction heartbeatStateMockedConstruction = mockConstruction( StreamsGroupHeartbeatRequestManager.HeartbeatState.class); final LogCaptureAppender logAppender = LogCaptureAppender.createAndRegister(StreamsGroupHeartbeatRequestManager.class) @@ -1312,9 +1282,7 @@ class StreamsGroupHeartbeatRequestManagerTest { try ( final MockedConstruction heartbeatRequestStateMockedConstruction = mockConstruction( HeartbeatRequestState.class, - (mock, context) -> { - when(mock.canSendRequest(time.milliseconds())).thenReturn(true); - }); + (mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true)); final MockedConstruction heartbeatStateMockedConstruction = mockConstruction( StreamsGroupHeartbeatRequestManager.HeartbeatState.class) ) { @@ -1343,9 +1311,7 @@ class StreamsGroupHeartbeatRequestManagerTest { try ( final MockedConstruction heartbeatRequestStateMockedConstruction = mockConstruction( HeartbeatRequestState.class, - (mock, context) -> { - when(mock.canSendRequest(time.milliseconds())).thenReturn(true); - }); + (mock, context) -> when(mock.canSendRequest(time.milliseconds())).thenReturn(true)); final MockedConstruction heartbeatStateMockedConstruction = mockConstruction( StreamsGroupHeartbeatRequestManager.HeartbeatState.class); final LogCaptureAppender logAppender = LogCaptureAppender.createAndRegister(StreamsGroupHeartbeatRequestManager.class) @@ -1424,14 +1390,11 @@ class StreamsGroupHeartbeatRequestManagerTest { @Test public void testMaximumTimeToWaitPollTimerExpired() { try ( - final MockedConstruction timerMockedConstruction = mockConstruction(Timer.class, (mock, context) -> { - when(mock.isExpired()).thenReturn(true); - }); + final MockedConstruction timerMockedConstruction = + mockConstruction(Timer.class, (mock, context) -> when(mock.isExpired()).thenReturn(true)); final MockedConstruction heartbeatRequestStateMockedConstruction = mockConstruction( HeartbeatRequestState.class, - (mock, context) -> { - when(mock.requestInFlight()).thenReturn(false); - }) + (mock, context) -> when(mock.requestInFlight()).thenReturn(false)) ) { final StreamsGroupHeartbeatRequestManager heartbeatRequestManager = createStreamsGroupHeartbeatRequestManager(); final Timer pollTimer = timerMockedConstruction.constructed().get(0); @@ -1450,9 +1413,7 @@ class StreamsGroupHeartbeatRequestManagerTest { final MockedConstruction timerMockedConstruction = mockConstruction(Timer.class); final MockedConstruction heartbeatRequestStateMockedConstruction = mockConstruction( HeartbeatRequestState.class, - (mock, context) -> { - when(mock.requestInFlight()).thenReturn(false); - }) + (mock, context) -> when(mock.requestInFlight()).thenReturn(false)) ) { final StreamsGroupHeartbeatRequestManager heartbeatRequestManager = createStreamsGroupHeartbeatRequestManager(); final Timer pollTimer = timerMockedConstruction.constructed().get(0); @@ -1473,9 +1434,8 @@ class StreamsGroupHeartbeatRequestManagerTest { final long remainingMs = 12L; final long timeToNextHeartbeatMs = 6L; try ( - final MockedConstruction timerMockedConstruction = mockConstruction(Timer.class, (mock, context) -> { - when(mock.remainingMs()).thenReturn(remainingMs); - }); + final MockedConstruction timerMockedConstruction = + mockConstruction(Timer.class, (mock, context) -> when(mock.remainingMs()).thenReturn(remainingMs)); final MockedConstruction heartbeatRequestStateMockedConstruction = mockConstruction( HeartbeatRequestState.class, (mock, context) -> { @@ -1500,14 +1460,11 @@ class StreamsGroupHeartbeatRequestManagerTest { public void testMaximumTimeToWaitSelectingMinimumWaitTime(final long remainingMs, final long timeToNextHeartbeatMs) { try ( - final MockedConstruction timerMockedConstruction = mockConstruction(Timer.class, (mock, context) -> { - when(mock.remainingMs()).thenReturn(remainingMs); - }); + final MockedConstruction timerMockedConstruction = + mockConstruction(Timer.class, (mock, context) -> when(mock.remainingMs()).thenReturn(remainingMs)); final MockedConstruction heartbeatRequestStateMockedConstruction = mockConstruction( HeartbeatRequestState.class, - (mock, context) -> { - when(mock.timeToNextHeartbeatMs(anyLong())).thenReturn(timeToNextHeartbeatMs); - }) + (mock, context) -> when(mock.timeToNextHeartbeatMs(anyLong())).thenReturn(timeToNextHeartbeatMs)) ) { final StreamsGroupHeartbeatRequestManager heartbeatRequestManager = createStreamsGroupHeartbeatRequestManager(); final Timer pollTimer = timerMockedConstruction.constructed().get(0); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceDataTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceDataTest.java index 9607a0e9e20..606ba0b7350 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceDataTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceDataTest.java @@ -62,7 +62,7 @@ public class StreamsRebalanceDataTest { public void testTaskIdCompareTo() { final StreamsRebalanceData.TaskId task = new StreamsRebalanceData.TaskId("subtopologyId1", 1); - assertTrue(task.compareTo(new StreamsRebalanceData.TaskId(task.subtopologyId(), task.partitionId())) == 0); + assertEquals(0, task.compareTo(new StreamsRebalanceData.TaskId(task.subtopologyId(), task.partitionId()))); assertTrue(task.compareTo(new StreamsRebalanceData.TaskId(task.subtopologyId() + "1", task.partitionId())) < 0); assertTrue(task.compareTo(new StreamsRebalanceData.TaskId(task.subtopologyId(), task.partitionId() + 1)) < 0); assertTrue(new StreamsRebalanceData.TaskId(task.subtopologyId() + "1", task.partitionId()).compareTo(task) > 0); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/metrics/AsyncConsumerMetricsTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/metrics/AsyncConsumerMetricsTest.java index d6dd8c30caa..876bc3ffa12 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/metrics/AsyncConsumerMetricsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/metrics/AsyncConsumerMetricsTest.java @@ -113,13 +113,13 @@ public class AsyncConsumerMetricsTest { // Then: assertEquals( + (double) 10, metrics.metric( metrics.metricName( "application-event-queue-size", groupName ) - ).metricValue(), - (double) 10 + ).metricValue() ); } @@ -156,13 +156,13 @@ public class AsyncConsumerMetricsTest { // Then: assertEquals( + (double) 10, metrics.metric( metrics.metricName( "unsent-requests-queue-size", groupName ) - ).metricValue(), - (double) 10 + ).metricValue() ); } @@ -187,13 +187,13 @@ public class AsyncConsumerMetricsTest { // Then: assertEquals( + (double) 10, metrics.metric( metrics.metricName( "background-event-queue-size", groupName ) - ).metricValue(), - (double) 10 + ).metricValue() ); } @@ -223,13 +223,13 @@ public class AsyncConsumerMetricsTest { private void assertMetricValue(final String name, final String groupName) { assertEquals( + (double) METRIC_VALUE, metrics.metric( metrics.metricName( name, groupName ) - ).metricValue(), - (double) METRIC_VALUE + ).metricValue() ); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java index 495578bad92..e66dcca5044 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java @@ -724,10 +724,10 @@ public class MockProducerTest { buildMockProducer(false); Future metadata = producer.send(record2, (md, exception) -> { assertNotNull(md); - assertEquals(md.offset(), -1L, "Invalid offset"); - assertEquals(md.timestamp(), RecordBatch.NO_TIMESTAMP, "Invalid timestamp"); - assertEquals(md.serializedKeySize(), -1L, "Invalid Serialized Key size"); - assertEquals(md.serializedValueSize(), -1L, "Invalid Serialized value size"); + assertEquals(-1L, md.offset(), "Invalid offset"); + assertEquals(RecordBatch.NO_TIMESTAMP, md.timestamp(), "Invalid timestamp"); + assertEquals(-1L, md.serializedKeySize(), "Invalid Serialized Key size"); + assertEquals(-1L, md.serializedValueSize(), "Invalid Serialized value size"); }); IllegalArgumentException e = new IllegalArgumentException("dummy exception"); assertTrue(producer.errorNext(e), "Complete the second request with an error"); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java index 128e15ed6c6..727368e8edd 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java @@ -219,7 +219,7 @@ public class BufferPoolTest { t1.join(); t2.join(); // both the allocate() called by threads t1 and t2 should have been interrupted and the waiters queue should be empty - assertEquals(pool.queued(), 0); + assertEquals(0, pool.queued()); } @Test @@ -332,7 +332,7 @@ public class BufferPoolTest { } - assertEquals(bufferPool.availableMemory(), 1024); + assertEquals(1024, bufferPool.availableMemory()); } public static class StressTestThread extends Thread { diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/KafkaProducerMetricsTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/KafkaProducerMetricsTest.java index 46d1ed329ee..383aa82ee2d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/KafkaProducerMetricsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/KafkaProducerMetricsTest.java @@ -121,8 +121,8 @@ class KafkaProducerMetricsTest { private void assertMetricValue(final String name) { assertEquals( - metrics.metric(metrics.metricName(name, KafkaProducerMetrics.GROUP)).metricValue(), - (double) METRIC_VALUE + (double) METRIC_VALUE, + metrics.metric(metrics.metricName(name, KafkaProducerMetrics.GROUP)).metricValue() ); } } \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java index 17d8676df1b..750440d2595 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java @@ -892,7 +892,7 @@ public class RecordAccumulatorTest { readyNodes = accum.ready(metadataCache, time.milliseconds()).readyNodes; assertEquals(Collections.singleton(node1), readyNodes, "Our partition's leader should be ready"); Map> drained = accum.drain(metadataCache, readyNodes, Integer.MAX_VALUE, time.milliseconds()); - assertEquals(drained.get(node1.id()).size(), 1, "There should be only one batch."); + assertEquals(1, drained.get(node1.id()).size(), "There should be only one batch."); time.sleep(1000L); accum.reenqueue(drained.get(node1.id()).get(0), time.milliseconds()); @@ -1788,6 +1788,6 @@ public class RecordAccumulatorTest { } // Verify all original records are accounted for (no data loss) - assertEquals(keyFoundMap.size(), 100, "All original 100 records should be present after splitting"); + assertEquals(100, keyFoundMap.size(), "All original 100 records should be present after splitting"); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index 6b2d50a52cc..cd984ac2a34 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -524,7 +524,7 @@ public class SenderTest { // Verify node is throttled a little bit. In real-life Apache Kafka, we observe that this can happen // as done above by throttling or with a disconnect / backoff. long currentPollDelay = client.pollDelayMs(nodeToThrottle, startTime); - assertEquals(currentPollDelay, throttleTimeMs); + assertEquals(throttleTimeMs, currentPollDelay); txnManager.beginTransaction(); txnManager.maybeAddPartition(tp0); @@ -3268,7 +3268,7 @@ public class SenderTest { fail("Expected abortable error to be thrown for commit"); } catch (KafkaException e) { assertTrue(transactionManager.hasAbortableError()); - assertEquals(commitResult.error().getClass(), TransactionAbortableException.class); + assertEquals(TransactionAbortableException.class, commitResult.error().getClass()); } // Abort API with TRANSACTION_ABORTABLE error should convert to Fatal error i.e. KafkaException @@ -3287,7 +3287,7 @@ public class SenderTest { // Verify TM is in FATAL_ERROR state assertTrue(transactionManager.hasFatalError()); assertFalse(e instanceof TransactionAbortableException); - assertEquals(abortResult.error().getClass(), KafkaException.class); + assertEquals(KafkaException.class, abortResult.error().getClass()); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java index 494c715df79..7815b751d80 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java @@ -623,9 +623,9 @@ public class TransactionManagerTest { @ValueSource(booleans = {true, false}) public void testDefaultSequenceNumber(boolean transactionV2Enabled) { initializeTransactionManager(Optional.empty(), transactionV2Enabled); - assertEquals(transactionManager.sequenceNumber(tp0), 0); + assertEquals(0, transactionManager.sequenceNumber(tp0)); transactionManager.incrementSequenceNumber(tp0, 3); - assertEquals(transactionManager.sequenceNumber(tp0), 3); + assertEquals(3, transactionManager.sequenceNumber(tp0)); } @ParameterizedTest @@ -849,13 +849,13 @@ public class TransactionManagerTest { @ValueSource(booleans = {true, false}) public void testSequenceNumberOverflow(boolean transactionV2Enabled) { initializeTransactionManager(Optional.empty(), transactionV2Enabled); - assertEquals(transactionManager.sequenceNumber(tp0), 0); + assertEquals(0, transactionManager.sequenceNumber(tp0)); transactionManager.incrementSequenceNumber(tp0, Integer.MAX_VALUE); - assertEquals(transactionManager.sequenceNumber(tp0), Integer.MAX_VALUE); + assertEquals(Integer.MAX_VALUE, transactionManager.sequenceNumber(tp0)); transactionManager.incrementSequenceNumber(tp0, 100); - assertEquals(transactionManager.sequenceNumber(tp0), 99); + assertEquals(99, transactionManager.sequenceNumber(tp0)); transactionManager.incrementSequenceNumber(tp0, Integer.MAX_VALUE); - assertEquals(transactionManager.sequenceNumber(tp0), 98); + assertEquals(98, transactionManager.sequenceNumber(tp0)); } @ParameterizedTest @@ -863,17 +863,17 @@ public class TransactionManagerTest { public void testProducerIdReset(boolean transactionV2Enabled) { initializeTransactionManager(Optional.empty(), transactionV2Enabled); initializeIdempotentProducerId(15L, Short.MAX_VALUE); - assertEquals(transactionManager.sequenceNumber(tp0), 0); - assertEquals(transactionManager.sequenceNumber(tp1), 0); + assertEquals(0, transactionManager.sequenceNumber(tp0)); + assertEquals(0, transactionManager.sequenceNumber(tp1)); transactionManager.incrementSequenceNumber(tp0, 3); - assertEquals(transactionManager.sequenceNumber(tp0), 3); + assertEquals(3, transactionManager.sequenceNumber(tp0)); transactionManager.incrementSequenceNumber(tp1, 3); - assertEquals(transactionManager.sequenceNumber(tp1), 3); + assertEquals(3, transactionManager.sequenceNumber(tp1)); transactionManager.requestIdempotentEpochBumpForPartition(tp0); transactionManager.bumpIdempotentEpochAndResetIdIfNeeded(); - assertEquals(transactionManager.sequenceNumber(tp0), 0); - assertEquals(transactionManager.sequenceNumber(tp1), 3); + assertEquals(0, transactionManager.sequenceNumber(tp0)); + assertEquals(3, transactionManager.sequenceNumber(tp1)); } @Test @@ -1101,7 +1101,7 @@ public class TransactionManagerTest { transactionManager.initializeTransactions(false); client.prepareUnsupportedVersionResponse(body -> { FindCoordinatorRequest findCoordinatorRequest = (FindCoordinatorRequest) body; - assertEquals(CoordinatorType.forId(findCoordinatorRequest.data().keyType()), CoordinatorType.TRANSACTION); + assertEquals(CoordinatorType.TRANSACTION, CoordinatorType.forId(findCoordinatorRequest.data().keyType())); assertTrue(findCoordinatorRequest.data().key().isEmpty()); assertEquals(1, findCoordinatorRequest.data().coordinatorKeys().size()); assertTrue(findCoordinatorRequest.data().coordinatorKeys().contains(transactionalId)); diff --git a/clients/src/test/java/org/apache/kafka/common/feature/SupportedVersionRangeTest.java b/clients/src/test/java/org/apache/kafka/common/feature/SupportedVersionRangeTest.java index 1d6679e62ce..9bc6f05106e 100644 --- a/clients/src/test/java/org/apache/kafka/common/feature/SupportedVersionRangeTest.java +++ b/clients/src/test/java/org/apache/kafka/common/feature/SupportedVersionRangeTest.java @@ -130,7 +130,7 @@ public class SupportedVersionRangeTest { public void testEquals() { SupportedVersionRange tested = new SupportedVersionRange((short) 1, (short) 1); assertEquals(tested, tested); - assertNotEquals(tested, new SupportedVersionRange((short) 1, (short) 2)); + assertNotEquals(new SupportedVersionRange((short) 1, (short) 2), tested); assertNotEquals(null, tested); } diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java index 4526b42e1aa..eda6648068c 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java @@ -684,8 +684,8 @@ public class MetricsTest { MetricName inheritedMetric = inherited.metricInstance(SampleMetrics.METRIC_WITH_INHERITED_TAGS, childTagsWithValues); Map filledOutTags = inheritedMetric.tags(); - assertEquals(filledOutTags.get("parent-tag"), "parent-tag-value", "parent-tag should be set properly"); - assertEquals(filledOutTags.get("child-tag"), "child-tag-value", "child-tag should be set properly"); + assertEquals("parent-tag-value", filledOutTags.get("parent-tag"), "parent-tag should be set properly"); + assertEquals("child-tag-value", filledOutTags.get("child-tag"), "child-tag should be set properly"); assertThrows(IllegalArgumentException.class, () -> inherited.metricInstance(SampleMetrics.METRIC_WITH_INHERITED_TAGS, parentTagsWithValues), diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/SensorTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/SensorTest.java index 5cdcebc858d..6b806c6bb7b 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/SensorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/SensorTest.java @@ -70,12 +70,12 @@ public class SensorTest { assertTrue(Sensor.RecordingLevel.DEBUG.shouldRecord(configLevel.id)); assertTrue(Sensor.RecordingLevel.TRACE.shouldRecord(configLevel.id)); - assertEquals(Sensor.RecordingLevel.valueOf(Sensor.RecordingLevel.DEBUG.toString()), - Sensor.RecordingLevel.DEBUG); - assertEquals(Sensor.RecordingLevel.valueOf(Sensor.RecordingLevel.INFO.toString()), - Sensor.RecordingLevel.INFO); - assertEquals(Sensor.RecordingLevel.valueOf(Sensor.RecordingLevel.TRACE.toString()), - Sensor.RecordingLevel.TRACE); + assertEquals(Sensor.RecordingLevel.DEBUG, + Sensor.RecordingLevel.valueOf(Sensor.RecordingLevel.DEBUG.toString())); + assertEquals(Sensor.RecordingLevel.INFO, + Sensor.RecordingLevel.valueOf(Sensor.RecordingLevel.INFO.toString())); + assertEquals(Sensor.RecordingLevel.TRACE, + Sensor.RecordingLevel.valueOf(Sensor.RecordingLevel.TRACE.toString())); } @Test diff --git a/clients/src/test/java/org/apache/kafka/common/requests/UpdateFeaturesRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/UpdateFeaturesRequestTest.java index 7450b155996..e63d1949c8a 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/UpdateFeaturesRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/UpdateFeaturesRequestTest.java @@ -84,9 +84,9 @@ public class UpdateFeaturesRequestTest { request = UpdateFeaturesRequest.parse(readable, UpdateFeaturesRequestData.LOWEST_SUPPORTED_VERSION); List updates = new ArrayList<>(request.featureUpdates()); - assertEquals(updates.size(), 2); - assertEquals(updates.get(0).upgradeType(), FeatureUpdate.UpgradeType.SAFE_DOWNGRADE); - assertEquals(updates.get(1).upgradeType(), FeatureUpdate.UpgradeType.UPGRADE); + assertEquals(2, updates.size()); + assertEquals(FeatureUpdate.UpgradeType.SAFE_DOWNGRADE, updates.get(0).upgradeType()); + assertEquals(FeatureUpdate.UpgradeType.UPGRADE, updates.get(1).upgradeType()); } @Test @@ -114,9 +114,9 @@ public class UpdateFeaturesRequestTest { request = UpdateFeaturesRequest.parse(readable, UpdateFeaturesRequestData.HIGHEST_SUPPORTED_VERSION); List updates = new ArrayList<>(request.featureUpdates()); - assertEquals(updates.size(), 2); - assertEquals(updates.get(0).upgradeType(), FeatureUpdate.UpgradeType.SAFE_DOWNGRADE); - assertEquals(updates.get(1).upgradeType(), FeatureUpdate.UpgradeType.UPGRADE); + assertEquals(2, updates.size()); + assertEquals(FeatureUpdate.UpgradeType.SAFE_DOWNGRADE, updates.get(0).upgradeType()); + assertEquals(FeatureUpdate.UpgradeType.UPGRADE, updates.get(1).upgradeType()); } diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticatorTest.java index 76c9109bd8c..710caeb150a 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticatorTest.java @@ -292,7 +292,7 @@ public class SaslServerAuthenticatorTest { when(saslServer.isComplete()).thenReturn(false).thenReturn(true); mockRequest(saslAuthenticateRequest(), transportLayer); - Throwable t = assertThrows(IllegalArgumentException.class, () -> authenticator.authenticate()); + Throwable t = assertThrows(IllegalArgumentException.class, authenticator::authenticate); assertEquals(ArithmeticException.class, t.getCause().getClass()); assertEquals("Cannot convert " + Long.MAX_VALUE + " millisecond to nanosecond due to arithmetic overflow", t.getMessage()); diff --git a/clients/src/test/java/org/apache/kafka/common/security/kerberos/KerberosRuleTest.java b/clients/src/test/java/org/apache/kafka/common/security/kerberos/KerberosRuleTest.java index 5980a0d3b3c..31c01849bc7 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/kerberos/KerberosRuleTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/kerberos/KerberosRuleTest.java @@ -26,12 +26,12 @@ public class KerberosRuleTest { @Test public void testReplaceParameters() throws BadFormatString { // positive test cases - assertEquals(KerberosRule.replaceParameters("", new String[0]), ""); - assertEquals(KerberosRule.replaceParameters("hello", new String[0]), "hello"); - assertEquals(KerberosRule.replaceParameters("", new String[]{"too", "many", "parameters", "are", "ok"}), ""); - assertEquals(KerberosRule.replaceParameters("hello", new String[]{"too", "many", "parameters", "are", "ok"}), "hello"); - assertEquals(KerberosRule.replaceParameters("hello $0", new String[]{"too", "many", "parameters", "are", "ok"}), "hello too"); - assertEquals(KerberosRule.replaceParameters("hello $0", new String[]{"no recursion $1"}), "hello no recursion $1"); + assertEquals("", KerberosRule.replaceParameters("", new String[0])); + assertEquals("hello", KerberosRule.replaceParameters("hello", new String[0])); + assertEquals("", KerberosRule.replaceParameters("", new String[]{"too", "many", "parameters", "are", "ok"})); + assertEquals("hello", KerberosRule.replaceParameters("hello", new String[]{"too", "many", "parameters", "are", "ok"})); + assertEquals("hello too", KerberosRule.replaceParameters("hello $0", new String[]{"too", "many", "parameters", "are", "ok"})); + assertEquals("hello no recursion $1", KerberosRule.replaceParameters("hello $0", new String[]{"no recursion $1"})); // negative test cases assertThrows( diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/unsecured/OAuthBearerUnsecuredLoginCallbackHandlerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/unsecured/OAuthBearerUnsecuredLoginCallbackHandlerTest.java index abbe2ef28f9..89e6de42c1d 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/unsecured/OAuthBearerUnsecuredLoginCallbackHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/unsecured/OAuthBearerUnsecuredLoginCallbackHandlerTest.java @@ -152,7 +152,7 @@ public class OAuthBearerUnsecuredLoginCallbackHandlerTest { private static void confirmCorrectValues(OAuthBearerUnsecuredJws jws, String user, long startMs, long lifetimeSeconds) throws OAuthBearerIllegalTokenException { Map header = jws.header(); - assertEquals(header.size(), 1); + assertEquals(1, header.size()); assertEquals("none", header.get("alg")); assertEquals(user != null ? user : "", jws.principalName()); assertEquals(Long.valueOf(startMs), jws.startTimeMs()); diff --git a/clients/src/test/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporterTest.java b/clients/src/test/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporterTest.java index c06e853b073..935c02dbf83 100644 --- a/clients/src/test/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporterTest.java +++ b/clients/src/test/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporterTest.java @@ -533,7 +533,7 @@ public class ClientTelemetryReporterTest { assertEquals(ClientTelemetryState.PUSH_NEEDED, telemetrySender.state()); - assertThrows(KafkaException.class, () -> telemetrySender.createRequest()); + assertThrows(KafkaException.class, telemetrySender::createRequest); assertEquals(ClientTelemetryState.TERMINATED, telemetrySender.state()); // === Test 3: After termination, no more requests === diff --git a/clients/src/test/java/org/apache/kafka/server/policy/AlterConfigPolicyTest.java b/clients/src/test/java/org/apache/kafka/server/policy/AlterConfigPolicyTest.java index 06d5a4e93eb..5a6d8b291b0 100644 --- a/clients/src/test/java/org/apache/kafka/server/policy/AlterConfigPolicyTest.java +++ b/clients/src/test/java/org/apache/kafka/server/policy/AlterConfigPolicyTest.java @@ -38,8 +38,8 @@ public class AlterConfigPolicyTest { assertEquals(requestMetadata, requestMetadata); - assertNotEquals(requestMetadata, null); - assertNotEquals(requestMetadata, new Object()); + assertNotEquals(null, requestMetadata); + assertNotEquals(new Object(), requestMetadata); assertNotEquals(requestMetadata, new RequestMetadata( new ConfigResource(Type.BROKER, "1"), Collections.singletonMap("foo", "bar") diff --git a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java index 47a26aa697d..889ebcbc607 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java @@ -444,14 +444,19 @@ public class TestSslUtils { SubjectPublicKeyInfo subPubKeyInfo = SubjectPublicKeyInfo.getInstance(keyPair.getPublic().getEncoded()); BcContentSignerBuilder signerBuilder; String keyAlgorithm = keyPair.getPublic().getAlgorithm(); - if (keyAlgorithm.equals("RSA")) - signerBuilder = new BcRSAContentSignerBuilder(sigAlgId, digAlgId); - else if (keyAlgorithm.equals("DSA")) - signerBuilder = new BcDSAContentSignerBuilder(sigAlgId, digAlgId); - else if (keyAlgorithm.equals("EC")) - signerBuilder = new BcECContentSignerBuilder(sigAlgId, digAlgId); - else - throw new IllegalArgumentException("Unsupported algorithm " + keyAlgorithm); + switch (keyAlgorithm) { + case "RSA": + signerBuilder = new BcRSAContentSignerBuilder(sigAlgId, digAlgId); + break; + case "DSA": + signerBuilder = new BcDSAContentSignerBuilder(sigAlgId, digAlgId); + break; + case "EC": + signerBuilder = new BcECContentSignerBuilder(sigAlgId, digAlgId); + break; + default: + throw new IllegalArgumentException("Unsupported algorithm " + keyAlgorithm); + } ContentSigner sigGen = signerBuilder.build(privateKeyAsymKeyParam); // Negative numbers for "days" can be used to generate expired certificates Date now = new Date(); @@ -520,14 +525,19 @@ public class TestSslUtils { SubjectPublicKeyInfo.getInstance(keyPair.getPublic().getEncoded()); BcContentSignerBuilder signerBuilder; String keyAlgorithm = keyPair.getPublic().getAlgorithm(); - if (keyAlgorithm.equals("RSA")) - signerBuilder = new BcRSAContentSignerBuilder(sigAlgId, digAlgId); - else if (keyAlgorithm.equals("DSA")) - signerBuilder = new BcDSAContentSignerBuilder(sigAlgId, digAlgId); - else if (keyAlgorithm.equals("EC")) - signerBuilder = new BcECContentSignerBuilder(sigAlgId, digAlgId); - else - throw new IllegalArgumentException("Unsupported algorithm " + keyAlgorithm); + switch (keyAlgorithm) { + case "RSA": + signerBuilder = new BcRSAContentSignerBuilder(sigAlgId, digAlgId); + break; + case "DSA": + signerBuilder = new BcDSAContentSignerBuilder(sigAlgId, digAlgId); + break; + case "EC": + signerBuilder = new BcECContentSignerBuilder(sigAlgId, digAlgId); + break; + default: + throw new IllegalArgumentException("Unsupported algorithm " + keyAlgorithm); + } ContentSigner sigGen = signerBuilder.build(privateKeyAsymKeyParam); // Negative numbers for "days" can be used to generate expired certificates Date now = new Date(); From f4e00e9cf082525d7c1d9fa62c23f2400b62bb1d Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Thu, 25 Sep 2025 23:52:46 +0800 Subject: [PATCH 077/100] MINOR: Remove unnecessary check in ReplicaManager (#20588) `setTopics` is executed at before, so the check is unnecessary. Reviewers: Ken Huang , Chia-Ping Tsai --- core/src/main/scala/kafka/server/ReplicaManager.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 070b3e544a6..10b41b88bd0 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -1261,11 +1261,11 @@ class ReplicaManager(val config: KafkaConfig, } val describeLogDirsResult = new DescribeLogDirsResponseData.DescribeLogDirsResult() - .setLogDir(absolutePath).setTopics(topicInfos) + .setLogDir(absolutePath) + .setTopics(topicInfos) .setErrorCode(Errors.NONE.code) - .setTotalBytes(totalBytes).setUsableBytes(usableBytes) - if (!topicInfos.isEmpty) - describeLogDirsResult.setTopics(topicInfos) + .setTotalBytes(totalBytes) + .setUsableBytes(usableBytes) describeLogDirsResult } catch { From 857b1e92cc5c75eb178e48613e5963755bc1b03b Mon Sep 17 00:00:00 2001 From: Kevin Wu Date: Thu, 25 Sep 2025 11:56:16 -0500 Subject: [PATCH 078/100] KAFKA-19719: --no-initial-controllers should not assume kraft.version=1 (#20551) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Just because a controller node sets --no-initial-controllers flag does not mean it is necessarily running kraft.version=1. The more precise meaning is that the controller node being formatted does not know what kraft version the cluster should be in, and therefore it is only safe to assume kraft.version=0. Only by setting --standalone,--initial-controllers, or --no-initial-controllers AND not specifying the controller.quorum.voters static config, is it known kraft.version > 0. For example, it is a valid configuration (although confusing) to run a static quorum defined by controller.quorum.voters but have all the controllers format with --no-initial-controllers. In this case, specifying --no-initial-controllers alongside a metadata version that does not support kraft.version=1 causes formatting to fail, which is a regression. Additionally, the formatter should not check the kraft.version against the release version, since kraft.version does not actually depend on any release version. It should only check the kraft.version against the static voters config/format arguments. This PR also cleans up the integration test framework to match the semantics of formatting an actual cluster. Reviewers: TengYao Chi , Kuan-Po Tseng , Chia-Ping Tsai , José Armando García Sancio --- .../main/scala/kafka/tools/StorageTool.scala | 27 ++-- .../ReconfigurableQuorumIntegrationTest.java | 51 +++++--- .../kafka/server/KRaftClusterTest.scala | 3 +- .../unit/kafka/tools/StorageToolTest.scala | 9 +- docs/ops.html | 10 +- .../kafka/metadata/storage/Formatter.java | 29 ++--- .../kafka/metadata/storage/FormatterTest.java | 103 ++++++--------- .../kafka/server/common/KRaftVersion.java | 7 +- .../common/test/KafkaClusterTestKit.java | 122 +++++++++--------- .../kafka/common/test/TestKitNodes.java | 5 - 10 files changed, 171 insertions(+), 195 deletions(-) diff --git a/core/src/main/scala/kafka/tools/StorageTool.scala b/core/src/main/scala/kafka/tools/StorageTool.scala index c342ddfe071..d8048d4d0aa 100644 --- a/core/src/main/scala/kafka/tools/StorageTool.scala +++ b/core/src/main/scala/kafka/tools/StorageTool.scala @@ -144,8 +144,9 @@ object StorageTool extends Logging { }) val initialControllers = namespace.getString("initial_controllers") val isStandalone = namespace.getBoolean("standalone") - if (!config.quorumConfig.voters().isEmpty && - (Option(initialControllers).isDefined || isStandalone)) { + val staticVotersEmpty = config.quorumConfig.voters().isEmpty + formatter.setHasDynamicQuorum(staticVotersEmpty) + if (!staticVotersEmpty && (Option(initialControllers).isDefined || isStandalone)) { throw new TerseFailure("You cannot specify " + QuorumConfig.QUORUM_VOTERS_CONFIG + " and format the node " + "with --initial-controllers or --standalone. " + @@ -158,16 +159,13 @@ object StorageTool extends Logging { if (isStandalone) { formatter.setInitialControllers(createStandaloneDynamicVoters(config)) } - if (namespace.getBoolean("no_initial_controllers")) { - formatter.setNoInitialControllersFlag(true) - } else { - if (config.processRoles.contains(ProcessRole.ControllerRole)) { - if (config.quorumConfig.voters().isEmpty && formatter.initialVoters().isEmpty) { + if (!namespace.getBoolean("no_initial_controllers") && + config.processRoles.contains(ProcessRole.ControllerRole) && + staticVotersEmpty && + formatter.initialVoters().isEmpty) { throw new TerseFailure("Because " + QuorumConfig.QUORUM_VOTERS_CONFIG + " is not set on this controller, you must specify one of the following: " + "--standalone, --initial-controllers, or --no-initial-controllers."); - } - } } Option(namespace.getList("add_scram")). foreach(scramArgs => formatter.setScramArguments(scramArgs.asInstanceOf[util.List[String]])) @@ -336,18 +334,21 @@ object StorageTool extends Logging { val reconfigurableQuorumOptions = formatParser.addMutuallyExclusiveGroup() reconfigurableQuorumOptions.addArgument("--standalone", "-s") - .help("Used to initialize a controller as a single-node dynamic quorum.") + .help("Used to initialize a controller as a single-node dynamic quorum. When setting this flag, " + + "the controller.quorum.voters config must not be set, and controller.quorum.bootstrap.servers is set instead.") .action(storeTrue()) reconfigurableQuorumOptions.addArgument("--no-initial-controllers", "-N") - .help("Used to initialize a server without a dynamic quorum topology.") + .help("Used to initialize a server without specifying a dynamic quorum. When setting this flag, " + + "the controller.quorum.voters config should not be set, and controller.quorum.bootstrap.servers is set instead.") .action(storeTrue()) reconfigurableQuorumOptions.addArgument("--initial-controllers", "-I") - .help("Used to initialize a server with a specific dynamic quorum topology. The argument " + + .help("Used to initialize a server with the specified dynamic quorum. The argument " + "is a comma-separated list of id@hostname:port:directory. The same values must be used to " + "format all nodes. For example:\n0@example.com:8082:JEXY6aqzQY-32P5TStzaFg,1@example.com:8083:" + - "MvDxzVmcRsaTz33bUuRU6A,2@example.com:8084:07R5amHmR32VDA6jHkGbTA\n") + "MvDxzVmcRsaTz33bUuRU6A,2@example.com:8084:07R5amHmR32VDA6jHkGbTA\n. When setting this flag, " + + "the controller.quorum.voters config must not be set, and controller.quorum.bootstrap.servers is set instead.") .action(store()) } diff --git a/core/src/test/java/kafka/server/ReconfigurableQuorumIntegrationTest.java b/core/src/test/java/kafka/server/ReconfigurableQuorumIntegrationTest.java index ad4193a0cb9..c67e941dd7a 100644 --- a/core/src/test/java/kafka/server/ReconfigurableQuorumIntegrationTest.java +++ b/core/src/test/java/kafka/server/ReconfigurableQuorumIntegrationTest.java @@ -84,9 +84,8 @@ public class ReconfigurableQuorumIntegrationTest { new TestKitNodes.Builder(). setNumBrokerNodes(1). setNumControllerNodes(1). - setFeature(KRaftVersion.FEATURE_NAME, KRaftVersion.KRAFT_VERSION_1.featureLevel()). build() - ).build()) { + ).setStandalone(true).build()) { cluster.format(); cluster.startup(); try (Admin admin = Admin.create(cluster.clientProperties())) { @@ -108,13 +107,23 @@ public class ReconfigurableQuorumIntegrationTest { @Test public void testRemoveController() throws Exception { - try (KafkaClusterTestKit cluster = new KafkaClusterTestKit.Builder( - new TestKitNodes.Builder(). - setNumBrokerNodes(1). - setNumControllerNodes(3). - setFeature(KRaftVersion.FEATURE_NAME, KRaftVersion.KRAFT_VERSION_1.featureLevel()). - build() - ).build()) { + final var nodes = new TestKitNodes.Builder(). + setNumBrokerNodes(1). + setNumControllerNodes(3). + build(); + + final Map initialVoters = new HashMap<>(); + for (final var controllerNode : nodes.controllerNodes().values()) { + initialVoters.put( + controllerNode.id(), + controllerNode.metadataDirectoryId() + ); + } + + try (KafkaClusterTestKit cluster = new KafkaClusterTestKit.Builder(nodes). + setInitialVoterSet(initialVoters). + build() + ) { cluster.format(); cluster.startup(); try (Admin admin = Admin.create(cluster.clientProperties())) { @@ -133,12 +142,22 @@ public class ReconfigurableQuorumIntegrationTest { @Test public void testRemoveAndAddSameController() throws Exception { - try (KafkaClusterTestKit cluster = new KafkaClusterTestKit.Builder( - new TestKitNodes.Builder(). - setNumBrokerNodes(1). - setNumControllerNodes(4). - setFeature(KRaftVersion.FEATURE_NAME, KRaftVersion.KRAFT_VERSION_1.featureLevel()). - build()).build() + final var nodes = new TestKitNodes.Builder(). + setNumBrokerNodes(1). + setNumControllerNodes(4). + build(); + + final Map initialVoters = new HashMap<>(); + for (final var controllerNode : nodes.controllerNodes().values()) { + initialVoters.put( + controllerNode.id(), + controllerNode.metadataDirectoryId() + ); + } + + try (KafkaClusterTestKit cluster = new KafkaClusterTestKit.Builder(nodes). + setInitialVoterSet(initialVoters). + build() ) { cluster.format(); cluster.startup(); @@ -173,7 +192,6 @@ public class ReconfigurableQuorumIntegrationTest { final var nodes = new TestKitNodes.Builder(). setNumBrokerNodes(1). setNumControllerNodes(3). - setFeature(KRaftVersion.FEATURE_NAME, KRaftVersion.KRAFT_VERSION_1.featureLevel()). build(); try (KafkaClusterTestKit cluster = new KafkaClusterTestKit.Builder(nodes). setConfigProp(QuorumConfig.QUORUM_AUTO_JOIN_ENABLE_CONFIG, true). @@ -199,7 +217,6 @@ public class ReconfigurableQuorumIntegrationTest { final var nodes = new TestKitNodes.Builder(). setNumBrokerNodes(1). setNumControllerNodes(3). - setFeature(KRaftVersion.FEATURE_NAME, KRaftVersion.KRAFT_VERSION_1.featureLevel()). build(); // Configure the initial voters with one voter having a different directory ID. diff --git a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala index 27b06daed21..6f552a8ebe9 100644 --- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala +++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala @@ -1035,8 +1035,7 @@ class KRaftClusterTest { val cluster = new KafkaClusterTestKit.Builder( new TestKitNodes.Builder(). setNumBrokerNodes(1). - setNumControllerNodes(1). - setFeature(KRaftVersion.FEATURE_NAME, 1.toShort).build()).build() + setNumControllerNodes(1).build()).setStandalone(true).build() try { cluster.format() cluster.startup() diff --git a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala index 1e938ea9cd8..a36ad51572a 100644 --- a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala +++ b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala @@ -605,19 +605,14 @@ Found problem: Seq("--release-version", "3.9-IV0"))).getMessage) } - @ParameterizedTest - @ValueSource(booleans = Array(false, true)) - def testFormatWithNoInitialControllersSucceedsOnController(setKraftVersionFeature: Boolean): Unit = { + @Test + def testFormatWithNoInitialControllersSucceedsOnController(): Unit = { val availableDirs = Seq(TestUtils.tempDir()) val properties = new Properties() properties.putAll(defaultDynamicQuorumProperties) properties.setProperty("log.dirs", availableDirs.mkString(",")) val stream = new ByteArrayOutputStream() val arguments = ListBuffer[String]("--release-version", "3.9-IV0", "--no-initial-controllers") - if (setKraftVersionFeature) { - arguments += "--feature" - arguments += "kraft.version=1" - } assertEquals(0, runFormatCommand(stream, properties, arguments.toSeq)) assertTrue(stream.toString(). contains("Formatting metadata directory %s".format(availableDirs.head)), diff --git a/docs/ops.html b/docs/ops.html index 803b429f18f..2d050ec76da 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -4099,14 +4099,8 @@ Feature: metadata.version SupportedMinVersion: 3.3-IV3 SupportedMaxVers

    The static versus dynamic nature of the quorum is determined at the time of formatting. Specifically, the quorum will be formatted as dynamic if controller.quorum.voters is - not present, and if the software version is Apache Kafka 3.9 or newer. If you have - followed the instructions earlier in this document, you will get a dynamic quorum.

    - - If you would like the formatting process to fail if a dynamic quorum cannot be achieved, format your - controllers using the --feature kraft.version=1. (Note that you should not supply - this flag when formatting brokers -- only when formatting controllers.)

    - -

    $ bin/kafka-storage.sh format -t KAFKA_CLUSTER_ID --feature kraft.version=1 -c controller.properties
    + not present, and one of --standalone, --initial-controllers, or --no-initial-controllers is set. + If you have followed the instructions earlier in this document, you will get a dynamic quorum.

    Note: To migrate from static voter set to dynamic voter set, please refer to the Upgrade section. diff --git a/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java b/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java index daac6630550..a036192fabb 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java @@ -131,7 +131,7 @@ public class Formatter { * The initial KIP-853 voters. */ private Optional initialControllers = Optional.empty(); - private boolean noInitialControllersFlag = false; + private boolean hasDynamicQuorum = false; public Formatter setPrintStream(PrintStream printStream) { this.printStream = printStream; @@ -217,8 +217,8 @@ public class Formatter { return this; } - public Formatter setNoInitialControllersFlag(boolean noInitialControllersFlag) { - this.noInitialControllersFlag = noInitialControllersFlag; + public Formatter setHasDynamicQuorum(boolean hasDynamicQuorum) { + this.hasDynamicQuorum = hasDynamicQuorum; return this; } @@ -227,7 +227,7 @@ public class Formatter { } boolean hasDynamicQuorum() { - return initialControllers.isPresent() || noInitialControllersFlag; + return hasDynamicQuorum; } public BootstrapMetadata bootstrapMetadata() { @@ -337,8 +337,8 @@ public class Formatter { /** * Calculate the effective feature level for kraft.version. In order to keep existing * command-line invocations of StorageTool working, we default this to 0 if no dynamic - * voter quorum arguments were provided. As a convenience, if dynamic voter quorum arguments - * were passed, we set the latest kraft.version. (Currently there is only 1 non-zero version). + * voter quorum arguments were provided. As a convenience, if the static voters config is + * empty, we set the latest kraft.version. (Currently there is only 1 non-zero version). * * @param configuredKRaftVersionLevel The configured level for kraft.version * @return The effective feature level. @@ -348,20 +348,19 @@ public class Formatter { if (configuredKRaftVersionLevel.get() == 0) { if (hasDynamicQuorum()) { throw new FormatterException( - "Cannot set kraft.version to " + - configuredKRaftVersionLevel.get() + - " if one of the flags --standalone, --initial-controllers, or --no-initial-controllers is used. " + - "For dynamic controllers support, try removing the --feature flag for kraft.version." + "Cannot set kraft.version to 0 if controller.quorum.voters is empty and one of the flags " + + "--standalone, --initial-controllers, or --no-initial-controllers is used. For dynamic " + + "controllers support, try removing the --feature flag for kraft.version." ); } } else { if (!hasDynamicQuorum()) { throw new FormatterException( - "Cannot set kraft.version to " + - configuredKRaftVersionLevel.get() + - " unless one of the flags --standalone, --initial-controllers, or --no-initial-controllers is used. " + - "For dynamic controllers support, try using one of --standalone, --initial-controllers, or " + - "--no-initial-controllers." + "Cannot set kraft.version to " + configuredKRaftVersionLevel.get() + + " unless controller.quorum.voters is empty and one of the flags --standalone, " + + "--initial-controllers, or --no-initial-controllers is used. " + + "For dynamic controllers support, try using one of --standalone, --initial-controllers, " + + "or --no-initial-controllers and removing controller.quorum.voters." ); } } diff --git a/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java b/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java index d4e13b4ccab..e57002abb8e 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java @@ -32,6 +32,7 @@ import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.EligibleLeaderReplicasVersion; import org.apache.kafka.server.common.Feature; import org.apache.kafka.server.common.GroupVersion; +import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.common.TestFeatureVersion; import org.apache.kafka.server.common.TransactionVersion; @@ -200,6 +201,7 @@ public class FormatterTest { String newDirectoryId = Uuid.randomUuid().toString(); formatter1.formatter .setInitialControllers(DynamicVoters.parse("1@localhost:8020:" + originalDirectoryId)) + .setHasDynamicQuorum(true) .run(); assertEquals("Bootstrap metadata: " + formatter1.formatter.bootstrapMetadata() + "\nFormatting dynamic metadata voter directory " + testEnv.directory(0) + @@ -422,13 +424,14 @@ public class FormatterTest { try (TestEnv testEnv = new TestEnv(2)) { FormatterContext formatter1 = testEnv.newFormatter(); if (specifyKRaftVersion) { - formatter1.formatter.setFeatureLevel("kraft.version", (short) 1); + formatter1.formatter.setFeatureLevel(KRaftVersion.FEATURE_NAME, (short) 1); } formatter1.formatter.setUnstableFeatureVersionsEnabled(true); formatter1.formatter.setInitialControllers(DynamicVoters. parse("1@localhost:8020:4znU-ou9Taa06bmEJxsjnw")); + formatter1.formatter.setHasDynamicQuorum(true); formatter1.formatter.run(); - assertEquals((short) 1, formatter1.formatter.featureLevels.getOrDefault("kraft.version", (short) 0)); + assertEquals((short) 1, formatter1.formatter.featureLevels.get(KRaftVersion.FEATURE_NAME)); assertEquals(List.of( "Bootstrap metadata: " + formatter1.formatter.bootstrapMetadata(), String.format("Formatting data directory %s with %s %s.", @@ -456,49 +459,66 @@ public class FormatterTest { public void testFormatWithInitialVotersFailsWithOlderKraftVersion() throws Exception { try (TestEnv testEnv = new TestEnv(2)) { FormatterContext formatter1 = testEnv.newFormatter(); - formatter1.formatter.setFeatureLevel("kraft.version", (short) 0); + formatter1.formatter.setFeatureLevel(KRaftVersion.FEATURE_NAME, (short) 0); formatter1.formatter.setUnstableFeatureVersionsEnabled(true); formatter1.formatter.setInitialControllers(DynamicVoters. parse("1@localhost:8020:4znU-ou9Taa06bmEJxsjnw")); + formatter1.formatter.setHasDynamicQuorum(true); assertTrue(formatter1.formatter.hasDynamicQuorum()); assertEquals( - "Cannot set kraft.version to 0 if one of the flags --standalone, --initial-controllers, or " + - "--no-initial-controllers is used. For dynamic controllers support, try removing the " + - "--feature flag for kraft.version.", + "Cannot set kraft.version to 0 if controller.quorum.voters is empty " + + "and one of the flags --standalone, --initial-controllers, or --no-initial-controllers is used. " + + "For dynamic controllers support, try removing the --feature flag for kraft.version.", assertThrows(FormatterException.class, formatter1.formatter::run).getMessage() ); } } @Test - public void testFormatWithoutInitialVotersFailsWithNewerKraftVersion() throws Exception { + public void testFormatWithStaticQuorumFailsWithNewerKraftVersion() throws Exception { try (TestEnv testEnv = new TestEnv(2)) { FormatterContext formatter1 = testEnv.newFormatter(); - formatter1.formatter.setFeatureLevel("kraft.version", (short) 1); + formatter1.formatter.setFeatureLevel(KRaftVersion.FEATURE_NAME, (short) 1); formatter1.formatter.setUnstableFeatureVersionsEnabled(true); assertFalse(formatter1.formatter.hasDynamicQuorum()); assertEquals( - "Cannot set kraft.version to 1 unless one of the flags --standalone, --initial-controllers, or " + - "--no-initial-controllers is used. For dynamic controllers support, try using one of " + - "--standalone, --initial-controllers, or --no-initial-controllers.", + "Cannot set kraft.version to 1 unless controller.quorum.voters is empty and " + + "one of the flags --standalone, --initial-controllers, or --no-initial-controllers is used. " + + "For dynamic controllers support, try using one of --standalone, --initial-controllers, " + + "or --no-initial-controllers and removing controller.quorum.voters.", assertThrows(FormatterException.class, formatter1.formatter::run).getMessage() ); } } @Test - public void testFormatWithInitialVotersFailsWithOlderMetadataVersion() throws Exception { + public void testFormatWithInitialVotersWithOlderMetadataVersion() throws Exception { try (TestEnv testEnv = new TestEnv(2)) { FormatterContext formatter1 = testEnv.newFormatter(); formatter1.formatter.setReleaseVersion(MetadataVersion.IBP_3_8_IV0); - formatter1.formatter.setFeatureLevel("kraft.version", (short) 1); + formatter1.formatter.setFeatureLevel(KRaftVersion.FEATURE_NAME, (short) 1); formatter1.formatter.setInitialControllers(DynamicVoters. parse("1@localhost:8020:4znU-ou9Taa06bmEJxsjnw")); formatter1.formatter.setUnstableFeatureVersionsEnabled(true); - assertEquals("kraft.version could not be set to 1 because it depends on " + - "metadata.version level 21", - assertThrows(IllegalArgumentException.class, - formatter1.formatter::run).getMessage()); + formatter1.formatter.setHasDynamicQuorum(true); + formatter1.formatter.run(); + assertEquals((short) 1, formatter1.formatter.featureLevels.get(KRaftVersion.FEATURE_NAME)); + } + } + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testFormatWithNoInitialControllersWithOlderMetadataVersion(boolean hasDynamicQuorum) throws Exception { + try (TestEnv testEnv = new TestEnv(2)) { + FormatterContext formatter1 = testEnv.newFormatter(); + formatter1.formatter.setReleaseVersion(MetadataVersion.IBP_3_8_IV0); + formatter1.formatter.setHasDynamicQuorum(hasDynamicQuorum); + formatter1.formatter.run(); + if (hasDynamicQuorum) { + assertEquals((short) 1, formatter1.formatter.featureLevels.get(KRaftVersion.FEATURE_NAME)); + } else { + assertEquals((short) 0, formatter1.formatter.featureLevels.get(KRaftVersion.FEATURE_NAME)); + } } } @@ -519,6 +539,7 @@ public class FormatterTest { formatter1.formatter.setFeatureLevel(EligibleLeaderReplicasVersion.FEATURE_NAME, (short) 1); formatter1.formatter.setInitialControllers(DynamicVoters. parse("1@localhost:8020:4znU-ou9Taa06bmEJxsjnw")); + formatter1.formatter.setHasDynamicQuorum(true); if (metadataVersion.isAtLeast(MetadataVersion.IBP_4_0_IV1)) { assertDoesNotThrow(formatter1.formatter::run); } else { @@ -530,20 +551,14 @@ public class FormatterTest { } } - @ParameterizedTest - @ValueSource(booleans = {false, true}) - public void testFormatWithNoInitialControllers(boolean specifyKRaftVersion) throws Exception { + @Test + public void testFormatWithNoInitialControllers() throws Exception { try (TestEnv testEnv = new TestEnv(2)) { FormatterContext formatter1 = testEnv.newFormatter(); - if (specifyKRaftVersion) { - formatter1.formatter.setFeatureLevel("kraft.version", (short) 1); - } formatter1.formatter.setUnstableFeatureVersionsEnabled(true); - formatter1.formatter.setNoInitialControllersFlag(true); - assertTrue(formatter1.formatter.hasDynamicQuorum()); - + assertFalse(formatter1.formatter.hasDynamicQuorum()); formatter1.formatter.run(); - assertEquals((short) 1, formatter1.formatter.featureLevels.getOrDefault("kraft.version", (short) 0)); + assertEquals((short) 0, formatter1.formatter.featureLevels.get(KRaftVersion.FEATURE_NAME)); assertEquals(List.of( "Bootstrap metadata: " + formatter1.formatter.bootstrapMetadata(), String.format("Formatting data directory %s with %s %s.", @@ -564,38 +579,4 @@ public class FormatterTest { assertNotNull(logDirProps1); } } - - @Test - public void testFormatWithoutNoInitialControllersFailsWithNewerKraftVersion() throws Exception { - try (TestEnv testEnv = new TestEnv(2)) { - FormatterContext formatter1 = testEnv.newFormatter(); - formatter1.formatter.setFeatureLevel("kraft.version", (short) 1); - formatter1.formatter.setUnstableFeatureVersionsEnabled(true); - formatter1.formatter.setNoInitialControllersFlag(false); - assertFalse(formatter1.formatter.hasDynamicQuorum()); - assertEquals( - "Cannot set kraft.version to 1 unless one of the flags --standalone, --initial-controllers, or " + - "--no-initial-controllers is used. For dynamic controllers support, try using one of " + - "--standalone, --initial-controllers, or --no-initial-controllers.", - assertThrows(FormatterException.class, formatter1.formatter::run).getMessage() - ); - } - } - - @Test - public void testFormatWithNoInitialControllersFailsWithOlderKraftVersion() throws Exception { - try (TestEnv testEnv = new TestEnv(2)) { - FormatterContext formatter1 = testEnv.newFormatter(); - formatter1.formatter.setFeatureLevel("kraft.version", (short) 0); - formatter1.formatter.setUnstableFeatureVersionsEnabled(true); - formatter1.formatter.setNoInitialControllersFlag(true); - assertTrue(formatter1.formatter.hasDynamicQuorum()); - assertEquals( - "Cannot set kraft.version to 0 if one of the flags --standalone, --initial-controllers, or " + - "--no-initial-controllers is used. For dynamic controllers support, try removing the " + - "--feature flag for kraft.version.", - assertThrows(FormatterException.class, formatter1.formatter::run).getMessage() - ); - } - } } diff --git a/server-common/src/main/java/org/apache/kafka/server/common/KRaftVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/KRaftVersion.java index 463cc2a015c..d797880c776 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/KRaftVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/KRaftVersion.java @@ -72,12 +72,7 @@ public enum KRaftVersion implements FeatureVersion { @Override public Map dependencies() { - if (this.featureLevel == 0) { - return Map.of(); - } else { - return Map.of( - MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_9_IV0.featureLevel()); - } + return Map.of(); } public boolean isAtLeast(KRaftVersion otherVersion) { 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 59041c7a66a..a8440ff32fd 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 @@ -45,7 +45,6 @@ import org.apache.kafka.raft.DynamicVoters; import org.apache.kafka.raft.MetadataLogConfig; import org.apache.kafka.raft.QuorumConfig; import org.apache.kafka.server.common.ApiMessageAndVersion; -import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.server.config.KRaftConfigs; import org.apache.kafka.server.config.ServerConfigs; import org.apache.kafka.server.fault.FaultHandler; @@ -182,18 +181,31 @@ public class KafkaClusterTestKit implements AutoCloseable { props.putIfAbsent(INTER_BROKER_LISTENER_NAME_CONFIG, brokerListenerName); props.putIfAbsent(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, controllerListenerName); - StringBuilder quorumVoterStringBuilder = new StringBuilder(); - String prefix = ""; - for (int nodeId : nodes.controllerNodes().keySet()) { - quorumVoterStringBuilder.append(prefix). - append(nodeId). - append("@"). - append("localhost"). - append(":"). - append(socketFactoryManager.getOrCreatePortForListener(nodeId, controllerListenerName)); - prefix = ","; + if (!standalone && initialVoterSet.isEmpty()) { + StringBuilder quorumVoterStringBuilder = new StringBuilder(); + String prefix = ""; + for (int nodeId : nodes.controllerNodes().keySet()) { + quorumVoterStringBuilder.append(prefix). + append(nodeId). + append("@"). + append("localhost"). + append(":"). + append(socketFactoryManager.getOrCreatePortForListener(nodeId, controllerListenerName)); + prefix = ","; + } + props.put(QuorumConfig.QUORUM_VOTERS_CONFIG, quorumVoterStringBuilder.toString()); + } else { + StringBuilder bootstrapServersStringBuilder = new StringBuilder(); + String prefix = ""; + for (int nodeId : nodes.controllerNodes().keySet()) { + bootstrapServersStringBuilder.append(prefix). + append("localhost"). + append(":"). + append(socketFactoryManager.getOrCreatePortForListener(nodeId, controllerListenerName)); + prefix = ","; + } + props.put(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, bootstrapServersStringBuilder.toString()); } - props.put(QuorumConfig.QUORUM_VOTERS_CONFIG, quorumVoterStringBuilder.toString()); // reduce log cleaner offset map memory usage props.putIfAbsent(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "2097152"); @@ -277,7 +289,7 @@ public class KafkaClusterTestKit implements AutoCloseable { Time.SYSTEM, new Metrics(), CompletableFuture.completedFuture(QuorumConfig.parseVoterConnections(config.quorumConfig().voters())), - List.of(), + QuorumConfig.parseBootstrapServers(config.quorumConfig().bootstrapServers()), faultHandlerFactory, socketFactoryManager.getOrCreateSocketFactory(node.id()) ); @@ -305,7 +317,7 @@ public class KafkaClusterTestKit implements AutoCloseable { Time.SYSTEM, new Metrics(), CompletableFuture.completedFuture(QuorumConfig.parseVoterConnections(config.quorumConfig().voters())), - List.of(), + QuorumConfig.parseBootstrapServers(config.quorumConfig().bootstrapServers()), faultHandlerFactory, socketFactoryManager.getOrCreateSocketFactory(node.id()) ); @@ -467,8 +479,6 @@ public class KafkaClusterTestKit implements AutoCloseable { return; } formatter.setReleaseVersion(nodes.bootstrapMetadata().metadataVersion()); - formatter.setFeatureLevel(KRaftVersion.FEATURE_NAME, - nodes.bootstrapMetadata().featureLevel(KRaftVersion.FEATURE_NAME)); formatter.setUnstableFeatureVersionsEnabled(true); formatter.setIgnoreFormatted(false); formatter.setControllerListenerName(controllerListenerName); @@ -477,53 +487,43 @@ public class KafkaClusterTestKit implements AutoCloseable { } else { formatter.setMetadataLogDirectory(Optional.empty()); } - if (nodes.bootstrapMetadata().featureLevel(KRaftVersion.FEATURE_NAME) > 0) { - StringBuilder dynamicVotersBuilder = new StringBuilder(); - String prefix = ""; - if (standalone) { - if (nodeId == TestKitDefaults.CONTROLLER_ID_OFFSET) { - final var controllerNode = nodes.controllerNodes().get(nodeId); - dynamicVotersBuilder.append( - String.format( - "%d@localhost:%d:%s", - controllerNode.id(), - socketFactoryManager. - getOrCreatePortForListener(controllerNode.id(), controllerListenerName), - controllerNode.metadataDirectoryId() - ) - ); - formatter.setInitialControllers(DynamicVoters.parse(dynamicVotersBuilder.toString())); - } else { - formatter.setNoInitialControllersFlag(true); - } - } else if (initialVoterSet.isPresent()) { - for (final var controllerNode : initialVoterSet.get().entrySet()) { - final var voterId = controllerNode.getKey(); - final var voterDirectoryId = controllerNode.getValue(); - dynamicVotersBuilder.append(prefix); - prefix = ","; - dynamicVotersBuilder.append( - String.format( - "%d@localhost:%d:%s", - voterId, - socketFactoryManager. - getOrCreatePortForListener(voterId, controllerListenerName), - voterDirectoryId - ) - ); - } - formatter.setInitialControllers(DynamicVoters.parse(dynamicVotersBuilder.toString())); - } else { - for (TestKitNode controllerNode : nodes.controllerNodes().values()) { - int port = socketFactoryManager. - getOrCreatePortForListener(controllerNode.id(), controllerListenerName); - dynamicVotersBuilder.append(prefix); - prefix = ","; - dynamicVotersBuilder.append(String.format("%d@localhost:%d:%s", - controllerNode.id(), port, controllerNode.metadataDirectoryId())); - } + StringBuilder dynamicVotersBuilder = new StringBuilder(); + String prefix = ""; + if (standalone) { + if (nodeId == TestKitDefaults.BROKER_ID_OFFSET + TestKitDefaults.CONTROLLER_ID_OFFSET) { + final var controllerNode = nodes.controllerNodes().get(nodeId); + dynamicVotersBuilder.append( + String.format( + "%d@localhost:%d:%s", + controllerNode.id(), + socketFactoryManager. + getOrCreatePortForListener(controllerNode.id(), controllerListenerName), + controllerNode.metadataDirectoryId() + ) + ); formatter.setInitialControllers(DynamicVoters.parse(dynamicVotersBuilder.toString())); } + // when the nodeId != TestKitDefaults.CONTROLLER_ID_OFFSET, the node is formatting with + // the --no-initial-controllers flag + formatter.setHasDynamicQuorum(true); + } else if (initialVoterSet.isPresent()) { + for (final var controllerNode : initialVoterSet.get().entrySet()) { + final var voterId = controllerNode.getKey(); + final var voterDirectoryId = controllerNode.getValue(); + dynamicVotersBuilder.append(prefix); + prefix = ","; + dynamicVotersBuilder.append( + String.format( + "%d@localhost:%d:%s", + voterId, + socketFactoryManager. + getOrCreatePortForListener(voterId, controllerListenerName), + voterDirectoryId + ) + ); + } + formatter.setInitialControllers(DynamicVoters.parse(dynamicVotersBuilder.toString())); + formatter.setHasDynamicQuorum(true); } formatter.run(); } catch (Exception e) { diff --git a/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/TestKitNodes.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/TestKitNodes.java index a9667dbd631..3622430f487 100644 --- a/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/TestKitNodes.java +++ b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/TestKitNodes.java @@ -93,11 +93,6 @@ public class TestKitNodes { return this; } - public Builder setFeature(String featureName, short level) { - this.bootstrapMetadata = bootstrapMetadata.copyWithFeatureRecord(featureName, level); - return this; - } - public Builder setCombined(boolean combined) { this.combined = combined; return this; From 9e9d2a23efa32deea6d930cd8a4bd07048656ac4 Mon Sep 17 00:00:00 2001 From: Lianet Magrans <98415067+lianetm@users.noreply.github.com> Date: Thu, 25 Sep 2025 15:29:35 -0400 Subject: [PATCH 079/100] MINOR: fix flaky sys test for static membership (#20594) Fixing flakiness seen on this test, where static consumers could not join as expected after shutting down previous consumers with the same instance ID, and logs showed `UnreleasedInstanceIdException`. I expect the flakiness could happen if a consumer with instanceId1 is closed but not effectively removed from the group due to leave group fail/delayed (the leave group request is sent on a best effort, not retried if fails or times out). Fix by adding check to ensure the group is empty before attempting to reuse the instance ID Reviewers: Matthias J. Sax --- tests/kafkatest/tests/client/consumer_test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/kafkatest/tests/client/consumer_test.py b/tests/kafkatest/tests/client/consumer_test.py index c6ecfc371a6..cb964e4c303 100644 --- a/tests/kafkatest/tests/client/consumer_test.py +++ b/tests/kafkatest/tests/client/consumer_test.py @@ -313,7 +313,7 @@ class OffsetValidationTest(VerifiableConsumerTest): num_rebalances = consumer.num_rebalances() conflict_consumer.start() if group_protocol == consumer_group.classic_group_protocol: - # Classic protocol: conflicting members should join, and the intial ones with conflicting instance id should fail. + # Classic protocol: conflicting members should join, and the initial ones with conflicting instance id should fail. self.await_members(conflict_consumer, num_conflict_consumers) self.await_members(consumer, len(consumer.nodes) - num_conflict_consumers) @@ -332,6 +332,11 @@ class OffsetValidationTest(VerifiableConsumerTest): wait_until(lambda: len(consumer.dead_nodes()) == len(consumer.nodes), timeout_sec=60, err_msg="Timed out waiting for the consumer to shutdown") + # Wait until the group becomes empty to ensure the instance ID is released. + # We use the 50-second timeout because the consumer session timeout is 45 seconds. + wait_until(lambda: self.group_id in self.kafka.list_consumer_groups(state="empty"), + timeout_sec=50, + err_msg="Timed out waiting for the consumers to be removed from the group.") conflict_consumer.start() self.await_members(conflict_consumer, num_conflict_consumers) From 527467d0538924cb4a5b99be890ef54c9063ee8a Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Fri, 26 Sep 2025 16:31:14 +0800 Subject: [PATCH 080/100] KAFKA-18356: Explicitly set up instrumentation for inline mocking (Java 21+) (#18339) Reviewers: Mickael Maison , Ismael Juma --- build.gradle | 39 ++++++++++++++++++++++++++++++++++++--- 1 file changed, 36 insertions(+), 3 deletions(-) diff --git a/build.gradle b/build.gradle index 753a86cfc35..29537dd77dc 100644 --- a/build.gradle +++ b/build.gradle @@ -157,7 +157,7 @@ ext { libs.log4j2Api, libs.log4j2Core ] - + } allprojects { @@ -484,11 +484,36 @@ subprojects { } } + // Workaround for Mockito Java Agent restrictions in Java 21+ + // Starting with Java 21, the JDK restricts libraries from attaching a Java agent + // to their own JVM. As a result, Mockito’s inline mock maker (mockito-core) + // fails without explicit instrumentation, and the JVM consistently emits warnings. + // See also: https://javadoc.io/doc/org.mockito/mockito-core/latest/org.mockito/org/mockito/Mockito.html#mockito-instrumentation + afterEvaluate { subproject -> + def hasMockitoCore = subproject.configurations.findAll { + it.canBeResolved + }.any { config -> + config.incoming.dependencies.any { dependency -> + "$dependency" == libs.mockitoCore + } + } + + if (hasMockitoCore) { + subproject.configurations { + mockitoAgent { + transitive = false + } + } + subproject.dependencies { + mockitoAgent libs.mockitoCore + } + } + } + // The suites are for running sets of tests in IDEs. // Gradle will run each test class, so we exclude the suites to avoid redundantly running the tests twice. def testsToExclude = ['**/*Suite.class'] - // This task will copy JUnit XML files out of the sub-project's build directory and into // a top-level build/junit-xml directory. This is necessary to avoid reporting on tests which // were not run, but instead were restored via FROM-CACHE. See KAFKA-17479 for more details. @@ -518,6 +543,14 @@ subprojects { } test { + + doFirst { + def mockitoAgentConfig = configurations.findByName('mockitoAgent') + if (mockitoAgentConfig) { + jvmArgs("-javaagent:${mockitoAgentConfig.asPath}") + } + } + maxParallelForks = maxTestForks ignoreFailures = userIgnoreFailures @@ -551,7 +584,7 @@ subprojects { maxFailures = userMaxTestRetryFailures } } - + finalizedBy("copyTestXml") } From ac495f9ef754dcd6c325f7ea46deaa5e86fba2ce Mon Sep 17 00:00:00 2001 From: Sanskar Jhajharia Date: Fri, 26 Sep 2025 16:14:02 +0530 Subject: [PATCH 081/100] MINOR: Clean Javadoc for BrokerReconfigurable interface (#20593) Reviewers: Mickael Maison , TengYao Chi --- .../main/java/org/apache/kafka/config/BrokerReconfigurable.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server-common/src/main/java/org/apache/kafka/config/BrokerReconfigurable.java b/server-common/src/main/java/org/apache/kafka/config/BrokerReconfigurable.java index b7076f05ef3..f17590317bf 100644 --- a/server-common/src/main/java/org/apache/kafka/config/BrokerReconfigurable.java +++ b/server-common/src/main/java/org/apache/kafka/config/BrokerReconfigurable.java @@ -32,7 +32,6 @@ import java.util.Set; *

  • Validating the new configuration before applying it via {@link #validateReconfiguration(AbstractConfig)}
  • *
  • Applying the new configuration via {@link #reconfigure(AbstractConfig, AbstractConfig)}
  • * - * Note: Since Kafka is eliminating Scala, developers should implement this interface instead of {@link kafka.server.BrokerReconfigurable} */ public interface BrokerReconfigurable { /** From fb0518c34e7b5c28dbeb234e504117a06ae20970 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Fri, 26 Sep 2025 19:40:47 +0200 Subject: [PATCH 082/100] KAFKA-19730: StreamsGroupDescribe result is missing topology (#20574) When toology not configured. In the streams group heartbeat, we validate the topology set for the group against the topic metadata, to generate the "configured topology" which has a specific number of partitions for each topic. In streams group describe, we use the configured topology to expose this information to the user. However, we leave the topology information as null in the streams group describe response, if the topology is not configured. This triggers an IllegalStateException in the admin client implementation. Instead, we should expose the unconfigured topology when the configured topology is not available, which will still expose useful information. Reviewers: Matthias J. Sax --------- Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- .../api/PlaintextAdminIntegrationTest.scala | 41 ++++++ .../group/streams/StreamsGroup.java | 11 +- .../group/streams/StreamsTopology.java | 41 ++++++ .../group/GroupMetadataManagerTest.java | 53 +++++++- .../group/streams/StreamsGroupTest.java | 120 ++++++++++++++++++ .../group/streams/StreamsTopologyTest.java | 84 ++++++++++++ 6 files changed, 346 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 1c3a7ed42e3..4a686f3d4d8 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -4463,6 +4463,47 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { } } + @Test + def testDescribeStreamsGroupsNotReady(): Unit = { + val streamsGroupId = "stream_group_id" + val testTopicName = "test_topic" + + val config = createConfig + client = Admin.create(config) + + val streams = createStreamsGroup( + inputTopic = testTopicName, + streamsGroupId = streamsGroupId + ) + streams.poll(JDuration.ofMillis(500L)) + + try { + TestUtils.waitUntilTrue(() => { + val firstGroup = client.listGroups().all().get().stream() + .filter(g => g.groupId() == streamsGroupId).findFirst().orElse(null) + firstGroup.groupState().orElse(null) == GroupState.NOT_READY && firstGroup.groupId() == streamsGroupId + }, "Stream group not NOT_READY yet") + + // Verify the describe call works correctly + val describedGroups = client.describeStreamsGroups(util.List.of(streamsGroupId)).all().get() + val group = describedGroups.get(streamsGroupId) + assertNotNull(group) + assertEquals(streamsGroupId, group.groupId()) + assertFalse(group.members().isEmpty) + assertNotNull(group.subtopologies()) + assertFalse(group.subtopologies().isEmpty) + + // Verify the topology contains the expected source and sink topics + val subtopologies = group.subtopologies().asScala + assertTrue(subtopologies.exists(subtopology => + subtopology.sourceTopics().contains(testTopicName))) + + } finally { + Utils.closeQuietly(streams, "streams") + Utils.closeQuietly(client, "adminClient") + } + } + @Test def testDeleteStreamsGroups(): Unit = { val testTopicName = "test_topic" diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java index 72d4386321e..61f61d101f1 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java @@ -1039,7 +1039,16 @@ public class StreamsGroup implements Group { .setGroupEpoch(groupEpoch.get(committedOffset)) .setGroupState(state.get(committedOffset).toString()) .setAssignmentEpoch(targetAssignmentEpoch.get(committedOffset)) - .setTopology(configuredTopology.get(committedOffset).map(ConfiguredTopology::asStreamsGroupDescribeTopology).orElse(null)); + .setTopology( + configuredTopology.get(committedOffset) + .filter(ConfiguredTopology::isReady) + .map(ConfiguredTopology::asStreamsGroupDescribeTopology) + .orElse( + topology.get(committedOffset) + .map(StreamsTopology::asStreamsGroupDescribeTopology) + .orElseThrow(() -> new IllegalStateException("There should always be a topology for a streams group.")) + ) + ); members.entrySet(committedOffset).forEach( entry -> describedGroup.members().add( entry.getValue().asStreamsGroupDescribeMember( diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsTopology.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsTopology.java index 498ff00a595..25ea4376331 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsTopology.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsTopology.java @@ -16,12 +16,14 @@ */ package org.apache.kafka.coordinator.group.streams; +import org.apache.kafka.common.message.StreamsGroupDescribeResponseData; import org.apache.kafka.common.message.StreamsGroupHeartbeatRequestData; import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue; import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue.Subtopology; import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue.TopicInfo; import java.util.Collections; +import java.util.Comparator; import java.util.Map; import java.util.Objects; import java.util.Set; @@ -95,4 +97,43 @@ public record StreamsTopology(int topologyEpoch, .collect(Collectors.toMap(StreamsGroupTopologyValue.Subtopology::subtopologyId, x -> x)); return new StreamsTopology(topology.epoch(), subtopologyMap); } + + public StreamsGroupDescribeResponseData.Topology asStreamsGroupDescribeTopology() { + return new StreamsGroupDescribeResponseData.Topology() + .setEpoch(topologyEpoch) + .setSubtopologies( + subtopologies.entrySet().stream() + .sorted(Map.Entry.comparingByKey()) + .map(entry -> asStreamsGroupDescribeSubtopology(entry.getKey(), entry.getValue())) + .toList() + ); + } + + private StreamsGroupDescribeResponseData.Subtopology asStreamsGroupDescribeSubtopology(String subtopologyId, StreamsGroupTopologyValue.Subtopology subtopology) { + return new StreamsGroupDescribeResponseData.Subtopology() + .setSubtopologyId(subtopologyId) + .setSourceTopics(subtopology.sourceTopics().stream().sorted().toList()) + .setRepartitionSinkTopics(subtopology.repartitionSinkTopics().stream().sorted().toList()) + .setRepartitionSourceTopics(subtopology.repartitionSourceTopics().stream() + .map(this::asStreamsGroupDescribeTopicInfo) + .sorted(Comparator.comparing(StreamsGroupDescribeResponseData.TopicInfo::name)).toList()) + .setStateChangelogTopics(subtopology.stateChangelogTopics().stream() + .map(this::asStreamsGroupDescribeTopicInfo) + .sorted(Comparator.comparing(StreamsGroupDescribeResponseData.TopicInfo::name)).toList()); + } + + private StreamsGroupDescribeResponseData.TopicInfo asStreamsGroupDescribeTopicInfo(StreamsGroupTopologyValue.TopicInfo topicInfo) { + return new StreamsGroupDescribeResponseData.TopicInfo() + .setName(topicInfo.name()) + .setPartitions(topicInfo.partitions()) + .setReplicationFactor(topicInfo.replicationFactor()) + .setTopicConfigs( + topicInfo.topicConfigs().stream().map( + topicConfig -> new StreamsGroupDescribeResponseData.KeyValue() + .setKey(topicConfig.key()) + .setValue(topicConfig.value()) + ).toList() + ); + } + } 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 8d4ae4fbe07..957ae7e8147 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 @@ -9645,19 +9645,43 @@ public class GroupMetadataManagerTest { .setProcessId("processId") .setMemberEpoch(epoch) .setPreviousMemberEpoch(epoch - 1); + String subtopology1 = "subtopology1"; + String fooTopicName = "foo"; + StreamsTopology topology = new StreamsTopology( + 0, + Map.of(subtopology1, + new StreamsGroupTopologyValue.Subtopology() + .setSubtopologyId(subtopology1) + .setSourceTopics(List.of(fooTopicName)) + ) + ); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withStreamsGroup(new StreamsGroupBuilder(streamsGroupIds.get(0), epoch)) + .withStreamsGroup(new StreamsGroupBuilder(streamsGroupIds.get(0), epoch) + .withTopology(topology) + ) .withStreamsGroup(new StreamsGroupBuilder(streamsGroupIds.get(1), epoch) - .withMember(memberBuilder.build())) + .withMember(memberBuilder.build()) + .withTopology(topology) + ) .build(); + StreamsGroupDescribeResponseData.Topology expectedTopology = + new StreamsGroupDescribeResponseData.Topology() + .setEpoch(0) + .setSubtopologies(List.of( + new StreamsGroupDescribeResponseData.Subtopology() + .setSubtopologyId(subtopology1) + .setSourceTopics(List.of(fooTopicName)) + )); + List expected = Arrays.asList( new StreamsGroupDescribeResponseData.DescribedGroup() .setGroupEpoch(epoch) .setGroupId(streamsGroupIds.get(0)) .setGroupState(StreamsGroupState.EMPTY.toString()) - .setAssignmentEpoch(0), + .setAssignmentEpoch(0) + .setTopology(expectedTopology), new StreamsGroupDescribeResponseData.DescribedGroup() .setGroupEpoch(epoch) .setGroupId(streamsGroupIds.get(1)) @@ -9666,6 +9690,7 @@ public class GroupMetadataManagerTest { TasksTuple.EMPTY ) )) + .setTopology(expectedTopology) .setGroupState(StreamsGroupState.NOT_READY.toString()) ); List actual = context.sendStreamsGroupDescribe(streamsGroupIds); @@ -9695,6 +9720,16 @@ public class GroupMetadataManagerTest { String memberId1 = "memberId1"; String memberId2 = "memberId2"; String subtopologyId = "subtopology1"; + String fooTopicName = "foo"; + StreamsGroupTopologyValue topology = new StreamsGroupTopologyValue() + .setEpoch(0) + .setSubtopologies( + List.of( + new StreamsGroupTopologyValue.Subtopology() + .setSubtopologyId(subtopologyId) + .setSourceTopics(List.of(fooTopicName)) + ) + ); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder().build(); @@ -9702,6 +9737,7 @@ public class GroupMetadataManagerTest { context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupMemberRecord(streamsGroupId, memberBuilder1.build())); context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(streamsGroupId, memberBuilder1.build())); context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(streamsGroupId, epoch + 1, 0)); + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupTopologyRecord(streamsGroupId, topology)); TasksTuple assignment = new TasksTuple( Map.of(subtopologyId, Set.of(0, 1)), @@ -9733,6 +9769,17 @@ public class GroupMetadataManagerTest { memberBuilder1.build().asStreamsGroupDescribeMember(TasksTuple.EMPTY), memberBuilder2.build().asStreamsGroupDescribeMember(assignment) )) + .setTopology( + new StreamsGroupDescribeResponseData.Topology() + .setEpoch(0) + .setSubtopologies( + List.of( + new StreamsGroupDescribeResponseData.Subtopology() + .setSubtopologyId(subtopologyId) + .setSourceTopics(List.of(fooTopicName)) + ) + ) + ) .setGroupState(StreamsGroup.StreamsGroupState.NOT_READY.toString()) .setGroupEpoch(epoch + 2); assertEquals(1, actual.size()); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java index 71feb2a1e90..ba24abd2b80 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java @@ -1019,4 +1019,124 @@ public class StreamsGroupTest { streamsGroup.removeMember(memberId2); assertEquals(Optional.empty(), streamsGroup.getShutdownRequestMemberId()); } + + @Test + public void testAsDescribedGroupWithStreamsTopologyHavingSubtopologies() { + SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); + StreamsGroup group = new StreamsGroup(LOG_CONTEXT, snapshotRegistry, "group-id-with-topology"); + snapshotRegistry.idempotentCreateSnapshot(0); + + // Create a topology with subtopologies + Map subtopologies = Map.of( + "sub-1", new StreamsGroupTopologyValue.Subtopology() + .setSubtopologyId("sub-1") + .setSourceTopics(List.of("input-topic")) + .setRepartitionSourceTopics(List.of( + new StreamsGroupTopologyValue.TopicInfo().setName("repartition-topic") + )) + .setStateChangelogTopics(List.of( + new StreamsGroupTopologyValue.TopicInfo().setName("changelog-topic") + )) + ); + + group.setGroupEpoch(2); + group.setTopology(new StreamsTopology(2, subtopologies)); + group.setTargetAssignmentEpoch(2); + group.updateMember(new StreamsGroupMember.Builder("member1") + .setMemberEpoch(2) + .setPreviousMemberEpoch(1) + .setState(MemberState.STABLE) + .setInstanceId("instance1") + .setRackId("rack1") + .setClientId("client1") + .setClientHost("host1") + .setRebalanceTimeoutMs(1000) + .setTopologyEpoch(2) + .setProcessId("process1") + .setUserEndpoint(new StreamsGroupMemberMetadataValue.Endpoint().setHost("host1").setPort(9092)) + .setClientTags(Map.of("tag1", "value1")) + .setAssignedTasks(new TasksTuple(Map.of(), Map.of(), Map.of())) + .setTasksPendingRevocation(new TasksTuple(Map.of(), Map.of(), Map.of())) + .build()); + snapshotRegistry.idempotentCreateSnapshot(1); + + StreamsGroupDescribeResponseData.DescribedGroup describedGroup = group.asDescribedGroup(1); + + assertEquals("group-id-with-topology", describedGroup.groupId()); + assertEquals(StreamsGroup.StreamsGroupState.NOT_READY.toString(), describedGroup.groupState()); + assertEquals(2, describedGroup.groupEpoch()); + assertEquals(2, describedGroup.assignmentEpoch()); + + // Verify topology is correctly described + assertNotNull(describedGroup.topology()); + assertEquals(2, describedGroup.topology().epoch()); + assertEquals(1, describedGroup.topology().subtopologies().size()); + + StreamsGroupDescribeResponseData.Subtopology subtopology = describedGroup.topology().subtopologies().get(0); + assertEquals("sub-1", subtopology.subtopologyId()); + assertEquals(List.of("input-topic"), subtopology.sourceTopics()); + assertEquals(1, subtopology.repartitionSourceTopics().size()); + assertEquals("repartition-topic", subtopology.repartitionSourceTopics().get(0).name()); + assertEquals(1, subtopology.stateChangelogTopics().size()); + assertEquals("changelog-topic", subtopology.stateChangelogTopics().get(0).name()); + + assertEquals(1, describedGroup.members().size()); + assertEquals("member1", describedGroup.members().get(0).memberId()); + } + + @Test + public void testAsDescribedGroupPrefersConfiguredTopologyOverStreamsTopology() { + SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); + StreamsGroup group = new StreamsGroup(LOG_CONTEXT, snapshotRegistry, "group-id-configured"); + snapshotRegistry.idempotentCreateSnapshot(0); + + // Create both StreamsTopology and ConfiguredTopology + Map subtopologies = Map.of( + "sub-1", new StreamsGroupTopologyValue.Subtopology() + .setSubtopologyId("sub-1") + .setSourceTopics(List.of("streams-topic")) + ); + + group.setGroupEpoch(3); + group.setTopology(new StreamsTopology(2, subtopologies)); + group.setConfiguredTopology(new ConfiguredTopology(3, 0, Optional.of(new TreeMap<>()), Map.of(), Optional.empty())); + group.setTargetAssignmentEpoch(3); + snapshotRegistry.idempotentCreateSnapshot(1); + + StreamsGroupDescribeResponseData.DescribedGroup describedGroup = group.asDescribedGroup(1); + + // Should prefer ConfiguredTopology over StreamsTopology + assertNotNull(describedGroup.topology()); + assertEquals(3, describedGroup.topology().epoch()); // ConfiguredTopology epoch + assertEquals(0, describedGroup.topology().subtopologies().size()); // Empty configured topology + } + + @Test + public void testAsDescribedGroupFallbackToStreamsTopologyWhenConfiguredTopologyEmpty() { + SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); + StreamsGroup group = new StreamsGroup(LOG_CONTEXT, snapshotRegistry, "group-id-fallback"); + snapshotRegistry.idempotentCreateSnapshot(0); + + // Create StreamsTopology with subtopologies + Map subtopologies = Map.of( + "sub-1", new StreamsGroupTopologyValue.Subtopology() + .setSubtopologyId("sub-1") + .setSourceTopics(List.of("fallback-topic")) + ); + + group.setGroupEpoch(4); + group.setTopology(new StreamsTopology(4, subtopologies)); + // No ConfiguredTopology set, so should fallback to StreamsTopology + group.setTargetAssignmentEpoch(4); + snapshotRegistry.idempotentCreateSnapshot(1); + + StreamsGroupDescribeResponseData.DescribedGroup describedGroup = group.asDescribedGroup(1); + + // Should use StreamsTopology when ConfiguredTopology is not available + assertNotNull(describedGroup.topology()); + assertEquals(4, describedGroup.topology().epoch()); // StreamsTopology epoch + assertEquals(1, describedGroup.topology().subtopologies().size()); + assertEquals("sub-1", describedGroup.topology().subtopologies().get(0).subtopologyId()); + assertEquals(List.of("fallback-topic"), describedGroup.topology().subtopologies().get(0).sourceTopics()); + } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsTopologyTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsTopologyTest.java index a9d269f2120..83ea799cdc9 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsTopologyTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsTopologyTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.coordinator.group.streams; +import org.apache.kafka.common.message.StreamsGroupDescribeResponseData; import org.apache.kafka.common.message.StreamsGroupHeartbeatRequestData; import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue; import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue.Subtopology; @@ -24,6 +25,7 @@ import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue.To import org.junit.jupiter.api.Test; import java.util.Arrays; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Set; @@ -120,6 +122,88 @@ public class StreamsTopologyTest { assertEquals(mkSubtopology2(), topology.subtopologies().get(SUBTOPOLOGY_ID_2)); } + @Test + public void asStreamsGroupDescribeTopologyShouldReturnCorrectStructure() { + Map subtopologies = mkMap( + mkEntry(SUBTOPOLOGY_ID_1, mkSubtopology1()), + mkEntry(SUBTOPOLOGY_ID_2, mkSubtopology2()) + ); + StreamsTopology topology = new StreamsTopology(1, subtopologies); + + StreamsGroupDescribeResponseData.Topology describeTopology = topology.asStreamsGroupDescribeTopology(); + + assertEquals(1, describeTopology.epoch()); + assertEquals(2, describeTopology.subtopologies().size()); + + // Verify subtopologies are correctly converted and sorted + List sortedSubtopologies = + describeTopology.subtopologies().stream() + .sorted(Comparator.comparing(StreamsGroupDescribeResponseData.Subtopology::subtopologyId)) + .toList(); + + // Verify first subtopology + StreamsGroupDescribeResponseData.Subtopology sub1 = sortedSubtopologies.get(0); + assertEquals(SUBTOPOLOGY_ID_1, sub1.subtopologyId()); + // Source topics are sorted alphabetically + assertEquals(List.of(REPARTITION_TOPIC_1, REPARTITION_TOPIC_2, SOURCE_TOPIC_1, SOURCE_TOPIC_2), + sub1.sourceTopics()); + assertEquals(List.of(REPARTITION_TOPIC_3), sub1.repartitionSinkTopics()); + assertEquals(2, sub1.repartitionSourceTopics().size()); + assertEquals(2, sub1.stateChangelogTopics().size()); + + // Verify second subtopology + StreamsGroupDescribeResponseData.Subtopology sub2 = sortedSubtopologies.get(1); + assertEquals(SUBTOPOLOGY_ID_2, sub2.subtopologyId()); + // Source topics are sorted alphabetically + assertEquals(List.of(REPARTITION_TOPIC_3, SOURCE_TOPIC_3), sub2.sourceTopics()); + assertEquals(List.of(), sub2.repartitionSinkTopics()); + assertEquals(1, sub2.repartitionSourceTopics().size()); + assertEquals(1, sub2.stateChangelogTopics().size()); + } + + @Test + public void asStreamsGroupDescribeTopicInfoShouldConvertCorrectly() { + Map subtopologies = mkMap( + mkEntry(SUBTOPOLOGY_ID_1, mkSubtopology1()) + ); + StreamsTopology topology = new StreamsTopology(1, subtopologies); + + StreamsGroupDescribeResponseData.Topology describeTopology = topology.asStreamsGroupDescribeTopology(); + StreamsGroupDescribeResponseData.Subtopology describedSub = describeTopology.subtopologies().get(0); + + // Verify repartition source topics are correctly converted + List repartitionTopics = describedSub.repartitionSourceTopics(); + assertEquals(2, repartitionTopics.size()); + + // Find the first repartition topic (they should be sorted by name) + StreamsGroupDescribeResponseData.TopicInfo firstTopic = repartitionTopics.stream() + .filter(topic -> topic.name().equals(REPARTITION_TOPIC_1)) + .findFirst() + .orElseThrow(); + assertEquals(REPARTITION_TOPIC_1, firstTopic.name()); + + // Verify changelog topics are correctly converted + List changelogTopics = describedSub.stateChangelogTopics(); + assertEquals(2, changelogTopics.size()); + + // Find the first changelog topic (they should be sorted by name) + StreamsGroupDescribeResponseData.TopicInfo firstChangelog = changelogTopics.stream() + .filter(topic -> topic.name().equals(CHANGELOG_TOPIC_1)) + .findFirst() + .orElseThrow(); + assertEquals(CHANGELOG_TOPIC_1, firstChangelog.name()); + } + + @Test + public void asStreamsGroupDescribeTopologyWithEmptySubtopologies() { + StreamsTopology topology = new StreamsTopology(0, Map.of()); + + StreamsGroupDescribeResponseData.Topology describeTopology = topology.asStreamsGroupDescribeTopology(); + + assertEquals(0, describeTopology.epoch()); + assertEquals(0, describeTopology.subtopologies().size()); + } + private Subtopology mkSubtopology1() { return new Subtopology() .setSubtopologyId(SUBTOPOLOGY_ID_1) From 41611b4bd2e708a3a30c7b8f5d89aa274a64676c Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Sun, 28 Sep 2025 19:06:06 +0800 Subject: [PATCH 083/100] MINOR: Followup KAFKA-19112 document updated (#20492) Some sections are not very clear, and we need to update the documentation. Reviewers: TengYao Chi , Jun Rao , Chia-Ping Tsai --- .../main/scala/kafka/cluster/Partition.scala | 2 +- .../unit/kafka/cluster/PartitionTest.scala | 46 ++++++++++++++++++- docs/upgrade.html | 10 +++- .../storage/internals/log/UnifiedLog.java | 4 +- 4 files changed, 56 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 3b45a08b067..44b28a1f07e 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -1658,7 +1658,7 @@ class Partition(val topicPartition: TopicPartition, def deleteRecordsOnLeader(offset: Long): LogDeleteRecordsResult = inReadLock(leaderIsrUpdateLock) { leaderLogIfLocal match { case Some(leaderLog) => - if (!leaderLog.config.delete) + if (!leaderLog.config.delete && leaderLog.config.compact) throw new PolicyViolationException(s"Records of partition $topicPartition can not be deleted due to the configured policy") val convertedOffset = if (offset == DeleteRecordsRequest.HIGH_WATERMARK) diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index 8e512ad4d01..5662c2d2276 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -21,7 +21,7 @@ import com.yammer.metrics.core.Metric import kafka.log.LogManager import kafka.server._ import kafka.utils._ -import org.apache.kafka.common.errors.{ApiException, FencedLeaderEpochException, InconsistentTopicIdException, InvalidTxnStateException, NotLeaderOrFollowerException, OffsetNotAvailableException, OffsetOutOfRangeException, UnknownLeaderEpochException} +import org.apache.kafka.common.errors.{ApiException, FencedLeaderEpochException, InconsistentTopicIdException, InvalidTxnStateException, NotLeaderOrFollowerException, OffsetNotAvailableException, OffsetOutOfRangeException, PolicyViolationException, UnknownLeaderEpochException} import org.apache.kafka.common.message.{AlterPartitionResponseData, FetchResponseData} import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.FileRecords.TimestampAndOffset @@ -61,7 +61,7 @@ import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams, Unexpec import org.apache.kafka.server.util.{KafkaScheduler, MockTime} import org.apache.kafka.storage.internals.checkpoint.OffsetCheckpoints import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache -import org.apache.kafka.storage.internals.log.{AppendOrigin, CleanerConfig, EpochEntry, LocalLog, LogAppendInfo, LogDirFailureChannel, LogLoader, LogOffsetMetadata, LogOffsetsListener, LogReadInfo, LogSegments, LogStartOffsetIncrementReason, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog, VerificationGuard} +import org.apache.kafka.storage.internals.log.{AppendOrigin, CleanerConfig, EpochEntry, LocalLog, LogAppendInfo, LogConfig, LogDirFailureChannel, LogLoader, LogOffsetMetadata, LogOffsetsListener, LogReadInfo, LogSegments, LogStartOffsetIncrementReason, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog, VerificationGuard} import org.apache.kafka.storage.log.metrics.BrokerTopicStats import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -4030,4 +4030,46 @@ class PartitionTest extends AbstractPartitionTest { alterPartitionManager) partition.tryCompleteDelayedRequests() } + + @Test + def testDeleteRecordsOnLeaderWithEmptyPolicy(): Unit = { + val leaderEpoch = 5 + val partition = setupPartitionWithMocks(leaderEpoch, isLeader = true) + + val emptyPolicyConfig = new LogConfig(util.Map.of( + TopicConfig.CLEANUP_POLICY_CONFIG, "" + )) + + val mockLog = mock(classOf[UnifiedLog]) + when(mockLog.config).thenReturn(emptyPolicyConfig) + when(mockLog.logEndOffset).thenReturn(2L) + when(mockLog.logStartOffset).thenReturn(0L) + when(mockLog.highWatermark).thenReturn(2L) + when(mockLog.maybeIncrementLogStartOffset(any(), any())).thenReturn(true) + + partition.setLog(mockLog, false) + + val result = partition.deleteRecordsOnLeader(1L) + assertEquals(1L, result.requestedOffset) + } + + @Test + def testDeleteRecordsOnLeaderWithCompactPolicy(): Unit = { + val leaderEpoch = 5 + val partition = setupPartitionWithMocks(leaderEpoch, isLeader = true) + + val emptyPolicyConfig = new LogConfig(util.Map.of( + TopicConfig.CLEANUP_POLICY_CONFIG, "compact" + )) + + val mockLog = mock(classOf[UnifiedLog]) + when(mockLog.config).thenReturn(emptyPolicyConfig) + when(mockLog.logEndOffset).thenReturn(2L) + when(mockLog.logStartOffset).thenReturn(0L) + when(mockLog.highWatermark).thenReturn(2L) + when(mockLog.maybeIncrementLogStartOffset(any(), any())).thenReturn(true) + + partition.setLog(mockLog, false) + assertThrows(classOf[PolicyViolationException], () => partition.deleteRecordsOnLeader(1L)) + } } diff --git a/docs/upgrade.html b/docs/upgrade.html index 66e05d90a5d..81af2d65261 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -136,9 +136,17 @@ settings.
  • - The cleanup.policy is empty and remote.storage.enable is set to true, the + cleanup.policy now supports empty values, which means infinite retention. + This is equivalent to setting retention.ms=-1 and retention.bytes=-1 +
    + If cleanup.policy is empty and remote.storage.enable is set to true, the local log segments will be cleaned based on the values of log.local.retention.bytes and log.local.retention.ms. +
    + If cleanup.policy is empty and remote.storage.enable is set to false, + local log segments will not be deleted automatically. However, records can still be deleted + explicitly through deleteRecords API calls, which will advance the log start offset + and remove the corresponding log segments.
  • diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java index d190c861998..c683ffc6c55 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java @@ -1934,8 +1934,8 @@ public class UnifiedLog implements AutoCloseable { deleteRetentionSizeBreachedSegments() + deleteRetentionMsBreachedSegments(); } else { - // If cleanup.policy is empty and remote storage is disabled, we should not delete any local - // log segments + // If cleanup.policy is empty and remote storage is disabled, we should not delete any local log segments + // unless the log start offset advances through deleteRecords return deleteLogStartOffsetBreachedSegments(); } } From d2a699954d964d0b0e675fa29fadb697c68622f3 Mon Sep 17 00:00:00 2001 From: Sanskar Jhajharia Date: Sun, 28 Sep 2025 20:45:28 +0530 Subject: [PATCH 084/100] MINOR: Cleanup `toString` methods in Storage Module (#20432) Getting rid of a bunch of `toString` functions in record classes in Storage Module. Reviewers: Chia-Ping Tsai --- .../kafka/storage/internals/log/EpochEntry.java | 8 -------- .../kafka/storage/internals/log/LastRecord.java | 8 -------- .../storage/internals/log/LoadedLogOffsets.java | 9 --------- .../storage/internals/log/LogOffsetSnapshot.java | 10 ---------- .../kafka/storage/internals/log/LogReadInfo.java | 12 ------------ .../kafka/storage/internals/log/OffsetPosition.java | 8 -------- .../internals/log/RemoteStorageFetchInfo.java | 11 ----------- .../kafka/storage/internals/log/RollParams.java | 13 ------------- .../storage/internals/log/TimestampOffset.java | 7 ------- 9 files changed, 86 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/EpochEntry.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/EpochEntry.java index a024919b665..f51c7c81939 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/EpochEntry.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/EpochEntry.java @@ -18,12 +18,4 @@ package org.apache.kafka.storage.internals.log; // Mapping of epoch to the first offset of the subsequent epoch public record EpochEntry(int epoch, long startOffset) { - - @Override - public String toString() { - return "EpochEntry(" + - "epoch=" + epoch + - ", startOffset=" + startOffset + - ')'; - } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LastRecord.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LastRecord.java index e000c77864d..69e9cfe34ef 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LastRecord.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LastRecord.java @@ -27,12 +27,4 @@ public record LastRecord(OptionalLong lastDataOffset, short producerEpoch) { public LastRecord { Objects.requireNonNull(lastDataOffset, "lastDataOffset must be non null"); } - - @Override - public String toString() { - return "LastRecord(" + - "lastDataOffset=" + lastDataOffset + - ", producerEpoch=" + producerEpoch + - ')'; - } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LoadedLogOffsets.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LoadedLogOffsets.java index 30bcf312588..aec4c88fe51 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LoadedLogOffsets.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LoadedLogOffsets.java @@ -26,13 +26,4 @@ public record LoadedLogOffsets(long logStartOffset, long recoveryPoint, LogOffse this.recoveryPoint = recoveryPoint; this.nextOffsetMetadata = Objects.requireNonNull(nextOffsetMetadata, "nextOffsetMetadata should not be null"); } - - @Override - public String toString() { - return "LoadedLogOffsets(" + - "logStartOffset=" + logStartOffset + - ", recoveryPoint=" + recoveryPoint + - ", nextOffsetMetadata=" + nextOffsetMetadata + - ')'; - } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogOffsetSnapshot.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogOffsetSnapshot.java index b96f68ba8dd..ecadf16f4ae 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogOffsetSnapshot.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogOffsetSnapshot.java @@ -23,14 +23,4 @@ package org.apache.kafka.storage.internals.log; */ public record LogOffsetSnapshot(long logStartOffset, LogOffsetMetadata logEndOffset, LogOffsetMetadata highWatermark, LogOffsetMetadata lastStableOffset) { - - @Override - public String toString() { - return "LogOffsetSnapshot(" + - "logStartOffset=" + logStartOffset + - ", logEndOffset=" + logEndOffset + - ", highWatermark=" + highWatermark + - ", lastStableOffset=" + lastStableOffset + - ')'; - } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadInfo.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadInfo.java index d50f00ddba1..6a603e5aa76 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadInfo.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadInfo.java @@ -25,16 +25,4 @@ import java.util.Optional; */ public record LogReadInfo(FetchDataInfo fetchedData, Optional divergingEpoch, long highWatermark, long logStartOffset, long logEndOffset, long lastStableOffset) { - - @Override - public String toString() { - return "LogReadInfo(" + - "fetchedData=" + fetchedData + - ", divergingEpoch=" + divergingEpoch + - ", highWatermark=" + highWatermark + - ", logStartOffset=" + logStartOffset + - ", logEndOffset=" + logEndOffset + - ", lastStableOffset=" + lastStableOffset + - ')'; - } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/OffsetPosition.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/OffsetPosition.java index 31ca0183276..ab32a1212e0 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/OffsetPosition.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/OffsetPosition.java @@ -32,12 +32,4 @@ public record OffsetPosition(long offset, int position) implements IndexEntry { public long indexValue() { return position; } - - @Override - public String toString() { - return "OffsetPosition(" + - "offset=" + offset + - ", position=" + position + - ')'; - } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteStorageFetchInfo.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteStorageFetchInfo.java index 3e70f8c7120..da37c98d892 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteStorageFetchInfo.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteStorageFetchInfo.java @@ -22,15 +22,4 @@ import org.apache.kafka.server.storage.log.FetchIsolation; public record RemoteStorageFetchInfo(int fetchMaxBytes, boolean minOneMessage, TopicIdPartition topicIdPartition, FetchRequest.PartitionData fetchInfo, FetchIsolation fetchIsolation) { - - @Override - public String toString() { - return "RemoteStorageFetchInfo{" + - "fetchMaxBytes=" + fetchMaxBytes + - ", minOneMessage=" + minOneMessage + - ", topicIdPartition=" + topicIdPartition + - ", fetchInfo=" + fetchInfo + - ", fetchIsolation=" + fetchIsolation + - '}'; - } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/RollParams.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/RollParams.java index cbb934a725c..ca5874b61b9 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/RollParams.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/RollParams.java @@ -21,17 +21,4 @@ package org.apache.kafka.storage.internals.log; */ public record RollParams(long maxSegmentMs, int maxSegmentBytes, long maxTimestampInMessages, long maxOffsetInMessages, int messagesSize, long now) { - - @Override - public String toString() { - return "RollParams(" + - "maxSegmentMs=" + maxSegmentMs + - ", maxSegmentBytes=" + maxSegmentBytes + - ", maxTimestampInMessages=" + maxTimestampInMessages + - ", maxOffsetInMessages=" + maxOffsetInMessages + - ", messagesSize=" + messagesSize + - ", now=" + now + - ')'; - } - } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/TimestampOffset.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/TimestampOffset.java index fecf458ed46..a09a1910868 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/TimestampOffset.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/TimestampOffset.java @@ -36,11 +36,4 @@ public record TimestampOffset(long timestamp, long offset) implements IndexEntry public long indexValue() { return offset; } - - @Override - public String toString() { - return String.format("TimestampOffset(offset = %d, timestamp = %d)", - offset, - timestamp); - } } From 60ad638a35ee0be6878e93cc1abd4b2123c63b9d Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Sun, 28 Sep 2025 23:50:25 +0800 Subject: [PATCH 085/100] KAFKA-19617: ConsumerPerformance#ConsumerPerfRebListener get corrupted value when the number of partitions is increased (#20388) With changes to the consumer protocol, rebalance may not necessarily result in a "stop the world". Thus, the method for calculating pause time in `ConsumerPerformance#ConsumerPerfRebListener` needs to be modified. Stop time is only recorded if `assignedPartitions` is empty. Reviewers: Andrew Schofield --- .../kafka/tools/ConsumerPerformance.java | 19 ++++-- .../kafka/tools/ConsumerPerformanceTest.java | 59 +++++++++++++++++++ 2 files changed, 73 insertions(+), 5 deletions(-) diff --git a/tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java b/tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java index 0334af83aa1..f4a987e77d5 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java +++ b/tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java @@ -36,6 +36,7 @@ import java.io.IOException; import java.text.SimpleDateFormat; import java.time.Duration; import java.util.Collection; +import java.util.HashSet; import java.util.List; import java.util.Optional; import java.util.Properties; @@ -165,7 +166,7 @@ public class ConsumerPerformance { if (showDetailedStats) printConsumerProgress(0, bytesRead, lastBytesRead, recordsRead, lastRecordsRead, lastReportTimeMs, currentTimeMs, dateFormat, joinTimeMsInSingleRound.get()); - joinTimeMsInSingleRound = new AtomicLong(0); + joinTimeMsInSingleRound.set(0); lastReportTimeMs = currentTimeMs; lastRecordsRead = recordsRead; lastBytesRead = bytesRead; @@ -230,24 +231,32 @@ public class ConsumerPerformance { public static class ConsumerPerfRebListener implements ConsumerRebalanceListener { private final AtomicLong joinTimeMs; private final AtomicLong joinTimeMsInSingleRound; + private final Collection assignedPartitions; private long joinStartMs; public ConsumerPerfRebListener(AtomicLong joinTimeMs, long joinStartMs, AtomicLong joinTimeMsInSingleRound) { this.joinTimeMs = joinTimeMs; this.joinStartMs = joinStartMs; this.joinTimeMsInSingleRound = joinTimeMsInSingleRound; + this.assignedPartitions = new HashSet<>(); } @Override public void onPartitionsRevoked(Collection partitions) { - joinStartMs = System.currentTimeMillis(); + assignedPartitions.removeAll(partitions); + if (assignedPartitions.isEmpty()) { + joinStartMs = System.currentTimeMillis(); + } } @Override public void onPartitionsAssigned(Collection partitions) { - long elapsedMs = System.currentTimeMillis() - joinStartMs; - joinTimeMs.addAndGet(elapsedMs); - joinTimeMsInSingleRound.addAndGet(elapsedMs); + if (assignedPartitions.isEmpty()) { + long elapsedMs = System.currentTimeMillis() - joinStartMs; + joinTimeMs.addAndGet(elapsedMs); + joinTimeMsInSingleRound.addAndGet(elapsedMs); + } + assignedPartitions.addAll(partitions); } } diff --git a/tools/src/test/java/org/apache/kafka/tools/ConsumerPerformanceTest.java b/tools/src/test/java/org/apache/kafka/tools/ConsumerPerformanceTest.java index 497deb7808d..9e385878cb9 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ConsumerPerformanceTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ConsumerPerformanceTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.MockConsumer; import org.apache.kafka.clients.consumer.internals.AutoOffsetResetStrategy; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Utils; @@ -35,9 +36,12 @@ import java.nio.file.Files; import java.nio.file.Path; import java.text.SimpleDateFormat; import java.util.Properties; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertTrue; public class ConsumerPerformanceTest { @@ -302,6 +306,61 @@ public class ConsumerPerformanceTest { assertTrue(Utils.isBlank(err), "Should be no stderr message, but was \"" + err + "\""); } + @Test + public void testConsumerListenerWithAllPartitionRevokedAndAssigned() throws InterruptedException { + String topicName = "topic"; + TopicPartition tp0 = new TopicPartition(topicName, 0); + TopicPartition tp1 = new TopicPartition(topicName, 1); + AtomicLong joinTimeMs = new AtomicLong(0); + AtomicLong joinTimeMsInSingleRound = new AtomicLong(0); + ConsumerPerformance.ConsumerPerfRebListener listener = new ConsumerPerformance.ConsumerPerfRebListener(joinTimeMs, 0, joinTimeMsInSingleRound); + listener.onPartitionsAssigned(Set.of(tp0)); + long lastJoinTimeMs = joinTimeMs.get(); + + // All assigned partitions have been revoked. + listener.onPartitionsRevoked(Set.of(tp0)); + Thread.sleep(100); + listener.onPartitionsAssigned(Set.of(tp1)); + + assertNotEquals(lastJoinTimeMs, joinTimeMs.get()); + } + + @Test + public void testConsumerListenerWithPartialPartitionRevokedAndAssigned() throws InterruptedException { + String topicName = "topic"; + TopicPartition tp0 = new TopicPartition(topicName, 0); + TopicPartition tp1 = new TopicPartition(topicName, 1); + AtomicLong joinTimeMs = new AtomicLong(0); + AtomicLong joinTimeMsInSingleRound = new AtomicLong(0); + ConsumerPerformance.ConsumerPerfRebListener listener = new ConsumerPerformance.ConsumerPerfRebListener(joinTimeMs, 0, joinTimeMsInSingleRound); + listener.onPartitionsAssigned(Set.of(tp0, tp1)); + long lastJoinTimeMs = joinTimeMs.get(); + + // The assigned partitions were partially revoked. + listener.onPartitionsRevoked(Set.of(tp0)); + Thread.sleep(100); + listener.onPartitionsAssigned(Set.of(tp0)); + + assertEquals(lastJoinTimeMs, joinTimeMs.get()); + } + + @Test + public void testConsumerListenerWithoutPartitionRevoked() throws InterruptedException { + String topicName = "topic"; + TopicPartition tp0 = new TopicPartition(topicName, 0); + TopicPartition tp1 = new TopicPartition(topicName, 1); + AtomicLong joinTimeMs = new AtomicLong(0); + AtomicLong joinTimeMsInSingleRound = new AtomicLong(0); + ConsumerPerformance.ConsumerPerfRebListener listener = new ConsumerPerformance.ConsumerPerfRebListener(joinTimeMs, 0, joinTimeMsInSingleRound); + listener.onPartitionsAssigned(Set.of(tp0)); + long lastJoinTimeMs = joinTimeMs.get(); + + Thread.sleep(100); + listener.onPartitionsAssigned(Set.of(tp1)); + + assertEquals(lastJoinTimeMs, joinTimeMs.get()); + } + private void testHeaderMatchContent(boolean detailed, int expectedOutputLineCount, Runnable runnable) { String out = ToolsTestUtils.captureStandardOut(() -> { ConsumerPerformance.printHeader(detailed); From 7d098cfbbd26ff60a3edb8f15e3068c8999a48b7 Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Mon, 29 Sep 2025 01:24:38 +0800 Subject: [PATCH 086/100] KAFKA-17876/ KAFKA-19150 Rename AssignmentsManager and RemoteStorageThreadPool metrics (#20265) Rename org.apache.kafka.server:type=AssignmentsManager and org.apache.kafka.storage.internals.log.RemoteStorageThreadPool metrics for the consist, these metrics should be - `kafka.log.remote:type=...` - `kafka.server:type=...` Reviewers: Chia-Ping Tsai --- docs/upgrade.html | 12 +++++++++ .../kafka/server/AssignmentsManager.java | 27 ++++++++++++------- .../kafka/server/AssignmentsManagerTest.java | 10 +++++++ .../remote/storage/RemoteStorageMetrics.java | 12 +++++++-- .../log/RemoteStorageThreadPool.java | 18 ++++++++++--- 5 files changed, 63 insertions(+), 16 deletions(-) diff --git a/docs/upgrade.html b/docs/upgrade.html index 81af2d65261..d28898590f8 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -170,6 +170,18 @@ For further details, please refer to KIP-1120. +
  • + The metrics org.apache.kafka.server:type=AssignmentsManager.QueuedReplicaToDirAssignments, + org.apache.kafka.storage.internals.log:type=RemoteStorageThreadPool.RemoteLogReaderTaskQueueSize, and + org.apache.kafka.storage.internals.log:type=RemoteStorageThreadPool.RemoteLogReaderAvgIdlePercent + have been deprecated and will be removed in Kafka 5.0. + + As replacements, the following metrics have been introduced, which report the same information: + kafka.server:type=AssignmentsManager.QueuedReplicaToDirAssignments, + kafka.log.remote:type=RemoteStorageThreadPool.RemoteLogReaderTaskQueueSize, and + kafka.log.remote:type=RemoteStorageThreadPool.RemoteLogReaderAvgIdlePercent. + For further details, please refer to KIP-1100. +
  • Upgrading to 4.1.0

    diff --git a/server/src/main/java/org/apache/kafka/server/AssignmentsManager.java b/server/src/main/java/org/apache/kafka/server/AssignmentsManager.java index 5b20e1475a2..34a0584e394 100644 --- a/server/src/main/java/org/apache/kafka/server/AssignmentsManager.java +++ b/server/src/main/java/org/apache/kafka/server/AssignmentsManager.java @@ -71,11 +71,15 @@ public final class AssignmentsManager { */ static final long MIN_NOISY_FAILURE_INTERVAL_NS = TimeUnit.MINUTES.toNanos(2); + @Deprecated(since = "4.2") + static final MetricName DEPRECATED_QUEUED_REPLICA_TO_DIR_ASSIGNMENTS_METRIC = + KafkaYammerMetrics.getMetricName("org.apache.kafka.server", "AssignmentsManager", "QueuedReplicaToDirAssignments"); + /** * The metric reflecting the number of pending assignments. */ static final MetricName QUEUED_REPLICA_TO_DIR_ASSIGNMENTS_METRIC = - metricName("QueuedReplicaToDirAssignments"); + KafkaYammerMetrics.getMetricName("kafka.server", "AssignmentsManager", "QueuedReplicaToDirAssignments"); /** * The event at which we send assignments, if appropriate. @@ -142,10 +146,6 @@ public final class AssignmentsManager { */ private final KafkaEventQueue eventQueue; - static MetricName metricName(String name) { - return KafkaYammerMetrics.getMetricName("org.apache.kafka.server", "AssignmentsManager", name); - } - public AssignmentsManager( Time time, NodeToControllerChannelManager channelManager, @@ -182,12 +182,18 @@ public final class AssignmentsManager { this.ready = new ConcurrentHashMap<>(); this.inflight = Map.of(); this.metricsRegistry = metricsRegistry; + this.metricsRegistry.newGauge(DEPRECATED_QUEUED_REPLICA_TO_DIR_ASSIGNMENTS_METRIC, new Gauge() { + @Override + public Integer value() { + return numPending(); + } + }); this.metricsRegistry.newGauge(QUEUED_REPLICA_TO_DIR_ASSIGNMENTS_METRIC, new Gauge() { - @Override - public Integer value() { - return numPending(); - } - }); + @Override + public Integer value() { + return numPending(); + } + }); this.previousGlobalFailures = 0; this.eventQueue = new KafkaEventQueue(time, new LogContext("[AssignmentsManager id=" + nodeId + "]"), @@ -248,6 +254,7 @@ public final class AssignmentsManager { log.error("Unexpected exception shutting down NodeToControllerChannelManager", e); } try { + metricsRegistry.removeMetric(DEPRECATED_QUEUED_REPLICA_TO_DIR_ASSIGNMENTS_METRIC); metricsRegistry.removeMetric(QUEUED_REPLICA_TO_DIR_ASSIGNMENTS_METRIC); } catch (Exception e) { log.error("Unexpected exception removing metrics.", e); diff --git a/server/src/test/java/org/apache/kafka/server/AssignmentsManagerTest.java b/server/src/test/java/org/apache/kafka/server/AssignmentsManagerTest.java index 3397a7488ff..4c533dd5737 100644 --- a/server/src/test/java/org/apache/kafka/server/AssignmentsManagerTest.java +++ b/server/src/test/java/org/apache/kafka/server/AssignmentsManagerTest.java @@ -250,6 +250,13 @@ public class AssignmentsManagerTest { return queuedReplicaToDirAssignments.value(); } + @SuppressWarnings("unchecked") // do not warn about Gauge typecast. + int deprecatedQueuedReplicaToDirAssignments() { + Gauge queuedReplicaToDirAssignments = + (Gauge) findMetric(AssignmentsManager.DEPRECATED_QUEUED_REPLICA_TO_DIR_ASSIGNMENTS_METRIC); + return queuedReplicaToDirAssignments.value(); + } + @Override public void close() throws Exception { try { @@ -279,10 +286,12 @@ public class AssignmentsManagerTest { public void testSuccessfulAssignment() throws Exception { try (TestEnv testEnv = new TestEnv()) { assertEquals(0, testEnv.queuedReplicaToDirAssignments()); + assertEquals(0, testEnv.deprecatedQueuedReplicaToDirAssignments()); testEnv.onAssignment(new TopicIdPartition(TOPIC_1, 0), DIR_1); TestUtils.retryOnExceptionWithTimeout(60_000, () -> { assertEquals(1, testEnv.assignmentsManager.numPending()); assertEquals(1, testEnv.queuedReplicaToDirAssignments()); + assertEquals(1, testEnv.deprecatedQueuedReplicaToDirAssignments()); }); assertEquals(0, testEnv.assignmentsManager.previousGlobalFailures()); assertEquals(1, testEnv.assignmentsManager.numInFlight()); @@ -290,6 +299,7 @@ public class AssignmentsManagerTest { TestUtils.retryOnExceptionWithTimeout(60_000, () -> { assertEquals(0, testEnv.assignmentsManager.numPending()); assertEquals(0, testEnv.queuedReplicaToDirAssignments()); + assertEquals(0, testEnv.deprecatedQueuedReplicaToDirAssignments()); assertEquals(1, testEnv.success(new TopicIdPartition(TOPIC_1, 0))); }); assertEquals(0, testEnv.assignmentsManager.previousGlobalFailures()); diff --git a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageMetrics.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageMetrics.java index 7922d88d831..8e47a674681 100644 --- a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageMetrics.java +++ b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageMetrics.java @@ -90,10 +90,16 @@ public class RemoteStorageMetrics { "kafka.server", "BrokerTopicMetrics", REMOTE_DELETE_LAG_SEGMENTS); public static final MetricName REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT_METRIC = getMetricName( "kafka.log.remote", "RemoteLogManager", REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT); - public static final MetricName REMOTE_LOG_READER_TASK_QUEUE_SIZE_METRIC = getMetricName( + @Deprecated(since = "4.2") + public static final MetricName DEPRECATE_REMOTE_LOG_READER_TASK_QUEUE_SIZE_METRIC = getMetricName( "org.apache.kafka.storage.internals.log", "RemoteStorageThreadPool", REMOTE_LOG_READER_TASK_QUEUE_SIZE); - public static final MetricName REMOTE_LOG_READER_AVG_IDLE_PERCENT_METRIC = getMetricName( + @Deprecated(since = "4.2") + public static final MetricName DEPRECATE_REMOTE_LOG_READER_AVG_IDLE_PERCENT_METRIC = getMetricName( "org.apache.kafka.storage.internals.log", "RemoteStorageThreadPool", REMOTE_LOG_READER_AVG_IDLE_PERCENT); + public static final MetricName REMOTE_LOG_READER_TASK_QUEUE_SIZE_METRIC = getMetricName( + "kafka.log.remote", "RemoteStorageThreadPool", REMOTE_LOG_READER_TASK_QUEUE_SIZE); + public static final MetricName REMOTE_LOG_READER_AVG_IDLE_PERCENT_METRIC = getMetricName( + "kafka.log.remote", "RemoteStorageThreadPool", REMOTE_LOG_READER_AVG_IDLE_PERCENT); public static final MetricName REMOTE_LOG_READER_FETCH_RATE_AND_TIME_METRIC = getMetricName( "kafka.log.remote", "RemoteLogManager", REMOTE_LOG_READER_FETCH_RATE_AND_TIME_MS); @@ -115,6 +121,8 @@ public class RemoteStorageMetrics { metrics.add(REMOTE_DELETE_LAG_BYTES_METRIC); metrics.add(REMOTE_DELETE_LAG_SEGMENTS_METRIC); metrics.add(REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT_METRIC); + metrics.add(DEPRECATE_REMOTE_LOG_READER_AVG_IDLE_PERCENT_METRIC); + metrics.add(DEPRECATE_REMOTE_LOG_READER_TASK_QUEUE_SIZE_METRIC); metrics.add(REMOTE_LOG_READER_AVG_IDLE_PERCENT_METRIC); metrics.add(REMOTE_LOG_READER_TASK_QUEUE_SIZE_METRIC); metrics.add(REMOTE_LOG_METADATA_COUNT_METRIC); diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteStorageThreadPool.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteStorageThreadPool.java index a09b558b124..692afbddaf2 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteStorageThreadPool.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteStorageThreadPool.java @@ -17,6 +17,7 @@ package org.apache.kafka.storage.internals.log; import org.apache.kafka.common.utils.ThreadUtils; +import org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics; import org.apache.kafka.server.metrics.KafkaMetricsGroup; import org.slf4j.Logger; @@ -32,8 +33,12 @@ import static org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics.RE public final class RemoteStorageThreadPool extends ThreadPoolExecutor { private static final Logger LOGGER = LoggerFactory.getLogger(RemoteStorageThreadPool.class); - private final KafkaMetricsGroup metricsGroup = new KafkaMetricsGroup(this.getClass()); + @Deprecated(since = "4.2") + // This metrics group is used to register deprecated metrics. It will be removed in Kafka 5.0 + private final KafkaMetricsGroup deprecatedLogMetricsGroup = new KafkaMetricsGroup("org.apache.kafka.storage.internals.log", "RemoteStorageThreadPool"); + private final KafkaMetricsGroup logRemoteMetricsGroup = new KafkaMetricsGroup("kafka.log.remote", "RemoteStorageThreadPool"); + @SuppressWarnings("deprecation") public RemoteStorageThreadPool(String threadNamePattern, int numThreads, int maxPendingTasks) { @@ -45,9 +50,13 @@ public final class RemoteStorageThreadPool extends ThreadPoolExecutor { ThreadUtils.createThreadFactory(threadNamePattern, false, (t, e) -> LOGGER.error("Uncaught exception in thread '{}':", t.getName(), e)) ); - metricsGroup.newGauge(REMOTE_LOG_READER_TASK_QUEUE_SIZE_METRIC.getName(), + deprecatedLogMetricsGroup.newGauge(RemoteStorageMetrics.DEPRECATE_REMOTE_LOG_READER_TASK_QUEUE_SIZE_METRIC.getName(), () -> getQueue().size()); - metricsGroup.newGauge(REMOTE_LOG_READER_AVG_IDLE_PERCENT_METRIC.getName(), + deprecatedLogMetricsGroup.newGauge(RemoteStorageMetrics.DEPRECATE_REMOTE_LOG_READER_AVG_IDLE_PERCENT_METRIC.getName(), + () -> 1 - (double) getActiveCount() / (double) getCorePoolSize()); + logRemoteMetricsGroup.newGauge(REMOTE_LOG_READER_TASK_QUEUE_SIZE_METRIC.getName(), + () -> getQueue().size()); + logRemoteMetricsGroup.newGauge(REMOTE_LOG_READER_AVG_IDLE_PERCENT_METRIC.getName(), () -> 1 - (double) getActiveCount() / (double) getCorePoolSize()); } @@ -59,6 +68,7 @@ public final class RemoteStorageThreadPool extends ThreadPoolExecutor { } public void removeMetrics() { - REMOTE_STORAGE_THREAD_POOL_METRICS.forEach(metricsGroup::removeMetric); + REMOTE_STORAGE_THREAD_POOL_METRICS.forEach(deprecatedLogMetricsGroup::removeMetric); + REMOTE_STORAGE_THREAD_POOL_METRICS.forEach(logRemoteMetricsGroup::removeMetric); } } From e27ea8d4dbb17681a05672dd6fedb8a85c37a28f Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Mon, 29 Sep 2025 01:32:27 +0800 Subject: [PATCH 087/100] KAFKA-19702 Move MetadataVersionConfigValidator and related test code to metadata module (#20526) 1. Move `MetadataVersionConfigValidator` to metadata module. 2. Move `MetadataVersionConfigValidatorTest` to metadata module. 3. Remove `KafkaConfig#validateWithMetadataVersion`. Reviewers: Chia-Ping Tsai --- .../scala/kafka/server/BrokerServer.scala | 7 +- .../main/scala/kafka/server/KafkaConfig.scala | 13 -- .../MetadataVersionConfigValidatorTest.java | 103 ------------- .../scala/unit/kafka/log/LogConfigTest.scala | 18 --- .../MetadataVersionConfigValidator.java | 37 +++-- .../MetadataVersionConfigValidatorTest.java | 140 ++++++++++++++++++ 6 files changed, 169 insertions(+), 149 deletions(-) delete mode 100644 core/src/test/java/kafka/server/MetadataVersionConfigValidatorTest.java rename {core/src/main/java/kafka/server => metadata/src/main/java/org/apache/kafka/metadata}/MetadataVersionConfigValidator.java (64%) create mode 100644 metadata/src/test/java/org/apache/kafka/metadata/MetadataVersionConfigValidatorTest.java diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 689c62b8687..a9217c4d023 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -41,7 +41,7 @@ import org.apache.kafka.coordinator.share.metrics.{ShareCoordinatorMetrics, Shar import org.apache.kafka.coordinator.share.{ShareCoordinator, ShareCoordinatorRecordSerde, ShareCoordinatorService} import org.apache.kafka.coordinator.transaction.ProducerIdManager import org.apache.kafka.image.publisher.{BrokerRegistrationTracker, MetadataPublisher} -import org.apache.kafka.metadata.{BrokerState, ListenerInfo} +import org.apache.kafka.metadata.{BrokerState, ListenerInfo, MetadataVersionConfigValidator} import org.apache.kafka.metadata.publisher.{AclPublisher, DelegationTokenPublisher, ScramPublisher} import org.apache.kafka.security.{CredentialProvider, DelegationTokenManager} import org.apache.kafka.server.authorizer.Authorizer @@ -469,7 +469,10 @@ class BrokerServer( socketServer.dataPlaneRequestChannel, dataPlaneRequestProcessor, time, config.numIoThreads, "RequestHandlerAvgIdlePercent") - metadataPublishers.add(new MetadataVersionConfigValidator(config, sharedServer.metadataPublishingFaultHandler)) + metadataPublishers.add(new MetadataVersionConfigValidator(config.brokerId, + () => config.processRoles.contains(ProcessRole.BrokerRole) && config.logDirs().size() > 1, + sharedServer.metadataPublishingFaultHandler + )) brokerMetadataPublisher = new BrokerMetadataPublisher(config, metadataCache, logManager, diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 671bee32d28..d9b8c5cd91f 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -43,7 +43,6 @@ import org.apache.kafka.raft.{MetadataLogConfig, QuorumConfig} import org.apache.kafka.security.authorizer.AuthorizerUtils import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.authorizer.Authorizer -import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.config.AbstractKafkaConfig.getMap import org.apache.kafka.server.config.{AbstractKafkaConfig, KRaftConfigs, QuotaConfig, ReplicationConfigs, ServerConfigs, ServerLogConfigs} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig @@ -653,18 +652,6 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) s"${BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG} must implement KafkaPrincipalSerde") } - /** - * Validate some configurations for new MetadataVersion. A new MetadataVersion can take place when - * a FeatureLevelRecord for "metadata.version" is read from the cluster metadata. - */ - def validateWithMetadataVersion(metadataVersion: MetadataVersion): Unit = { - if (processRoles.contains(ProcessRole.BrokerRole) && logDirs.size > 1) { - require(metadataVersion.isDirectoryAssignmentSupported, - s"Multiple log directories (aka JBOD) are not supported in the current MetadataVersion ${metadataVersion}. " + - s"Need ${MetadataVersion.IBP_3_7_IV2} or higher") - } - } - /** * Copy the subset of properties that are relevant to Logs. The individual properties * are listed here since the names are slightly different in each Config class... diff --git a/core/src/test/java/kafka/server/MetadataVersionConfigValidatorTest.java b/core/src/test/java/kafka/server/MetadataVersionConfigValidatorTest.java deleted file mode 100644 index daa0aacca7d..00000000000 --- a/core/src/test/java/kafka/server/MetadataVersionConfigValidatorTest.java +++ /dev/null @@ -1,103 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.server; - -import org.apache.kafka.common.metadata.FeatureLevelRecord; -import org.apache.kafka.image.MetadataDelta; -import org.apache.kafka.image.MetadataImage; -import org.apache.kafka.image.MetadataProvenance; -import org.apache.kafka.image.loader.LogDeltaManifest; -import org.apache.kafka.raft.LeaderAndEpoch; -import org.apache.kafka.server.common.MetadataVersion; -import org.apache.kafka.server.fault.FaultHandler; - -import org.junit.jupiter.api.Test; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.BDDMockito.willAnswer; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoMoreInteractions; -import static org.mockito.Mockito.when; - -public class MetadataVersionConfigValidatorTest { - - private static final LogDeltaManifest TEST_MANIFEST = LogDeltaManifest.newBuilder() - .provenance(MetadataProvenance.EMPTY) - .leaderAndEpoch(LeaderAndEpoch.UNKNOWN) - .numBatches(1) - .elapsedNs(90) - .numBytes(88) - .build(); - public static final MetadataProvenance TEST_PROVENANCE = - new MetadataProvenance(50, 3, 8000, true); - - void testWith(MetadataVersion metadataVersion, KafkaConfig config, FaultHandler faultHandler) throws Exception { - try (MetadataVersionConfigValidator validator = new MetadataVersionConfigValidator(config, faultHandler)) { - MetadataDelta delta = new MetadataDelta.Builder() - .setImage(MetadataImage.EMPTY) - .build(); - if (metadataVersion != null) { - delta.replay(new FeatureLevelRecord(). - setName(MetadataVersion.FEATURE_NAME). - setFeatureLevel(metadataVersion.featureLevel())); - } - MetadataImage image = delta.apply(TEST_PROVENANCE); - - validator.onMetadataUpdate(delta, image, TEST_MANIFEST); - } - } - - @Test - void testValidatesConfigOnMetadataChange() throws Exception { - MetadataVersion metadataVersion = MetadataVersion.IBP_3_7_IV2; - KafkaConfig config = mock(KafkaConfig.class); - FaultHandler faultHandler = mock(FaultHandler.class); - - when(config.brokerId()).thenReturn(8); - - testWith(metadataVersion, config, faultHandler); - - verify(config, times(1)).validateWithMetadataVersion(eq(metadataVersion)); - verifyNoMoreInteractions(faultHandler); - } - - @SuppressWarnings("ThrowableNotThrown") - @Test - void testInvokesFaultHandlerOnException() throws Exception { - MetadataVersion metadataVersion = MetadataVersion.IBP_3_7_IV2; - Exception exception = new Exception(); - KafkaConfig config = mock(KafkaConfig.class); - FaultHandler faultHandler = mock(FaultHandler.class); - - when(faultHandler.handleFault(any(), any())).thenReturn(new RuntimeException("returned exception")); - when(config.brokerId()).thenReturn(8); - willAnswer(invocation -> { - throw exception; - }).given(config).validateWithMetadataVersion(eq(metadataVersion)); - - testWith(metadataVersion, config, faultHandler); - - verify(config, times(1)).validateWithMetadataVersion(eq(metadataVersion)); - verify(faultHandler, times(1)).handleFault( - eq("Broker configuration does not support the cluster MetadataVersion"), - eq(exception)); - } -} diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index e23e16fa40a..e942e7e3380 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -23,7 +23,6 @@ import org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM import org.apache.kafka.common.config.ConfigDef.Type.INT import org.apache.kafka.common.config.{ConfigException, SslConfigs, TopicConfig} import org.apache.kafka.common.errors.InvalidConfigurationException -import org.apache.kafka.server.common.MetadataVersion import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -429,21 +428,4 @@ class LogConfigTest { logProps.put(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, deleteOnDisable.toString) LogConfig.validate(logProps) } - - @Test - def testValidateWithMetadataVersionJbodSupport(): Unit = { - def validate(metadataVersion: MetadataVersion, jbodConfig: Boolean): Unit = - KafkaConfig.fromProps( - TestUtils.createBrokerConfig(nodeId = 0, logDirCount = if (jbodConfig) 2 else 1) - ).validateWithMetadataVersion(metadataVersion) - - validate(MetadataVersion.IBP_3_6_IV2, jbodConfig = false) - validate(MetadataVersion.IBP_3_7_IV0, jbodConfig = false) - validate(MetadataVersion.IBP_3_7_IV2, jbodConfig = false) - assertThrows(classOf[IllegalArgumentException], () => - validate(MetadataVersion.IBP_3_6_IV2, jbodConfig = true)) - assertThrows(classOf[IllegalArgumentException], () => - validate(MetadataVersion.IBP_3_7_IV0, jbodConfig = true)) - validate(MetadataVersion.IBP_3_7_IV2, jbodConfig = true) - } } diff --git a/core/src/main/java/kafka/server/MetadataVersionConfigValidator.java b/metadata/src/main/java/org/apache/kafka/metadata/MetadataVersionConfigValidator.java similarity index 64% rename from core/src/main/java/kafka/server/MetadataVersionConfigValidator.java rename to metadata/src/main/java/org/apache/kafka/metadata/MetadataVersionConfigValidator.java index 6f53ec092f6..2df73463c25 100644 --- a/core/src/main/java/kafka/server/MetadataVersionConfigValidator.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/MetadataVersionConfigValidator.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package kafka.server; +package org.apache.kafka.metadata; import org.apache.kafka.image.MetadataDelta; import org.apache.kafka.image.MetadataImage; @@ -24,18 +24,20 @@ import org.apache.kafka.image.publisher.MetadataPublisher; import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.fault.FaultHandler; +import java.util.function.Supplier; + public class MetadataVersionConfigValidator implements MetadataPublisher { private final String name; - private final KafkaConfig config; + private final Supplier hasMultiLogDirs; private final FaultHandler faultHandler; public MetadataVersionConfigValidator( - KafkaConfig config, - FaultHandler faultHandler + int id, + Supplier hasMultiLogDirs, + FaultHandler faultHandler ) { - int id = config.brokerId(); this.name = "MetadataVersionPublisher(id=" + id + ")"; - this.config = config; + this.hasMultiLogDirs = hasMultiLogDirs; this.faultHandler = faultHandler; } @@ -46,9 +48,9 @@ public class MetadataVersionConfigValidator implements MetadataPublisher { @Override public void onMetadataUpdate( - MetadataDelta delta, - MetadataImage newImage, - LoaderManifest manifest + MetadataDelta delta, + MetadataImage newImage, + LoaderManifest manifest ) { if (delta.featuresDelta() != null) { if (delta.metadataVersionChanged().isPresent()) { @@ -57,13 +59,22 @@ public class MetadataVersionConfigValidator implements MetadataPublisher { } } + /** + * Validate some configurations for the new MetadataVersion. A new MetadataVersion can take place when + * a FeatureLevelRecord for "metadata.version" is read from the cluster metadata. + */ @SuppressWarnings("ThrowableNotThrown") private void onMetadataVersionChanged(MetadataVersion metadataVersion) { - try { - this.config.validateWithMetadataVersion(metadataVersion); - } catch (Throwable t) { + if (this.hasMultiLogDirs.get() && !metadataVersion.isDirectoryAssignmentSupported()) { + String errorMsg = String.format( + "Multiple log directories (aka JBOD) are not supported in the current MetadataVersion %s. Need %s or higher", + metadataVersion, MetadataVersion.IBP_3_7_IV2 + ); + this.faultHandler.handleFault( - "Broker configuration does not support the cluster MetadataVersion", t); + "Broker configuration does not support the cluster MetadataVersion", + new IllegalArgumentException(errorMsg) + ); } } } diff --git a/metadata/src/test/java/org/apache/kafka/metadata/MetadataVersionConfigValidatorTest.java b/metadata/src/test/java/org/apache/kafka/metadata/MetadataVersionConfigValidatorTest.java new file mode 100644 index 00000000000..50ad1b07ccd --- /dev/null +++ b/metadata/src/test/java/org/apache/kafka/metadata/MetadataVersionConfigValidatorTest.java @@ -0,0 +1,140 @@ +/* + * 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.metadata; + +import org.apache.kafka.common.metadata.FeatureLevelRecord; +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.image.MetadataProvenance; +import org.apache.kafka.image.loader.LogDeltaManifest; +import org.apache.kafka.raft.LeaderAndEpoch; +import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.fault.FaultHandler; + +import org.junit.jupiter.api.Test; + +import java.util.function.Supplier; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + +@SuppressWarnings({"unchecked", "ThrowableNotThrown"}) +public class MetadataVersionConfigValidatorTest { + + private static final LogDeltaManifest TEST_MANIFEST = LogDeltaManifest.newBuilder() + .provenance(MetadataProvenance.EMPTY) + .leaderAndEpoch(LeaderAndEpoch.UNKNOWN) + .numBatches(1) + .elapsedNs(90) + .numBytes(88) + .build(); + public static final MetadataProvenance TEST_PROVENANCE = + new MetadataProvenance(50, 3, 8000, true); + + void executeMetadataUpdate( + MetadataVersion metadataVersion, + Supplier multiLogDirSupplier, + FaultHandler faultHandler + ) throws Exception { + try (MetadataVersionConfigValidator validator = new MetadataVersionConfigValidator(0, multiLogDirSupplier, faultHandler)) { + MetadataDelta delta = new MetadataDelta.Builder() + .setImage(MetadataImage.EMPTY) + .build(); + if (metadataVersion != null) { + delta.replay(new FeatureLevelRecord(). + setName(MetadataVersion.FEATURE_NAME). + setFeatureLevel(metadataVersion.featureLevel())); + } + MetadataImage image = delta.apply(TEST_PROVENANCE); + + validator.onMetadataUpdate(delta, image, TEST_MANIFEST); + } + } + + @Test + void testValidatesConfigOnMetadataChange() throws Exception { + MetadataVersion metadataVersion = MetadataVersion.IBP_3_7_IV2; + FaultHandler faultHandler = mock(FaultHandler.class); + Supplier multiLogDirSupplier = mock(Supplier.class); + when(multiLogDirSupplier.get()).thenReturn(false); + + executeMetadataUpdate(metadataVersion, multiLogDirSupplier, faultHandler); + + verify(multiLogDirSupplier, times(1)).get(); + verifyNoMoreInteractions(faultHandler); + } + + @Test + void testInvokesFaultHandlerOnException() throws Exception { + MetadataVersion metadataVersion = MetadataVersion.IBP_3_7_IV1; + Supplier multiLogDirSupplier = mock(Supplier.class); + FaultHandler faultHandler = mock(FaultHandler.class); + + when(multiLogDirSupplier.get()).thenReturn(true); + + executeMetadataUpdate(metadataVersion, multiLogDirSupplier, faultHandler); + + verify(multiLogDirSupplier, times(1)).get(); + verify(faultHandler, times(1)).handleFault( + eq("Broker configuration does not support the cluster MetadataVersion"), + any(IllegalArgumentException.class)); + } + + @Test + void testValidateWithMetadataVersionJbodSupport() throws Exception { + FaultHandler faultHandler = mock(FaultHandler.class); + validate(MetadataVersion.IBP_3_6_IV2, false, faultHandler); + verifyNoMoreInteractions(faultHandler); + + faultHandler = mock(FaultHandler.class); + validate(MetadataVersion.IBP_3_7_IV0, false, faultHandler); + verifyNoMoreInteractions(faultHandler); + + faultHandler = mock(FaultHandler.class); + validate(MetadataVersion.IBP_3_7_IV2, false, faultHandler); + verifyNoMoreInteractions(faultHandler); + + faultHandler = mock(FaultHandler.class); + validate(MetadataVersion.IBP_3_6_IV2, true, faultHandler); + verify(faultHandler, times(1)).handleFault( + eq("Broker configuration does not support the cluster MetadataVersion"), + any(IllegalArgumentException.class)); + + faultHandler = mock(FaultHandler.class); + validate(MetadataVersion.IBP_3_7_IV0, true, faultHandler); + verify(faultHandler, times(1)).handleFault( + eq("Broker configuration does not support the cluster MetadataVersion"), + any(IllegalArgumentException.class)); + + faultHandler = mock(FaultHandler.class); + validate(MetadataVersion.IBP_3_7_IV2, true, faultHandler); + verifyNoMoreInteractions(faultHandler); + } + + private void validate(MetadataVersion metadataVersion, boolean jbodConfig, FaultHandler faultHandler) + throws Exception { + Supplier multiLogDirSupplier = () -> jbodConfig; + + executeMetadataUpdate(metadataVersion, multiLogDirSupplier, faultHandler); + } +} From c2aeec46a2899c6f8ad27bab65baeeaa87f1dacf Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Mon, 29 Sep 2025 01:37:58 +0800 Subject: [PATCH 088/100] MINOR: Remove logContext arrtibute from StreamsGroup and CoordinatorRuntime (#20572) The `logContext` attribute in `StreamsGroup` and `CoordinatorRuntime` is not used anymore. This patch removes it. Reviewers: Ken Huang , TengYao Chi , Chia-Ping Tsai --- .../kafka/common/network/ChannelBuilders.java | 2 +- .../kafka/common/network/SslChannelBuilder.java | 4 +--- .../kafka/common/network/SslSelectorTest.java | 9 ++++----- .../common/network/SslTransportLayerTest.java | 14 ++++++-------- .../common/network/SslTransportTls12Tls13Test.java | 5 ++--- .../common/runtime/CoordinatorRuntime.java | 6 ------ .../coordinator/group/streams/StreamsGroup.java | 2 -- 7 files changed, 14 insertions(+), 28 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java index aea38c72cac..847f887a324 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java +++ b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java @@ -125,7 +125,7 @@ public class ChannelBuilders { switch (securityProtocol) { case SSL: requireNonNullMode(connectionMode, securityProtocol); - channelBuilder = new SslChannelBuilder(connectionMode, listenerName, isInterBrokerListener, logContext); + channelBuilder = new SslChannelBuilder(connectionMode, listenerName, isInterBrokerListener); break; case SASL_SSL: case SASL_PLAINTEXT: diff --git a/clients/src/main/java/org/apache/kafka/common/network/SslChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/SslChannelBuilder.java index a35a0b8b209..249fcad163a 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SslChannelBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SslChannelBuilder.java @@ -26,7 +26,6 @@ import org.apache.kafka.common.security.auth.KafkaPrincipalSerde; import org.apache.kafka.common.security.auth.SslAuthenticationContext; import org.apache.kafka.common.security.ssl.SslFactory; import org.apache.kafka.common.security.ssl.SslPrincipalMapper; -import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Utils; import java.io.Closeable; @@ -53,8 +52,7 @@ public class SslChannelBuilder implements ChannelBuilder, ListenerReconfigurable */ public SslChannelBuilder(ConnectionMode connectionMode, ListenerName listenerName, - boolean isInterBrokerListener, - LogContext logContext) { + boolean isInterBrokerListener) { this.connectionMode = connectionMode; this.listenerName = listenerName; this.isInterBrokerListener = isInterBrokerListener; diff --git a/clients/src/test/java/org/apache/kafka/common/network/SslSelectorTest.java b/clients/src/test/java/org/apache/kafka/common/network/SslSelectorTest.java index 8a9704c1621..572ec443e08 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SslSelectorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SslSelectorTest.java @@ -74,11 +74,10 @@ public abstract class SslSelectorTest extends SelectorTest { this.server.start(); this.time = new MockTime(); sslClientConfigs = createSslClientConfigs(trustStoreFile); - LogContext logContext = new LogContext(); - this.channelBuilder = new SslChannelBuilder(ConnectionMode.CLIENT, null, false, logContext); + this.channelBuilder = new SslChannelBuilder(ConnectionMode.CLIENT, null, false); this.channelBuilder.configure(sslClientConfigs); this.metrics = new Metrics(); - this.selector = new Selector(5000, metrics, time, "MetricGroup", channelBuilder, logContext); + this.selector = new Selector(5000, metrics, time, "MetricGroup", channelBuilder, new LogContext()); } protected abstract Map createSslClientConfigs(File trustStoreFile) throws GeneralSecurityException, IOException; @@ -255,7 +254,7 @@ public abstract class SslSelectorTest extends SelectorTest { .tlsProtocol(tlsProtocol) .createNewTrustStore(trustStoreFile) .build(); - channelBuilder = new SslChannelBuilder(ConnectionMode.SERVER, null, false, new LogContext()); + channelBuilder = new SslChannelBuilder(ConnectionMode.SERVER, null, false); channelBuilder.configure(sslServerConfigs); selector = new Selector(NetworkReceive.UNLIMITED, 5000, metrics, time, "MetricGroup", new HashMap<>(), true, false, channelBuilder, pool, new LogContext()); @@ -342,7 +341,7 @@ public abstract class SslSelectorTest extends SelectorTest { private static class TestSslChannelBuilder extends SslChannelBuilder { public TestSslChannelBuilder(ConnectionMode connectionMode) { - super(connectionMode, null, false, new LogContext()); + super(connectionMode, null, false); } @Override diff --git a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java index 72f130ca4e3..9525ccfbc82 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java @@ -767,11 +767,10 @@ public class SslTransportLayerTest { @ParameterizedTest @ArgumentsSource(SslTransportLayerArgumentsProvider.class) public void testNetworkThreadTimeRecorded(Args args) throws Exception { - LogContext logContext = new LogContext(); - ChannelBuilder channelBuilder = new SslChannelBuilder(ConnectionMode.CLIENT, null, false, logContext); + ChannelBuilder channelBuilder = new SslChannelBuilder(ConnectionMode.CLIENT, null, false); channelBuilder.configure(args.sslClientConfigs); try (Selector selector = new Selector(NetworkReceive.UNLIMITED, Selector.NO_IDLE_TIMEOUT_MS, new Metrics(), Time.SYSTEM, - "MetricGroup", new HashMap<>(), false, true, channelBuilder, MemoryPool.NONE, logContext)) { + "MetricGroup", new HashMap<>(), false, true, channelBuilder, MemoryPool.NONE, new LogContext())) { String node = "0"; server = createEchoServer(args, SecurityProtocol.SSL); @@ -967,7 +966,7 @@ public class SslTransportLayerTest { } private SslChannelBuilder newClientChannelBuilder() { - return new SslChannelBuilder(ConnectionMode.CLIENT, null, false, new LogContext()); + return new SslChannelBuilder(ConnectionMode.CLIENT, null, false); } private void testClose(Args args, SecurityProtocol securityProtocol, ChannelBuilder clientChannelBuilder) throws Exception { @@ -1311,10 +1310,9 @@ public class SslTransportLayerTest { } private Selector createSelector(Args args) { - LogContext logContext = new LogContext(); - ChannelBuilder channelBuilder = new SslChannelBuilder(ConnectionMode.CLIENT, null, false, logContext); + ChannelBuilder channelBuilder = new SslChannelBuilder(ConnectionMode.CLIENT, null, false); channelBuilder.configure(args.sslClientConfigs); - selector = new Selector(5000, new Metrics(), TIME, "MetricGroup", channelBuilder, logContext); + selector = new Selector(5000, new Metrics(), TIME, "MetricGroup", channelBuilder, new LogContext()); return selector; } @@ -1371,7 +1369,7 @@ public class SslTransportLayerTest { int flushDelayCount = 0; public TestSslChannelBuilder(ConnectionMode connectionMode) { - super(connectionMode, null, false, new LogContext()); + super(connectionMode, null, false); } public void configureBufferSizes(Integer netReadBufSize, Integer netWriteBufSize, Integer appBufSize) { diff --git a/clients/src/test/java/org/apache/kafka/common/network/SslTransportTls12Tls13Test.java b/clients/src/test/java/org/apache/kafka/common/network/SslTransportTls12Tls13Test.java index 0d67c8aee63..0ad81d17b5b 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SslTransportTls12Tls13Test.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SslTransportTls12Tls13Test.java @@ -50,10 +50,9 @@ public class SslTransportTls12Tls13Test { sslServerConfigs = serverCertStores.getTrustingConfig(clientCertStores); sslClientConfigs = clientCertStores.getTrustingConfig(serverCertStores); - LogContext logContext = new LogContext(); - ChannelBuilder channelBuilder = new SslChannelBuilder(ConnectionMode.CLIENT, null, false, logContext); + ChannelBuilder channelBuilder = new SslChannelBuilder(ConnectionMode.CLIENT, null, false); channelBuilder.configure(sslClientConfigs); - this.selector = new Selector(5000, new Metrics(), TIME, "MetricGroup", channelBuilder, logContext); + this.selector = new Selector(5000, new Metrics(), TIME, "MetricGroup", channelBuilder, new LogContext()); } @AfterEach 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 27dd6518f1a..52d3f27f3cf 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 @@ -1923,11 +1923,6 @@ public class CoordinatorRuntime, U> implements Aut */ private final String logPrefix; - /** - * The log context. - */ - private final LogContext logContext; - /** * The logger. */ @@ -2054,7 +2049,6 @@ public class CoordinatorRuntime, U> implements Aut ExecutorService executorService ) { this.logPrefix = logPrefix; - this.logContext = logContext; this.log = logContext.logger(CoordinatorRuntime.class); this.time = time; this.timer = timer; diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java index 61f61d101f1..7ec3596628e 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java @@ -113,7 +113,6 @@ public class StreamsGroup implements Group { } } - private final LogContext logContext; private final Logger log; /** @@ -217,7 +216,6 @@ public class StreamsGroup implements Group { String groupId ) { this.log = logContext.logger(StreamsGroup.class); - this.logContext = logContext; this.snapshotRegistry = Objects.requireNonNull(snapshotRegistry); this.groupId = Objects.requireNonNull(groupId); this.state = new TimelineObject<>(snapshotRegistry, EMPTY); From 3c0843961bd77f721a91f5e92e1624144859fe42 Mon Sep 17 00:00:00 2001 From: Patrik Nagy <77003490+pnagy-cldr@users.noreply.github.com> Date: Mon, 29 Sep 2025 12:30:17 +0200 Subject: [PATCH 089/100] KAFKA-19739 Upgrade commons-validator to 1.10.0 (#20601) In [KAFKA-19359](https://issues.apache.org/jira/browse/KAFKA-19359), the commons-beanutils transitive dependency was force bumped in the project to avoid related CVEs. The commons-validator already has a new release, which solves this problem: https://github.com/apache/commons-validator/tags The workaround could be deleted as part of the version bump. Reviewers: Chia-Ping Tsai --- LICENSE-binary | 2 +- build.gradle | 2 -- gradle/dependencies.gradle | 4 +--- 3 files changed, 2 insertions(+), 6 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index c8fa1e8207a..c6078a48bca 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -211,7 +211,7 @@ License Version 2.0: - commons-digester-2.1 - commons-lang3-3.18.0 - commons-logging-1.3.5 -- commons-validator-1.9.0 +- commons-validator-1.10.0 - hash4j-0.22.0 - jackson-annotations-2.19.0 - jackson-core-2.19.0 diff --git a/build.gradle b/build.gradle index 29537dd77dc..8d18d0ea7b8 100644 --- a/build.gradle +++ b/build.gradle @@ -193,8 +193,6 @@ 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, libs.commonsLang ) diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 390c7e830c8..31adcd2c268 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -57,9 +57,8 @@ versions += [ caffeine: "3.2.0", bndlib: "7.1.0", checkstyle: project.hasProperty('checkstyleVersion') ? checkstyleVersion : "10.20.2", - commonsBeanutils: "1.11.0", commonsLang: "3.18.0", - commonsValidator: "1.9.0", + commonsValidator: "1.10.0", classgraph: "4.8.179", gradle: "8.14.3", grgit: "4.1.1", @@ -151,7 +150,6 @@ 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", commonsLang: "org.apache.commons:commons-lang3:$versions.commonsLang", commonsValidator: "commons-validator:commons-validator:$versions.commonsValidator", jacksonAnnotations: "com.fasterxml.jackson.core:jackson-annotations:$versions.jackson", From 92169b8f08c21826612a0aba01e5eda3464923c7 Mon Sep 17 00:00:00 2001 From: YuChia Ma <79797958+Mirai1129@users.noreply.github.com> Date: Mon, 29 Sep 2025 23:06:56 +0800 Subject: [PATCH 090/100] KAFKA-19357 AsyncConsumer#close hangs as commitAsync never completes when coordinator is missing (#19914) Problem: When AsyncConsumer is closing, CoordinatorRequestManager stops looking for coordinator by returning EMPTY in poll() method when closing flag is true. This prevents commitAsync() and other coordinator-dependent operations from completing, causing close() to hang until timeout. Solution: Modified the closing flag check in poll() method of CommitRequestManager to be more targeted: - When both coordinators are unknown and the consumer is closing, only return EMPTY - When this condition is met, proactively fail all pending commit requests with CommitFailedException - This allows coordinator lookup to continue when coordinator is available during shutdown, while preventing indefinite hangs when coordinator is unreachable Reviewers: PoAn Yang , Andrew Schofield , TengYao Chi , Kirk True , Jhen-Yung Hsu , Lan Ding , TaiJuWu , Ken Huang , KuoChe , Chia-Ping Tsai --- .../consumer/PlaintextConsumerCommitTest.java | 37 +++++++++++++++++ .../internals/CommitRequestManager.java | 8 ++++ .../internals/CommitRequestManagerTest.java | 41 +++++++++++++++++++ 3 files changed, 86 insertions(+) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java index b5bd27cf41b..c00d1ddab90 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java @@ -28,6 +28,7 @@ import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.test.MockConsumerInterceptor; +import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.BeforeEach; @@ -452,6 +453,40 @@ public class PlaintextConsumerCommitTest { } } + /** + * This is testing when closing the consumer but commit request has already been sent. + * During the closing, the consumer won't find the coordinator anymore. + */ + @ClusterTest + public void testCommitAsyncFailsWhenCoordinatorUnavailableDuringClose() throws InterruptedException { + try (Producer producer = cluster.producer(); + var consumer = createConsumer(GroupProtocol.CONSUMER, false) + ) { + sendRecords(producer, tp, 3, System.currentTimeMillis()); + consumer.assign(List.of(tp)); + + var callback = new CountConsumerCommitCallback(); + + // Close the coordinator before committing because otherwise the commit will fail to find the coordinator. + cluster.brokerIds().forEach(cluster::shutdownBroker); + + TestUtils.waitForCondition(() -> cluster.aliveBrokers().isEmpty(), "All brokers should be shut down"); + + consumer.poll(Duration.ofMillis(500)); + consumer.commitAsync(Map.of(tp, new OffsetAndMetadata(1L)), callback); + + long startTime = System.currentTimeMillis(); + consumer.close(CloseOptions.timeout(Duration.ofMillis(500))); + long closeDuration = System.currentTimeMillis() - startTime; + + assertTrue(closeDuration < 1000, "The closing process for the consumer was too long: " + closeDuration + " ms"); + assertTrue(callback.lastError.isPresent()); + assertEquals(CommitFailedException.class, callback.lastError.get().getClass()); + assertEquals("Failed to commit offsets: Coordinator unknown and consumer is closing", callback.lastError.get().getMessage()); + assertEquals(1, callback.exceptionCount); + } + } + // TODO: This only works in the new consumer, but should be fixed for the old consumer as well @ClusterTest public void testCommitAsyncCompletedBeforeConsumerCloses() throws InterruptedException { @@ -575,6 +610,7 @@ public class PlaintextConsumerCommitTest { private static class CountConsumerCommitCallback implements OffsetCommitCallback { private int successCount = 0; + private int exceptionCount = 0; private Optional lastError = Optional.empty(); @Override @@ -582,6 +618,7 @@ public class PlaintextConsumerCommitTest { if (exception == null) { successCount += 1; } else { + exceptionCount += 1; lastError = Optional.of(exception); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java index fe4d3806f2a..6aae084fd47 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java @@ -181,6 +181,14 @@ public class CommitRequestManager implements RequestManager, MemberStateListener // poll when the coordinator node is known and fatal error is not present if (coordinatorRequestManager.coordinator().isEmpty()) { pendingRequests.maybeFailOnCoordinatorFatalError(); + + if (closing && pendingRequests.hasUnsentRequests()) { + CommitFailedException exception = new CommitFailedException( + "Failed to commit offsets: Coordinator unknown and consumer is closing"); + pendingRequests.drainPendingCommits() + .forEach(request -> request.future().completeExceptionally(exception)); + } + return EMPTY; } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java index 26d39715d27..afbb81eb53f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java @@ -1494,6 +1494,47 @@ public class CommitRequestManagerTest { assertTrue(commitRequestManager.pendingRequests.unsentOffsetCommits.isEmpty()); } + @Test + public void testPollWithFatalErrorDuringCoordinatorIsEmptyAndClosing() { + CommitRequestManager commitRequestManager = create(true, 100); + + Map offsets = Map.of(new TopicPartition("topic", 1), + new OffsetAndMetadata(0)); + + var commitFuture = commitRequestManager.commitAsync(offsets); + + commitRequestManager.signalClose(); + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); + when(coordinatorRequestManager.fatalError()) + .thenReturn(Optional.of(new GroupAuthorizationException("Fatal error"))); + + assertEquals(NetworkClientDelegate.PollResult.EMPTY, commitRequestManager.poll(time.milliseconds())); + + assertTrue(commitFuture.isCompletedExceptionally()); + + TestUtils.assertFutureThrows(GroupAuthorizationException.class, commitFuture, "Fatal error"); + } + + @Test + public void testPollWithClosingAndPendingRequests() { + CommitRequestManager commitRequestManager = create(true, 100); + + Map offsets = Map.of(new TopicPartition("topic", 1), + new OffsetAndMetadata(0)); + + var commitFuture = commitRequestManager.commitAsync(offsets); + + commitRequestManager.signalClose(); + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); + + assertEquals(NetworkClientDelegate.PollResult.EMPTY, commitRequestManager.poll(time.milliseconds())); + + assertTrue(commitFuture.isCompletedExceptionally()); + + TestUtils.assertFutureThrows(CommitFailedException.class, commitFuture, + "Failed to commit offsets: Coordinator unknown and consumer is closing"); + } + // Supplies (error, isRetriable) private static Stream partitionDataErrorSupplier() { return Stream.of( From 71c5a426b8a02b80b5b7d2494ac278467d2c2f29 Mon Sep 17 00:00:00 2001 From: Deep Golani <54791570+deepgolani4@users.noreply.github.com> Date: Mon, 29 Sep 2025 11:23:05 -0400 Subject: [PATCH 091/100] KAFKA-12506: Strengthen AdjustStreamThreadCountTest with stateful counting and higher throughput (#20540) Add count store and output topic; produce 1,000 records across 50 keys to better exercise concurrency. Reviewers: Matthias J. Sax --- .../AdjustStreamThreadCountTest.java | 43 ++++++++++++++++++- 1 file changed, 41 insertions(+), 2 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/AdjustStreamThreadCountTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/AdjustStreamThreadCountTest.java index df96837458c..d9779b5a9c5 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/AdjustStreamThreadCountTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/AdjustStreamThreadCountTest.java @@ -17,6 +17,9 @@ package org.apache.kafka.streams.integration; import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.LogCaptureAppender; @@ -28,6 +31,9 @@ import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThr import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.kstream.Named; import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorContext; @@ -93,6 +99,7 @@ public class AdjustStreamThreadCountTest { private final List stateTransitionHistory = new ArrayList<>(); private static String inputTopic; + private static String outputTopic; private static StreamsBuilder builder; private static Properties properties; private static String appId = ""; @@ -103,10 +110,21 @@ public class AdjustStreamThreadCountTest { final String testId = safeUniqueTestName(testInfo); appId = "appId_" + testId; inputTopic = "input" + testId; + outputTopic = "output" + testId; IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, inputTopic); builder = new StreamsBuilder(); - builder.stream(inputTopic); + // Build a simple stateful topology to exercise concurrency with state stores + final KStream source = builder.stream(inputTopic); + final KTable counts = source + .groupByKey() + .count(Named.as("counts"), Materialized.as("counts-store")); + counts + .toStream() + .mapValues(Object::toString) + .to(outputTopic); + + produceTestRecords(inputTopic, CLUSTER); properties = mkObjectProperties( mkMap( @@ -121,6 +139,21 @@ public class AdjustStreamThreadCountTest { ); } + private void produceTestRecords(final String inputTopic, final EmbeddedKafkaCluster cluster) { + final Properties props = new Properties(); + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); + props.put(ProducerConfig.CLIENT_ID_CONFIG, "test-client"); + props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); + props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); + try (KafkaProducer producer = new KafkaProducer<>(props)) { + for (int i = 0; i < 1000; i++) { + final String key = "key-" + (i % 50); + final String value = "value-" + i; + producer.send(new ProducerRecord<>(inputTopic, key, value)); + } + } + } + private void startStreamsAndWaitForRunning(final KafkaStreams kafkaStreams) throws InterruptedException { kafkaStreams.start(); waitForRunning(); @@ -251,7 +284,13 @@ public class AdjustStreamThreadCountTest { assertTrue(latch.await(30, TimeUnit.SECONDS)); one.join(); two.join(); - + waitForCondition( + () -> kafkaStreams.metadataForLocalThreads().size() == oldThreadCount && + kafkaStreams.state() == KafkaStreams.State.RUNNING, + DEFAULT_DURATION.toMillis(), + "Kafka Streams did not stabilize at the expected thread count and RUNNING state." + ); + threadMetadata = kafkaStreams.metadataForLocalThreads(); assertThat(threadMetadata.size(), equalTo(oldThreadCount)); } catch (final AssertionError e) { From 1ebca7817b537b6503941276a1d654ff5a5ae615 Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Mon, 29 Sep 2025 23:26:49 +0800 Subject: [PATCH 092/100] KAFKA-19539: Kafka Streams should also purge internal topics based on user commit requests (#20234) Repartition topic records should be purged up to the currently committed offset once `repartition.purge.interval.ms` duration has passed. Reviewers: Matthias J. Sax --- .../processor/internals/StreamThread.java | 12 +++---- .../processor/internals/StreamThreadTest.java | 33 +++++++++++++++++++ 2 files changed, 39 insertions(+), 6 deletions(-) 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 649a1ec666c..91511da5ee0 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 @@ -1837,12 +1837,6 @@ public class StreamThread extends Thread implements ProcessingThread { .collect(Collectors.toSet()) ); - if ((now - lastPurgeMs) > purgeTimeMs) { - // try to purge the committed records for repartition topics if possible - taskManager.maybePurgeCommittedRecords(); - lastPurgeMs = now; - } - if (committed == -1) { log.debug("Unable to commit as we are in the middle of a rebalance, will try again when it completes."); } else { @@ -1853,6 +1847,12 @@ public class StreamThread extends Thread implements ProcessingThread { committed = taskManager.maybeCommitActiveTasksPerUserRequested(); } + if ((now - lastPurgeMs) > purgeTimeMs) { + // try to purge the committed records for repartition topics if possible + taskManager.maybePurgeCommittedRecords(); + lastPurgeMs = now; + } + return committed; } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index 8fcc44993a5..e53474db74a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -624,6 +624,39 @@ public class StreamThreadTest { verify(taskManager).maybePurgeCommittedRecords(); } + @ParameterizedTest + @MethodSource("data") + public void shouldAlsoPurgeBeforeTheCommitInterval(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) { + final long purgeInterval = 1000L; + final long commitInterval = Long.MAX_VALUE; + final Properties props = configProps(false, stateUpdaterEnabled, processingThreadsEnabled); + props.setProperty(StreamsConfig.STATE_DIR_CONFIG, stateDir); + props.setProperty(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, Long.toString(commitInterval)); + props.setProperty(StreamsConfig.REPARTITION_PURGE_INTERVAL_MS_CONFIG, Long.toString(purgeInterval)); + + final StreamsConfig config = new StreamsConfig(props); + @SuppressWarnings("unchecked") + final Consumer consumer = mock(Consumer.class); + final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class); + when(consumer.groupMetadata()).thenReturn(consumerGroupMetadata); + when(consumerGroupMetadata.groupInstanceId()).thenReturn(Optional.empty()); + final TaskManager taskManager = mock(TaskManager.class); + + final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); + topologyMetadata.buildAndRewriteTopology(); + thread = buildStreamThread(consumer, taskManager, config, topologyMetadata); + + thread.setNow(mockTime.milliseconds()); + thread.maybeCommit(); + + mockTime.sleep(purgeInterval + 1); + + thread.setNow(mockTime.milliseconds()); + thread.maybeCommit(); + + verify(taskManager, times(2)).maybePurgeCommittedRecords(); + } + @ParameterizedTest @MethodSource("data") public void shouldNotProcessWhenPartitionRevoked(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) { From d1a821226c1beebae6db66e02a7ae8ada28ffabc Mon Sep 17 00:00:00 2001 From: lorcan Date: Mon, 29 Sep 2025 17:02:32 +0100 Subject: [PATCH 093/100] KAFKA-15873: Filter topics before sorting (#19304) Partially addresses KAFKA-15873. When filtering and sorting, we should be applying the filter before the sort of topics. Order that unauthorizedForDescribeTopicMetadata is added to not relevant as it is a HashSet. Reviewers: TaiJuWu , Calvin Liu , Matthias J. Sax --- .../handlers/DescribeTopicPartitionsRequestHandler.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/kafka/server/handlers/DescribeTopicPartitionsRequestHandler.java b/core/src/main/java/kafka/server/handlers/DescribeTopicPartitionsRequestHandler.java index d02fd9d3a7d..9b6a04ee979 100644 --- a/core/src/main/java/kafka/server/handlers/DescribeTopicPartitionsRequestHandler.java +++ b/core/src/main/java/kafka/server/handlers/DescribeTopicPartitionsRequestHandler.java @@ -89,7 +89,7 @@ public class DescribeTopicPartitionsRequestHandler { // Do not disclose the existence of topics unauthorized for Describe, so we've not even checked if they exist or not Set unauthorizedForDescribeTopicMetadata = new HashSet<>(); - Stream authorizedTopicsStream = topics.stream().sorted().filter(topicName -> { + Stream authorizedTopicsStream = topics.stream().filter(topicName -> { boolean isAuthorized = authHelper.authorize( abstractRequest.context(), DESCRIBE, TOPIC, topicName, true, true, 1); if (!fetchAllTopics && !isAuthorized) { @@ -99,7 +99,7 @@ public class DescribeTopicPartitionsRequestHandler { ); } return isAuthorized; - }); + }).sorted(); DescribeTopicPartitionsResponseData response = metadataCache.describeTopicResponse( authorizedTopicsStream.iterator(), From 423330ebe7d76697f6cb108530fcd4cb8be1db48 Mon Sep 17 00:00:00 2001 From: Logan Zhu Date: Tue, 30 Sep 2025 21:17:41 +0800 Subject: [PATCH 094/100] KAFKA-19692 improve the docs of "clusterId" for AddRaftVoterOptions and RemoveRaftVoterOptions (#20555) Improves the documentation of the clusterId field in AddRaftVoterOptions and RemoveRaftVoterOptions. The changes include: 1. Adding Javadoc to both addRaftVoter and removeRaftVoter methods to explain the behavior of the optional clusterId. 2. Integration tests have been added to verify the correct behavior of add and remove voter operations with and without clusterId, including scenarios with inconsistent cluster ids. Reviewers: TengYao Chi , Chia-Ping Tsai --- .../ReconfigurableQuorumIntegrationTest.java | 132 ++++++++++++++++++ .../clients/admin/AddRaftVoterOptions.java | 9 ++ .../org/apache/kafka/clients/admin/Admin.java | 19 ++- .../clients/admin/RemoveRaftVoterOptions.java | 9 ++ 4 files changed, 167 insertions(+), 2 deletions(-) create mode 100644 clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/ReconfigurableQuorumIntegrationTest.java diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/ReconfigurableQuorumIntegrationTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/ReconfigurableQuorumIntegrationTest.java new file mode 100644 index 00000000000..f02db36c061 --- /dev/null +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/ReconfigurableQuorumIntegrationTest.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.clients.admin; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.errors.InconsistentClusterIdException; +import org.apache.kafka.common.test.KafkaClusterTestKit; +import org.apache.kafka.common.test.TestKitNodes; +import org.apache.kafka.test.TestUtils; + +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +@Tag("integration") +public class ReconfigurableQuorumIntegrationTest { + + static Map descVoterDirs(Admin admin) throws ExecutionException, InterruptedException { + var quorumInfo = admin.describeMetadataQuorum().quorumInfo().get(); + return quorumInfo.voters().stream().collect(Collectors.toMap(QuorumInfo.ReplicaState::replicaId, QuorumInfo.ReplicaState::replicaDirectoryId)); + } + + @Test + public void testRemoveAndAddVoterWithValidClusterId() throws Exception { + final var nodes = new TestKitNodes.Builder() + .setClusterId("test-cluster") + .setNumBrokerNodes(1) + .setNumControllerNodes(3) + .build(); + + final Map initialVoters = new HashMap<>(); + for (final var controllerNode : nodes.controllerNodes().values()) { + initialVoters.put( + controllerNode.id(), + controllerNode.metadataDirectoryId() + ); + } + + try (var cluster = new KafkaClusterTestKit.Builder(nodes).setInitialVoterSet(initialVoters).build()) { + cluster.format(); + cluster.startup(); + try (Admin admin = Admin.create(cluster.clientProperties())) { + TestUtils.waitForCondition(() -> { + Map voters = descVoterDirs(admin); + assertEquals(Set.of(3000, 3001, 3002), voters.keySet()); + return voters.values().stream().noneMatch(directory -> directory.equals(Uuid.ZERO_UUID)); + }, "Initial quorum voters should be {3000, 3001, 3002} and all should have non-zero directory IDs"); + + Uuid dirId = cluster.nodes().controllerNodes().get(3000).metadataDirectoryId(); + admin.removeRaftVoter( + 3000, + dirId, + new RemoveRaftVoterOptions().setClusterId(Optional.of("test-cluster")) + ).all().get(); + TestUtils.waitForCondition(() -> { + Map voters = descVoterDirs(admin); + assertEquals(Set.of(3001, 3002), voters.keySet()); + return voters.values().stream().noneMatch(directory -> directory.equals(Uuid.ZERO_UUID)); + }, "After removing voter 3000, remaining voters should be {3001, 3002} with non-zero directory IDs"); + + admin.addRaftVoter( + 3000, + dirId, + Set.of(new RaftVoterEndpoint("CONTROLLER", "example.com", 8080)), + new AddRaftVoterOptions().setClusterId(Optional.of("test-cluster")) + ).all().get(); + } + } + } + + @Test + public void testRemoveAndAddVoterWithInconsistentClusterId() throws Exception { + final var nodes = new TestKitNodes.Builder() + .setClusterId("test-cluster") + .setNumBrokerNodes(1) + .setNumControllerNodes(3) + .build(); + + final Map initialVoters = new HashMap<>(); + for (final var controllerNode : nodes.controllerNodes().values()) { + initialVoters.put( + controllerNode.id(), + controllerNode.metadataDirectoryId() + ); + } + + try (var cluster = new KafkaClusterTestKit.Builder(nodes).setInitialVoterSet(initialVoters).build()) { + cluster.format(); + cluster.startup(); + try (Admin admin = Admin.create(cluster.clientProperties())) { + Uuid dirId = cluster.nodes().controllerNodes().get(3000).metadataDirectoryId(); + var removeFuture = admin.removeRaftVoter( + 3000, + dirId, + new RemoveRaftVoterOptions().setClusterId(Optional.of("inconsistent")) + ).all(); + TestUtils.assertFutureThrows(InconsistentClusterIdException.class, removeFuture); + + var addFuture = admin.addRaftVoter( + 3000, + dirId, + Set.of(new RaftVoterEndpoint("CONTROLLER", "example.com", 8080)), + new AddRaftVoterOptions().setClusterId(Optional.of("inconsistent")) + ).all(); + TestUtils.assertFutureThrows(InconsistentClusterIdException.class, addFuture); + } + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/AddRaftVoterOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/AddRaftVoterOptions.java index e28a03d541c..81e889db30d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/AddRaftVoterOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/AddRaftVoterOptions.java @@ -17,11 +17,20 @@ package org.apache.kafka.clients.admin; import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.common.protocol.Errors; import java.util.Optional; /** * Options for {@link Admin#addRaftVoter}. + * + *

    + * The clusterId is optional. + *

    + * If provided, the request will only succeed if the cluster id matches the id of the current cluster. + * If the cluster id does not match, the request will fail with {@link Errors#INCONSISTENT_CLUSTER_ID}. + *

    + * If not provided, the cluster id check is skipped. */ @InterfaceStability.Stable public class AddRaftVoterOptions extends AbstractOptions { diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java index e596754f62f..06ede9f620d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java @@ -32,6 +32,7 @@ import org.apache.kafka.common.acl.AclBindingFilter; import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.errors.FeatureUpdateFailedException; +import org.apache.kafka.common.errors.InconsistentClusterIdException; import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.quota.ClientQuotaAlteration; @@ -1866,10 +1867,17 @@ public interface Admin extends AutoCloseable { /** * Add a new voter node to the KRaft metadata quorum. * + *

    + * The clusterId in {@link AddRaftVoterOptions} is optional. + * If provided, the operation will only succeed if the cluster id matches the id + * of the current cluster. If the cluster id does not match, the operation + * will fail with {@link InconsistentClusterIdException}. + * If not provided, the cluster id check is skipped. + * * @param voterId The node ID of the voter. * @param voterDirectoryId The directory ID of the voter. * @param endpoints The endpoints that the new voter has. - * @param options The options to use when adding the new voter node. + * @param options Additional options for the operation, including optional cluster ID. */ AddRaftVoterResult addRaftVoter( int voterId, @@ -1894,9 +1902,16 @@ public interface Admin extends AutoCloseable { /** * Remove a voter node from the KRaft metadata quorum. * + *

    + * The clusterId in {@link AddRaftVoterOptions} is optional. + * If provided, the operation will only succeed if the cluster id matches the id + * of the current cluster. If the cluster id does not match, the operation + * will fail with {@link InconsistentClusterIdException}. + * If not provided, the cluster id check is skipped. + * * @param voterId The node ID of the voter. * @param voterDirectoryId The directory ID of the voter. - * @param options The options to use when removing the voter node. + * @param options Additional options for the operation, including optional cluster ID. */ RemoveRaftVoterResult removeRaftVoter( int voterId, diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/RemoveRaftVoterOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/RemoveRaftVoterOptions.java index cb5fe563c19..da6e965ebe0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/RemoveRaftVoterOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/RemoveRaftVoterOptions.java @@ -17,11 +17,20 @@ package org.apache.kafka.clients.admin; import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.common.protocol.Errors; import java.util.Optional; /** * Options for {@link Admin#removeRaftVoter}. + * + *

    + * The clusterId is optional. + *

    + * If provided, the request will only succeed if the cluster id matches the id of the current cluster. + * If the cluster id does not match, the request will fail with {@link Errors#INCONSISTENT_CLUSTER_ID}. + *

    + * If not provided, the cluster id check is skipped. */ @InterfaceStability.Stable public class RemoveRaftVoterOptions extends AbstractOptions { From 7426629ba4e5f4e3feb177e3987d54c1c078ff17 Mon Sep 17 00:00:00 2001 From: Genseric Ghiro Date: Wed, 1 Oct 2025 11:51:45 -0400 Subject: [PATCH 095/100] MINOR: Moving quota test to core directory (#20582) ## Summary Quota test isn't testing anything on the client side, but rather enforcing server-side quotas, so moving it out of the clients directory into the core directory. Reviewers: Lianet Magrans --- tests/kafkatest/tests/{client => core}/quota_test.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename tests/kafkatest/tests/{client => core}/quota_test.py (100%) diff --git a/tests/kafkatest/tests/client/quota_test.py b/tests/kafkatest/tests/core/quota_test.py similarity index 100% rename from tests/kafkatest/tests/client/quota_test.py rename to tests/kafkatest/tests/core/quota_test.py From 0ddc69da7094fcc4163b6beb1056fc46a8e15a68 Mon Sep 17 00:00:00 2001 From: Jhen-Yung Hsu Date: Thu, 2 Oct 2025 03:17:47 +0800 Subject: [PATCH 096/100] KAFKA-19721: Update streams documentation with KIP-1147 changes (#20606) Update KIP-1147 changes (renaming --property to --formatter-property) in the ops and streams documentation. Reviewers: Andrew Schofield --- docs/ops.html | 2 +- docs/streams/developer-guide/datatypes.html | 14 ++++----- docs/streams/quickstart.html | 32 ++++++++++----------- 3 files changed, 24 insertions(+), 24 deletions(-) diff --git a/docs/ops.html b/docs/ops.html index 2d050ec76da..fcef480f2c9 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -4315,7 +4315,7 @@ $ bin/kafka-topics.sh --create --topic tieredTopic --bootstrap-server localhost:

    Lastly, we can try to consume some data from the beginning and print offset number, to make sure it will successfully fetch offset 0 from the remote storage.

    -
    $ bin/kafka-console-consumer.sh --topic tieredTopic --from-beginning --max-messages 1 --bootstrap-server localhost:9092 --property print.offset=true
    +
    $ bin/kafka-console-consumer.sh --topic tieredTopic --from-beginning --max-messages 1 --bootstrap-server localhost:9092 --formatter-property print.offset=true

    In KRaft mode, you can disable tiered storage at the topic level, to make the remote logs as read-only logs, or completely delete all remote logs.

    diff --git a/docs/streams/developer-guide/datatypes.html b/docs/streams/developer-guide/datatypes.html index 2bc2d7d5d0e..6e93de6e1e1 100644 --- a/docs/streams/developer-guide/datatypes.html +++ b/docs/streams/developer-guide/datatypes.html @@ -217,15 +217,15 @@ TimeWindowedDeserializer<String> deserializer = new TimeWindowedDeserializ

    Usage in Command Line

    When using command-line tools (like bin/kafka-console-consumer.sh), you can configure windowed deserializers by passing the inner class and window size via configuration properties. The property names use a prefix pattern:

    # Time windowed deserializer configuration
    ---property print.key=true \
    ---property key.deserializer=org.apache.kafka.streams.kstream.TimeWindowedDeserializer \
    ---property key.deserializer.windowed.inner.deserializer.class=org.apache.kafka.common.serialization.StringDeserializer \
    ---property key.deserializer.window.size.ms=500
    +--formatter-property print.key=true \
    +--formatter-property key.deserializer=org.apache.kafka.streams.kstream.TimeWindowedDeserializer \
    +--formatter-property key.deserializer.windowed.inner.deserializer.class=org.apache.kafka.common.serialization.StringDeserializer \
    +--formatter-property key.deserializer.window.size.ms=500
     
     # Session windowed deserializer configuration  
    ---property print.key=true \
    ---property key.deserializer=org.apache.kafka.streams.kstream.SessionWindowedDeserializer \
    ---property key.deserializer.windowed.inner.deserializer.class=org.apache.kafka.common.serialization.StringDeserializer
    +--formatter-property print.key=true \ +--formatter-property key.deserializer=org.apache.kafka.streams.kstream.SessionWindowedDeserializer \ +--formatter-property key.deserializer.windowed.inner.deserializer.class=org.apache.kafka.common.serialization.StringDeserializer

    Deprecated Configs

    The following StreamsConfig parameters are deprecated in favor of passing parameters directly to serializer/deserializer constructors:

    diff --git a/docs/streams/quickstart.html b/docs/streams/quickstart.html index 56ab128e2a5..b6bd6a51e63 100644 --- a/docs/streams/quickstart.html +++ b/docs/streams/quickstart.html @@ -175,10 +175,10 @@ and inspect the output of the WordCount demo application by reading from its out
    $ bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \
         --topic streams-wordcount-output \
         --from-beginning \
    -    --property print.key=true \
    -    --property print.value=true \
    -    --property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
    -    --property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer
    + --formatter-property print.key=true \ + --formatter-property print.value=true \ + --formatter-property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \ + --formatter-property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer

    Step 5: Process some data

    @@ -197,10 +197,10 @@ This message will be processed by the Wordcount application and the following ou
    $ bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \
         --topic streams-wordcount-output \
         --from-beginning \
    -    --property print.key=true \
    -    --property print.value=true \
    -    --property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
    -    --property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer
    +    --formatter-property print.key=true \
    +    --formatter-property print.value=true \
    +    --formatter-property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
    +    --formatter-property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer
     
     all	    1
     streams	1
    @@ -225,10 +225,10 @@ In your other terminal in which the console consumer is running, you will observ
     
    $ bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \
         --topic streams-wordcount-output \
         --from-beginning \
    -    --property print.key=true \
    -    --property print.value=true \
    -    --property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
    -    --property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer
    +    --formatter-property print.key=true \
    +    --formatter-property print.value=true \
    +    --formatter-property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
    +    --formatter-property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer
     
     all	    1
     streams	1
    @@ -255,10 +255,10 @@ The streams-wordcount-output topic will subsequently show the correspondi
     
    $ bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \
         --topic streams-wordcount-output \
         --from-beginning \
    -    --property print.key=true \
    -    --property print.value=true \
    -    --property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
    -    --property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer
    +    --formatter-property print.key=true \
    +    --formatter-property print.value=true \
    +    --formatter-property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
    +    --formatter-property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer
     
     all	    1
     streams	1
    
    From 7f65b1fa9608d49355e41701b7d6453d158932c5 Mon Sep 17 00:00:00 2001
    From: Andrew Schofield 
    Date: Wed, 1 Oct 2025 20:19:50 +0100
    Subject: [PATCH 097/100] MINOR: Typo in ListShareGroupOffsetsResults javadoc
     (#20623)
    
    Fixed a tiny javadoc typo.
    
    Reviewers: Apoorv Mittal 
    ---
     .../apache/kafka/clients/admin/ListShareGroupOffsetsResult.java | 2 +-
     1 file changed, 1 insertion(+), 1 deletion(-)
    
    diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ListShareGroupOffsetsResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/ListShareGroupOffsetsResult.java
    index 6fecddc69ef..1a2c8869c6c 100644
    --- a/clients/src/main/java/org/apache/kafka/clients/admin/ListShareGroupOffsetsResult.java
    +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ListShareGroupOffsetsResult.java
    @@ -46,7 +46,7 @@ public class ListShareGroupOffsetsResult {
         /**
          * Return the future when the requests for all groups succeed.
          *
    -     * @return Future which yields all {@code Map>} objects, if requests for all the groups succeed.
    +     * @return Future which yields all {@code Map>} objects, if requests for all the groups succeed.
          */
         public KafkaFuture>> all() {
             return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])).thenApply(
    
    From 33cd1143756a2eb31de81eaa8da8b38563f5927e Mon Sep 17 00:00:00 2001
    From: Chang-Yu Huang 
    Date: Thu, 2 Oct 2025 06:01:53 -0400
    Subject: [PATCH 098/100] KAFKA-9825 Kafka protocol BNF format should have some
     way to display tagged fields (#20536)
    MIME-Version: 1.0
    Content-Type: text/plain; charset=UTF-8
    Content-Transfer-Encoding: 8bit
    
    # Description
    The [protocol guide](https://kafka.apache.org/protocol) 1) doesn't
    display
    tagged fields in BNF, and 2) includes empty tagged fields and redundant
    nested tables in tables.
    
    # Change
    ## BNF
    Now tagged fields are displayed as FIELD_NAME
    
    Old:  Screenshot 2025-09-13 at 5 34
    28 PM
    
    New:  Screenshot 2025-09-24 at 12 50
    34 PM
    
    Array Field: Screenshot 2025-09-24 at
    12 52 19 PM
    
    ## Table
    Empty tagged fields are removed from the table.
    
    Nested table for tagged fie  Old: Screenshot 2025-09-28 at 11 07 01 PM  lds are removed. Tag of the field is shown in the "Field" column.
    
    New:   Screenshot 2025-09-28 at 11
    10 30 PM
    
    Reviewers: Andrew Schofield 
    ---
     .../kafka/common/protocol/Protocol.java       | 49 +++++++++++--------
     1 file changed, 28 insertions(+), 21 deletions(-)
    
    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 c23aa1782d6..1b051d58bf0 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
    @@ -19,6 +19,7 @@ package org.apache.kafka.common.protocol;
     import org.apache.kafka.common.message.RequestHeaderData;
     import org.apache.kafka.common.message.ResponseHeaderData;
     import org.apache.kafka.common.protocol.types.BoundField;
    +import org.apache.kafka.common.protocol.types.Field;
     import org.apache.kafka.common.protocol.types.Schema;
     import org.apache.kafka.common.protocol.types.TaggedFields;
     import org.apache.kafka.common.protocol.types.Type;
    @@ -27,6 +28,7 @@ import java.util.LinkedHashMap;
     import java.util.LinkedHashSet;
     import java.util.Map;
     import java.util.Set;
    +import java.util.TreeMap;
     
     public class Protocol {
     
    @@ -49,7 +51,23 @@ public class Protocol {
                         subTypes.put(field.def.name, type.arrayElementType().get());
                     }
                 } else if (type instanceof TaggedFields) {
    -                b.append("_tagged_fields ");
    +                Map taggedFields = new TreeMap<>(((TaggedFields) type).fields());
    +                taggedFields.forEach((tag, taggedField) -> {
    +                    if (taggedField.type.isArray()) {
    +                        b.append("[");
    +                        b.append(taggedField.name);
    +                        b.append("]");
    +                        if (!subTypes.containsKey(taggedField.name))
    +                            subTypes.put(taggedField.name + "<tag: " + tag.toString() + ">", taggedField.type.arrayElementType().get());
    +                    } else {
    +                        b.append(taggedField.name);
    +                        if (!subTypes.containsKey(taggedField.name))
    +                            subTypes.put(taggedField.name + "<tag: " + tag.toString() + ">", taggedField.type);
    +                    }
    +                    b.append("<tag: ");
    +                    b.append(tag);
    +                    b.append("> ");
    +                });
                 } else {
                     b.append(field.def.name);
                     b.append(" ");
    @@ -90,6 +108,12 @@ public class Protocol {
             }
         }
     
    +    private static void appendFieldNameToTable(String name, StringBuilder b) {
    +        b.append("");
    +        b.append(name);
    +        b.append("");
    +    }
    +
         private static void schemaToFieldTableHtml(Schema schema, StringBuilder b) {
             Set fields = new LinkedHashSet<>();
             populateSchemaFields(schema, fields);
    @@ -101,28 +125,12 @@ public class Protocol {
             b.append("");
             for (BoundField field : fields) {
                 b.append("\n");
    -            b.append("");
    -            b.append(field.def.name);
    -            b.append("");
    -            b.append("");
                 if (field.def.type instanceof TaggedFields) {
                     TaggedFields taggedFields = (TaggedFields) field.def.type;
                     // Only include the field in the table if there are actually tags defined
                     if (taggedFields.numFields() > 0) {
    -                    b.append("\n");
    -                    b.append("");
    -                    b.append("\n");
    -                    b.append("\n");
    -                    b.append("\n");
    -                    b.append("");
                         taggedFields.fields().forEach((tag, taggedField) -> {
    -                        b.append("\n");
    -                        b.append("");
    -                        b.append("");
    +                        appendFieldNameToTable(taggedField.name + "<tag: " + tag.toString() + ">", b);
                             b.append("");
                             b.append("\n");
                         });
    -                    b.append("
    TagTagged fieldDescription
    "); - b.append(tag); - b.append(""); - b.append(taggedField.name); - b.append(""); b.append(taggedField.docString); if (taggedField.type.isArray()) { @@ -136,11 +144,10 @@ public class Protocol { b.append("
    \n"); - } else { - b.append(field.def.docString); } } else { + appendFieldNameToTable(field.def.name, b); + b.append(""); b.append(field.def.docString); } b.append(""); From 8468317dac490e8c3ba9c7b35ac8ff64f5f1fbfe Mon Sep 17 00:00:00 2001 From: Mahsa Seifikar Date: Thu, 2 Oct 2025 14:02:47 -0400 Subject: [PATCH 099/100] KAFKA-19467; Add a metric for controller thread idleness (#20422) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This change adds the metric ControllerEventManager::AvgIdleRatio which measures the amount of time the controller spends blocked waiting for events vs the amount of time spent processing events. A value of 1.0 means that the controller spent the entire interval blocked waiting for events. Reviewers: José Armando García Sancio , Kevin Wu , Alyssa Huang , TengYao Chi , Reviewers: Chia-Ping Tsai --- checkstyle/import-control.xml | 1 + docs/upgrade.html | 4 ++ .../kafka/controller/QuorumController.java | 9 +++- .../metrics/QuorumControllerMetrics.java | 23 +++++++++- .../metrics/QuorumControllerMetricsTest.java | 30 ++++++++++++ .../raft/internals/KafkaRaftMetrics.java | 1 + .../apache/kafka/queue/KafkaEventQueue.java | 46 +++++++++++++------ .../kafka/server/metrics}/TimeRatio.java | 31 ++++++++++--- .../kafka/queue/KafkaEventQueueTest.java | 44 ++++++++++++++++++ .../kafka/server/metrics}/TimeRatioTest.java | 2 +- 10 files changed, 168 insertions(+), 23 deletions(-) rename {raft/src/main/java/org/apache/kafka/raft/internals => server-common/src/main/java/org/apache/kafka/server/metrics}/TimeRatio.java (80%) rename {raft/src/test/java/org/apache/kafka/raft/internals => server-common/src/test/java/org/apache/kafka/server/metrics}/TimeRatioTest.java (98%) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index b1ef62ca3a2..c7f9eaad7ea 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -497,6 +497,7 @@ + diff --git a/docs/upgrade.html b/docs/upgrade.html index d28898590f8..b5501bd2a74 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -182,6 +182,10 @@ kafka.log.remote:type=RemoteStorageThreadPool.RemoteLogReaderAvgIdlePercent. For further details, please refer to KIP-1100. +
  • + A new metric AvgIdleRatio has been added to the ControllerEventManager group. This metric measures the average idle ratio of the controller event queue thread, + providing visibility into how much time the controller spends waiting for events versus processing them. The metric value ranges from 0.0 (always busy) to 1.0 (always idle). +
  • Upgrading to 4.1.0

    diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index 3e1dd69723b..dfde76ecba5 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -406,7 +406,14 @@ public final class QuorumController implements Controller { KafkaEventQueue queue = null; try { - queue = new KafkaEventQueue(time, logContext, threadNamePrefix); + queue = new KafkaEventQueue( + time, + logContext, + threadNamePrefix, + EventQueue.VoidEvent.INSTANCE, + controllerMetrics::updateIdleTime + ); + return new QuorumController( nonFatalFaultHandler, fatalFaultHandler, diff --git a/metadata/src/main/java/org/apache/kafka/controller/metrics/QuorumControllerMetrics.java b/metadata/src/main/java/org/apache/kafka/controller/metrics/QuorumControllerMetrics.java index 310a2c1dd61..4a251faafc4 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/metrics/QuorumControllerMetrics.java +++ b/metadata/src/main/java/org/apache/kafka/controller/metrics/QuorumControllerMetrics.java @@ -19,6 +19,7 @@ package org.apache.kafka.controller.metrics; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.metrics.KafkaYammerMetrics; +import org.apache.kafka.server.metrics.TimeRatio; import com.yammer.metrics.core.Gauge; import com.yammer.metrics.core.Histogram; @@ -48,6 +49,8 @@ public class QuorumControllerMetrics implements AutoCloseable { "ControllerEventManager", "EventQueueTimeMs"); private static final MetricName EVENT_QUEUE_PROCESSING_TIME_MS = getMetricName( "ControllerEventManager", "EventQueueProcessingTimeMs"); + private static final MetricName AVERAGE_IDLE_RATIO = getMetricName( + "ControllerEventManager", "AvgIdleRatio"); private static final MetricName LAST_APPLIED_RECORD_OFFSET = getMetricName( "KafkaController", "LastAppliedRecordOffset"); private static final MetricName LAST_COMMITTED_RECORD_OFFSET = getMetricName( @@ -64,6 +67,7 @@ public class QuorumControllerMetrics implements AutoCloseable { "KafkaController", "EventQueueOperationsTimedOutCount"); private static final MetricName NEW_ACTIVE_CONTROLLERS_COUNT = getMetricName( "KafkaController", "NewActiveControllersCount"); + private static final String TIME_SINCE_LAST_HEARTBEAT_RECEIVED_METRIC_NAME = "TimeSinceLastHeartbeatReceivedMs"; private static final String BROKER_ID_TAG = "broker"; @@ -75,6 +79,7 @@ public class QuorumControllerMetrics implements AutoCloseable { private final AtomicLong lastAppliedRecordTimestamp = new AtomicLong(0); private final Consumer eventQueueTimeUpdater; private final Consumer eventQueueProcessingTimeUpdater; + private final TimeRatio avgIdleTimeRatio; private final AtomicLong timedOutHeartbeats = new AtomicLong(0); private final AtomicLong operationsStarted = new AtomicLong(0); @@ -109,6 +114,7 @@ public class QuorumControllerMetrics implements AutoCloseable { this.eventQueueTimeUpdater = newHistogram(EVENT_QUEUE_TIME_MS, true); this.eventQueueProcessingTimeUpdater = newHistogram(EVENT_QUEUE_PROCESSING_TIME_MS, true); this.sessionTimeoutMs = sessionTimeoutMs; + this.avgIdleTimeRatio = new TimeRatio(1); registry.ifPresent(r -> r.newGauge(LAST_APPLIED_RECORD_OFFSET, new Gauge() { @Override public Long value() { @@ -157,6 +163,20 @@ public class QuorumControllerMetrics implements AutoCloseable { return newActiveControllers(); } })); + registry.ifPresent(r -> r.newGauge(AVERAGE_IDLE_RATIO, new Gauge() { + @Override + public Double value() { + synchronized (avgIdleTimeRatio) { + return avgIdleTimeRatio.measure(); + } + } + })); + } + + public void updateIdleTime(long idleDurationMs) { + synchronized (avgIdleTimeRatio) { + avgIdleTimeRatio.record((double) idleDurationMs, time.milliseconds()); + } } public void addTimeSinceLastHeartbeatMetric(int brokerId) { @@ -291,7 +311,8 @@ public class QuorumControllerMetrics implements AutoCloseable { TIMED_OUT_BROKER_HEARTBEAT_COUNT, EVENT_QUEUE_OPERATIONS_STARTED_COUNT, EVENT_QUEUE_OPERATIONS_TIMED_OUT_COUNT, - NEW_ACTIVE_CONTROLLERS_COUNT + NEW_ACTIVE_CONTROLLERS_COUNT, + AVERAGE_IDLE_RATIO ).forEach(r::removeMetric)); removeTimeSinceLastHeartbeatMetrics(); } diff --git a/metadata/src/test/java/org/apache/kafka/controller/metrics/QuorumControllerMetricsTest.java b/metadata/src/test/java/org/apache/kafka/controller/metrics/QuorumControllerMetricsTest.java index 491d22f1cd8..4aa50a561df 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/metrics/QuorumControllerMetricsTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/metrics/QuorumControllerMetricsTest.java @@ -45,6 +45,7 @@ public class QuorumControllerMetricsTest { Set expected = Set.of( "kafka.controller:type=ControllerEventManager,name=EventQueueProcessingTimeMs", "kafka.controller:type=ControllerEventManager,name=EventQueueTimeMs", + "kafka.controller:type=ControllerEventManager,name=AvgIdleRatio", "kafka.controller:type=KafkaController,name=ActiveControllerCount", "kafka.controller:type=KafkaController,name=EventQueueOperationsStartedCount", "kafka.controller:type=KafkaController,name=EventQueueOperationsTimedOutCount", @@ -189,6 +190,35 @@ public class QuorumControllerMetricsTest { } } + @Test + public void testAvgIdleRatio() { + final double delta = 0.001; + MetricsRegistry registry = new MetricsRegistry(); + MockTime time = new MockTime(); + try (QuorumControllerMetrics metrics = new QuorumControllerMetrics(Optional.of(registry), time, 9000)) { + Gauge avgIdleRatio = (Gauge) registry.allMetrics().get(metricName("ControllerEventManager", "AvgIdleRatio")); + + // No idle time recorded yet; returns default ratio of 1.0 + assertEquals(1.0, avgIdleRatio.value(), delta); + + // First recording is dropped to establish the interval start time + // This is because TimeRatio needs an initial timestamp to measure intervals from + metrics.updateIdleTime(10); + time.sleep(40); + metrics.updateIdleTime(20); + // avgIdleRatio = (20ms idle) / (40ms interval) = 0.5 + assertEquals(0.5, avgIdleRatio.value(), delta); + + time.sleep(20); + metrics.updateIdleTime(1); + // avgIdleRatio = (1ms idle) / (20ms interval) = 0.05 + assertEquals(0.05, avgIdleRatio.value(), delta); + + } finally { + registry.shutdown(); + } + } + private static void assertMetricHistogram(MetricsRegistry registry, MetricName metricName, long count, double sum) { Histogram histogram = (Histogram) registry.allMetrics().get(metricName); diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java b/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java index 87c5b217d8e..a90928d35f3 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java @@ -29,6 +29,7 @@ import org.apache.kafka.raft.LogOffsetMetadata; import org.apache.kafka.raft.QuorumState; import org.apache.kafka.raft.ReplicaKey; import org.apache.kafka.server.common.OffsetAndEpoch; +import org.apache.kafka.server.metrics.TimeRatio; import java.util.List; import java.util.OptionalLong; diff --git a/server-common/src/main/java/org/apache/kafka/queue/KafkaEventQueue.java b/server-common/src/main/java/org/apache/kafka/queue/KafkaEventQueue.java index 42ebdd37d20..ad2c916e3fc 100644 --- a/server-common/src/main/java/org/apache/kafka/queue/KafkaEventQueue.java +++ b/server-common/src/main/java/org/apache/kafka/queue/KafkaEventQueue.java @@ -33,6 +33,7 @@ import java.util.TreeMap; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Consumer; import java.util.function.Function; import java.util.function.UnaryOperator; @@ -278,22 +279,22 @@ public final class KafkaEventQueue implements EventQueue { remove(toRun); continue; } - if (awaitNs == Long.MAX_VALUE) { - try { + + long startIdleMs = time.milliseconds(); + try { + if (awaitNs == Long.MAX_VALUE) { cond.await(); - } catch (InterruptedException e) { - log.warn("Interrupted while waiting for a new event. " + - "Shutting down event queue"); - interrupted = true; - } - } else { - try { + } else { cond.awaitNanos(awaitNs); - } catch (InterruptedException e) { - log.warn("Interrupted while waiting for a deferred event. " + - "Shutting down event queue"); - interrupted = true; } + } catch (InterruptedException e) { + log.warn( + "Interrupted while waiting for a {} event. Shutting down event queue", + (awaitNs == Long.MAX_VALUE) ? "new" : "deferred" + ); + interrupted = true; + } finally { + idleTimeCallback.accept(Math.max(time.milliseconds() - startIdleMs, 0)); } } finally { lock.unlock(); @@ -440,12 +441,18 @@ public final class KafkaEventQueue implements EventQueue { */ private boolean interrupted; + /** + * Optional callback for queue idle time tracking. + */ + private final Consumer idleTimeCallback; + + public KafkaEventQueue( Time time, LogContext logContext, String threadNamePrefix ) { - this(time, logContext, threadNamePrefix, VoidEvent.INSTANCE); + this(time, logContext, threadNamePrefix, VoidEvent.INSTANCE, __ -> { }); } public KafkaEventQueue( @@ -453,6 +460,16 @@ public final class KafkaEventQueue implements EventQueue { LogContext logContext, String threadNamePrefix, Event cleanupEvent + ) { + this(time, logContext, threadNamePrefix, cleanupEvent, __ -> { }); + } + + public KafkaEventQueue( + Time time, + LogContext logContext, + String threadNamePrefix, + Event cleanupEvent, + Consumer idleTimeCallback ) { this.time = time; this.cleanupEvent = Objects.requireNonNull(cleanupEvent); @@ -463,6 +480,7 @@ public final class KafkaEventQueue implements EventQueue { this.eventHandler, false); this.shuttingDown = false; this.interrupted = false; + this.idleTimeCallback = Objects.requireNonNull(idleTimeCallback); this.eventHandlerThread.start(); } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/TimeRatio.java b/server-common/src/main/java/org/apache/kafka/server/metrics/TimeRatio.java similarity index 80% rename from raft/src/main/java/org/apache/kafka/raft/internals/TimeRatio.java rename to server-common/src/main/java/org/apache/kafka/server/metrics/TimeRatio.java index 357682b6fe2..8a1572c0273 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/TimeRatio.java +++ b/server-common/src/main/java/org/apache/kafka/server/metrics/TimeRatio.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.raft.internals; +package org.apache.kafka.server.metrics; import org.apache.kafka.common.metrics.MeasurableStat; import org.apache.kafka.common.metrics.MetricConfig; @@ -46,11 +46,26 @@ public class TimeRatio implements MeasurableStat { @Override public double measure(MetricConfig config, long currentTimestampMs) { + return measure(); + } + + @Override + public void record(MetricConfig config, double value, long currentTimestampMs) { + record(value, currentTimestampMs); + } + + /** + * Measures the ratio of recorded duration to the interval duration + * since the last measurement. + * + * @return The measured ratio value between 0.0 and 1.0 + */ + public double measure() { if (lastRecordedTimestampMs < 0) { // Return the default value if no recordings have been captured. return defaultRatio; } else { - // We measure the ratio over the + // We measure the ratio over the interval double intervalDurationMs = Math.max(lastRecordedTimestampMs - intervalStartTimestampMs, 0); final double ratio; if (intervalDurationMs == 0) { @@ -61,15 +76,20 @@ public class TimeRatio implements MeasurableStat { ratio = totalRecordedDurationMs / intervalDurationMs; } - // The next interval begins at the + // The next interval begins at the last recorded timestamp intervalStartTimestampMs = lastRecordedTimestampMs; totalRecordedDurationMs = 0; return ratio; } } - @Override - public void record(MetricConfig config, double value, long currentTimestampMs) { + /** + * Records a duration value at the specified timestamp. + * + * @param value The duration value to record + * @param currentTimestampMs The current timestamp in milliseconds + */ + public void record(double value, long currentTimestampMs) { if (intervalStartTimestampMs < 0) { // Discard the initial value since the value occurred prior to the interval start intervalStartTimestampMs = currentTimestampMs; @@ -78,5 +98,4 @@ public class TimeRatio implements MeasurableStat { lastRecordedTimestampMs = currentTimestampMs; } } - } diff --git a/server-common/src/test/java/org/apache/kafka/queue/KafkaEventQueueTest.java b/server-common/src/test/java/org/apache/kafka/queue/KafkaEventQueueTest.java index 54fc65a604a..d2d4526eef7 100644 --- a/server-common/src/test/java/org/apache/kafka/queue/KafkaEventQueueTest.java +++ b/server-common/src/test/java/org/apache/kafka/queue/KafkaEventQueueTest.java @@ -424,4 +424,48 @@ public class KafkaEventQueueTest { assertEquals(InterruptedException.class, ieTrapper2.exception.get().getClass()); } } + + @Test + public void testIdleTimeCallback() throws Exception { + MockTime time = new MockTime(); + AtomicLong lastIdleTimeMs = new AtomicLong(0); + + try (KafkaEventQueue queue = new KafkaEventQueue( + time, + logContext, + "testIdleTimeCallback", + EventQueue.VoidEvent.INSTANCE, + lastIdleTimeMs::set)) { + time.sleep(2); + assertEquals(0, lastIdleTimeMs.get(), "Last idle time should be 0ms"); + + // Test 1: Two events with a wait in between using FutureEvent + CompletableFuture event1 = new CompletableFuture<>(); + queue.append(new FutureEvent<>(event1, () -> { + time.sleep(1); + return "event1-processed"; + })); + assertEquals("event1-processed", event1.get()); + + long waitTime5Ms = 5; + time.sleep(waitTime5Ms); + CompletableFuture event2 = new CompletableFuture<>(); + queue.append(new FutureEvent<>(event2, () -> { + time.sleep(1); + return "event2-processed"; + })); + assertEquals("event2-processed", event2.get()); + assertEquals(waitTime5Ms, lastIdleTimeMs.get(), "Idle time should be " + waitTime5Ms + "ms, was: " + lastIdleTimeMs.get()); + + // Test 2: Deferred event + long waitTime2Ms = 2; + CompletableFuture deferredEvent2 = new CompletableFuture<>(); + queue.scheduleDeferred("deferred2", + __ -> OptionalLong.of(time.nanoseconds() + TimeUnit.MILLISECONDS.toNanos(waitTime2Ms)), + () -> deferredEvent2.complete(null)); + time.sleep(waitTime2Ms); + deferredEvent2.get(); + assertEquals(waitTime2Ms, lastIdleTimeMs.get(), "Idle time should be " + waitTime2Ms + "ms, was: " + lastIdleTimeMs.get()); + } + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/TimeRatioTest.java b/server-common/src/test/java/org/apache/kafka/server/metrics/TimeRatioTest.java similarity index 98% rename from raft/src/test/java/org/apache/kafka/raft/internals/TimeRatioTest.java rename to server-common/src/test/java/org/apache/kafka/server/metrics/TimeRatioTest.java index 94e8844734d..2c194a1448a 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/TimeRatioTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/metrics/TimeRatioTest.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.raft.internals; +package org.apache.kafka.server.metrics; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.utils.MockTime; From 28e7803037fb48770cd4877c9bf12e42b7c52441 Mon Sep 17 00:00:00 2001 From: Nikita Shupletsov Date: Thu, 2 Oct 2025 21:24:36 -0700 Subject: [PATCH 100/100] KAFKA-19744: Move restore time calculation to ChangelogMetadata (#20613) - Move restore time calculation to ChangelogMetadata. - Introduced a new interface to propagate the calculated value to the stores to avoid modifications in the public interface. Reviewers: Matthias J. Sax --- .../integration/RestoreIntegrationTest.java | 51 ++++++++++++++++- .../utils/IntegrationTestUtils.java | 9 +++ .../internals/StoreChangelogReader.java | 11 ++++ .../state/internals/MeteredKeyValueStore.java | 15 +++-- .../state/internals/MeteredSessionStore.java | 14 +++-- .../state/internals/MeteredStateStore.java | 22 ++++++++ .../state/internals/MeteredWindowStore.java | 12 +++- .../internals/StoreChangelogReaderTest.java | 55 +++++++++++++++++++ .../internals/MeteredKeyValueStoreTest.java | 10 ++-- .../internals/MeteredSessionStoreTest.java | 9 ++- .../MeteredVersionedKeyValueStoreTest.java | 9 +-- .../internals/MeteredWindowStoreTest.java | 11 +++- 12 files changed, 196 insertions(+), 32 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredStateStore.java diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java index e85ac344157..7ffeb6ac035 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java @@ -112,6 +112,9 @@ import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.wa import static org.apache.kafka.streams.utils.TestUtils.safeUniqueTestName; import static org.apache.kafka.test.TestUtils.waitForCondition; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.core.IsEqual.equalTo; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; @@ -685,6 +688,52 @@ public class RestoreIntegrationTest { } } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void shouldRecordRestoreMetrics(final boolean useNewProtocol) throws Exception { + final AtomicInteger numReceived = new AtomicInteger(0); + final StreamsBuilder builder = new StreamsBuilder(); + + final Properties props = props(); + + if (useNewProtocol) { + props.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.name()); + } + + props.put(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, "DEBUG"); + + createStateForRestoration(inputStream, 10000); + + final CountDownLatch shutdownLatch = new CountDownLatch(1); + builder.table(inputStream, Consumed.with(Serdes.Integer(), Serdes.Integer()), Materialized.as("store")) + .toStream() + .foreach((key, value) -> { + if (numReceived.incrementAndGet() == numberOfKeys) { + shutdownLatch.countDown(); + } + }); + + kafkaStreams = new KafkaStreams(builder.build(), props); + + final AtomicLong restored = new AtomicLong(0); + final TrackingStateRestoreListener restoreListener = new TrackingStateRestoreListener(restored); + kafkaStreams.setGlobalStateRestoreListener(restoreListener); + kafkaStreams.start(); + + assertTrue(shutdownLatch.await(30, TimeUnit.SECONDS)); + assertThat(numReceived.get(), equalTo(numberOfKeys)); + + final Map taskIdToMetricValue = kafkaStreams.metrics().entrySet().stream() + .filter(e -> e.getKey().name().equals("restore-latency-max")) + .collect(Collectors.toMap(e -> e.getKey().tags().get("task-id"), e -> ((Double) e.getValue().metricValue()).longValue())); + + for (final Map.Entry entry : restoreListener.changelogToRestoreTime().entrySet()) { + final long lowerBound = entry.getValue() - TimeUnit.NANOSECONDS.convert(1, TimeUnit.SECONDS); + final long upperBound = entry.getValue() + TimeUnit.NANOSECONDS.convert(1, TimeUnit.SECONDS); + assertThat(taskIdToMetricValue.get("0_" + entry.getKey().partition()), allOf(greaterThanOrEqualTo(lowerBound), lessThanOrEqualTo(upperBound))); + } + } + private void validateReceivedMessages(final List> expectedRecords, final String outputTopic) throws Exception { final Properties consumerProperties = new Properties(); @@ -971,4 +1020,4 @@ public class RestoreIntegrationTest { } } } -} \ No newline at end of file +} diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java index a7b3f838f2a..44ea3c7fc89 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java @@ -1337,6 +1337,8 @@ public class IntegrationTestUtils { public final Map changelogToStartOffset = new ConcurrentHashMap<>(); public final Map changelogToEndOffset = new ConcurrentHashMap<>(); public final Map changelogToTotalNumRestored = new ConcurrentHashMap<>(); + private final Map changelogToRestoreStartTime = new ConcurrentHashMap<>(); + private final Map changelogToRestoreEndTime = new ConcurrentHashMap<>(); private final AtomicLong restored; public TrackingStateRestoreListener() { @@ -1355,6 +1357,7 @@ public class IntegrationTestUtils { changelogToStartOffset.put(topicPartition, new AtomicLong(startingOffset)); changelogToEndOffset.put(topicPartition, new AtomicLong(endingOffset)); changelogToTotalNumRestored.put(topicPartition, new AtomicLong(0L)); + changelogToRestoreStartTime.put(topicPartition, new AtomicLong(System.nanoTime())); } @Override @@ -1372,6 +1375,7 @@ public class IntegrationTestUtils { if (restored != null) { restored.addAndGet(totalRestored); } + changelogToRestoreEndTime.put(topicPartition, new AtomicLong(System.nanoTime())); } public long totalNumRestored() { @@ -1381,6 +1385,11 @@ public class IntegrationTestUtils { } return totalNumRestored; } + + public Map changelogToRestoreTime() { + return changelogToRestoreStartTime.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> changelogToRestoreEndTime.get(e.getKey()).get() - e.getValue().get())); + } } public static class TrackingStandbyUpdateListener implements StandbyUpdateListener { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index c91e9b38b99..5e09ceb62da 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -41,6 +41,7 @@ import org.apache.kafka.streams.processor.StateRestoreListener; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.ProcessorStateManager.StateStoreMetadata; import org.apache.kafka.streams.processor.internals.Task.TaskType; +import org.apache.kafka.streams.state.internals.MeteredStateStore; import org.slf4j.Logger; @@ -138,6 +139,8 @@ public class StoreChangelogReader implements ChangelogReader { // either due to limit offset (standby) or committed end offset (active) private int bufferedLimitIndex; + private long restoreStartTimeNs; + private ChangelogMetadata(final StateStoreMetadata storeMetadata, final ProcessorStateManager stateManager) { this.changelogState = ChangelogState.REGISTERED; this.storeMetadata = storeMetadata; @@ -188,6 +191,10 @@ public class StoreChangelogReader implements ChangelogReader { int bufferedLimitIndex() { return bufferedLimitIndex; } + + long calculateRestoreTime(final long restoreEndTimeNs) { + return restoreEndTimeNs - restoreStartTimeNs; + } } private static final long DEFAULT_OFFSET_UPDATE_MS = Duration.ofMinutes(5L).toMillis(); @@ -695,6 +702,9 @@ public class StoreChangelogReader implements ChangelogReader { changelogMetadata.transitTo(ChangelogState.COMPLETED); pauseChangelogsFromRestoreConsumer(Collections.singleton(partition)); + if (storeMetadata.store() instanceof MeteredStateStore) { + ((MeteredStateStore) storeMetadata.store()).recordRestoreTime(changelogMetadata.calculateRestoreTime(time.nanoseconds())); + } try { stateRestoreListener.onRestoreEnd(partition, storeName, changelogMetadata.totalRestored); @@ -1026,6 +1036,7 @@ public class StoreChangelogReader implements ChangelogReader { // no records to restore; in this case we just initialize the sensor to zero final long recordsToRestore = Math.max(changelogMetadata.restoreEndOffset - startOffset, 0L); task.recordRestoration(time, recordsToRestore, true); + changelogMetadata.restoreStartTimeNs = time.nanoseconds(); } else if (changelogMetadata.stateManager.taskType() == TaskType.STANDBY) { try { standbyUpdateListener.onUpdateStart(partition, storeName, startOffset); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java index 0962033b7ef..9c033d6bbd5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java @@ -69,7 +69,7 @@ import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetric */ public class MeteredKeyValueStore extends WrappedStateStore, K, V> - implements KeyValueStore { + implements KeyValueStore, MeteredStateStore { final Serde keySerde; final Serde valueSerde; @@ -91,6 +91,7 @@ public class MeteredKeyValueStore protected InternalProcessorContext internalContext; private StreamsMetricsImpl streamsMetrics; private TaskId taskId; + private Sensor restoreSensor; protected OpenIterators openIterators; @@ -128,11 +129,10 @@ public class MeteredKeyValueStore streamsMetrics = (StreamsMetricsImpl) stateStoreContext.metrics(); registerMetrics(); - final Sensor restoreSensor = - StateStoreMetrics.restoreSensor(taskId.toString(), metricsScope, name(), streamsMetrics); - // register and possibly restore the state from the logs - maybeMeasureLatency(() -> super.init(stateStoreContext, root), time, restoreSensor); + restoreSensor = StateStoreMetrics.restoreSensor(taskId.toString(), metricsScope, name(), streamsMetrics); + + super.init(stateStoreContext, root); } private void registerMetrics() { @@ -152,6 +152,11 @@ public class MeteredKeyValueStore openIterators = new OpenIterators(taskId, metricsScope, name(), streamsMetrics); } + @Override + public void recordRestoreTime(final long restoreTimeNs) { + restoreSensor.record(restoreTimeNs); + } + protected Serde prepareValueSerdeForStore(final Serde valueSerde, final SerdeGetter getter) { return WrappingNullableUtils.prepareValueSerde(valueSerde, getter); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java index 234ac1220f7..7794a6ebc51 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java @@ -57,7 +57,7 @@ import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetric public class MeteredSessionStore extends WrappedStateStore, Windowed, V> - implements SessionStore { + implements SessionStore, MeteredStateStore { private final String metricsScope; private final Serde keySerde; @@ -73,6 +73,7 @@ public class MeteredSessionStore private Sensor iteratorDurationSensor; private InternalProcessorContext internalContext; private TaskId taskId; + private Sensor restoreSensor; private final LongAdder numOpenIterators = new LongAdder(); private final NavigableSet openIterators = new ConcurrentSkipListSet<>(Comparator.comparingLong(MeteredIterator::startTimestamp)); @@ -108,11 +109,9 @@ public class MeteredSessionStore streamsMetrics = (StreamsMetricsImpl) stateStoreContext.metrics(); registerMetrics(); - final Sensor restoreSensor = - StateStoreMetrics.restoreSensor(taskId.toString(), metricsScope, name(), streamsMetrics); + restoreSensor = StateStoreMetrics.restoreSensor(taskId.toString(), metricsScope, name(), streamsMetrics); - // register and possibly restore the state from the logs - maybeMeasureLatency(() -> super.init(stateStoreContext, root), time, restoreSensor); + super.init(stateStoreContext, root); } private void registerMetrics() { @@ -132,6 +131,11 @@ public class MeteredSessionStore ); } + @Override + public void recordRestoreTime(final long restoreTimeNs) { + restoreSensor.record(restoreTimeNs); + } + private void initStoreSerde(final StateStoreContext context) { final String storeName = name(); final String changelogTopic = ProcessorContextUtils.changelogFor(context, storeName, Boolean.FALSE); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredStateStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredStateStore.java new file mode 100644 index 00000000000..6b9f4eeb8f2 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredStateStore.java @@ -0,0 +1,22 @@ +/* + * 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.streams.state.internals; + +public interface MeteredStateStore { + + void recordRestoreTime(final long restoreTimeNs); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java index 2da877453ce..1ba37da6dab 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java @@ -60,7 +60,7 @@ import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetric public class MeteredWindowStore extends WrappedStateStore, Windowed, V> - implements WindowStore { + implements WindowStore, MeteredStateStore { private final long windowSizeMs; private final String metricsScope; @@ -76,6 +76,7 @@ public class MeteredWindowStore private Sensor iteratorDurationSensor; private InternalProcessorContext internalContext; private TaskId taskId; + private Sensor restoreSensor; private final LongAdder numOpenIterators = new LongAdder(); private final NavigableSet openIterators = new ConcurrentSkipListSet<>(Comparator.comparingLong(MeteredIterator::startTimestamp)); @@ -124,8 +125,8 @@ public class MeteredWindowStore streamsMetrics = (StreamsMetricsImpl) stateStoreContext.metrics(); registerMetrics(); - final Sensor restoreSensor = - StateStoreMetrics.restoreSensor(taskId.toString(), metricsScope, name(), streamsMetrics); + + restoreSensor = StateStoreMetrics.restoreSensor(taskId.toString(), metricsScope, name(), streamsMetrics); // register and possibly restore the state from the logs maybeMeasureLatency(() -> super.init(stateStoreContext, root), time, restoreSensor); @@ -150,6 +151,11 @@ public class MeteredWindowStore ); } + @Override + public void recordRestoreTime(final long restoreTimeNs) { + restoreSensor.record(restoreTimeNs); + } + private void initStoreSerde(final StateStoreContext context) { final String storeName = name(); final String changelogTopic = ProcessorContextUtils.changelogFor(context, storeName, Boolean.FALSE); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java index 69655b4d642..99a2b2519f9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java @@ -41,6 +41,7 @@ import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.ProcessorStateManager.StateStoreMetadata; +import org.apache.kafka.streams.state.internals.MeteredKeyValueStore; import org.apache.kafka.test.MockStandbyUpdateListener; import org.apache.kafka.test.MockStateRestoreListener; import org.apache.kafka.test.StreamsTestUtils; @@ -89,7 +90,9 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) @@ -1364,6 +1367,58 @@ public class StoreChangelogReaderTest { } } + @Test + public void shouldCallRecordRestoreTimeAtTheEndOfRestore() { + setupActiveStateManager(); + + final MeteredKeyValueStore meteredStateStore = mock(MeteredKeyValueStore.class); + + when(storeMetadata.changelogPartition()).thenReturn(tp); + when(storeMetadata.store()).thenReturn(meteredStateStore); + when(meteredStateStore.name()).thenReturn(storeName); + final TaskId taskId = new TaskId(0, 0); + + when(storeMetadata.offset()).thenReturn(0L); + when(activeStateManager.taskId()).thenReturn(taskId); + + setupConsumer(2, tp); + consumer.updateEndOffsets(Collections.singletonMap(tp, 2L)); + adminClient.updateEndOffsets(Collections.singletonMap(tp, 2L)); + + changelogReader.register(tp, activeStateManager); + + changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))); + + assertEquals(1L, changelogReader.changelogMetadata(tp).totalRestored()); + verify(meteredStateStore).recordRestoreTime(anyLong()); + } + + @Test + public void shouldNotCallRecordRestoreTimeIfRestoreDoesNotComplete() { + setupActiveStateManager(); + + final MeteredKeyValueStore meteredStateStore = mock(MeteredKeyValueStore.class); + + when(storeMetadata.changelogPartition()).thenReturn(tp); + when(storeMetadata.store()).thenReturn(meteredStateStore); + when(meteredStateStore.name()).thenReturn(storeName); + final TaskId taskId = new TaskId(0, 0); + + when(storeMetadata.offset()).thenReturn(0L); + when(activeStateManager.taskId()).thenReturn(taskId); + + setupConsumer(2, tp); + consumer.updateEndOffsets(Collections.singletonMap(tp, 3L)); + adminClient.updateEndOffsets(Collections.singletonMap(tp, 3L)); + + changelogReader.register(tp, activeStateManager); + + changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))); + + assertEquals(1L, changelogReader.changelogMetadata(tp).totalRestored()); + verify(meteredStateStore, never()).recordRestoreTime(anyLong()); + } + private void setupConsumer(final long messages, final TopicPartition topicPartition) { assignPartition(messages, topicPartition); addRecords(messages, topicPartition); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java index 294af3944f2..1ba655a75ce 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java @@ -58,7 +58,6 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.empty; -import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.not; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNull; @@ -210,16 +209,19 @@ public class MeteredKeyValueStoreTest { } @Test - public void shouldRecordRestoreLatencyOnInit() { + public void shouldRecordRestoreLatencyOnRecordRestoreTime() { setUp(); doNothing().when(inner).init(context, metered); init(); + final long restoreTimeNs = 1000L; + metered.recordRestoreTime(restoreTimeNs); + // it suffices to verify one restore metric since all restore metrics are recorded by the same sensor // and the sensor is tested elsewhere - final KafkaMetric metric = metric("restore-rate"); - assertThat((Double) metric.metricValue(), greaterThan(0.0)); + final KafkaMetric metric = metric("restore-latency-max"); + assertThat((Double) metric.metricValue(), equalTo((double) restoreTimeNs)); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java index ee1b686dade..f8b08a532d1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java @@ -458,14 +458,17 @@ public class MeteredSessionStoreTest { } @Test - public void shouldRecordRestoreTimeOnInit() { + public void shouldRecordRestoreLatencyOnRecordRestoreTime() { setUp(); init(); + final long restoreTimeNs = 1000L; + store.recordRestoreTime(restoreTimeNs); + // it suffices to verify one restore metric since all restore metrics are recorded by the same sensor // and the sensor is tested elsewhere - final KafkaMetric metric = metric("restore-rate"); - assertTrue((Double) metric.metricValue() > 0); + final KafkaMetric metric = metric("restore-latency-max"); + assertThat((Double) metric.metricValue(), equalTo((double) restoreTimeNs)); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStoreTest.java index 8e8e02b2722..5c4509bc7a3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStoreTest.java @@ -183,13 +183,6 @@ public class MeteredVersionedKeyValueStoreTest { verify(valueSerializer).serialize(changelogTopicName, VALUE); } - @Test - public void shouldRecordMetricsOnInit() { - // init is called in setUp(). it suffices to verify one restore metric since all restore - // metrics are recorded by the same sensor, and the sensor is tested elsewhere. - assertThat((Double) getMetric("restore-rate").metricValue(), greaterThan(0.0)); - } - @Test public void shouldDelegateAndRecordMetricsOnPut() { when(inner.put(RAW_KEY, RAW_VALUE, TIMESTAMP)).thenReturn(PUT_RETURN_CODE_VALID_TO_UNDEFINED); @@ -473,4 +466,4 @@ public class MeteredVersionedKeyValueStoreTest { .filter(name -> name.group().equals(STORE_LEVEL_GROUP) && name.tags().equals(tags)) .collect(Collectors.toList()); } -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java index 1c8935d1e1c..2726ce26aa7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java @@ -210,14 +210,19 @@ public class MeteredWindowStoreTest { } @Test - public void shouldRecordRestoreLatencyOnInit() { + public void shouldRecordRestoreLatencyOnRecordRestoreTime() { + setUp(); doNothing().when(innerStoreMock).init(context, store); + store.init(context, store); + final long restoreTimeNs = 1000L; + store.recordRestoreTime(restoreTimeNs); + // it suffices to verify one restore metric since all restore metrics are recorded by the same sensor // and the sensor is tested elsewhere - final KafkaMetric metric = metric("restore-rate"); - assertThat((Double) metric.metricValue(), greaterThan(0.0)); + final KafkaMetric metric = metric("restore-latency-max"); + assertThat((Double) metric.metricValue(), equalTo((double) restoreTimeNs)); } @Test