From d3acdf0b9a2965a5867510f5f8cb3d99944f0aca Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Tue, 7 May 2024 11:51:47 +0100 Subject: [PATCH 01/40] KAFKA-10551: Add topic id support to produce request and response --- .../clients/producer/internals/Sender.java | 10 +- .../kafka/common/requests/ProduceRequest.java | 14 +-- .../common/requests/ProduceResponse.java | 15 +-- .../org/apache/kafka/common/utils/Utils.java | 9 ++ .../common/message/ProduceRequest.json | 6 +- .../common/message/ProduceResponse.json | 6 +- .../producer/internals/SenderTest.java | 69 +++++++------ .../internals/TransactionManagerTest.java | 5 +- .../kafka/common/message/MessageTest.java | 12 ++- .../common/requests/ProduceResponseTest.java | 19 ++-- .../common/requests/RequestResponseTest.java | 17 ++-- .../group/CoordinatorPartitionWriter.scala | 8 +- .../group/GroupMetadataManager.scala | 25 ++--- .../transaction/TransactionStateManager.scala | 15 +-- .../scala/kafka/server/DelayedProduce.scala | 10 +- .../main/scala/kafka/server/KafkaApis.scala | 58 +++++++---- .../scala/kafka/server/ReplicaManager.scala | 59 ++++++----- .../kafka/api/AuthorizerIntegrationTest.scala | 3 +- .../server/LocalLeaderEndPointTest.scala | 27 ++--- .../AbstractCoordinatorConcurrencyTest.scala | 10 +- .../CoordinatorPartitionWriterTest.scala | 63 +++++++----- .../GroupCoordinatorConcurrencyTest.scala | 4 + .../group/GroupCoordinatorTest.scala | 35 ++++--- .../group/GroupMetadataManagerTest.scala | 69 ++++++++----- ...ransactionCoordinatorConcurrencyTest.scala | 7 +- .../TransactionStateManagerTest.scala | 43 +++++--- .../unit/kafka/server/KafkaApisTest.scala | 99 ++++++++++++------- .../ReplicaManagerConcurrencyTest.scala | 8 +- .../kafka/server/ReplicaManagerTest.scala | 78 ++++++++++----- .../producer/ProducerResponseBenchmark.java | 9 +- 30 files changed, 499 insertions(+), 313 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index c4e2b73e8b91b..cd23e0501ab4d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -33,6 +33,7 @@ import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.AuthenticationException; import org.apache.kafka.common.errors.ClusterAuthorizationException; import org.apache.kafka.common.errors.FencedLeaderEpochException; @@ -610,7 +611,9 @@ private void handleProduceResponse(ClientResponse response, Map partitionsWithUpdatedLeaderInfo = new HashMap<>(); produceResponse.data().responses().forEach(r -> r.partitionResponses().forEach(p -> { - TopicPartition tp = new TopicPartition(r.name(), p.index()); + // Version 12 drop topic name and add support to topic id. However, metadata can be used to map topic id to topic name. + String topicName = (r.name() == null || r.name().isEmpty()) ? metadata.topicNames().get(r.topicId()) : r.name(); + TopicPartition tp = new TopicPartition(topicName, p.index()); ProduceResponse.PartitionResponse partResp = new ProduceResponse.PartitionResponse( Errors.forCode(p.errorCode()), p.baseOffset(), @@ -880,6 +883,7 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo ProduceRequestData.TopicProduceDataCollection tpd = new ProduceRequestData.TopicProduceDataCollection(); for (ProducerBatch batch : batches) { TopicPartition tp = batch.topicPartition; + Uuid topicId = metadata.topicIds().getOrDefault(tp.topic(), Uuid.ZERO_UUID); MemoryRecords records = batch.records(); // down convert if necessary to the minimum magic used. In general, there can be a delay between the time @@ -891,9 +895,9 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo // which is supporting the new magic version to one which doesn't, then we will need to convert. if (!records.hasMatchingMagic(minUsedMagic)) records = batch.records().downConvert(minUsedMagic, 0, time).records(); - ProduceRequestData.TopicProduceData tpData = tpd.find(tp.topic()); + ProduceRequestData.TopicProduceData tpData = tpd.find(tp.topic(), topicId); if (tpData == null) { - tpData = new ProduceRequestData.TopicProduceData().setName(tp.topic()); + tpData = new ProduceRequestData.TopicProduceData().setName(tp.topic()).setTopicId(topicId); tpd.add(tpData); } tpData.partitionData().add(new ProduceRequestData.PartitionProduceData() diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 4724ce4789ccb..9f4672d446858 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -17,7 +17,7 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.InvalidRecordException; -import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.errors.UnsupportedCompressionTypeException; import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.message.ProduceResponseData; @@ -117,7 +117,7 @@ public String toString() { // Care should be taken in methods that use this field. private volatile ProduceRequestData data; // the partitionSizes is lazily initialized since it is used by server-side in production. - private volatile Map partitionSizes; + private volatile Map partitionSizes; public ProduceRequest(ProduceRequestData produceRequestData, short version) { super(ApiKeys.PRODUCE, version); @@ -128,15 +128,15 @@ public ProduceRequest(ProduceRequestData produceRequestData, short version) { } // visible for testing - Map partitionSizes() { + Map partitionSizes() { if (partitionSizes == null) { // this method may be called by different thread (see the comment on data) synchronized (this) { if (partitionSizes == null) { - Map tmpPartitionSizes = new HashMap<>(); + Map tmpPartitionSizes = new HashMap<>(); data.topicData().forEach(topicData -> topicData.partitionData().forEach(partitionData -> - tmpPartitionSizes.compute(new TopicPartition(topicData.name(), partitionData.index()), + tmpPartitionSizes.compute(new TopicIdPartition(topicData.topicId(), partitionData.index(), topicData.name()), (ignored, previousValue) -> partitionData.records().sizeInBytes() + (previousValue == null ? 0 : previousValue)) ) @@ -183,9 +183,9 @@ public ProduceResponse getErrorResponse(int throttleTimeMs, Throwable e) { ApiError apiError = ApiError.fromThrowable(e); ProduceResponseData data = new ProduceResponseData().setThrottleTimeMs(throttleTimeMs); partitionSizes().forEach((tp, ignored) -> { - ProduceResponseData.TopicProduceResponse tpr = data.responses().find(tp.topic()); + ProduceResponseData.TopicProduceResponse tpr = data.responses().find(tp.topic(), tp.topicId()); if (tpr == null) { - tpr = new ProduceResponseData.TopicProduceResponse().setName(tp.topic()); + tpr = new ProduceResponseData.TopicProduceResponse().setName(tp.topic()).setTopicId(tp.topicId()); data.responses().add(tpr); } tpr.partitionResponses().add(new ProduceResponseData.PartitionProduceResponse() diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 186ad9b80a19f..4241260d77416 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -17,7 +17,7 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.Node; -import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.message.ProduceResponseData; import org.apache.kafka.common.message.ProduceResponseData.LeaderIdAndEpoch; import org.apache.kafka.common.protocol.ApiKeys; @@ -33,6 +33,7 @@ import java.util.Objects; import java.util.stream.Collectors; + /** * This wrapper supports both v0 and v8 of ProduceResponse. * @@ -72,7 +73,7 @@ public ProduceResponse(ProduceResponseData produceResponseData) { * @param responses Produced data grouped by topic-partition */ @Deprecated - public ProduceResponse(Map responses) { + public ProduceResponse(Map responses) { this(responses, DEFAULT_THROTTLE_TIME, Collections.emptyList()); } @@ -83,7 +84,7 @@ public ProduceResponse(Map responses) { * @param throttleTimeMs Time in milliseconds the response was throttled */ @Deprecated - public ProduceResponse(Map responses, int throttleTimeMs) { + public ProduceResponse(Map responses, int throttleTimeMs) { this(toData(responses, throttleTimeMs, Collections.emptyList())); } @@ -96,16 +97,16 @@ public ProduceResponse(Map responses, int thr * @param nodeEndpoints List of node endpoints */ @Deprecated - public ProduceResponse(Map responses, int throttleTimeMs, List nodeEndpoints) { + public ProduceResponse(Map responses, int throttleTimeMs, List nodeEndpoints) { this(toData(responses, throttleTimeMs, nodeEndpoints)); } - private static ProduceResponseData toData(Map responses, int throttleTimeMs, List nodeEndpoints) { + private static ProduceResponseData toData(Map responses, int throttleTimeMs, List nodeEndpoints) { ProduceResponseData data = new ProduceResponseData().setThrottleTimeMs(throttleTimeMs); responses.forEach((tp, response) -> { - ProduceResponseData.TopicProduceResponse tpr = data.responses().find(tp.topic()); + ProduceResponseData.TopicProduceResponse tpr = data.responses().find(tp.topic(), tp.topicId()); if (tpr == null) { - tpr = new ProduceResponseData.TopicProduceResponse().setName(tp.topic()); + tpr = new ProduceResponseData.TopicProduceResponse().setName(tp.topic()).setTopicId(tp.topicId()); data.responses().add(tpr); } tpr.partitionResponses() 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 f62aee76cbc05..5098ed2a57f87 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 @@ -1694,6 +1694,15 @@ public static void require(boolean requirement) { throw new IllegalArgumentException("requirement failed"); } + /** + * Convert map keys to another type. + */ + public static Map convertKeys(Map originalMap, Function converter) { + Map newMap = new HashMap<>(); + originalMap.forEach((key, value) -> newMap.put(converter.apply(key), value)); + return newMap; + } + /** * A runnable that can throw checked exception. */ diff --git a/clients/src/main/resources/common/message/ProduceRequest.json b/clients/src/main/resources/common/message/ProduceRequest.json index ae01fe5c8c08a..81ab87989ac38 100644 --- a/clients/src/main/resources/common/message/ProduceRequest.json +++ b/clients/src/main/resources/common/message/ProduceRequest.json @@ -37,7 +37,8 @@ // Version 10 is the same as version 9 (KIP-951). // // Version 11 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). - "validVersions": "0-11", + // Version 12 replaces topic names with topic IDs (KIP-516). May return UNKNOWN_TOPIC_ID error code. + "validVersions": "0-12", "deprecatedVersions": "0-6", "flexibleVersions": "9+", "fields": [ @@ -49,8 +50,9 @@ "about": "The timeout to await a response in milliseconds." }, { "name": "TopicData", "type": "[]TopicProduceData", "versions": "0+", "about": "Each topic to produce to.", "fields": [ - { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", "mapKey": true, + { "name": "Name", "type": "string", "versions": "0-11", "entityType": "topicName", "mapKey": true, "ignorable": true, "about": "The topic name." }, + { "name": "TopicId", "type": "uuid", "versions": "12+", "mapKey": true, "ignorable": true, "about": "The unique topic ID" }, { "name": "PartitionData", "type": "[]PartitionProduceData", "versions": "0+", "about": "Each partition to produce to.", "fields": [ { "name": "Index", "type": "int32", "versions": "0+", diff --git a/clients/src/main/resources/common/message/ProduceResponse.json b/clients/src/main/resources/common/message/ProduceResponse.json index 92c7a2223da09..dbe4a18a9b27f 100644 --- a/clients/src/main/resources/common/message/ProduceResponse.json +++ b/clients/src/main/resources/common/message/ProduceResponse.json @@ -36,13 +36,15 @@ // Version 10 adds 'CurrentLeader' and 'NodeEndpoints' as tagged fields (KIP-951) // // Version 11 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). - "validVersions": "0-11", + // Version 12 replaces topic names with topic IDs (KIP-516). May return UNKNOWN_TOPIC_ID error code. + "validVersions": "0-12", "flexibleVersions": "9+", "fields": [ { "name": "Responses", "type": "[]TopicProduceResponse", "versions": "0+", "about": "Each produce response", "fields": [ - { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", "mapKey": true, + { "name": "Name", "type": "string", "versions": "0-11", "entityType": "topicName", "mapKey": true, "ignorable": true, "about": "The topic name" }, + { "name": "TopicId", "type": "uuid", "versions": "12+", "mapKey": true, "ignorable": true, "about": "The unique topic ID" }, { "name": "PartitionResponses", "type": "[]PartitionProduceResponse", "versions": "0+", "about": "Each partition that we produced to within the topic.", "fields": [ { "name": "Index", "type": "int32", "versions": "0+", 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 eb01d1d5841d7..efbdff84e30b7 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 @@ -31,7 +31,9 @@ import org.apache.kafka.common.MetricName; import org.apache.kafka.common.MetricNameTemplate; import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.ClusterAuthorizationException; import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.errors.InvalidTxnStateException; @@ -149,10 +151,15 @@ public class SenderTest { private static final int DELIVERY_TIMEOUT_MS = 1500; private static final long TOPIC_IDLE_MS = 60 * 1000; - private TopicPartition tp0 = new TopicPartition("test", 0); - private TopicPartition tp1 = new TopicPartition("test", 1); + private static final String TOPIC_NAME = "test"; + private static final Uuid TOPIC_ID = Uuid.randomUuid(); + private TopicPartition tp0 = new TopicPartition(TOPIC_NAME, 0); + private TopicPartition tp1 = new TopicPartition(TOPIC_NAME, 1); - private TopicPartition tp2 = new TopicPartition("test", 2); + private TopicPartition tp2 = new TopicPartition(TOPIC_NAME, 2); + private TopicIdPartition topicIdPartition0 = new TopicIdPartition(TOPIC_ID, tp0); + private TopicIdPartition topicIdPartition1 = new TopicIdPartition(TOPIC_ID, tp0); + private TopicIdPartition topicIdPartition2 = new TopicIdPartition(TOPIC_ID, tp0); private MockTime time = new MockTime(); private int batchSize = 16 * 1024; private ProducerMetadata metadata = new ProducerMetadata(0, 0, Long.MAX_VALUE, TOPIC_IDLE_MS, @@ -260,9 +267,9 @@ public void testDownConversionForMismatchedMagicValues() throws Exception { apiVersions.update("0", NodeApiVersions.create()); ProduceResponse.PartitionResponse resp = new ProduceResponse.PartitionResponse(Errors.NONE, offset, RecordBatch.NO_TIMESTAMP, 100); - Map partResp = new HashMap<>(); - partResp.put(tp0, resp); - partResp.put(tp1, resp); + Map partResp = new HashMap<>(); + partResp.put(new TopicIdPartition(TOPIC_ID, tp0), resp); + partResp.put(new TopicIdPartition(TOPIC_ID, tp1), resp); ProduceResponse produceResponse = new ProduceResponse(partResp, 0); client.prepareResponse(body -> { @@ -295,7 +302,7 @@ public void testDownConversionForMismatchedMagicValues() throws Exception { public void testQuotaMetrics() { MockSelector selector = new MockSelector(time); Sensor throttleTimeSensor = Sender.throttleTimeSensor(this.senderMetricsRegistry); - Cluster cluster = TestUtils.singletonCluster("test", 1); + Cluster cluster = TestUtils.singletonCluster(TOPIC_NAME, 1); Node node = cluster.nodes().get(0); NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, 1000, 1000, 64 * 1024, 64 * 1024, 1000, 10 * 1000, 127 * 1000, @@ -434,11 +441,11 @@ public void testSendInOrder() throws Exception { Sender sender = new Sender(logContext, client, metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries, senderMetrics, time, REQUEST_TIMEOUT, RETRY_BACKOFF_MS, null, apiVersions); // Create a two broker cluster, with partition 0 on broker 0 and partition 1 on broker 1 - MetadataResponse metadataUpdate1 = RequestTestUtils.metadataUpdateWith(2, Collections.singletonMap("test", 2)); + MetadataResponse metadataUpdate1 = RequestTestUtils.metadataUpdateWith(2, Collections.singletonMap(TOPIC_NAME, 2)); client.prepareMetadataUpdate(metadataUpdate1); // Send the first message. - TopicPartition tp2 = new TopicPartition("test", 1); + TopicPartition tp2 = new TopicPartition(TOPIC_NAME, 1); appendToAccumulator(tp2, 0L, "key1", "value1"); sender.runOnce(); // connect sender.runOnce(); // send produce request @@ -455,7 +462,7 @@ public void testSendInOrder() throws Exception { appendToAccumulator(tp2, 0L, "key2", "value2"); // Update metadata before sender receives response from broker 0. Now partition 2 moves to broker 0 - MetadataResponse metadataUpdate2 = RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 2)); + MetadataResponse metadataUpdate2 = RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(TOPIC_NAME, 2)); client.prepareMetadataUpdate(metadataUpdate2); // Sender should not send the second message to node 0. assertEquals(1, sender.inFlightBatches(tp2).size()); @@ -530,12 +537,12 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { @Test public void testMetadataTopicExpiry() throws Exception { long offset = 0; - client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 2))); + client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(TOPIC_NAME, 2))); Future future = appendToAccumulator(tp0); sender.runOnce(); assertTrue(metadata.containsTopic(tp0.topic()), "Topic not added to metadata"); - client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 2))); + client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(TOPIC_NAME, 2))); sender.runOnce(); // send produce request client.respond(produceResponse(tp0, offset, Errors.NONE, 0)); sender.runOnce(); @@ -547,12 +554,12 @@ public void testMetadataTopicExpiry() throws Exception { assertTrue(metadata.containsTopic(tp0.topic()), "Topic not retained in metadata list"); time.sleep(TOPIC_IDLE_MS); - client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 2))); + client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(TOPIC_NAME, 2))); assertFalse(metadata.containsTopic(tp0.topic()), "Unused topic has not been expired"); future = appendToAccumulator(tp0); sender.runOnce(); assertTrue(metadata.containsTopic(tp0.topic()), "Topic not added to metadata"); - client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 2))); + client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(TOPIC_NAME, 2))); sender.runOnce(); // send produce request client.respond(produceResponse(tp0, offset + 1, Errors.NONE, 0)); sender.runOnce(); @@ -2444,8 +2451,8 @@ private void testSplitBatchAndSend(TransactionManager txnManager, assertEquals(1, client.inFlightRequestCount()); assertTrue(client.isReady(node, time.milliseconds()), "Client ready status should be true"); - Map responseMap = new HashMap<>(); - responseMap.put(tp, new ProduceResponse.PartitionResponse(Errors.MESSAGE_TOO_LARGE)); + Map responseMap = new HashMap<>(); + responseMap.put(new TopicIdPartition(TOPIC_ID, tp), new ProduceResponse.PartitionResponse(Errors.MESSAGE_TOO_LARGE)); client.respond(new ProduceResponse(responseMap)); sender.runOnce(); // split and reenqueue assertEquals(2, txnManager.sequenceNumber(tp), "The next sequence should be 2"); @@ -2462,7 +2469,7 @@ private void testSplitBatchAndSend(TransactionManager txnManager, assertEquals(1, client.inFlightRequestCount()); assertTrue(client.isReady(node, time.milliseconds()), "Client ready status should be true"); - responseMap.put(tp, new ProduceResponse.PartitionResponse(Errors.NONE, 0L, 0L, 0L)); + responseMap.put(new TopicIdPartition(TOPIC_ID, tp), new ProduceResponse.PartitionResponse(Errors.NONE, 0L, 0L, 0L)); client.respond(produceRequestMatcher(tp, producerIdAndEpoch, 0, txnManager.isTransactional()), new ProduceResponse(responseMap)); @@ -2479,7 +2486,7 @@ private void testSplitBatchAndSend(TransactionManager txnManager, assertEquals(1, client.inFlightRequestCount()); assertTrue(client.isReady(node, time.milliseconds()), "Client ready status should be true"); - responseMap.put(tp, new ProduceResponse.PartitionResponse(Errors.NONE, 1L, 0L, 0L)); + responseMap.put(new TopicIdPartition(TOPIC_ID, tp), new ProduceResponse.PartitionResponse(Errors.NONE, 1L, 0L, 0L)); client.respond(produceRequestMatcher(tp, producerIdAndEpoch, 1, txnManager.isTransactional()), new ProduceResponse(responseMap)); @@ -2532,8 +2539,8 @@ public void testInflightBatchesExpireOnDeliveryTimeout() throws InterruptedExcep assertEquals(1, client.inFlightRequestCount()); assertEquals(1, sender.inFlightBatches(tp0).size(), "Expect one in-flight batch in accumulator"); - Map responseMap = new HashMap<>(); - responseMap.put(tp0, new ProduceResponse.PartitionResponse(Errors.NONE, 0L, 0L, 0L)); + Map responseMap = new HashMap<>(); + responseMap.put(new TopicIdPartition(TOPIC_ID, tp0), new ProduceResponse.PartitionResponse(Errors.NONE, 0L, 0L, 0L)); client.respond(new ProduceResponse(responseMap)); time.sleep(deliveryTimeoutMs); @@ -2710,8 +2717,8 @@ public void testExpiredBatchesInMultiplePartitions() throws Exception { assertEquals(1, client.inFlightRequestCount()); assertEquals(1, sender.inFlightBatches(tp0).size(), "Expect one in-flight batch in accumulator"); - Map responseMap = new HashMap<>(); - responseMap.put(tp0, new ProduceResponse.PartitionResponse(Errors.NONE, 0L, 0L, 0L)); + Map responseMap = new HashMap<>(); + responseMap.put(new TopicIdPartition(TOPIC_ID, tp0), new ProduceResponse.PartitionResponse(Errors.NONE, 0L, 0L, 0L)); client.respond(new ProduceResponse(responseMap)); // Successfully expire both batches. @@ -3211,7 +3218,7 @@ public void testProducerBatchRetriesWhenPartitionLeaderChanges() throws Exceptio int tp0LeaderEpoch = 100; int epoch = tp0LeaderEpoch; this.client.updateMetadata( - RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 2), + RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(TOPIC_NAME, 2), tp -> { if (tp0.equals(tp)) { return epoch; @@ -3238,7 +3245,7 @@ public void testProducerBatchRetriesWhenPartitionLeaderChanges() throws Exceptio // Update leader epoch for tp0 int newEpoch = ++tp0LeaderEpoch; this.client.updateMetadata( - RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 2), + RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(TOPIC_NAME, 2), tp -> { if (tp0.equals(tp)) { return newEpoch; @@ -3326,7 +3333,7 @@ public void testWhenProduceResponseReturnsWithALeaderShipChangeErrorButNoNewLead int tp1LeaderEpoch = 200; int tp2LeaderEpoch = 300; this.client.updateMetadata( - RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 3), + RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(TOPIC_NAME, 3), tp -> { if (tp0.equals(tp)) { return tp0LeaderEpoch; @@ -3406,7 +3413,7 @@ public void testWhenProduceResponseReturnsWithALeaderShipChangeErrorAndNewLeader int tp1LeaderEpoch = 200; int tp2LeaderEpoch = 300; this.client.updateMetadata( - RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 3), + RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(TOPIC_NAME, 3), tp -> { if (tp0.equals(tp)) { return tp0LeaderEpoch; @@ -3603,7 +3610,7 @@ private FutureRecordMetadata appendToAccumulator(TopicPartition tp, long timesta private ProduceResponse produceResponse(TopicPartition tp, long offset, Errors error, int throttleTimeMs, long logStartOffset, String errorMessage) { ProduceResponse.PartitionResponse resp = new ProduceResponse.PartitionResponse(error, offset, RecordBatch.NO_TIMESTAMP, logStartOffset, Collections.emptyList(), errorMessage); - Map partResp = Collections.singletonMap(tp, resp); + Map partResp = Collections.singletonMap(new TopicIdPartition(TOPIC_ID, tp), resp); return new ProduceResponse(partResp, throttleTimeMs); } @@ -3619,9 +3626,9 @@ private ProduceResponse produceResponse(Map resp for (Map.Entry entry : responses.entrySet()) { TopicPartition topicPartition = entry.getKey(); - ProduceResponseData.TopicProduceResponse topicData = data.responses().find(topicPartition.topic()); + ProduceResponseData.TopicProduceResponse topicData = data.responses().find(topicPartition.topic(), TOPIC_ID); if (topicData == null) { - topicData = new ProduceResponseData.TopicProduceResponse().setName(topicPartition.topic()); + topicData = new ProduceResponseData.TopicProduceResponse().setName(topicPartition.topic()).setTopicId(TOPIC_ID); data.responses().add(topicData); } @@ -3703,9 +3710,9 @@ private void setupWithTransactionState( this.sender = new Sender(logContext, this.client, this.metadata, this.accumulator, guaranteeOrder, MAX_REQUEST_SIZE, ACKS_ALL, retries, this.senderMetricsRegistry, this.time, REQUEST_TIMEOUT, RETRY_BACKOFF_MS, transactionManager, apiVersions); - metadata.add("test", time.milliseconds()); + metadata.add(TOPIC_NAME, time.milliseconds()); if (updateMetadata) - this.client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 2))); + this.client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(TOPIC_NAME, 2))); } private void assertSuccessfulSend() throws InterruptedException { 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 51299ad337ea1..c88c8df29aadc 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 @@ -29,7 +29,9 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.FencedInstanceIdException; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.InvalidTxnStateException; @@ -131,6 +133,7 @@ public class TransactionManagerTest { private final int transactionTimeoutMs = 1121; private final String topic = "test"; + private static final Uuid TOPIC_ID = Uuid.randomUuid(); private final TopicPartition tp0 = new TopicPartition(topic, 0); private final TopicPartition tp1 = new TopicPartition(topic, 1); private final long producerId = 13131L; @@ -3882,7 +3885,7 @@ private ProduceResponse produceResponse(TopicPartition tp, long offset, Errors e @SuppressWarnings("deprecation") private ProduceResponse produceResponse(TopicPartition tp, long offset, Errors error, int throttleTimeMs, int logStartOffset) { ProduceResponse.PartitionResponse resp = new ProduceResponse.PartitionResponse(error, offset, RecordBatch.NO_TIMESTAMP, logStartOffset); - Map partResp = singletonMap(tp, resp); + Map partResp = singletonMap(new TopicIdPartition(TOPIC_ID, tp), resp); return new ProduceResponse(partResp, throttleTimeMs); } diff --git a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java index 478bfa0668d3a..35066d7f4c093 100644 --- a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java +++ b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java @@ -928,7 +928,6 @@ public void testProduceResponseVersions() throws Exception { testAllMessageRoundTrips(new ProduceResponseData() .setResponses(new ProduceResponseData.TopicProduceResponseCollection(singletonList( new ProduceResponseData.TopicProduceResponse() - .setName(topicName) .setPartitionResponses(singletonList( new ProduceResponseData.PartitionProduceResponse() .setIndex(partitionIndex) @@ -938,7 +937,6 @@ public void testProduceResponseVersions() throws Exception { Supplier response = () -> new ProduceResponseData() .setResponses(new ProduceResponseData.TopicProduceResponseCollection(singletonList( new ProduceResponseData.TopicProduceResponse() - .setName(topicName) .setPartitionResponses(singletonList( new ProduceResponseData.PartitionProduceResponse() .setIndex(partitionIndex) @@ -973,10 +971,20 @@ public void testProduceResponseVersions() throws Exception { responseData.setThrottleTimeMs(0); } + if (version <= 11) { + responseData.responses().iterator().next().setName(topicName); + } + + if (version >= 12) { + responseData.responses().iterator().next().setTopicId(Uuid.randomUuid()); + } + if (version >= 3 && version <= 4) { testAllMessageRoundTripsBetweenVersions(version, (short) 5, responseData, responseData); } else if (version >= 6 && version <= 7) { testAllMessageRoundTripsBetweenVersions(version, (short) 8, responseData, responseData); + } else if (version < 12) { + testAllMessageRoundTripsBetweenVersions(version, (short) 12, responseData, responseData); } else { testEquivalentMessageRoundTrip(version, responseData); } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java index d854eb023012b..1bae3fb46af44 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java @@ -17,8 +17,9 @@ package org.apache.kafka.common.requests; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.ProduceResponseData; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.RecordBatch; @@ -42,8 +43,8 @@ public class ProduceResponseTest { @SuppressWarnings("deprecation") @Test public void produceResponseV5Test() { - Map responseData = new HashMap<>(); - TopicPartition tp0 = new TopicPartition("test", 0); + Map responseData = new HashMap<>(); + TopicIdPartition tp0 = new TopicIdPartition(Uuid.ZERO_UUID, 0, "test"); responseData.put(tp0, new ProduceResponse.PartitionResponse(Errors.NONE, 10000, RecordBatch.NO_TIMESTAMP, 100)); ProduceResponse v5Response = new ProduceResponse(responseData, 10); @@ -58,7 +59,7 @@ public void produceResponseV5Test() { ProduceResponseData.TopicProduceResponse topicProduceResponse = v5FromBytes.data().responses().iterator().next(); assertEquals(1, topicProduceResponse.partitionResponses().size()); ProduceResponseData.PartitionProduceResponse partitionProduceResponse = topicProduceResponse.partitionResponses().iterator().next(); - TopicPartition tp = new TopicPartition(topicProduceResponse.name(), partitionProduceResponse.index()); + TopicIdPartition tp = new TopicIdPartition(topicProduceResponse.topicId(), partitionProduceResponse.index(), topicProduceResponse.name()); assertEquals(tp0, tp); assertEquals(100, partitionProduceResponse.logStartOffset()); @@ -72,8 +73,9 @@ public void produceResponseV5Test() { @SuppressWarnings("deprecation") @Test public void produceResponseVersionTest() { - Map responseData = new HashMap<>(); - responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE, 10000, RecordBatch.NO_TIMESTAMP, 100)); + Map responseData = new HashMap<>(); + TopicIdPartition topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), 0, "test"); + responseData.put(topicIdPartition, new ProduceResponse.PartitionResponse(Errors.NONE, 10000, RecordBatch.NO_TIMESTAMP, 100)); ProduceResponse v0Response = new ProduceResponse(responseData); ProduceResponse v1Response = new ProduceResponse(responseData, 10); ProduceResponse v2Response = new ProduceResponse(responseData, 10); @@ -93,14 +95,15 @@ public void produceResponseVersionTest() { assertEquals(Errors.NONE, Errors.forCode(partitionProduceResponse.errorCode())); assertNull(partitionProduceResponse.errorMessage()); assertTrue(partitionProduceResponse.recordErrors().isEmpty()); + assertEquals(topicIdPartition.topicId(), topicProduceResponse.topicId()); } } @SuppressWarnings("deprecation") @Test public void produceResponseRecordErrorsTest() { - Map responseData = new HashMap<>(); - TopicPartition tp = new TopicPartition("test", 0); + Map responseData = new HashMap<>(); + TopicIdPartition tp = new TopicIdPartition(Uuid.randomUuid(), 0, "test"); ProduceResponse.PartitionResponse partResponse = new ProduceResponse.PartitionResponse(Errors.NONE, 10000, RecordBatch.NO_TIMESTAMP, 100, Collections.singletonList(new ProduceResponse.RecordError(3, "Record error")), diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index b1fdf35d8e383..986a3fc50bed4 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -445,8 +445,9 @@ public void cannotUseFindCoordinatorV0ToFindTransactionCoordinator() { @Test public void testProduceRequestPartitionSize() { - TopicPartition tp0 = new TopicPartition("test", 0); - TopicPartition tp1 = new TopicPartition("test", 1); + Uuid topicId = Uuid.randomUuid(); + TopicIdPartition tp0 = new TopicIdPartition(topicId, 0, "test"); + TopicIdPartition tp1 = new TopicIdPartition(topicId, 1, "test"); MemoryRecords records0 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, new SimpleRecord("woot".getBytes())); MemoryRecords records1 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, @@ -454,9 +455,9 @@ public void testProduceRequestPartitionSize() { ProduceRequest request = ProduceRequest.forMagic(RecordBatch.MAGIC_VALUE_V2, new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection(asList( - new ProduceRequestData.TopicProduceData().setName(tp0.topic()).setPartitionData( + new ProduceRequestData.TopicProduceData().setName(tp0.topic()).setTopicId(topicId).setPartitionData( singletonList(new ProduceRequestData.PartitionProduceData().setIndex(tp0.partition()).setRecords(records0))), - new ProduceRequestData.TopicProduceData().setName(tp1.topic()).setPartitionData( + new ProduceRequestData.TopicProduceData().setName(tp1.topic()).setTopicId(topicId).setPartitionData( singletonList(new ProduceRequestData.PartitionProduceData().setIndex(tp1.partition()).setRecords(records1)))) .iterator())) .setAcks((short) 1) @@ -2366,16 +2367,16 @@ private ProduceRequest createProduceRequest(short version) { @SuppressWarnings("deprecation") private ProduceResponse createProduceResponse() { - Map responseData = new HashMap<>(); - responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE, + Map responseData = new HashMap<>(); + responseData.put(new TopicIdPartition(Uuid.randomUuid(), 0, "test"), new ProduceResponse.PartitionResponse(Errors.NONE, 10000, RecordBatch.NO_TIMESTAMP, 100)); return new ProduceResponse(responseData, 0); } @SuppressWarnings("deprecation") private ProduceResponse createProduceResponseWithErrorMessage() { - Map responseData = new HashMap<>(); - responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE, + Map responseData = new HashMap<>(); + responseData.put(new TopicIdPartition(Uuid.randomUuid(), 0, "test"), new ProduceResponse.PartitionResponse(Errors.NONE, 10000, RecordBatch.NO_TIMESTAMP, 100, singletonList(new ProduceResponse.RecordError(0, "error message")), "global error message")); return new ProduceResponse(responseData, 0); diff --git a/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala b/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala index b912881847602..e911f0a228e3b 100644 --- a/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala +++ b/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala @@ -18,7 +18,7 @@ package kafka.coordinator.group import kafka.cluster.PartitionListener import kafka.server.{ActionQueue, ReplicaManager, RequestLocal, defaultError, genericError} -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{TopicIdPartition, TopicPartition} import org.apache.kafka.common.errors.RecordTooLargeException import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.{CompressionType, ControlRecordType, EndTransactionMarker, MemoryRecords, RecordBatch, TimestampType} @@ -217,13 +217,13 @@ class CoordinatorPartitionWriter[T]( memoryRecords: MemoryRecords, verificationGuard: VerificationGuard = VerificationGuard.SENTINEL ): Long = { - var appendResults: Map[TopicPartition, PartitionResponse] = Map.empty + var appendResults: Map[TopicIdPartition, PartitionResponse] = Map.empty replicaManager.appendRecords( timeout = 0L, requiredAcks = 1, internalTopicsAllowed = true, origin = AppendOrigin.COORDINATOR, - entriesPerPartition = Map(tp -> memoryRecords), + entriesPerPartition = Map(replicaManager.getTopicIdPartition(tp) -> memoryRecords), responseCallback = results => appendResults = results, requestLocal = RequestLocal.NoCaching, verificationGuards = Map(tp -> verificationGuard), @@ -233,7 +233,7 @@ class CoordinatorPartitionWriter[T]( actionQueue = directActionQueue ) - val partitionResult = appendResults.getOrElse(tp, + val partitionResult = appendResults.getOrElse(replicaManager.getTopicIdPartition(tp), throw new IllegalStateException(s"Append status $appendResults should have partition $tp.")) if (partitionResult.error != Errors.NONE) { diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala index b9dda91e25c16..06db912be7611 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala @@ -267,18 +267,19 @@ class GroupMetadataManager(brokerId: Int, } val groupMetadataPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, partitionFor(group.groupId)) - val groupMetadataRecords = Map(groupMetadataPartition -> records) + val groupMetadataTopicIdPartition = replicaManager.getTopicIdPartition(groupMetadataPartition) + val groupMetadataRecords = Map(groupMetadataTopicIdPartition -> records) val generationId = group.generationId // set the callback function to insert the created group into cache after log append completed - def putCacheCallback(responseStatus: Map[TopicPartition, PartitionResponse]): Unit = { + def putCacheCallback(responseStatus: Map[TopicIdPartition, PartitionResponse]): Unit = { // the append response should only contain the topics partition - if (responseStatus.size != 1 || !responseStatus.contains(groupMetadataPartition)) + if (responseStatus.size != 1 || !responseStatus.contains(groupMetadataTopicIdPartition)) throw new IllegalStateException("Append status %s should only have one partition %s" .format(responseStatus, groupMetadataPartition)) // construct the error status in the propagated assignment response in the cache - val status = responseStatus(groupMetadataPartition) + val status = responseStatus(groupMetadataTopicIdPartition) val responseError = if (status.error == Errors.NONE) { Errors.NONE @@ -329,9 +330,9 @@ class GroupMetadataManager(brokerId: Int, // This method should be called under the group lock to ensure atomicity of the update to the the in-memory and persisted state. private def appendForGroup( group: GroupMetadata, - records: Map[TopicPartition, MemoryRecords], + records: Map[TopicIdPartition, MemoryRecords], requestLocal: RequestLocal, - callback: Map[TopicPartition, PartitionResponse] => Unit, + callback: Map[TopicIdPartition, PartitionResponse] => Unit, verificationGuards: Map[TopicPartition, VerificationGuard] = Map.empty ): Unit = { // call replica manager to append the group message @@ -384,18 +385,19 @@ class GroupMetadataManager(brokerId: Int, responseCallback: immutable.Map[TopicIdPartition, Errors] => Unit, producerId: Long, records: Map[TopicPartition, MemoryRecords], - preAppendErrors: Map[TopicPartition, LogAppendResult] = Map.empty): Map[TopicPartition, PartitionResponse] => Unit = { + preAppendErrors: Map[TopicPartition, LogAppendResult] = Map.empty): Map[TopicIdPartition, PartitionResponse] => Unit = { val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, partitionFor(group.groupId)) + val offsetTopicIdPartition = replicaManager.getTopicIdPartition(offsetTopicPartition) // set the callback function to insert offsets into cache after log append completed - def putCacheCallback(responseStatus: Map[TopicPartition, PartitionResponse]): Unit = { + def putCacheCallback(responseStatus: Map[TopicIdPartition, PartitionResponse]): Unit = { // the append response should only contain the topics partition - if (responseStatus.size != 1 || !responseStatus.contains(offsetTopicPartition)) + if (responseStatus.size != 1 || !responseStatus.contains(offsetTopicIdPartition)) throw new IllegalStateException("Append status %s should only have one partition %s" .format(responseStatus, offsetTopicPartition)) // construct the commit response status and insert // the offset and metadata to cache if the append status has no error - val status = responseStatus(offsetTopicPartition) + val status = responseStatus(offsetTopicIdPartition) val responseError = group.inLock { if (status.error == Errors.NONE) { @@ -499,7 +501,8 @@ class GroupMetadataManager(brokerId: Int, group.prepareOffsetCommit(offsetMetadata) } - appendForGroup(group, records, requestLocal, putCacheCallback, verificationGuards) + val topicIdPartitionsToRecords = Utils.convertKeys(records.asJava, replicaManager.getTopicIdPartition).asScala + appendForGroup(group, topicIdPartitionsToRecords, requestLocal, putCacheCallback, verificationGuards) } /** diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala index d7d6c95ba8730..3e124661fc3af 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala @@ -34,7 +34,7 @@ import org.apache.kafka.common.record.{FileRecords, MemoryRecords, MemoryRecords import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.TransactionResult import org.apache.kafka.common.utils.{Time, Utils} -import org.apache.kafka.common.{KafkaException, TopicPartition} +import org.apache.kafka.common.{KafkaException, TopicIdPartition, TopicPartition} import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} import org.apache.kafka.server.config.Defaults import org.apache.kafka.server.record.BrokerCompressionType @@ -240,7 +240,7 @@ class TransactionStateManager(brokerId: Int, expiredForPartition: Iterable[TransactionalIdCoordinatorEpochAndMetadata], tombstoneRecords: MemoryRecords ): Unit = { - def removeFromCacheCallback(responses: collection.Map[TopicPartition, PartitionResponse]): Unit = { + def removeFromCacheCallback(responses: collection.Map[TopicIdPartition, PartitionResponse]): Unit = { responses.forKeyValue { (topicPartition, response) => inReadLock(stateLock) { transactionMetadataCache.get(topicPartition.partition).foreach { txnMetadataCacheEntry => @@ -275,7 +275,7 @@ class TransactionStateManager(brokerId: Int, requiredAcks = TransactionLog.EnforcedRequiredAcks, internalTopicsAllowed = true, origin = AppendOrigin.COORDINATOR, - entriesPerPartition = Map(transactionPartition -> tombstoneRecords), + entriesPerPartition = Map(replicaManager.getTopicIdPartition(transactionPartition) -> tombstoneRecords), responseCallback = removeFromCacheCallback, requestLocal = RequestLocal.NoCaching) } @@ -623,16 +623,17 @@ class TransactionStateManager(brokerId: Int, val records = MemoryRecords.withRecords(TransactionLog.EnforcedCompressionType, new SimpleRecord(timestamp, keyBytes, valueBytes)) val topicPartition = new TopicPartition(Topic.TRANSACTION_STATE_TOPIC_NAME, partitionFor(transactionalId)) - val recordsPerPartition = Map(topicPartition -> records) + val transactionStateTopicIdPartition = replicaManager.getTopicIdPartition(topicPartition) + val recordsPerPartition = Map(transactionStateTopicIdPartition -> records) // set the callback function to update transaction status in cache after log append completed - def updateCacheCallback(responseStatus: collection.Map[TopicPartition, PartitionResponse]): Unit = { + def updateCacheCallback(responseStatus: collection.Map[TopicIdPartition, PartitionResponse]): Unit = { // the append response should only contain the topics partition - if (responseStatus.size != 1 || !responseStatus.contains(topicPartition)) + if (responseStatus.size != 1 || !responseStatus.contains(transactionStateTopicIdPartition)) throw new IllegalStateException("Append status %s should only have one partition %s" .format(responseStatus, topicPartition)) - val status = responseStatus(topicPartition) + val status = responseStatus(transactionStateTopicIdPartition) var responseError = if (status.error == Errors.NONE) { Errors.NONE diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index 7a21a86260cb5..9c08c383aafe0 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -23,7 +23,7 @@ import com.typesafe.scalalogging.Logger import com.yammer.metrics.core.Meter import kafka.utils.Implicits._ import kafka.utils.Pool -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{TopicIdPartition, TopicPartition} import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.server.metrics.KafkaMetricsGroup @@ -42,7 +42,7 @@ case class ProducePartitionStatus(requiredOffset: Long, responseStatus: Partitio * The produce metadata maintained by the delayed produce operation */ case class ProduceMetadata(produceRequiredAcks: Short, - produceStatus: Map[TopicPartition, ProducePartitionStatus]) { + produceStatus: Map[TopicIdPartition, ProducePartitionStatus]) { override def toString = s"[requiredAcks: $produceRequiredAcks, partitionStatus: $produceStatus]" } @@ -58,7 +58,7 @@ object DelayedProduce { class DelayedProduce(delayMs: Long, produceMetadata: ProduceMetadata, replicaManager: ReplicaManager, - responseCallback: Map[TopicPartition, PartitionResponse] => Unit, + responseCallback: Map[TopicIdPartition, PartitionResponse] => Unit, lockOpt: Option[Lock] = None) extends DelayedOperation(delayMs, lockOpt) { @@ -94,7 +94,7 @@ class DelayedProduce(delayMs: Long, trace(s"Checking produce satisfaction for $topicPartition, current status $status") // skip those partitions that have already been satisfied if (status.acksPending) { - val (hasEnough, error) = replicaManager.getPartitionOrError(topicPartition) match { + val (hasEnough, error) = replicaManager.getPartitionOrError(topicPartition.topicPartition()) match { case Left(err) => // Case A (false, err) @@ -122,7 +122,7 @@ class DelayedProduce(delayMs: Long, produceMetadata.produceStatus.forKeyValue { (topicPartition, status) => if (status.acksPending) { debug(s"Expiring produce request for partition $topicPartition with status $status") - DelayedProduceMetrics.recordExpiration(topicPartition) + DelayedProduceMetrics.recordExpiration(topicPartition.topicPartition()) } } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 66482133bcccd..73a8fce82fcc6 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -604,40 +604,54 @@ class KafkaApis(val requestChannel: RequestChannel, } } - val unauthorizedTopicResponses = mutable.Map[TopicPartition, PartitionResponse]() - val nonExistingTopicResponses = mutable.Map[TopicPartition, PartitionResponse]() - val invalidRequestResponses = mutable.Map[TopicPartition, PartitionResponse]() - val authorizedRequestInfo = mutable.Map[TopicPartition, MemoryRecords]() + val unauthorizedTopicResponses = mutable.Map[TopicIdPartition, PartitionResponse]() + val nonExistingTopicResponses = mutable.Map[TopicIdPartition, PartitionResponse]() + val invalidRequestResponses = mutable.Map[TopicIdPartition, PartitionResponse]() + val authorizedRequestInfo = mutable.Map[TopicIdPartition, MemoryRecords]() + val partitionDatas = new mutable.ArrayBuffer[(TopicIdPartition, ProduceRequestData.PartitionProduceData)] + + produceRequest.data.topicData.forEach { topic => + topic.partitionData.forEach { partition => + val topicIdIsMissing = topic.topicId == null || topic.topicId == Uuid.ZERO_UUID + + val topicName: String = metadataCache.getTopicName(topic.topicId).getOrElse(topic.name) + val topicId: Uuid = if (topicIdIsMissing) metadataCache.getTopicId(topicName) else topic.topicId + + val topicPartition = new TopicPartition(topicName, partition.index()) + if (topicName == null || topicName.isEmpty) + nonExistingTopicResponses += new TopicIdPartition(topicId, topicPartition) -> new PartitionResponse(Errors.UNKNOWN_TOPIC_ID) + else if (!metadataCache.contains(topicPartition)) + nonExistingTopicResponses += new TopicIdPartition(topicId, topicPartition) -> new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION) + else + partitionDatas += new TopicIdPartition(topicId, topicPartition) -> partition + } + } // cache the result to avoid redundant authorization calls - val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC, - produceRequest.data().topicData().asScala)(_.name()) + val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC, partitionDatas)(_._1.topic) - produceRequest.data.topicData.forEach(topic => topic.partitionData.forEach { partition => - val topicPartition = new TopicPartition(topic.name, partition.index) + partitionDatas.foreach { case (topicIdPartition, partition) => // This caller assumes the type is MemoryRecords and that is true on current serialization // We cast the type to avoid causing big change to code base. // https://issues.apache.org/jira/browse/KAFKA-10698 val memoryRecords = partition.records.asInstanceOf[MemoryRecords] - if (!authorizedTopics.contains(topicPartition.topic)) - unauthorizedTopicResponses += topicPartition -> new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED) - else if (!metadataCache.contains(topicPartition)) - nonExistingTopicResponses += topicPartition -> new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION) + if (!authorizedTopics.contains(topicIdPartition.topic)) + unauthorizedTopicResponses += topicIdPartition -> new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED) else try { ProduceRequest.validateRecords(request.header.apiVersion, memoryRecords) - authorizedRequestInfo += (topicPartition -> memoryRecords) + authorizedRequestInfo += (topicIdPartition -> memoryRecords) } catch { case e: ApiException => - invalidRequestResponses += topicPartition -> new PartitionResponse(Errors.forException(e)) + invalidRequestResponses += topicIdPartition -> new PartitionResponse(Errors.forException(e)) } - }) + } // the callback for sending a produce response // The construction of ProduceResponse is able to accept auto-generated protocol data so // KafkaApis#handleProduceRequest should apply auto-generated protocol to avoid extra conversion. // https://issues.apache.org/jira/browse/KAFKA-10730 @nowarn("cat=deprecation") - def sendResponseCallback(responseStatus: Map[TopicPartition, PartitionResponse]): Unit = { + def sendResponseCallback(responseStatus: Map[TopicIdPartition, PartitionResponse]): Unit = { val mergedResponseStatus = responseStatus ++ unauthorizedTopicResponses ++ nonExistingTopicResponses ++ invalidRequestResponses var errorInResponse = false @@ -654,7 +668,7 @@ class KafkaApis(val requestChannel: RequestChannel, if (request.header.apiVersion >= 10) { status.error match { case Errors.NOT_LEADER_OR_FOLLOWER => - val leaderNode = getCurrentLeader(topicPartition, request.context.listenerName) + val leaderNode = getCurrentLeader(topicPartition.topicPartition(), request.context.listenerName) leaderNode.node.foreach { node => nodeEndpoints.put(node.id(), node) } @@ -730,7 +744,9 @@ class KafkaApis(val requestChannel: RequestChannel, transactionalId = produceRequest.transactionalId, entriesPerPartition = authorizedRequestInfo, responseCallback = sendResponseCallback, - recordValidationStatsCallback = processingStatsCallback, + recordValidationStatsCallback = result => processingStatsCallback(result.map { + case (partition, response) => (partition.topicPartition(), response) + }), requestLocal = requestLocal, supportedOperation = supportedOperation) @@ -2454,7 +2470,7 @@ class KafkaApis(val requestChannel: RequestChannel, } } - val controlRecords = mutable.Map.empty[TopicPartition, MemoryRecords] + val controlRecords = mutable.Map.empty[TopicIdPartition, MemoryRecords] partitionsWithCompatibleMessageFormat.foreach { partition => if (config.isNewGroupCoordinatorEnabled && partition.topic == GROUP_METADATA_TOPIC_NAME) { // When the new group coordinator is used, writing the end marker is fully delegated @@ -2485,7 +2501,7 @@ class KafkaApis(val requestChannel: RequestChannel, } else { // Otherwise, the regular appendRecords path is used for all the non __consumer_offsets // partitions or for all partitions when the new group coordinator is disabled. - controlRecords += partition -> MemoryRecords.withEndTransactionMarker( + controlRecords += replicaManager.getTopicIdPartition(partition) -> MemoryRecords.withEndTransactionMarker( producerId, marker.producerEpoch, new EndTransactionMarker(controlRecordType, marker.coordinatorEpoch) @@ -2502,7 +2518,7 @@ class KafkaApis(val requestChannel: RequestChannel, requestLocal = requestLocal, responseCallback = errors => { errors.forKeyValue { (tp, partitionResponse) => - markerResults.put(tp, partitionResponse.error) + markerResults.put(tp.topicPartition(), partitionResponse.error) } maybeComplete() } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index eba50ff304d3d..a272fd6366cdc 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -621,6 +621,11 @@ class ReplicaManager(val config: KafkaConfig, errorMap } + def getTopicIdPartition(topicPartition: TopicPartition): TopicIdPartition = { + val topicId = metadataCache.getTopicId(topicPartition.topic()) + new TopicIdPartition(topicId, topicPartition) + } + def getPartition(topicPartition: TopicPartition): HostedPartition = { Option(allPartitions.get(topicPartition)).getOrElse(HostedPartition.None) } @@ -744,10 +749,10 @@ class ReplicaManager(val config: KafkaConfig, requiredAcks: Short, internalTopicsAllowed: Boolean, origin: AppendOrigin, - entriesPerPartition: Map[TopicPartition, MemoryRecords], - responseCallback: Map[TopicPartition, PartitionResponse] => Unit, + entriesPerPartition: Map[TopicIdPartition, MemoryRecords], + responseCallback: Map[TopicIdPartition, PartitionResponse] => Unit, delayedProduceLock: Option[Lock] = None, - recordValidationStatsCallback: Map[TopicPartition, RecordValidationStats] => Unit = _ => (), + recordValidationStatsCallback: Map[TopicIdPartition, RecordValidationStats] => Unit = _ => (), requestLocal: RequestLocal = RequestLocal.NoCaching, actionQueue: ActionQueue = this.defaultActionQueue, verificationGuards: Map[TopicPartition, VerificationGuard] = Map.empty): Unit = { @@ -801,20 +806,21 @@ class ReplicaManager(val config: KafkaConfig, requiredAcks: Short, internalTopicsAllowed: Boolean, transactionalId: String, - entriesPerPartition: Map[TopicPartition, MemoryRecords], - responseCallback: Map[TopicPartition, PartitionResponse] => Unit, - recordValidationStatsCallback: Map[TopicPartition, RecordValidationStats] => Unit = _ => (), + entriesPerPartition: Map[TopicIdPartition, MemoryRecords], + responseCallback: Map[TopicIdPartition, PartitionResponse] => Unit, + recordValidationStatsCallback: Map[TopicIdPartition, RecordValidationStats] => Unit = _ => (), requestLocal: RequestLocal = RequestLocal.NoCaching, actionQueue: ActionQueue = this.defaultActionQueue, supportedOperation: SupportedOperation): Unit = { val transactionalProducerInfo = mutable.HashSet[(Long, Short)]() val topicPartitionBatchInfo = mutable.Map[TopicPartition, Int]() + val topicIds = entriesPerPartition.keys.map(tp => tp.topic() -> tp.topicId()).toMap entriesPerPartition.forKeyValue { (topicPartition, records) => // Produce requests (only requests that require verification) should only have one batch per partition in "batches" but check all just to be safe. val transactionalBatches = records.batches.asScala.filter(batch => batch.hasProducerId && batch.isTransactional) transactionalBatches.foreach(batch => transactionalProducerInfo.add(batch.producerId, batch.producerEpoch)) - if (transactionalBatches.nonEmpty) topicPartitionBatchInfo.put(topicPartition, records.firstBatch.baseSequence) + if (transactionalBatches.nonEmpty) topicPartitionBatchInfo.put(topicPartition.topicPartition(), records.firstBatch.baseSequence) } if (transactionalProducerInfo.size > 1) { throw new InvalidPidMappingException("Transactional records contained more than one producer ID") @@ -841,17 +847,16 @@ class ReplicaManager(val config: KafkaConfig, s"Unable to verify the partition has been added to the transaction. Underlying error: ${error.toString}")) case _ => None } - topicPartition -> LogAppendResult( + new TopicIdPartition(topicIds.getOrElse(topicPartition.topic(), Uuid.ZERO_UUID), topicPartition) -> LogAppendResult( LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, Some(customException.getOrElse(error.exception)), hasCustomErrorMessage = customException.isDefined ) } val entriesWithoutErrorsPerPartition = entriesPerPartition.filter { case (key, _) => !errorResults.contains(key) } - val preAppendPartitionResponses = buildProducePartitionStatus(errorResults).map { case (k, status) => k -> status.responseStatus } - def newResponseCallback(responses: Map[TopicPartition, PartitionResponse]): Unit = { + def newResponseCallback(responses: Map[TopicIdPartition, PartitionResponse]): Unit = { responseCallback(preAppendPartitionResponses ++ responses) } @@ -894,8 +899,8 @@ class ReplicaManager(val config: KafkaConfig, } private def buildProducePartitionStatus( - results: Map[TopicPartition, LogAppendResult] - ): Map[TopicPartition, ProducePartitionStatus] = { + results: Map[TopicIdPartition, LogAppendResult] + ): Map[TopicIdPartition, ProducePartitionStatus] = { results.map { case (topicPartition, result) => topicPartition -> ProducePartitionStatus( result.info.lastOffset + 1, // required offset @@ -914,7 +919,7 @@ class ReplicaManager(val config: KafkaConfig, private def addCompletePurgatoryAction( actionQueue: ActionQueue, - appendResults: Map[TopicPartition, LogAppendResult] + appendResults: Map[TopicIdPartition, LogAppendResult] ): Unit = { actionQueue.add { () => appendResults.foreach { case (topicPartition, result) => @@ -939,10 +944,10 @@ class ReplicaManager(val config: KafkaConfig, requiredAcks: Short, delayedProduceLock: Option[Lock], timeoutMs: Long, - entriesPerPartition: Map[TopicPartition, MemoryRecords], - initialAppendResults: Map[TopicPartition, LogAppendResult], - initialProduceStatus: Map[TopicPartition, ProducePartitionStatus], - responseCallback: Map[TopicPartition, PartitionResponse] => Unit, + entriesPerPartition: Map[TopicIdPartition, MemoryRecords], + initialAppendResults: Map[TopicIdPartition, LogAppendResult], + initialProduceStatus: Map[TopicIdPartition, ProducePartitionStatus], + responseCallback: Map[TopicIdPartition, PartitionResponse] => Unit, ): Unit = { if (delayedProduceRequestRequired(requiredAcks, entriesPerPartition, initialAppendResults)) { // create delayed produce operation @@ -963,8 +968,8 @@ class ReplicaManager(val config: KafkaConfig, } } - private def sendInvalidRequiredAcksResponse(entries: Map[TopicPartition, MemoryRecords], - responseCallback: Map[TopicPartition, PartitionResponse] => Unit): Unit = { + private def sendInvalidRequiredAcksResponse(entries: Map[TopicIdPartition, MemoryRecords], + responseCallback: Map[TopicIdPartition, PartitionResponse] => Unit): Unit = { // If required.acks is outside accepted range, something is wrong with the client // Just return an error and don't handle the request at all val responseStatus = entries.map { case (topicPartition, _) => @@ -1345,8 +1350,8 @@ class ReplicaManager(val config: KafkaConfig, // 2. there is data to append // 3. at least one partition append was successful (fewer errors than partitions) private def delayedProduceRequestRequired(requiredAcks: Short, - entriesPerPartition: Map[TopicPartition, MemoryRecords], - localProduceResults: Map[TopicPartition, LogAppendResult]): Boolean = { + entriesPerPartition: Map[TopicIdPartition, MemoryRecords], + localProduceResults: Map[TopicIdPartition, LogAppendResult]): Boolean = { requiredAcks == -1 && entriesPerPartition.nonEmpty && localProduceResults.values.count(_.exception.isDefined) < entriesPerPartition.size @@ -1361,10 +1366,10 @@ class ReplicaManager(val config: KafkaConfig, */ private def appendToLocalLog(internalTopicsAllowed: Boolean, origin: AppendOrigin, - entriesPerPartition: Map[TopicPartition, MemoryRecords], + entriesPerPartition: Map[TopicIdPartition, MemoryRecords], requiredAcks: Short, requestLocal: RequestLocal, - verificationGuards: Map[TopicPartition, VerificationGuard]): Map[TopicPartition, LogAppendResult] = { + verificationGuards: Map[TopicPartition, VerificationGuard]): Map[TopicIdPartition, LogAppendResult] = { val traceEnabled = isTraceEnabled def processFailedRecord(topicPartition: TopicPartition, t: Throwable) = { val logStartOffset = onlinePartition(topicPartition).map(_.logStartOffset).getOrElse(-1L) @@ -1390,9 +1395,9 @@ class ReplicaManager(val config: KafkaConfig, hasCustomErrorMessage = false)) } else { try { - val partition = getPartitionOrException(topicPartition) + val partition = getPartitionOrException(topicPartition.topicPartition()) val info = partition.appendRecordsToLeader(records, origin, requiredAcks, requestLocal, - verificationGuards.getOrElse(topicPartition, VerificationGuard.SENTINEL)) + verificationGuards.getOrElse(topicPartition.topicPartition(), VerificationGuard.SENTINEL)) val numAppendedMessages = info.numMessages // update stats for successfully appended bytes and messages as bytesInRate and messageInRate @@ -1417,12 +1422,12 @@ class ReplicaManager(val config: KafkaConfig, _: KafkaStorageException) => (topicPartition, LogAppendResult(LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, Some(e), hasCustomErrorMessage = false)) case rve: RecordValidationException => - val logStartOffset = processFailedRecord(topicPartition, rve.invalidException) + val logStartOffset = processFailedRecord(topicPartition.topicPartition(), rve.invalidException) val recordErrors = rve.recordErrors (topicPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithAdditionalInfo(logStartOffset, recordErrors), Some(rve.invalidException), hasCustomErrorMessage = true)) case t: Throwable => - val logStartOffset = processFailedRecord(topicPartition, t) + val logStartOffset = processFailedRecord(topicPartition.topicPartition(), t) (topicPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithLogStartOffset(logStartOffset), Some(t), hasCustomErrorMessage = false)) } diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 908b6084b1780..71364f689eb8c 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -92,9 +92,10 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { val requestKeyToError = (topicNames: Map[Uuid, String], version: Short) => Map[ApiKeys, Nothing => Errors]( ApiKeys.METADATA -> ((resp: requests.MetadataResponse) => resp.errors.asScala.find(_._1 == topic).getOrElse(("test", Errors.NONE))._2), ApiKeys.PRODUCE -> ((resp: requests.ProduceResponse) => { + val topicId: Uuid = topicNames.find(topicName => topicName._2 == topic).map(_._1).getOrElse(Uuid.ZERO_UUID) Errors.forCode( resp.data - .responses.find(topic) + .responses.find(topic, topicId) .partitionResponses.asScala.find(_.index == part).get .errorCode ) diff --git a/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala b/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala index ac259ce07d94b..2ab81534835cc 100644 --- a/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala +++ b/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala @@ -21,7 +21,7 @@ import kafka.cluster.BrokerEndPoint import kafka.server.QuotaFactory.QuotaManagers import kafka.server.checkpoints.LazyOffsetCheckpoints import kafka.utils.{CoreUtils, Logging, TestUtils} -import org.apache.kafka.common.{Node, TopicPartition, Uuid} +import org.apache.kafka.common.{Node, TopicIdPartition, Uuid} import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset @@ -47,7 +47,8 @@ class LocalLeaderEndPointTest extends Logging { val time = new MockTime val topicId: Uuid = Uuid.randomUuid() val topic = "test" - val topicPartition = new TopicPartition(topic, 5) + val topicIdPartition = new TopicIdPartition(topicId, 5, topic) + val topicPartition = topicIdPartition.topicPartition() val sourceBroker: BrokerEndPoint = BrokerEndPoint(0, "localhost", 9092) var replicaManager: ReplicaManager = _ var endPoint: LeaderEndPoint = _ @@ -90,25 +91,25 @@ class LocalLeaderEndPointTest extends Logging { @Test def testFetchLatestOffset(): Unit = { - appendRecords(replicaManager, topicPartition, records) + appendRecords(replicaManager, topicIdPartition, records) .onFire(response => assertEquals(Errors.NONE, response.error)) assertEquals(new OffsetAndEpoch(3L, 0), endPoint.fetchLatestOffset(topicPartition, currentLeaderEpoch = 0)) val leaderAndIsrRequest = buildLeaderAndIsrRequest(leaderEpoch = 4) replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest, (_, _) => ()) - appendRecords(replicaManager, topicPartition, records) + appendRecords(replicaManager, topicIdPartition, records) .onFire(response => assertEquals(Errors.NONE, response.error)) assertEquals(new OffsetAndEpoch(6L, 4), endPoint.fetchLatestOffset(topicPartition, currentLeaderEpoch = 7)) } @Test def testFetchEarliestOffset(): Unit = { - appendRecords(replicaManager, topicPartition, records) + appendRecords(replicaManager, topicIdPartition, records) .onFire(response => assertEquals(Errors.NONE, response.error)) assertEquals(new OffsetAndEpoch(0L, 0), endPoint.fetchEarliestOffset(topicPartition, currentLeaderEpoch = 0)) val leaderAndIsrRequest = buildLeaderAndIsrRequest(leaderEpoch = 4) replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest, (_, _) => ()) - appendRecords(replicaManager, topicPartition, records) + appendRecords(replicaManager, topicIdPartition, records) .onFire(response => assertEquals(Errors.NONE, response.error)) replicaManager.deleteRecords(timeout = 1000L, Map(topicPartition -> 3), _ => ()) assertEquals(new OffsetAndEpoch(3L, 4), endPoint.fetchEarliestOffset(topicPartition, currentLeaderEpoch = 7)) @@ -116,13 +117,13 @@ class LocalLeaderEndPointTest extends Logging { @Test def testFetchEarliestLocalOffset(): Unit = { - appendRecords(replicaManager, topicPartition, records) + appendRecords(replicaManager, topicIdPartition, records) .onFire(response => assertEquals(Errors.NONE, response.error)) assertEquals(new OffsetAndEpoch(0L, 0), endPoint.fetchEarliestLocalOffset(topicPartition, currentLeaderEpoch = 0)) val leaderAndIsrRequest = buildLeaderAndIsrRequest(leaderEpoch = 4) replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest, (_, _) => ()) - appendRecords(replicaManager, topicPartition, records) + appendRecords(replicaManager, topicIdPartition, records) .onFire(response => assertEquals(Errors.NONE, response.error)) replicaManager.logManager.getLog(topicPartition).foreach(log => log._localLogStartOffset = 3) assertEquals(new OffsetAndEpoch(0L, 0), endPoint.fetchEarliestOffset(topicPartition, currentLeaderEpoch = 7)) @@ -131,7 +132,7 @@ class LocalLeaderEndPointTest extends Logging { @Test def testFetchEpochEndOffsets(): Unit = { - appendRecords(replicaManager, topicPartition, records) + appendRecords(replicaManager, topicIdPartition, records) .onFire(response => assertEquals(Errors.NONE, response.error)) var result = endPoint.fetchEpochEndOffsets(Map( @@ -151,7 +152,7 @@ class LocalLeaderEndPointTest extends Logging { // Change leader epoch and end offset, and verify the behavior again. val leaderAndIsrRequest = buildLeaderAndIsrRequest(leaderEpoch = 4) replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest, (_, _) => ()) - appendRecords(replicaManager, topicPartition, records) + appendRecords(replicaManager, topicIdPartition, records) .onFire(response => assertEquals(Errors.NONE, response.error)) result = endPoint.fetchEpochEndOffsets(Map( @@ -238,12 +239,12 @@ class LocalLeaderEndPointTest extends Logging { } private def appendRecords(replicaManager: ReplicaManager, - partition: TopicPartition, + partition: TopicIdPartition, records: MemoryRecords, origin: AppendOrigin = AppendOrigin.CLIENT, requiredAcks: Short = -1): CallbackResult[PartitionResponse] = { val result = new CallbackResult[PartitionResponse]() - def appendCallback(responses: Map[TopicPartition, PartitionResponse]): Unit = { + def appendCallback(responses: Map[TopicIdPartition, PartitionResponse]): Unit = { val response = responses.get(partition) assertTrue(response.isDefined) result.fire(response.get) @@ -254,7 +255,7 @@ class LocalLeaderEndPointTest extends Logging { requiredAcks = requiredAcks, internalTopicsAllowed = false, origin = origin, - entriesPerPartition = Map(partition -> records), + entriesPerPartition = Map(topicIdPartition-> records), responseCallback = appendCallback) result diff --git a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala index 956db24c010ec..bf7acdbb5d3db 100644 --- a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala @@ -27,7 +27,7 @@ import kafka.server.QuotaFactory.QuotaManagers import kafka.server.{DelayedOperationPurgatory, KafkaConfig, _} import kafka.utils._ import kafka.zk.KafkaZkClient -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{TopicIdPartition, TopicPartition} import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordValidationStats} import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse @@ -182,7 +182,7 @@ object AbstractCoordinatorConcurrencyTest { logManager, None, quotaManagers, - null, + mock(classOf[MetadataCache]), null, null, delayedProducePurgatoryParam = Some(producePurgatory), @@ -213,10 +213,10 @@ object AbstractCoordinatorConcurrencyTest { requiredAcks: Short, internalTopicsAllowed: Boolean, origin: AppendOrigin, - entriesPerPartition: Map[TopicPartition, MemoryRecords], - responseCallback: Map[TopicPartition, PartitionResponse] => Unit, + entriesPerPartition: Map[TopicIdPartition, MemoryRecords], + responseCallback: Map[TopicIdPartition, PartitionResponse] => Unit, delayedProduceLock: Option[Lock] = None, - processingStatsCallback: Map[TopicPartition, RecordValidationStats] => Unit = _ => (), + processingStatsCallback: Map[TopicIdPartition, RecordValidationStats] => Unit = _ => (), requestLocal: RequestLocal = RequestLocal.NoCaching, actionQueue: ActionQueue = null, verificationGuards: Map[TopicPartition, VerificationGuard] = Map.empty): Unit = { diff --git a/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala index 7a3c5f194f9a5..4adb64f2e21da 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala @@ -18,7 +18,7 @@ package kafka.coordinator.group import kafka.server.ReplicaManager import kafka.utils.TestUtils -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.{NotLeaderOrFollowerException, RecordTooLargeException} import org.apache.kafka.common.protocol.{ApiKeys, Errors} @@ -99,11 +99,13 @@ class CoordinatorPartitionWriterTest { Collections.emptyMap(), new Properties() ))) + val topicId = Uuid.randomUuid() + when(replicaManager.getTopicIdPartition(tp)).thenReturn(new TopicIdPartition(topicId, tp)) - val recordsCapture: ArgumentCaptor[Map[TopicPartition, MemoryRecords]] = - ArgumentCaptor.forClass(classOf[Map[TopicPartition, MemoryRecords]]) - val callbackCapture: ArgumentCaptor[Map[TopicPartition, PartitionResponse] => Unit] = - ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) + val recordsCapture: ArgumentCaptor[Map[TopicIdPartition, MemoryRecords]] = + ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, MemoryRecords]]) + val callbackCapture: ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = + ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, PartitionResponse] => Unit]) when(replicaManager.appendRecords( ArgumentMatchers.eq(0L), @@ -119,7 +121,7 @@ class CoordinatorPartitionWriterTest { ArgumentMatchers.eq(Map(tp -> VerificationGuard.SENTINEL)), )).thenAnswer( _ => { callbackCapture.getValue.apply(Map( - tp -> new PartitionResponse( + new TopicIdPartition(topicId, tp) -> new PartitionResponse( Errors.NONE, 5, 10, @@ -145,8 +147,9 @@ class CoordinatorPartitionWriterTest { records.asJava )) - val batch = recordsCapture.getValue.getOrElse(tp, - throw new AssertionError(s"No records for $tp")) + val batch = recordsCapture.getValue.find(_._1.topicPartition() == tp) + .getOrElse(throw new AssertionError(s"No records for $tp")) + ._2 assertEquals(1, batch.batches().asScala.toList.size) val receivedRecords = batch.records.asScala.map { record => @@ -176,11 +179,13 @@ class CoordinatorPartitionWriterTest { Collections.emptyMap(), new Properties() ))) + val topicId = Uuid.randomUuid() + when(replicaManager.getTopicIdPartition(tp)).thenReturn(new TopicIdPartition(topicId, tp)) - val recordsCapture: ArgumentCaptor[Map[TopicPartition, MemoryRecords]] = - ArgumentCaptor.forClass(classOf[Map[TopicPartition, MemoryRecords]]) - val callbackCapture: ArgumentCaptor[Map[TopicPartition, PartitionResponse] => Unit] = - ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) + val recordsCapture: ArgumentCaptor[Map[TopicIdPartition, MemoryRecords]] = + ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, MemoryRecords]]) + val callbackCapture: ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = + ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, PartitionResponse] => Unit]) when(replicaManager.appendRecords( ArgumentMatchers.eq(0L), @@ -196,7 +201,7 @@ class CoordinatorPartitionWriterTest { ArgumentMatchers.eq(Map(tp -> verificationGuard)), )).thenAnswer(_ => { callbackCapture.getValue.apply(Map( - tp -> new PartitionResponse( + new TopicIdPartition(topicId, tp) -> new PartitionResponse( Errors.NONE, 5, 10, @@ -222,8 +227,8 @@ class CoordinatorPartitionWriterTest { records.asJava )) - val batch = recordsCapture.getValue.getOrElse(tp, - throw new AssertionError(s"No records for $tp")) + val batch = recordsCapture.getValue.find(_._1.topicPartition() == tp) + .getOrElse(throw new AssertionError(s"No records for $tp"))._2 assertEquals(1, batch.batches().asScala.toList.size) val firstBatch = batch.batches.asScala.head @@ -258,11 +263,13 @@ class CoordinatorPartitionWriterTest { Collections.emptyMap(), new Properties() ))) + val topicId = Uuid.randomUuid() + when(replicaManager.getTopicIdPartition(tp)).thenReturn(new TopicIdPartition(topicId, tp)) - val recordsCapture: ArgumentCaptor[Map[TopicPartition, MemoryRecords]] = - ArgumentCaptor.forClass(classOf[Map[TopicPartition, MemoryRecords]]) - val callbackCapture: ArgumentCaptor[Map[TopicPartition, PartitionResponse] => Unit] = - ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) + val recordsCapture: ArgumentCaptor[Map[TopicIdPartition, MemoryRecords]] = + ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, MemoryRecords]]) + val callbackCapture: ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = + ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, PartitionResponse] => Unit]) when(replicaManager.appendRecords( ArgumentMatchers.eq(0L), @@ -278,7 +285,7 @@ class CoordinatorPartitionWriterTest { ArgumentMatchers.eq(Map(tp -> VerificationGuard.SENTINEL)), )).thenAnswer(_ => { callbackCapture.getValue.apply(Map( - tp -> new PartitionResponse( + new TopicIdPartition(topicId, tp) -> new PartitionResponse( Errors.NONE, 5, 10, @@ -298,8 +305,8 @@ class CoordinatorPartitionWriterTest { if (controlRecordType == ControlRecordType.COMMIT) TransactionResult.COMMIT else TransactionResult.ABORT )) - val batch = recordsCapture.getValue.getOrElse(tp, - throw new AssertionError(s"No records for $tp")) + val batch = recordsCapture.getValue.find(_._1.topicPartition() == tp) + .getOrElse(throw new AssertionError(s"No records for $tp"))._2 assertEquals(1, batch.batches.asScala.toList.size) val firstBatch = batch.batches.asScala.head @@ -383,11 +390,13 @@ class CoordinatorPartitionWriterTest { Collections.emptyMap(), new Properties() ))) + val topicId = Uuid.randomUuid() + when(replicaManager.getTopicIdPartition(tp)).thenReturn(new TopicIdPartition(topicId, tp)) - val recordsCapture: ArgumentCaptor[Map[TopicPartition, MemoryRecords]] = - ArgumentCaptor.forClass(classOf[Map[TopicPartition, MemoryRecords]]) - val callbackCapture: ArgumentCaptor[Map[TopicPartition, PartitionResponse] => Unit] = - ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) + val recordsCapture: ArgumentCaptor[Map[TopicIdPartition, MemoryRecords]] = + ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, MemoryRecords]]) + val callbackCapture: ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = + ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, PartitionResponse] => Unit]) when(replicaManager.appendRecords( ArgumentMatchers.eq(0L), @@ -403,7 +412,7 @@ class CoordinatorPartitionWriterTest { ArgumentMatchers.eq(Map(tp -> VerificationGuard.SENTINEL)), )).thenAnswer(_ => { callbackCapture.getValue.apply(Map( - tp -> new PartitionResponse(Errors.NOT_LEADER_OR_FOLLOWER) + new TopicIdPartition(topicId, tp) -> new PartitionResponse(Errors.NOT_LEADER_OR_FOLLOWER) )) }) diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorConcurrencyTest.scala index 924b8393069a8..7aa1c983dfa92 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorConcurrencyTest.scala @@ -90,6 +90,10 @@ class GroupCoordinatorConcurrencyTest extends AbstractCoordinatorConcurrencyTest // Transactional appends attempt to schedule to the request handler thread using // a non request handler thread. Set this to avoid error. KafkaRequestHandler.setBypassThreadCheck(true) + + val groupMetadataTopicId = Uuid.randomUuid() + when(replicaManager.metadataCache.getTopicName(groupMetadataTopicId)).thenReturn(Some(Topic.GROUP_METADATA_TOPIC_NAME)) + when(replicaManager.metadataCache.getTopicId(Topic.GROUP_METADATA_TOPIC_NAME)).thenReturn(groupMetadataTopicId) } @AfterEach diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala index 58c6523db6ad6..06096de04ce8d 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala @@ -92,6 +92,7 @@ class GroupCoordinatorTest { private val protocolSuperset = List((protocolName, metadata), ("roundrobin", metadata)) private val requireStable = true private var groupPartitionId: Int = -1 + val groupMetadataTopicId = Uuid.randomUuid() // we use this string value since its hashcode % #.partitions is different private val otherGroupId = "otherGroup" @@ -126,6 +127,8 @@ class GroupCoordinatorTest { // add the partition into the owned partition list groupPartitionId = groupCoordinator.partitionFor(groupId) + val groupPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId) + when(replicaManager.getTopicIdPartition(groupPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, groupPartition)) groupCoordinator.groupManager.addOwnedPartition(groupPartitionId) } @@ -2778,8 +2781,12 @@ class GroupCoordinatorTest { val producerEpoch: Short = 3 val groupIds = List(groupId, otherGroupId) - val offsetTopicPartitions = List(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupCoordinator.partitionFor(groupId)), - new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupCoordinator.partitionFor(otherGroupId))) + val otherGroupMetadataPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupCoordinator.partitionFor(otherGroupId)) + val groupMetadataPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupCoordinator.partitionFor(groupId)) + val offsetTopicPartitions = List(groupMetadataPartition, otherGroupMetadataPartition) + + when(replicaManager.getTopicIdPartition(groupMetadataPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, groupMetadataPartition)) + when(replicaManager.getTopicIdPartition(otherGroupMetadataPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, otherGroupMetadataPartition)) groupCoordinator.groupManager.addOwnedPartition(offsetTopicPartitions(1).partition) val errors = mutable.ArrayBuffer[Errors]() @@ -3936,13 +3943,13 @@ class GroupCoordinatorTest { supportSkippingAssignment: Boolean): Future[JoinGroupResult] = { val (responseFuture, responseCallback) = setupJoinGroupCallback - val capturedArgument: ArgumentCaptor[scala.collection.Map[TopicPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[scala.collection.Map[TopicPartition, PartitionResponse] => Unit]) + val capturedArgument: ArgumentCaptor[scala.collection.Map[TopicIdPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[scala.collection.Map[TopicIdPartition, PartitionResponse] => Unit]) when(replicaManager.appendRecords(anyLong, anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), - any[Map[TopicPartition, MemoryRecords]], + any[Map[TopicIdPartition, MemoryRecords]], capturedArgument.capture(), any[Option[ReentrantLock]], any(), @@ -3951,7 +3958,7 @@ class GroupCoordinatorTest { any[Map[TopicPartition, VerificationGuard]] )).thenAnswer(_ => { capturedArgument.getValue.apply( - Map(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId) -> + Map(new TopicIdPartition(groupMetadataTopicId, groupPartitionId, Topic.GROUP_METADATA_TOPIC_NAME) -> new PartitionResponse(appendRecordError, 0L, RecordBatch.NO_TIMESTAMP, 0L) ) ) @@ -3972,13 +3979,13 @@ class GroupCoordinatorTest { assignment: Map[String, Array[Byte]]): Future[SyncGroupResult] = { val (responseFuture, responseCallback) = setupSyncGroupCallback - val capturedArgument: ArgumentCaptor[scala.collection.Map[TopicPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[scala.collection.Map[TopicPartition, PartitionResponse] => Unit]) + val capturedArgument: ArgumentCaptor[scala.collection.Map[TopicIdPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[scala.collection.Map[TopicIdPartition, PartitionResponse] => Unit]) when(replicaManager.appendRecords(anyLong, anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), - any[Map[TopicPartition, MemoryRecords]], + any[Map[TopicIdPartition, MemoryRecords]], capturedArgument.capture(), any[Option[ReentrantLock]], any(), @@ -3987,7 +3994,7 @@ class GroupCoordinatorTest { any[Map[TopicPartition, VerificationGuard]] )).thenAnswer(_ => { capturedArgument.getValue.apply( - Map(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId) -> + Map(new TopicIdPartition(groupMetadataTopicId, groupPartitionId, Topic.GROUP_METADATA_TOPIC_NAME) -> new PartitionResponse(Errors.NONE, 0L, RecordBatch.NO_TIMESTAMP, 0L) ) ) @@ -4119,13 +4126,13 @@ class GroupCoordinatorTest { groupInstanceId: Option[String] = None): CommitOffsetCallbackParams = { val (responseFuture, responseCallback) = setupCommitOffsetsCallback - val capturedArgument: ArgumentCaptor[scala.collection.Map[TopicPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[scala.collection.Map[TopicPartition, PartitionResponse] => Unit]) + val capturedArgument: ArgumentCaptor[scala.collection.Map[TopicIdPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[scala.collection.Map[TopicIdPartition, PartitionResponse] => Unit]) when(replicaManager.appendRecords(anyLong, anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), - any[Map[TopicPartition, MemoryRecords]], + any[Map[TopicIdPartition, MemoryRecords]], capturedArgument.capture(), any[Option[ReentrantLock]], any(), @@ -4134,7 +4141,7 @@ class GroupCoordinatorTest { any[Map[TopicPartition, VerificationGuard]] )).thenAnswer(_ => { capturedArgument.getValue.apply( - Map(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId) -> + Map(new TopicIdPartition(groupMetadataTopicId, groupPartitionId, Topic.GROUP_METADATA_TOPIC_NAME) -> new PartitionResponse(Errors.NONE, 0L, RecordBatch.NO_TIMESTAMP, 0L) ) ) @@ -4155,7 +4162,7 @@ class GroupCoordinatorTest { verificationError: Errors = Errors.NONE): CommitOffsetCallbackParams = { val (responseFuture, responseCallback) = setupCommitOffsetsCallback - val capturedArgument: ArgumentCaptor[scala.collection.Map[TopicPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[scala.collection.Map[TopicPartition, PartitionResponse] => Unit]) + val capturedArgument: ArgumentCaptor[scala.collection.Map[TopicIdPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[scala.collection.Map[TopicIdPartition, PartitionResponse] => Unit]) // Since transactional ID is only used for verification, we can use a dummy value. Ensure it passes through. val transactionalId = "dummy-txn-id" @@ -4183,7 +4190,7 @@ class GroupCoordinatorTest { anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), - any[Map[TopicPartition, MemoryRecords]], + any[Map[TopicIdPartition, MemoryRecords]], capturedArgument.capture(), any[Option[ReentrantLock]], any(), @@ -4192,7 +4199,7 @@ class GroupCoordinatorTest { any[Map[TopicPartition, VerificationGuard]] )).thenAnswer(_ => { capturedArgument.getValue.apply( - Map(offsetTopicPartition -> + Map(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition) -> new PartitionResponse(Errors.NONE, 0L, RecordBatch.NO_TIMESTAMP, 0L) ) ) diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala index 50e3c27069d41..530d405a4e710 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala @@ -92,6 +92,8 @@ class GroupMetadataManagerTest { config.offsetCommitRequiredAcks) } + val groupMetadataTopicId = Uuid.randomUuid() + @BeforeEach def setUp(): Unit = { defaultOffsetRetentionMs = offsetConfig.offsetsRetentionMs @@ -1112,9 +1114,12 @@ class GroupMetadataManagerTest { maybeError = Some(error) } + val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) + when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + groupMetadataManager.storeGroup(group, Map.empty, callback) assertEquals(Some(Errors.NONE), maybeError) - val records = capturedRecords.getValue()(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId)) + val records = capturedRecords.getValue()(new TopicIdPartition(groupMetadataTopicId, groupPartitionId, Topic.GROUP_METADATA_TOPIC_NAME)) .records.asScala.toList assertEquals(1, records.size) @@ -1136,9 +1141,12 @@ class GroupMetadataManagerTest { maybeError = Some(error) } + val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) + when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + groupMetadataManager.storeGroup(group, Map.empty, callback) assertEquals(Some(Errors.NONE), maybeError) - val records = capturedRecords.getValue()(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId)) + val records = capturedRecords.getValue()(new TopicIdPartition(groupMetadataTopicId, groupPartitionId, Topic.GROUP_METADATA_TOPIC_NAME)) .records.asScala.toList assertEquals(1, records.size) @@ -1167,6 +1175,9 @@ class GroupMetadataManagerTest { val group = new GroupMetadata(groupId, Empty, time) groupMetadataManager.addGroup(group) + val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) + when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + expectAppendMessage(appendError) var maybeError: Option[Errors] = None def callback(error: Errors): Unit = { @@ -1210,6 +1221,8 @@ class GroupMetadataManagerTest { def callback(error: Errors): Unit = { maybeError = Some(error) } + val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) + when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) groupMetadataManager.storeGroup(group, Map(memberId -> Array[Byte]()), callback) assertEquals(Some(Errors.NONE), maybeError) @@ -1266,6 +1279,8 @@ class GroupMetadataManagerTest { groupMetadataManager.addGroup(group) val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) + when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + val offsets = immutable.Map(topicIdPartition -> OffsetAndMetadata(offset, "", time.milliseconds())) expectAppendMessage(Errors.NONE) @@ -1323,12 +1338,14 @@ class GroupMetadataManagerTest { val offsetAndMetadata = OffsetAndMetadata(offset, "", time.milliseconds()) val offsets = immutable.Map(topicIdPartition -> offsetAndMetadata) - val capturedResponseCallback: ArgumentCaptor[Map[TopicPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) + val capturedResponseCallback: ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, PartitionResponse] => Unit]) when(replicaManager.getMagic(any())).thenReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE)) var commitErrors: Option[immutable.Map[TopicIdPartition, Errors]] = None def callback(errors: immutable.Map[TopicIdPartition, Errors]): Unit = { commitErrors = Some(errors) } + when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + val verificationGuard = new VerificationGuard() groupMetadataManager.storeOffsets(group, memberId, offsetTopicPartition, offsets, callback, producerId, producerEpoch, verificationGuard = Some(verificationGuard)) @@ -1339,7 +1356,7 @@ class GroupMetadataManagerTest { anyShort(), any(), any(), - any[Map[TopicPartition, MemoryRecords]], + any[Map[TopicIdPartition, MemoryRecords]], capturedResponseCallback.capture(), any[Option[ReentrantLock]], any(), @@ -1347,7 +1364,7 @@ class GroupMetadataManagerTest { any(), ArgumentMatchers.eq(Map(offsetTopicPartition -> verificationGuard))) verify(replicaManager).getMagic(any()) - capturedResponseCallback.getValue.apply(Map(groupTopicPartition -> + capturedResponseCallback.getValue.apply(Map(new TopicIdPartition(groupMetadataTopicId, groupTopicPartition) -> new PartitionResponse(Errors.NONE, 0L, RecordBatch.NO_TIMESTAMP, 0L))) assertTrue(group.hasOffsets) @@ -1374,7 +1391,7 @@ class GroupMetadataManagerTest { val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) val offsets = immutable.Map(topicIdPartition -> OffsetAndMetadata(offset, "", time.milliseconds())) - + when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) when(replicaManager.getMagic(any())).thenReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE)) var commitErrors: Option[immutable.Map[TopicIdPartition, Errors]] = None @@ -1387,7 +1404,7 @@ class GroupMetadataManagerTest { assertTrue(group.hasOffsets) assertTrue(group.allOffsets.isEmpty) val capturedResponseCallback = verifyAppendAndCaptureCallback() - capturedResponseCallback.getValue.apply(Map(groupTopicPartition -> + capturedResponseCallback.getValue.apply(Map(new TopicIdPartition(groupMetadataTopicId, groupTopicPartition) -> new PartitionResponse(Errors.NOT_ENOUGH_REPLICAS, 0L, RecordBatch.NO_TIMESTAMP, 0L))) assertFalse(group.hasOffsets) @@ -1401,7 +1418,7 @@ class GroupMetadataManagerTest { anyShort(), any(), any(), - any[Map[TopicPartition, MemoryRecords]], + any[Map[TopicIdPartition, MemoryRecords]], any(), any[Option[ReentrantLock]], any(), @@ -1427,6 +1444,7 @@ class GroupMetadataManagerTest { val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) val offsets = immutable.Map(topicIdPartition -> OffsetAndMetadata(offset, "", time.milliseconds())) + when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) when(replicaManager.getMagic(any())).thenReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE)) var commitErrors: Option[immutable.Map[TopicIdPartition, Errors]] = None @@ -1439,7 +1457,7 @@ class GroupMetadataManagerTest { assertTrue(group.hasOffsets) assertTrue(group.allOffsets.isEmpty) val capturedResponseCallback = verifyAppendAndCaptureCallback() - capturedResponseCallback.getValue.apply(Map(groupTopicPartition -> + capturedResponseCallback.getValue.apply(Map(new TopicIdPartition(groupMetadataTopicId, groupTopicPartition) -> new PartitionResponse(Errors.NONE, 0L, RecordBatch.NO_TIMESTAMP, 0L))) assertTrue(group.hasOffsets) @@ -1453,7 +1471,7 @@ class GroupMetadataManagerTest { anyShort(), any(), any(), - any[Map[TopicPartition, MemoryRecords]], + any[Map[TopicIdPartition, MemoryRecords]], any(), any[Option[ReentrantLock]], any(), @@ -1514,10 +1532,10 @@ class GroupMetadataManagerTest { val group = new GroupMetadata(groupId, Empty, time) groupMetadataManager.addGroup(group) - val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) val offsets = immutable.Map(topicIdPartition -> OffsetAndMetadata(offset, "", time.milliseconds())) + when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) when(replicaManager.getMagic(any())).thenReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE)) var commitErrors: Option[immutable.Map[TopicIdPartition, Errors]] = None @@ -1529,7 +1547,7 @@ class GroupMetadataManagerTest { groupMetadataManager.storeOffsets(group, memberId, offsetTopicPartition, offsets, callback, verificationGuard = None) assertTrue(group.hasOffsets) val capturedResponseCallback = verifyAppendAndCaptureCallback() - capturedResponseCallback.getValue.apply(Map(groupTopicPartition -> + capturedResponseCallback.getValue.apply(Map(new TopicIdPartition(groupMetadataTopicId, groupTopicPartition) -> new PartitionResponse(appendError, 0L, RecordBatch.NO_TIMESTAMP, 0L))) assertFalse(commitErrors.isEmpty) @@ -1563,8 +1581,9 @@ class GroupMetadataManagerTest { val group = new GroupMetadata(groupId, Empty, time) groupMetadataManager.addGroup(group) - val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) + when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + val offsets = immutable.Map( topicIdPartition -> OffsetAndMetadata(offset, "", time.milliseconds()), // This will failed @@ -1582,7 +1601,7 @@ class GroupMetadataManagerTest { groupMetadataManager.storeOffsets(group, memberId, offsetTopicPartition, offsets, callback, verificationGuard = None) assertTrue(group.hasOffsets) val capturedResponseCallback = verifyAppendAndCaptureCallback() - capturedResponseCallback.getValue.apply(Map(groupTopicPartition -> + capturedResponseCallback.getValue.apply(Map(new TopicIdPartition(groupMetadataTopicId, groupTopicPartition) -> new PartitionResponse(Errors.NONE, 0L, RecordBatch.NO_TIMESTAMP, 0L))) assertFalse(commitErrors.isEmpty) @@ -1608,7 +1627,7 @@ class GroupMetadataManagerTest { anyShort(), any(), any(), - any[Map[TopicPartition, MemoryRecords]], + any[Map[TopicIdPartition, MemoryRecords]], any(), any[Option[ReentrantLock]], any(), @@ -1674,6 +1693,7 @@ class GroupMetadataManagerTest { groupMetadataManager.addGroup(group) val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) + when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) // expire the offset after 1 millisecond val startMs = time.milliseconds val offsets = immutable.Map( @@ -1830,6 +1850,7 @@ class GroupMetadataManagerTest { groupMetadataManager.addGroup(group) val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) + when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) // expire the offset after 1 millisecond val startMs = time.milliseconds val offsets = immutable.Map( @@ -1914,6 +1935,7 @@ class GroupMetadataManagerTest { group.initNextGeneration() val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) + when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) val startMs = time.milliseconds // old clients, expiry timestamp is explicitly set val tp1OffsetAndMetadata = OffsetAndMetadata(offset, "", startMs, startMs + 1) @@ -2093,6 +2115,7 @@ class GroupMetadataManagerTest { groupMetadataManager.addGroup(group) val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) + when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) // expire the offset after 1 and 3 milliseconds (old clients) and after default retention (new clients) val startMs = time.milliseconds // old clients, expiry timestamp is explicitly set @@ -2202,6 +2225,8 @@ class GroupMetadataManagerTest { group.transitionTo(Stable) val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) + when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + val startMs = time.milliseconds val t1p0OffsetAndMetadata = OffsetAndMetadata(offset, "", startMs) @@ -2822,13 +2847,13 @@ class GroupMetadataManagerTest { assertEquals(Some(""), valueStringOpt) } - private def verifyAppendAndCaptureCallback(): ArgumentCaptor[Map[TopicPartition, PartitionResponse] => Unit] = { - val capturedArgument: ArgumentCaptor[Map[TopicPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) + private def verifyAppendAndCaptureCallback(): ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = { + val capturedArgument: ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, PartitionResponse] => Unit]) verify(replicaManager).appendRecords(anyLong(), anyShort(), any(), any(), - any[Map[TopicPartition, MemoryRecords]], + any[Map[TopicIdPartition, MemoryRecords]], capturedArgument.capture(), any[Option[ReentrantLock]], any(), @@ -2838,9 +2863,9 @@ class GroupMetadataManagerTest { capturedArgument } - private def expectAppendMessage(error: Errors): ArgumentCaptor[Map[TopicPartition, MemoryRecords]] = { - val capturedCallback: ArgumentCaptor[Map[TopicPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) - val capturedRecords: ArgumentCaptor[Map[TopicPartition, MemoryRecords]] = ArgumentCaptor.forClass(classOf[Map[TopicPartition, MemoryRecords]]) + private def expectAppendMessage(error: Errors): ArgumentCaptor[Map[TopicIdPartition, MemoryRecords]] = { + val capturedCallback: ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, PartitionResponse] => Unit]) + val capturedRecords: ArgumentCaptor[Map[TopicIdPartition, MemoryRecords]] = ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, MemoryRecords]]) when(replicaManager.appendRecords(anyLong(), anyShort(), any(), @@ -2854,7 +2879,7 @@ class GroupMetadataManagerTest { any() )).thenAnswer(_ => { capturedCallback.getValue.apply( - Map(groupTopicPartition -> + Map(new TopicIdPartition(groupMetadataTopicId, groupTopicPartition) -> new PartitionResponse(error, 0L, RecordBatch.NO_TIMESTAMP, 0L) ) )}) diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala index d57a8e974c623..dad00b815c3e9 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala @@ -26,6 +26,7 @@ import kafka.log.UnifiedLog import kafka.server.{KafkaConfig, MetadataCache, RequestLocal} import kafka.utils.{Pool, TestUtils} import org.apache.kafka.clients.{ClientResponse, NetworkClient} +import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.internals.Topic.TRANSACTION_STATE_TOPIC_NAME import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.ListenerName @@ -33,7 +34,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.{CompressionType, FileRecords, MemoryRecords, RecordBatch, SimpleRecord} import org.apache.kafka.common.requests._ import org.apache.kafka.common.utils.{LogContext, MockTime, ProducerIdAndEpoch} -import org.apache.kafka.common.{Node, TopicPartition} +import org.apache.kafka.common.{Node, TopicPartition, Uuid} import org.apache.kafka.storage.internals.log.{FetchDataInfo, FetchIsolation, LogConfig, LogOffsetMetadata} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} @@ -103,6 +104,10 @@ class TransactionCoordinatorConcurrencyTest extends AbstractCoordinatorConcurren txnStateManager, time) + val transactionStateTopicId = Uuid.randomUuid() + when(replicaManager.metadataCache.getTopicName(transactionStateTopicId)).thenReturn(Some(Topic.TRANSACTION_STATE_TOPIC_NAME)) + when(replicaManager.metadataCache.getTopicId(Topic.TRANSACTION_STATE_TOPIC_NAME)).thenReturn(transactionStateTopicId) + transactionCoordinator = new TransactionCoordinator( txnConfig, scheduler, diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala index 907e4c9c17e15..69f80c54e3dcd 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala @@ -27,7 +27,7 @@ import kafka.log.UnifiedLog import kafka.server.{ReplicaManager, RequestLocal} import kafka.utils.{Pool, TestUtils} import kafka.zk.KafkaZkClient -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.internals.Topic.TRANSACTION_STATE_TOPIC_NAME import org.apache.kafka.common.metrics.{JmxReporter, KafkaMetricsContext, Metrics} import org.apache.kafka.common.protocol.{Errors, MessageUtil} @@ -52,6 +52,7 @@ class TransactionStateManagerTest { val numPartitions = 2 val transactionTimeoutMs: Int = 1000 val topicPartition = new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, partitionId) + val transactionTopicId = Uuid.randomUuid() val coordinatorEpoch = 10 val txnRecords: mutable.ArrayBuffer[SimpleRecord] = mutable.ArrayBuffer[SimpleRecord]() @@ -86,6 +87,8 @@ class TransactionStateManagerTest { // make sure the transactional id hashes to the assigning partition id assertEquals(partitionId, transactionManager.partitionFor(transactionalId1)) assertEquals(partitionId, transactionManager.partitionFor(transactionalId2)) + when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 0))).thenReturn(new TopicIdPartition(transactionTopicId, 0, TRANSACTION_STATE_TOPIC_NAME)) + when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 1))).thenReturn(new TopicIdPartition(transactionTopicId, 1, TRANSACTION_STATE_TOPIC_NAME)) } @AfterEach @@ -651,10 +654,12 @@ class TransactionStateManagerTest { reset(replicaManager) expectLogConfig(partitionIds, maxBatchSize) - val attemptedAppends = mutable.Map.empty[TopicPartition, mutable.Buffer[MemoryRecords]] + val attemptedAppends = mutable.Map.empty[TopicIdPartition, mutable.Buffer[MemoryRecords]] expectTransactionalIdExpiration(Errors.MESSAGE_TOO_LARGE, attemptedAppends) assertEquals(allTransactionalIds, listExpirableTransactionalIds()) + when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 0))).thenReturn(new TopicIdPartition(transactionTopicId, 0, TRANSACTION_STATE_TOPIC_NAME)) + when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 1))).thenReturn(new TopicIdPartition(transactionTopicId, 1, TRANSACTION_STATE_TOPIC_NAME)) transactionManager.removeExpiredTransactionalIds() verify(replicaManager, atLeastOnce()).appendRecords( anyLong(), @@ -695,8 +700,9 @@ class TransactionStateManagerTest { // No log config returned for partition 0 since it is offline when(replicaManager.getLogConfig(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, offlinePartitionId))) .thenReturn(None) - - val appendedRecords = mutable.Map.empty[TopicPartition, mutable.Buffer[MemoryRecords]] + when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 0))).thenReturn(new TopicIdPartition(transactionTopicId, 0, TRANSACTION_STATE_TOPIC_NAME)) + when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 1))).thenReturn(new TopicIdPartition(transactionTopicId, 1, TRANSACTION_STATE_TOPIC_NAME)) + val appendedRecords = mutable.Map.empty[TopicIdPartition, mutable.Buffer[MemoryRecords]] expectTransactionalIdExpiration(Errors.NONE, appendedRecords) assertEquals(allTransactionalIds, listExpirableTransactionalIds()) @@ -738,10 +744,13 @@ class TransactionStateManagerTest { reset(replicaManager) expectLogConfig(partitionIds, maxBatchSize) - val appendedRecords = mutable.Map.empty[TopicPartition, mutable.Buffer[MemoryRecords]] + val appendedRecords = mutable.Map.empty[TopicIdPartition, mutable.Buffer[MemoryRecords]] expectTransactionalIdExpiration(Errors.NONE, appendedRecords) assertEquals(allTransactionalIds, listExpirableTransactionalIds()) + when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 0))).thenReturn(new TopicIdPartition(transactionTopicId, 0, TRANSACTION_STATE_TOPIC_NAME)) + when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 1))).thenReturn(new TopicIdPartition(transactionTopicId, 1, TRANSACTION_STATE_TOPIC_NAME)) + transactionManager.removeExpiredTransactionalIds() verify(replicaManager, atLeastOnce()).appendRecords( anyLong(), @@ -795,9 +804,11 @@ class TransactionStateManagerTest { time.sleep(txnConfig.transactionalIdExpirationMs + 1) reset(replicaManager) + when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 0))).thenReturn(new TopicIdPartition(transactionTopicId, 0, TRANSACTION_STATE_TOPIC_NAME)) + when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 1))).thenReturn(new TopicIdPartition(transactionTopicId, 1, TRANSACTION_STATE_TOPIC_NAME)) expectLogConfig(partitionIds, maxBatchSize) - val appendedRecords = mutable.Map.empty[TopicPartition, mutable.Buffer[MemoryRecords]] + val appendedRecords = mutable.Map.empty[TopicIdPartition, mutable.Buffer[MemoryRecords]] expectTransactionalIdExpiration(Errors.NONE, appendedRecords) transactionManager.removeExpiredTransactionalIds() @@ -820,7 +831,7 @@ class TransactionStateManagerTest { } private def collectTransactionalIdsFromTombstones( - appendedRecords: mutable.Map[TopicPartition, mutable.Buffer[MemoryRecords]] + appendedRecords: mutable.Map[TopicIdPartition, mutable.Buffer[MemoryRecords]] ): Set[String] = { val expiredTransactionalIds = mutable.Set.empty[String] appendedRecords.values.foreach { batches => @@ -946,10 +957,10 @@ class TransactionStateManagerTest { private def expectTransactionalIdExpiration( appendError: Errors, - capturedAppends: mutable.Map[TopicPartition, mutable.Buffer[MemoryRecords]] + capturedAppends: mutable.Map[TopicIdPartition, mutable.Buffer[MemoryRecords]] ): Unit = { - val recordsCapture: ArgumentCaptor[Map[TopicPartition, MemoryRecords]] = ArgumentCaptor.forClass(classOf[Map[TopicPartition, MemoryRecords]]) - val callbackCapture: ArgumentCaptor[Map[TopicPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) + val recordsCapture: ArgumentCaptor[Map[TopicIdPartition, MemoryRecords]] = ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, MemoryRecords]]) + val callbackCapture: ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, PartitionResponse] => Unit]) when(replicaManager.appendRecords( anyLong(), @@ -1012,7 +1023,7 @@ class TransactionStateManagerTest { txnMetadata2.txnLastUpdateTimestamp = time.milliseconds() transactionManager.putTransactionStateIfNotExists(txnMetadata2) - val appendedRecords = mutable.Map.empty[TopicPartition, mutable.Buffer[MemoryRecords]] + val appendedRecords = mutable.Map.empty[TopicIdPartition, mutable.Buffer[MemoryRecords]] expectTransactionalIdExpiration(error, appendedRecords) transactionManager.removeExpiredTransactionalIds() @@ -1024,7 +1035,7 @@ class TransactionStateManagerTest { if (stateAllowsExpiration) { val partitionId = transactionManager.partitionFor(transactionalId1) - val topicPartition = new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, partitionId) + val topicPartition = new TopicIdPartition(transactionTopicId, partitionId, TRANSACTION_STATE_TOPIC_NAME) val expectedTombstone = new SimpleRecord(time.milliseconds(), TransactionLog.keyToBytes(transactionalId1), null) val expectedRecords = MemoryRecords.withRecords(TransactionLog.EnforcedCompressionType, expectedTombstone) assertEquals(Set(topicPartition), appendedRecords.keySet) @@ -1103,12 +1114,12 @@ class TransactionStateManagerTest { private def prepareForTxnMessageAppend(error: Errors): Unit = { reset(replicaManager) - val capturedArgument: ArgumentCaptor[Map[TopicPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) + val capturedArgument: ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, PartitionResponse] => Unit]) when(replicaManager.appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), - any[Map[TopicPartition, MemoryRecords]], + any[Map[TopicIdPartition, MemoryRecords]], capturedArgument.capture(), any[Option[ReentrantLock]], any(), @@ -1116,11 +1127,13 @@ class TransactionStateManagerTest { any(), any() )).thenAnswer(_ => capturedArgument.getValue.apply( - Map(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, partitionId) -> + Map(new TopicIdPartition(transactionTopicId, partitionId, TRANSACTION_STATE_TOPIC_NAME) -> new PartitionResponse(error, 0L, RecordBatch.NO_TIMESTAMP, 0L))) ) when(replicaManager.getMagic(any())) .thenReturn(Some(RecordBatch.MAGIC_VALUE_V1)) + when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 0))).thenReturn(new TopicIdPartition(transactionTopicId, 0, TRANSACTION_STATE_TOPIC_NAME)) + when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 1))).thenReturn(new TopicIdPartition(transactionTopicId, 1, TRANSACTION_STATE_TOPIC_NAME)) } @Test diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 8826cad5c86a0..c387bb13c94c3 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -2458,20 +2458,22 @@ class KafkaApisTest extends Logging { @Test def shouldReplaceProducerFencedWithInvalidProducerEpochInProduceResponse(): Unit = { val topic = "topic" - addTopicToMetadataCache(topic, numPartitions = 2) + val topicId = Uuid.randomUuid() + val tp = new TopicIdPartition(topicId, 0, "topic") + addTopicToMetadataCache(topic, numPartitions = 2, topicId = topicId) for (version <- ApiKeys.PRODUCE.oldestVersion to ApiKeys.PRODUCE.latestVersion) { reset(replicaManager, clientQuotaManager, clientRequestQuotaManager, requestChannel, txnCoordinator) - val responseCallback: ArgumentCaptor[Map[TopicPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) - - val tp = new TopicPartition("topic", 0) + val responseCallback: ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, PartitionResponse] => Unit]) val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection( Collections.singletonList(new ProduceRequestData.TopicProduceData() - .setName(tp.topic).setPartitionData(Collections.singletonList( + .setName(tp.topic) + .setTopicId(topicId) + .setPartitionData(Collections.singletonList( new ProduceRequestData.PartitionProduceData() .setIndex(tp.partition) .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes)))))) @@ -2517,15 +2519,16 @@ class KafkaApisTest extends Logging { @Test def testProduceResponseContainsNewLeaderOnNotLeaderOrFollower(): Unit = { val topic = "topic" - addTopicToMetadataCache(topic, numPartitions = 2, numBrokers = 3) + val topicId = Uuid.randomUuid() + addTopicToMetadataCache(topic, numPartitions = 2, numBrokers = 3, topicId = topicId) for (version <- 10 to ApiKeys.PRODUCE.latestVersion) { reset(replicaManager, clientQuotaManager, clientRequestQuotaManager, requestChannel, txnCoordinator) - val responseCallback: ArgumentCaptor[Map[TopicPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) + val responseCallback: ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, PartitionResponse] => Unit]) - val tp = new TopicPartition(topic, 0) + val tp = new TopicIdPartition(topicId, 0, topic) val partition = mock(classOf[Partition]) val newLeaderId = 2 val newLeaderEpoch = 5 @@ -2533,7 +2536,9 @@ class KafkaApisTest extends Logging { val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection( Collections.singletonList(new ProduceRequestData.TopicProduceData() - .setName(tp.topic).setPartitionData(Collections.singletonList( + .setName(tp.topic) + .setTopicId(topicId) + .setPartitionData(Collections.singletonList( new ProduceRequestData.PartitionProduceData() .setIndex(tp.partition) .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes)))))) @@ -2555,7 +2560,7 @@ class KafkaApisTest extends Logging { any()) ).thenAnswer(_ => responseCallback.getValue.apply(Map(tp -> new PartitionResponse(Errors.NOT_LEADER_OR_FOLLOWER)))) - when(replicaManager.getPartitionOrError(tp)).thenAnswer(_ => Right(partition)) + when(replicaManager.getPartitionOrError(tp.topicPartition())).thenAnswer(_ => Right(partition)) when(partition.leaderReplicaIdOpt).thenAnswer(_ => Some(newLeaderId)) when(partition.getLeaderEpoch).thenAnswer(_ => newLeaderEpoch) @@ -2585,20 +2590,23 @@ class KafkaApisTest extends Logging { @Test def testProduceResponseReplicaManagerLookupErrorOnNotLeaderOrFollower(): Unit = { val topic = "topic" - addTopicToMetadataCache(topic, numPartitions = 2, numBrokers = 3) + val topicId = Uuid.randomUuid() + addTopicToMetadataCache(topic, numPartitions = 2, numBrokers = 3, topicId = topicId) for (version <- 10 to ApiKeys.PRODUCE.latestVersion) { reset(replicaManager, clientQuotaManager, clientRequestQuotaManager, requestChannel, txnCoordinator) - val responseCallback: ArgumentCaptor[Map[TopicPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) + val responseCallback: ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, PartitionResponse] => Unit]) - val tp = new TopicPartition(topic, 0) + val tp = new TopicIdPartition(topicId, 0, topic) val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection( Collections.singletonList(new ProduceRequestData.TopicProduceData() - .setName(tp.topic).setPartitionData(Collections.singletonList( + .setName(tp.topic) + .setTopicId(topicId) + .setPartitionData(Collections.singletonList( new ProduceRequestData.PartitionProduceData() .setIndex(tp.partition) .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes)))))) @@ -2620,7 +2628,7 @@ class KafkaApisTest extends Logging { any()) ).thenAnswer(_ => responseCallback.getValue.apply(Map(tp -> new PartitionResponse(Errors.NOT_LEADER_OR_FOLLOWER)))) - when(replicaManager.getPartitionOrError(tp)).thenAnswer(_ => Left(Errors.UNKNOWN_TOPIC_OR_PARTITION)) + when(replicaManager.getPartitionOrError(tp.topicPartition())).thenAnswer(_ => Left(Errors.UNKNOWN_TOPIC_OR_PARTITION)) when(clientRequestQuotaManager.maybeRecordAndGetThrottleTimeMs(any[RequestChannel.Request](), any[Long])).thenReturn(0) @@ -2649,23 +2657,31 @@ class KafkaApisTest extends Logging { @Test def testProduceResponseMetadataLookupErrorOnNotLeaderOrFollower(): Unit = { val topic = "topic" + val topicId = Uuid.randomUuid() metadataCache = mock(classOf[ZkMetadataCache]) for (version <- 10 to ApiKeys.PRODUCE.latestVersion) { reset(replicaManager, clientQuotaManager, clientRequestQuotaManager, requestChannel, txnCoordinator) - val responseCallback: ArgumentCaptor[Map[TopicPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) + val responseCallback: ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, PartitionResponse] => Unit]) - val tp = new TopicPartition(topic, 0) + val tp = new TopicIdPartition(topicId, 0, topic) + + val topicProduceData = new ProduceRequestData.TopicProduceData() + .setName(tp.topic) + + if (version >= 12 ) { + topicProduceData.setTopicId(topicId) + } val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection( - Collections.singletonList(new ProduceRequestData.TopicProduceData() - .setName(tp.topic).setPartitionData(Collections.singletonList( - new ProduceRequestData.PartitionProduceData() - .setIndex(tp.partition) - .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes)))))) + Collections.singletonList(topicProduceData + .setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData() + .setIndex(tp.partition) + .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes)))))) .iterator)) .setAcks(1.toShort) .setTimeoutMs(5000)) @@ -2684,13 +2700,15 @@ class KafkaApisTest extends Logging { any()) ).thenAnswer(_ => responseCallback.getValue.apply(Map(tp -> new PartitionResponse(Errors.NOT_LEADER_OR_FOLLOWER)))) - when(replicaManager.getPartitionOrError(tp)).thenAnswer(_ => Left(Errors.UNKNOWN_TOPIC_OR_PARTITION)) + when(replicaManager.getPartitionOrError(tp.topicPartition)).thenAnswer(_ => Left(Errors.UNKNOWN_TOPIC_OR_PARTITION)) when(clientRequestQuotaManager.maybeRecordAndGetThrottleTimeMs(any[RequestChannel.Request](), any[Long])).thenReturn(0) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) - when(metadataCache.contains(tp)).thenAnswer(_ => true) + when(metadataCache.contains(tp.topicPartition())).thenAnswer(_ => true) + when(metadataCache.getTopicName(any())).thenReturn(Some(topicProduceData.name())) + when(metadataCache.getTopicId(any())).thenReturn(topicProduceData.topicId()) when(metadataCache.getPartitionInfo(tp.topic(), tp.partition())).thenAnswer(_ => Option.empty) when(metadataCache.getAliveBrokerNode(any(), any())).thenReturn(Option.empty) kafkaApis = createKafkaApis() @@ -2714,18 +2732,20 @@ class KafkaApisTest extends Logging { val topic = "topic" val transactionalId = "txn1" - addTopicToMetadataCache(topic, numPartitions = 2) + val tp = new TopicIdPartition(Uuid.randomUuid(), 0, "topic") + addTopicToMetadataCache(topic, numPartitions = 2, topicId = tp.topicId()) for (version <- 3 to ApiKeys.PRODUCE.latestVersion) { reset(replicaManager, clientQuotaManager, clientRequestQuotaManager, requestChannel, txnCoordinator) - val tp = new TopicPartition("topic", 0) val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection( Collections.singletonList(new ProduceRequestData.TopicProduceData() - .setName(tp.topic).setPartitionData(Collections.singletonList( + .setName(tp.topic) + .setTopicId(tp.topicId()) + .setPartitionData(Collections.singletonList( new ProduceRequestData.PartitionProduceData() .setIndex(tp.partition) .setRecords(MemoryRecords.withTransactionalRecords(CompressionType.NONE, 0, 0, 0, new SimpleRecord("test".getBytes)))))) @@ -2868,11 +2888,12 @@ class KafkaApisTest extends Logging { def shouldRespondWithUnsupportedMessageFormatForBadPartitionAndNoErrorsForGoodPartition(): Unit = { val tp1 = new TopicPartition("t", 0) val tp2 = new TopicPartition("t1", 0) + val topicId = Uuid.randomUuid() val (_, request) = createWriteTxnMarkersRequest(asList(tp1, tp2)) val expectedErrors = Map(tp1 -> Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT, tp2 -> Errors.NONE).asJava val capturedResponse: ArgumentCaptor[WriteTxnMarkersResponse] = ArgumentCaptor.forClass(classOf[WriteTxnMarkersResponse]) - val responseCallback: ArgumentCaptor[Map[TopicPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) + val responseCallback: ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, PartitionResponse] => Unit]) when(replicaManager.getMagic(tp1)) .thenReturn(Some(RecordBatch.MAGIC_VALUE_V1)) @@ -2891,7 +2912,7 @@ class KafkaApisTest extends Logging { ArgumentMatchers.eq(requestLocal), any(), any() - )).thenAnswer(_ => responseCallback.getValue.apply(Map(tp2 -> new PartitionResponse(Errors.NONE)))) + )).thenAnswer(_ => responseCallback.getValue.apply(Map(new TopicIdPartition(topicId, tp2) -> new PartitionResponse(Errors.NONE)))) kafkaApis = createKafkaApis() kafkaApis.handleWriteTxnMarkersRequest(request, requestLocal) @@ -3000,11 +3021,12 @@ class KafkaApisTest extends Logging { def shouldRespondWithUnknownTopicOrPartitionForBadPartitionAndNoErrorsForGoodPartition(): Unit = { val tp1 = new TopicPartition("t", 0) val tp2 = new TopicPartition("t1", 0) + val topicId = Uuid.randomUuid() val (_, request) = createWriteTxnMarkersRequest(asList(tp1, tp2)) val expectedErrors = Map(tp1 -> Errors.UNKNOWN_TOPIC_OR_PARTITION, tp2 -> Errors.NONE).asJava val capturedResponse: ArgumentCaptor[WriteTxnMarkersResponse] = ArgumentCaptor.forClass(classOf[WriteTxnMarkersResponse]) - val responseCallback: ArgumentCaptor[Map[TopicPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) + val responseCallback: ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, PartitionResponse] => Unit]) when(replicaManager.getMagic(tp1)) .thenReturn(None) @@ -3023,7 +3045,7 @@ class KafkaApisTest extends Logging { ArgumentMatchers.eq(requestLocal), any(), any() - )).thenAnswer(_ => responseCallback.getValue.apply(Map(tp2 -> new PartitionResponse(Errors.NONE)))) + )).thenAnswer(_ => responseCallback.getValue.apply(Map(new TopicIdPartition(topicId,tp2) -> new PartitionResponse(Errors.NONE)))) kafkaApis = createKafkaApis() kafkaApis.handleWriteTxnMarkersRequest(request, requestLocal) verify(requestChannel).sendResponse( @@ -3066,6 +3088,9 @@ class KafkaApisTest extends Logging { val foo0 = new TopicPartition("foo", 0) val foo1 = new TopicPartition("foo", 1) + val topicIds = Map( + Topic.GROUP_METADATA_TOPIC_NAME -> Uuid.randomUuid(), + "foo" -> Uuid.randomUuid()) val allPartitions = List( offset0, offset1, @@ -3096,8 +3121,8 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(writeTxnMarkersRequest) allPartitions.foreach { tp => - when(replicaManager.getMagic(tp)) - .thenReturn(Some(RecordBatch.MAGIC_VALUE_V2)) + when(replicaManager.getMagic(tp)).thenReturn(Some(RecordBatch.MAGIC_VALUE_V2)) + when(replicaManager.getTopicIdPartition(tp)).thenReturn(new TopicIdPartition(topicIds.get(tp.topic()).getOrElse(Uuid.ZERO_UUID), tp)) } when(groupCoordinator.completeTransaction( @@ -3118,10 +3143,10 @@ class KafkaApisTest extends Logging { ArgumentMatchers.eq(Duration.ofMillis(Defaults.REQUEST_TIMEOUT_MS)) )).thenReturn(CompletableFuture.completedFuture[Void](null)) - val entriesPerPartition: ArgumentCaptor[Map[TopicPartition, MemoryRecords]] = - ArgumentCaptor.forClass(classOf[Map[TopicPartition, MemoryRecords]]) - val responseCallback: ArgumentCaptor[Map[TopicPartition, PartitionResponse] => Unit] = - ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) + val entriesPerPartition: ArgumentCaptor[Map[TopicIdPartition, MemoryRecords]] = + ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, MemoryRecords]]) + val responseCallback: ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = + ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, PartitionResponse] => Unit]) when(replicaManager.appendRecords( ArgumentMatchers.eq(Defaults.REQUEST_TIMEOUT_MS.toLong), diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala index cf9ac1f812c1a..657918702f7b5 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala @@ -289,11 +289,13 @@ class ReplicaManagerConcurrencyTest extends Logging { } val future = new CompletableFuture[ProduceResponse.PartitionResponse]() - def produceCallback(results: collection.Map[TopicPartition, ProduceResponse.PartitionResponse]): Unit = { + val topicIdPartition = replicaManager.getTopicIdPartition(topicPartition) + + def produceCallback(results: collection.Map[TopicIdPartition, ProduceResponse.PartitionResponse]): Unit = { try { assertEquals(1, results.size) val (topicPartition, result) = results.head - assertEquals(this.topicPartition, topicPartition) + assertEquals(topicIdPartition, topicPartition) assertEquals(Errors.NONE, result.error) future.complete(result) } catch { @@ -306,7 +308,7 @@ class ReplicaManagerConcurrencyTest extends Logging { requiredAcks = (-1).toShort, internalTopicsAllowed = false, origin = AppendOrigin.CLIENT, - entriesPerPartition = collection.Map(topicPartition -> TestUtils.records(records)), + entriesPerPartition = collection.Map(topicIdPartition -> TestUtils.records(records)), responseCallback = produceCallback ) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index afc1937e45081..c59acd21c4efa 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -221,7 +221,7 @@ class ReplicaManagerTest { alterPartitionManager = alterPartitionManager, threadNamePrefix = Option(this.getClass.getName)) try { - def callback(responseStatus: Map[TopicPartition, PartitionResponse]): Unit = { + def callback(responseStatus: Map[TopicIdPartition, PartitionResponse]): Unit = { assert(responseStatus.values.head.error == Errors.INVALID_REQUIRED_ACKS) } rm.appendRecords( @@ -229,7 +229,7 @@ class ReplicaManagerTest { requiredAcks = 3, internalTopicsAllowed = false, origin = AppendOrigin.CLIENT, - entriesPerPartition = Map(new TopicPartition("test1", 0) -> MemoryRecords.withRecords(CompressionType.NONE, + entriesPerPartition = Map(new TopicIdPartition(Uuid.randomUuid(), 0, "test1") -> MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("first message".getBytes))), responseCallback = callback) } finally { @@ -261,6 +261,9 @@ class ReplicaManagerTest { val config = KafkaConfig.fromProps(props) val logManager = TestUtils.createLogManager(config.logDirs.map(new File(_)), new LogConfig(new Properties())) val metadataCache: MetadataCache = mock(classOf[MetadataCache]) + topicNames.foreach { case (topicId, topicName) => + when(metadataCache.getTopicId(topicName)).thenReturn(topicId) + } mockGetAliveBrokerFunctions(metadataCache, Seq(new Node(0, "host0", 0))) when(metadataCache.metadataVersion()).thenReturn(config.interBrokerProtocolVersion) val rm = new ReplicaManager( @@ -2125,6 +2128,7 @@ class ReplicaManagerTest { try { val tp0 = new TopicPartition(topic, 0) + val topicIdPartition0 = new TopicIdPartition(Uuid.randomUuid(), tp0) val offsetCheckpoints = new LazyOffsetCheckpoints(replicaManager.highWatermarkCheckpoints) replicaManager.createPartition(tp0).createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None) val partition0Replicas = Seq[Integer](0, 1).asJava @@ -2144,7 +2148,7 @@ class ReplicaManagerTest { Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() replicaManager.becomeLeaderOrFollower(1, becomeLeaderRequest, (_, _) => ()) - val produceResult = sendProducerAppend(replicaManager, tp0, 3) + val produceResult = sendProducerAppend(replicaManager, topicIdPartition0, 3) assertNull(produceResult.get) when(replicaManager.metadataCache.contains(tp0)).thenReturn(true) @@ -2284,7 +2288,6 @@ class ReplicaManagerTest { replicaManager.becomeLeaderOrFollower(1, makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), LeaderAndIsr(1, List(0, 1))), (_, _) => ()) - // Start with sequence 6 val transactionalRecords = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence, new SimpleRecord("message".getBytes)) @@ -2777,11 +2780,11 @@ class ReplicaManagerTest { private def sendProducerAppend( replicaManager: ReplicaManager, - topicPartition: TopicPartition, + topicPartition: TopicIdPartition, numOfRecords: Int ): AtomicReference[PartitionResponse] = { val produceResult = new AtomicReference[PartitionResponse]() - def callback(response: Map[TopicPartition, PartitionResponse]): Unit = { + def callback(response: Map[TopicIdPartition, PartitionResponse]): Unit = { produceResult.set(response(topicPartition)) } @@ -3024,18 +3027,23 @@ class ReplicaManagerTest { origin: AppendOrigin = AppendOrigin.CLIENT, requiredAcks: Short = -1): CallbackResult[PartitionResponse] = { val result = new CallbackResult[PartitionResponse]() - def appendCallback(responses: Map[TopicPartition, PartitionResponse]): Unit = { - val response = responses.get(partition) + val topicIdPartition = new TopicIdPartition(topicId, partition) + def appendCallback(responses: Map[TopicIdPartition, PartitionResponse]): Unit = { + val response = responses.get(topicIdPartition) assertTrue(response.isDefined) result.fire(response.get) } + topicNames.foreach { case (topicId, topicName) => + when(replicaManager.metadataCache.getTopicId(topicName)).thenReturn(topicId) + } + replicaManager.appendRecords( timeout = 1000, requiredAcks = requiredAcks, internalTopicsAllowed = false, origin = origin, - entriesPerPartition = Map(partition -> records), + entriesPerPartition = Map(new TopicIdPartition(topicId, partition) -> records), responseCallback = appendCallback, ) @@ -3046,19 +3054,22 @@ class ReplicaManagerTest { entriesToAppend: Map[TopicPartition, MemoryRecords], transactionalId: String, origin: AppendOrigin = AppendOrigin.CLIENT, - requiredAcks: Short = -1): CallbackResult[Map[TopicPartition, PartitionResponse]] = { - val result = new CallbackResult[Map[TopicPartition, PartitionResponse]]() - def appendCallback(responses: Map[TopicPartition, PartitionResponse]): Unit = { + requiredAcks: Short = -1): CallbackResult[Map[TopicIdPartition, PartitionResponse]] = { + val result = new CallbackResult[Map[TopicIdPartition, PartitionResponse]]() + def appendCallback(responses: Map[TopicIdPartition, PartitionResponse]): Unit = { responses.foreach( response => assertTrue(responses.get(response._1).isDefined)) result.fire(responses) } + topicNames.foreach { case (topicId, topicName) => + when(replicaManager.metadataCache.getTopicId(topicName)).thenReturn(topicId) + } replicaManager.handleProduceAppend( timeout = 1000, requiredAcks = requiredAcks, internalTopicsAllowed = false, transactionalId = transactionalId, - entriesPerPartition = entriesToAppend, + entriesPerPartition = entriesToAppend.map(e => replicaManager.getTopicIdPartition(e._1) -> e._2), responseCallback = appendCallback, supportedOperation = supportedOperation ) @@ -3074,19 +3085,24 @@ class ReplicaManagerTest { transactionalId: String): CallbackResult[PartitionResponse] = { val result = new CallbackResult[PartitionResponse]() - def appendCallback(responses: Map[TopicPartition, PartitionResponse]): Unit = { - val response = responses.get(partition) + val topicIdPartition = new TopicIdPartition(topicIds.get(partition.topic()).getOrElse(Uuid.ZERO_UUID), partition) + def appendCallback(responses: Map[TopicIdPartition, PartitionResponse]): Unit = { + val response = responses.get(topicIdPartition) assertTrue(response.isDefined) result.fire(response.get) } + topicNames.foreach { case (topicId, topicName) => + when(replicaManager.metadataCache.getTopicId(topicName)).thenReturn(topicId) + } + val entriesPerPartition = Map(partition -> records) replicaManager.handleProduceAppend( timeout = 1000, requiredAcks = requiredAcks, internalTopicsAllowed = false, transactionalId = transactionalId, - entriesPerPartition = entriesPerPartition, + entriesPerPartition = entriesPerPartition.map(e => replicaManager.getTopicIdPartition(e._1) -> e._2), responseCallback = appendCallback, supportedOperation = supportedOperation ) @@ -3246,7 +3262,9 @@ class ReplicaManagerTest { config: KafkaConfig = config): ReplicaManager = { val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_))) val metadataCache = mock(classOf[MetadataCache]) - + topicNames.foreach { case (topicId, topicName) => + when(metadataCache.getTopicId(topicName)).thenReturn(topicId) + } val replicaManager = new ReplicaManager( metrics = metrics, config = config, @@ -3324,6 +3342,9 @@ class ReplicaManagerTest { when(metadataCache.topicNamesToIds()).thenReturn(topicIds.asJava) when(metadataCache.topicIdsToNames()).thenReturn(topicNames.asJava) when(metadataCache.metadataVersion()).thenReturn(config.interBrokerProtocolVersion) + topicNames.foreach { case (topicId, topicName) => + when(metadataCache.getTopicId(topicName)).thenReturn(topicId) + } mockGetAliveBrokerFunctions(metadataCache, aliveBrokers) val mockProducePurgatory = new DelayedOperationPurgatory[DelayedProduce]( purgatoryName = "Produce", timer, reaperEnabled = false) @@ -5122,7 +5143,8 @@ class ReplicaManagerTest { val localId = 1 val otherId = localId + 1 val numOfRecords = 3 - val topicPartition = new TopicPartition("foo", 0) + val topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), 0, "foo") + val topicPartition = topicIdPartition.topicPartition() val replicaManager = setupReplicaManagerWithMockedPurgatories(new MockTimer(time), localId, enableRemoteStorage = enableRemoteStorage) try { @@ -5148,7 +5170,7 @@ class ReplicaManagerTest { } // Send a produce request and advance the highwatermark - val leaderResponse = sendProducerAppend(replicaManager, topicPartition, numOfRecords) + val leaderResponse = sendProducerAppend(replicaManager, topicIdPartition, numOfRecords) fetchPartitionAsFollower( replicaManager, topicIdPartition, @@ -5163,7 +5185,7 @@ class ReplicaManagerTest { replicaManager.applyDelta(followerTopicsDelta, followerMetadataImage) // Append on a follower should fail - val followerResponse = sendProducerAppend(replicaManager, topicPartition, numOfRecords) + val followerResponse = sendProducerAppend(replicaManager, topicIdPartition, numOfRecords) assertEquals(Errors.NOT_LEADER_OR_FOLLOWER, followerResponse.get.error) // Check the state of that partition and fetcher @@ -5213,18 +5235,21 @@ class ReplicaManagerTest { assertEquals(Some(BrokerEndPoint(otherId, otherEndpoint.host(), otherEndpoint.port())), fetcher.map(_.leader.brokerEndPoint())) // Append on a follower should fail - val followerResponse = sendProducerAppend(replicaManager, topicPartition, numOfRecords) + val followerResponse = sendProducerAppend(replicaManager, + new TopicIdPartition(followerMetadataImage.topics().topicsByName().get("foo").id, topicPartition), + numOfRecords) assertEquals(Errors.NOT_LEADER_OR_FOLLOWER, followerResponse.get.error) // Change the local replica to leader val leaderTopicsDelta = topicsChangeDelta(followerMetadataImage.topics(), localId, true) val leaderMetadataImage = imageFromTopics(leaderTopicsDelta.apply()) - val topicId = leaderMetadataImage.topics().topicsByName.get("foo").id - val topicIdPartition = new TopicIdPartition(topicId, topicPartition) replicaManager.applyDelta(leaderTopicsDelta, leaderMetadataImage) + val topicIdPartition = new TopicIdPartition(leaderMetadataImage.topics().topicsByName().get("foo").id, topicPartition) // Send a produce request and advance the highwatermark - val leaderResponse = sendProducerAppend(replicaManager, topicPartition, numOfRecords) + val leaderResponse = sendProducerAppend(replicaManager, + topicIdPartition + , numOfRecords) fetchPartitionAsFollower( replicaManager, topicIdPartition, @@ -5490,7 +5515,8 @@ class ReplicaManagerTest { val localId = 1 val otherId = localId + 1 val numOfRecords = 3 - val topicPartition = new TopicPartition("foo", 0) + val topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), 0, "foo") + val topicPartition = topicIdPartition.topicPartition() val replicaManager = setupReplicaManagerWithMockedPurgatories(new MockTimer(time), localId, enableRemoteStorage = enableRemoteStorage) try { @@ -5513,7 +5539,7 @@ class ReplicaManagerTest { assertEquals(None, replicaManager.replicaFetcherManager.getFetcher(topicPartition)) // Send a produce request - val leaderResponse = sendProducerAppend(replicaManager, topicPartition, numOfRecords) + val leaderResponse = sendProducerAppend(replicaManager, topicIdPartition, numOfRecords) // Change the local replica to follower val followerTopicsDelta = topicsChangeDelta(leaderMetadataImage.topics(), localId, false) diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerResponseBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerResponseBenchmark.java index 431880a96dc18..8b86048c170a1 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerResponseBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerResponseBenchmark.java @@ -17,7 +17,8 @@ package org.apache.kafka.jmh.producer; -import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.AbstractResponse; import org.apache.kafka.common.requests.ProduceResponse; @@ -46,9 +47,11 @@ public class ProducerResponseBenchmark { private static final int NUMBER_OF_PARTITIONS = 3; private static final int NUMBER_OF_RECORDS = 3; - private static final Map PARTITION_RESPONSE_MAP = IntStream.range(0, NUMBER_OF_PARTITIONS) + private static final Uuid TOPIC_ID = Uuid.randomUuid(); + private static final String TOPIC_NAME = "tp"; + private static final Map PARTITION_RESPONSE_MAP = IntStream.range(0, NUMBER_OF_PARTITIONS) .mapToObj(partitionIndex -> new AbstractMap.SimpleEntry<>( - new TopicPartition("tp", partitionIndex), + new TopicIdPartition(TOPIC_ID, partitionIndex, TOPIC_NAME), new ProduceResponse.PartitionResponse( Errors.NONE, 0, From ddeac1b536b8cf25e7d5828439b24a55a4b68cd8 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Thu, 9 May 2024 17:04:18 +0100 Subject: [PATCH 02/40] KAFKA-10551: fix compatibility with 2.6 IBP --- .../clients/producer/internals/Sender.java | 24 +++++++++++++++---- .../kafka/common/requests/ProduceRequest.java | 7 +++++- 2 files changed, 25 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index cd23e0501ab4d..1556d8a5470cc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -880,10 +880,12 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo if (batch.magic() < minUsedMagic) minUsedMagic = batch.magic(); } + Map topicIds = getTopicIdsFromBatches(batches); + boolean canUseTopicId = !topicIds.entrySet().stream().anyMatch(e -> e.getValue() == Uuid.ZERO_UUID); + ProduceRequestData.TopicProduceDataCollection tpd = new ProduceRequestData.TopicProduceDataCollection(); for (ProducerBatch batch : batches) { TopicPartition tp = batch.topicPartition; - Uuid topicId = metadata.topicIds().getOrDefault(tp.topic(), Uuid.ZERO_UUID); MemoryRecords records = batch.records(); // down convert if necessary to the minimum magic used. In general, there can be a delay between the time @@ -895,11 +897,15 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo // which is supporting the new magic version to one which doesn't, then we will need to convert. if (!records.hasMatchingMagic(minUsedMagic)) records = batch.records().downConvert(minUsedMagic, 0, time).records(); - ProduceRequestData.TopicProduceData tpData = tpd.find(tp.topic(), topicId); - if (tpData == null) { - tpData = new ProduceRequestData.TopicProduceData().setName(tp.topic()).setTopicId(topicId); - tpd.add(tpData); + Optional topicProduceData = canUseTopicId ? + Optional.ofNullable(tpd.find(tp.topic(), topicIds.get(tp.topic()))): + tpd.stream().filter(data -> data.name() == tp.topic()).findFirst(); + + ProduceRequestData.TopicProduceData tpData = topicProduceData.orElse(new ProduceRequestData.TopicProduceData().setName(tp.topic())); + if (canUseTopicId) { + tpData.setTopicId(topicIds.get(tp.topic())); } + tpd.add(tpData); tpData.partitionData().add(new ProduceRequestData.PartitionProduceData() .setIndex(tp.partition()) .setRecords(records)); @@ -926,6 +932,14 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo log.trace("Sent produce request to {}: {}", nodeId, requestBuilder); } + private Map getTopicIdsFromBatches(List batches) { + return batches.stream() + .collect(Collectors.toMap( + b -> b.topicPartition.topic(), + b -> metadata.topicIds().getOrDefault(b.topicPartition.topic(), Uuid.ZERO_UUID)) + ); + } + /** * Wake up the selector associated with this send thread */ diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 9f4672d446858..bcc73411f8448 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.InvalidRecordException; import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.UnsupportedCompressionTypeException; import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.message.ProduceResponseData; @@ -48,7 +49,7 @@ public static Builder forMagic(byte magic, ProduceRequestData data) { final short minVersion; final short maxVersion; - if (magic < RecordBatch.MAGIC_VALUE_V2) { + if (magic < RecordBatch.MAGIC_VALUE_V2 || canNotSupportTopicId(data)) { minVersion = 2; maxVersion = 2; } else { @@ -58,6 +59,10 @@ public static Builder forMagic(byte magic, ProduceRequestData data) { return new Builder(minVersion, maxVersion, data); } + private static boolean canNotSupportTopicId(ProduceRequestData data) { + return data.topicData().stream().anyMatch(d -> d.topicId() == Uuid.ZERO_UUID); + } + public static Builder forCurrentMagic(ProduceRequestData data) { return forMagic(RecordBatch.CURRENT_MAGIC_VALUE, data); } From a92abe6ecc6e67aff7ad0e01912bc70732ff287d Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Tue, 14 May 2024 12:48:40 +0100 Subject: [PATCH 03/40] KAFKA-10551: refactor --- .../kafka/server/ReplicaManagerTest.scala | 23 +++++-------------- 1 file changed, 6 insertions(+), 17 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index c59acd21c4efa..41f58d660edf9 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -115,6 +115,7 @@ class ReplicaManagerTest { private var addPartitionsToTxnManager: AddPartitionsToTxnManager = _ private var brokerTopicStats: BrokerTopicStats = _ private val supportedOperation = genericError + private val metadataCache: MetadataCache = mock(classOf[MetadataCache]) // Constants defined for readability private val zkVersion = 0 @@ -139,6 +140,10 @@ class ReplicaManagerTest { val callback = invocationOnMock.getArgument(4, classOf[AddPartitionsToTxnManager.AppendCallback]) callback(Map.empty[TopicPartition, Errors].toMap) } + // make sure metadataCache can map between topic name and id + topicNames.foreach { case (topicId, topicName) => + when(metadataCache.getTopicId(topicName)).thenReturn(topicId) + } } @AfterEach @@ -260,10 +265,6 @@ class ReplicaManagerTest { props.put("log.dirs", dir1.getAbsolutePath + "," + dir2.getAbsolutePath) val config = KafkaConfig.fromProps(props) val logManager = TestUtils.createLogManager(config.logDirs.map(new File(_)), new LogConfig(new Properties())) - val metadataCache: MetadataCache = mock(classOf[MetadataCache]) - topicNames.foreach { case (topicId, topicName) => - when(metadataCache.getTopicId(topicName)).thenReturn(topicId) - } mockGetAliveBrokerFunctions(metadataCache, Seq(new Node(0, "host0", 0))) when(metadataCache.metadataVersion()).thenReturn(config.interBrokerProtocolVersion) val rm = new ReplicaManager( @@ -325,7 +326,6 @@ class ReplicaManagerTest { val logProps = new Properties() val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)), new LogConfig(logProps)) val aliveBrokers = Seq(new Node(0, "host0", 0), new Node(1, "host1", 1)) - val metadataCache: MetadataCache = mock(classOf[MetadataCache]) mockGetAliveBrokerFunctions(metadataCache, aliveBrokers) when(metadataCache.metadataVersion()).thenReturn(config.interBrokerProtocolVersion) val rm = new ReplicaManager( @@ -2897,7 +2897,6 @@ class ReplicaManagerTest { val aliveBrokerIds = Seq[Integer](followerBrokerId, leaderBrokerId) val aliveBrokers = aliveBrokerIds.map(brokerId => new Node(brokerId, s"host$brokerId", brokerId)) - val metadataCache: MetadataCache = mock(classOf[MetadataCache]) mockGetAliveBrokerFunctions(metadataCache, aliveBrokers) when(metadataCache.getPartitionReplicaEndpoints( any[TopicPartition], any[ListenerName])). @@ -3261,10 +3260,6 @@ class ReplicaManagerTest { transactionalTopicPartitions: List[TopicPartition], config: KafkaConfig = config): ReplicaManager = { val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_))) - val metadataCache = mock(classOf[MetadataCache]) - topicNames.foreach { case (topicId, topicName) => - when(metadataCache.getTopicId(topicName)).thenReturn(topicId) - } val replicaManager = new ReplicaManager( metrics = metrics, config = config, @@ -3337,14 +3332,10 @@ class ReplicaManagerTest { val aliveBrokers = aliveBrokerIds.map(brokerId => new Node(brokerId, s"host$brokerId", brokerId)) brokerTopicStats = new BrokerTopicStats(java.util.Optional.of(KafkaConfig.fromProps(props))) - val metadataCache: MetadataCache = mock(classOf[MetadataCache]) when(metadataCache.topicIdInfo()).thenReturn((topicIds.asJava, topicNames.asJava)) when(metadataCache.topicNamesToIds()).thenReturn(topicIds.asJava) when(metadataCache.topicIdsToNames()).thenReturn(topicNames.asJava) when(metadataCache.metadataVersion()).thenReturn(config.interBrokerProtocolVersion) - topicNames.foreach { case (topicId, topicName) => - when(metadataCache.getTopicId(topicName)).thenReturn(topicId) - } mockGetAliveBrokerFunctions(metadataCache, aliveBrokers) val mockProducePurgatory = new DelayedOperationPurgatory[DelayedProduce]( purgatoryName = "Produce", timer, reaperEnabled = false) @@ -5247,9 +5238,7 @@ class ReplicaManagerTest { val topicIdPartition = new TopicIdPartition(leaderMetadataImage.topics().topicsByName().get("foo").id, topicPartition) // Send a produce request and advance the highwatermark - val leaderResponse = sendProducerAppend(replicaManager, - topicIdPartition - , numOfRecords) + val leaderResponse = sendProducerAppend(replicaManager, topicIdPartition, numOfRecords) fetchPartitionAsFollower( replicaManager, topicIdPartition, From a8f0c913a78cd70690f8ef96582561c94922384f Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Tue, 14 May 2024 15:46:46 +0100 Subject: [PATCH 04/40] fix test --- .../clients/producer/internals/Sender.java | 3 +- .../kafka/common/requests/ProduceRequest.java | 7 +++-- .../org/apache/kafka/common/utils/Utils.java | 2 +- .../kafka/common/message/MessageTest.java | 9 +++--- .../common/requests/ProduceRequestTest.java | 28 +++++++++++++++-- .../common/requests/ProduceResponseTest.java | 6 ++-- .../common/requests/RequestResponseTest.java | 8 +++-- .../main/scala/kafka/server/KafkaApis.scala | 11 ++++--- .../CoordinatorPartitionWriterTest.scala | 2 +- .../group/GroupCoordinatorTest.scala | 2 +- .../unit/kafka/server/KafkaApisTest.scala | 26 ++++++++-------- .../kafka/server/ProduceRequestTest.scala | 10 +++++-- .../kafka/server/ReplicaManagerTest.scala | 30 ++++++++----------- 13 files changed, 88 insertions(+), 56 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 77ab4289d9928..166bc26d2e3eb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -936,7 +936,8 @@ private Map getTopicIdsFromBatches(List batches) { return batches.stream() .collect(Collectors.toMap( b -> b.topicPartition.topic(), - b -> metadata.topicIds().getOrDefault(b.topicPartition.topic(), Uuid.ZERO_UUID)) + b -> metadata.topicIds().getOrDefault(b.topicPartition.topic(), Uuid.ZERO_UUID), + (existing, replacement) -> replacement) ); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index bcc73411f8448..a09e35deda595 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -49,9 +49,12 @@ public static Builder forMagic(byte magic, ProduceRequestData data) { final short minVersion; final short maxVersion; - if (magic < RecordBatch.MAGIC_VALUE_V2 || canNotSupportTopicId(data)) { + if (magic < RecordBatch.MAGIC_VALUE_V2) { minVersion = 2; maxVersion = 2; + } else if (canNotSupportTopicId(data)) { + minVersion = 3; + maxVersion = 11; } else { minVersion = 3; maxVersion = ApiKeys.PRODUCE.latestVersion(); @@ -60,7 +63,7 @@ public static Builder forMagic(byte magic, ProduceRequestData data) { } private static boolean canNotSupportTopicId(ProduceRequestData data) { - return data.topicData().stream().anyMatch(d -> d.topicId() == Uuid.ZERO_UUID); + return data.topicData().stream().anyMatch(d -> d.topicId() == null || d.topicId() == Uuid.ZERO_UUID); } public static Builder forCurrentMagic(ProduceRequestData data) { 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 b9e8993a368f5..c6d26e2ebcd92 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 @@ -1655,7 +1655,7 @@ public static void require(boolean requirement) { } /** - * Convert map keys to another type. + * Convert a map's keys to another type. */ public static Map convertKeys(Map originalMap, Function converter) { Map newMap = new HashMap<>(); diff --git a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java index d18404d036734..34df62f9de511 100644 --- a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java +++ b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java @@ -915,6 +915,8 @@ private void testAllMessageRoundTripsOffsetFetchFromVersionV8AndAbove(short from @Test public void testProduceResponseVersions() throws Exception { String topicName = "topic"; + Uuid topicId = Uuid.fromString("klZ9sa2rSvig6QpgGXzALT"); + int partitionIndex = 0; short errorCode = Errors.INVALID_TOPIC_EXCEPTION.code(); long baseOffset = 12L; @@ -971,12 +973,11 @@ public void testProduceResponseVersions() throws Exception { responseData.setThrottleTimeMs(0); } - if (version <= 11) { - responseData.responses().iterator().next().setName(topicName); - } if (version >= 12) { - responseData.responses().iterator().next().setTopicId(Uuid.randomUuid()); + responseData.responses().iterator().next().setTopicId(topicId); + } else { + responseData.responses().iterator().next().setName(topicName); } if (version >= 3 && version <= 4) { diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java index 4ac61dbf5b3a3..61e8f3ac1e9c4 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java @@ -18,6 +18,7 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.InvalidRecordException; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.UnsupportedCompressionTypeException; import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.protocol.ApiKeys; @@ -120,8 +121,11 @@ public void testBuildWithCurrentMessageFormat() { ProduceRequest.Builder requestBuilder = ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE, new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( - new ProduceRequestData.TopicProduceData().setName("test").setPartitionData(Collections.singletonList( - new ProduceRequestData.PartitionProduceData().setIndex(9).setRecords(builder.build())))) + new ProduceRequestData.TopicProduceData() + .setName("test") + .setTopicId(Uuid.fromString("H3Emm3vW7AKKO4NTRPaCWt")) + .setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData().setIndex(9).setRecords(builder.build())))) .iterator())) .setAcks((short) 1) .setTimeoutMs(5000)); @@ -129,6 +133,26 @@ public void testBuildWithCurrentMessageFormat() { assertEquals(ApiKeys.PRODUCE.latestVersion(), requestBuilder.latestAllowedVersion()); } + @Test + public void testBuildWithCurrentMessageFormatWithoutTopicId() { + ByteBuffer buffer = ByteBuffer.allocate(256); + MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, + CompressionType.NONE, TimestampType.CREATE_TIME, 0L); + builder.append(10L, null, "a".getBytes()); + ProduceRequest.Builder requestBuilder = ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE, + new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( + new ProduceRequestData.TopicProduceData() + .setName("test") + .setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData().setIndex(9).setRecords(builder.build())))) + .iterator())) + .setAcks((short) 1) + .setTimeoutMs(5000)); + assertEquals(3, requestBuilder.oldestAllowedVersion()); + assertEquals(11, requestBuilder.latestAllowedVersion()); + } + @Test public void testV3AndAboveShouldContainOnlyOneRecordBatch() { ByteBuffer buffer = ByteBuffer.allocate(256); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java index 1bae3fb46af44..63fb6c89e33b0 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java @@ -74,7 +74,8 @@ public void produceResponseV5Test() { @Test public void produceResponseVersionTest() { Map responseData = new HashMap<>(); - TopicIdPartition topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), 0, "test"); + Uuid topicId = Uuid.fromString("5JkYABorYD4w0AQXe9TvBG"); + TopicIdPartition topicIdPartition = new TopicIdPartition(topicId, 0, "test"); responseData.put(topicIdPartition, new ProduceResponse.PartitionResponse(Errors.NONE, 10000, RecordBatch.NO_TIMESTAMP, 100)); ProduceResponse v0Response = new ProduceResponse(responseData); ProduceResponse v1Response = new ProduceResponse(responseData, 10); @@ -103,7 +104,8 @@ public void produceResponseVersionTest() { @Test public void produceResponseRecordErrorsTest() { Map responseData = new HashMap<>(); - TopicIdPartition tp = new TopicIdPartition(Uuid.randomUuid(), 0, "test"); + Uuid topicId = Uuid.fromString("4w0AQXe9TvBG5JkYABorYD"); + TopicIdPartition tp = new TopicIdPartition(topicId, 0, "test"); ProduceResponse.PartitionResponse partResponse = new ProduceResponse.PartitionResponse(Errors.NONE, 10000, RecordBatch.NO_TIMESTAMP, 100, Collections.singletonList(new ProduceResponse.RecordError(3, "Record error")), diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 986a3fc50bed4..002952b297ce4 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -445,7 +445,7 @@ public void cannotUseFindCoordinatorV0ToFindTransactionCoordinator() { @Test public void testProduceRequestPartitionSize() { - Uuid topicId = Uuid.randomUuid(); + Uuid topicId = Uuid.fromString("e9TvBGX5JkYAB0AQorYD4w"); TopicIdPartition tp0 = new TopicIdPartition(topicId, 0, "test"); TopicIdPartition tp1 = new TopicIdPartition(topicId, 1, "test"); MemoryRecords records0 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, @@ -2368,7 +2368,8 @@ private ProduceRequest createProduceRequest(short version) { @SuppressWarnings("deprecation") private ProduceResponse createProduceResponse() { Map responseData = new HashMap<>(); - responseData.put(new TopicIdPartition(Uuid.randomUuid(), 0, "test"), new ProduceResponse.PartitionResponse(Errors.NONE, + Uuid topicId = Uuid.fromString("0AQorYD4we9TvBGX5JkYAB"); + responseData.put(new TopicIdPartition(topicId, 0, "test"), new ProduceResponse.PartitionResponse(Errors.NONE, 10000, RecordBatch.NO_TIMESTAMP, 100)); return new ProduceResponse(responseData, 0); } @@ -2376,7 +2377,8 @@ private ProduceResponse createProduceResponse() { @SuppressWarnings("deprecation") private ProduceResponse createProduceResponseWithErrorMessage() { Map responseData = new HashMap<>(); - responseData.put(new TopicIdPartition(Uuid.randomUuid(), 0, "test"), new ProduceResponse.PartitionResponse(Errors.NONE, + Uuid topicId = Uuid.fromString("0AQorYD4we9TvBGX5JkYAB"); + responseData.put(new TopicIdPartition(topicId, 0, "test"), new ProduceResponse.PartitionResponse(Errors.NONE, 10000, RecordBatch.NO_TIMESTAMP, 100, singletonList(new ProduceResponse.RecordError(0, "error message")), "global error message")); return new ProduceResponse(responseData, 0); diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 7995b772b35c9..485f867f09652 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -608,13 +608,12 @@ class KafkaApis(val requestChannel: RequestChannel, val nonExistingTopicResponses = mutable.Map[TopicIdPartition, PartitionResponse]() val invalidRequestResponses = mutable.Map[TopicIdPartition, PartitionResponse]() val authorizedRequestInfo = mutable.Map[TopicIdPartition, MemoryRecords]() - val partitionDatas = new mutable.ArrayBuffer[(TopicIdPartition, ProduceRequestData.PartitionProduceData)] + val topicIdToPartitionData = new mutable.ArrayBuffer[(TopicIdPartition, ProduceRequestData.PartitionProduceData)] produceRequest.data.topicData.forEach { topic => topic.partitionData.forEach { partition => val topicIdIsMissing = topic.topicId == null || topic.topicId == Uuid.ZERO_UUID - - val topicName: String = metadataCache.getTopicName(topic.topicId).getOrElse(topic.name) + val topicName: String = if (topicIdIsMissing) topic.name else metadataCache.getTopicName(topic.topicId).getOrElse(topic.name) val topicId: Uuid = if (topicIdIsMissing) metadataCache.getTopicId(topicName) else topic.topicId val topicPartition = new TopicPartition(topicName, partition.index()) @@ -623,13 +622,13 @@ class KafkaApis(val requestChannel: RequestChannel, else if (!metadataCache.contains(topicPartition)) nonExistingTopicResponses += new TopicIdPartition(topicId, topicPartition) -> new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION) else - partitionDatas += new TopicIdPartition(topicId, topicPartition) -> partition + topicIdToPartitionData += new TopicIdPartition(topicId, topicPartition) -> partition } } // cache the result to avoid redundant authorization calls - val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC, partitionDatas)(_._1.topic) + val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC, topicIdToPartitionData)(_._1.topic) - partitionDatas.foreach { case (topicIdPartition, partition) => + topicIdToPartitionData.foreach { case (topicIdPartition, partition) => // This caller assumes the type is MemoryRecords and that is true on current serialization // We cast the type to avoid causing big change to code base. // https://issues.apache.org/jira/browse/KAFKA-10698 diff --git a/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala index 4adb64f2e21da..64528a6196f1d 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala @@ -99,7 +99,7 @@ class CoordinatorPartitionWriterTest { Collections.emptyMap(), new Properties() ))) - val topicId = Uuid.randomUuid() + val topicId = Uuid.fromString("JkYAB4AQe905orYDTvBGXw") when(replicaManager.getTopicIdPartition(tp)).thenReturn(new TopicIdPartition(topicId, tp)) val recordsCapture: ArgumentCaptor[Map[TopicIdPartition, MemoryRecords]] = diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala index 06096de04ce8d..17f70acad4fa9 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala @@ -92,7 +92,7 @@ class GroupCoordinatorTest { private val protocolSuperset = List((protocolName, metadata), ("roundrobin", metadata)) private val requireStable = true private var groupPartitionId: Int = -1 - val groupMetadataTopicId = Uuid.randomUuid() + val groupMetadataTopicId = Uuid.fromString("JaTH2JYK2ed2GzUapg8tgg") // we use this string value since its hashcode % #.partitions is different private val otherGroupId = "otherGroup" diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index d41277943945e..859ca36c96080 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -2458,7 +2458,7 @@ class KafkaApisTest extends Logging { @Test def shouldReplaceProducerFencedWithInvalidProducerEpochInProduceResponse(): Unit = { val topic = "topic" - val topicId = Uuid.randomUuid() + val topicId = Uuid.fromString("d2Gg8tgzJa2JYK2eTHUapg") val tp = new TopicIdPartition(topicId, 0, "topic") addTopicToMetadataCache(topic, numPartitions = 2, topicId = topicId) @@ -2519,7 +2519,7 @@ class KafkaApisTest extends Logging { @Test def testProduceResponseContainsNewLeaderOnNotLeaderOrFollower(): Unit = { val topic = "topic" - val topicId = Uuid.randomUuid() + val topicId = Uuid.fromString("d2Gg8tgzJa2JYK2eTHUapg") addTopicToMetadataCache(topic, numPartitions = 2, numBrokers = 3, topicId = topicId) for (version <- 10 to ApiKeys.PRODUCE.latestVersion) { @@ -2590,7 +2590,7 @@ class KafkaApisTest extends Logging { @Test def testProduceResponseReplicaManagerLookupErrorOnNotLeaderOrFollower(): Unit = { val topic = "topic" - val topicId = Uuid.randomUuid() + val topicId = Uuid.fromString("d2Gg8tgzJa2JYK2eTHUapg") addTopicToMetadataCache(topic, numPartitions = 2, numBrokers = 3, topicId = topicId) for (version <- 10 to ApiKeys.PRODUCE.latestVersion) { @@ -2657,7 +2657,7 @@ class KafkaApisTest extends Logging { @Test def testProduceResponseMetadataLookupErrorOnNotLeaderOrFollower(): Unit = { val topic = "topic" - val topicId = Uuid.randomUuid() + val topicId = Uuid.fromString("d2Gg8tgzJa2JYK2eTHUapg") metadataCache = mock(classOf[ZkMetadataCache]) for (version <- 10 to ApiKeys.PRODUCE.latestVersion) { @@ -2669,10 +2669,11 @@ class KafkaApisTest extends Logging { val tp = new TopicIdPartition(topicId, 0, topic) val topicProduceData = new ProduceRequestData.TopicProduceData() - .setName(tp.topic) if (version >= 12 ) { topicProduceData.setTopicId(topicId) + } else { + topicProduceData.setName(tp.topic) } val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData() @@ -2707,8 +2708,8 @@ class KafkaApisTest extends Logging { when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) when(metadataCache.contains(tp.topicPartition())).thenAnswer(_ => true) - when(metadataCache.getTopicName(any())).thenReturn(Some(topicProduceData.name())) - when(metadataCache.getTopicId(any())).thenReturn(topicProduceData.topicId()) + when(metadataCache.getTopicName(tp.topicId())).thenReturn(Some(tp.topic())) + when(metadataCache.getTopicId(tp.topic())).thenReturn(tp.topicId()) when(metadataCache.getPartitionInfo(tp.topic(), tp.partition())).thenAnswer(_ => Option.empty) when(metadataCache.getAliveBrokerNode(any(), any())).thenReturn(Option.empty) kafkaApis = createKafkaApis() @@ -2732,7 +2733,8 @@ class KafkaApisTest extends Logging { val topic = "topic" val transactionalId = "txn1" - val tp = new TopicIdPartition(Uuid.randomUuid(), 0, "topic") + val topicId = Uuid.fromString("d2Gg8tgzJa2JYK2eTHUapg") + val tp = new TopicIdPartition(topicId, 0, "topic") addTopicToMetadataCache(topic, numPartitions = 2, topicId = tp.topicId()) for (version <- 3 to ApiKeys.PRODUCE.latestVersion) { @@ -2913,7 +2915,7 @@ class KafkaApisTest extends Logging { def shouldRespondWithUnsupportedMessageFormatForBadPartitionAndNoErrorsForGoodPartition(): Unit = { val tp1 = new TopicPartition("t", 0) val tp2 = new TopicPartition("t1", 0) - val topicId = Uuid.randomUuid() + val topicId = Uuid.fromString("d2Gg8tgzJa2JYK2eTHUapg") val (_, request) = createWriteTxnMarkersRequest(asList(tp1, tp2)) val expectedErrors = Map(tp1 -> Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT, tp2 -> Errors.NONE).asJava @@ -3046,7 +3048,7 @@ class KafkaApisTest extends Logging { def shouldRespondWithUnknownTopicOrPartitionForBadPartitionAndNoErrorsForGoodPartition(): Unit = { val tp1 = new TopicPartition("t", 0) val tp2 = new TopicPartition("t1", 0) - val topicId = Uuid.randomUuid() + val topicId = Uuid.fromString("d2Gg8tgzJa2JYK2eTHUapg") val (_, request) = createWriteTxnMarkersRequest(asList(tp1, tp2)) val expectedErrors = Map(tp1 -> Errors.UNKNOWN_TOPIC_OR_PARTITION, tp2 -> Errors.NONE).asJava @@ -3131,8 +3133,8 @@ class KafkaApisTest extends Logging { val foo1 = new TopicPartition("foo", 1) val topicIds = Map( - Topic.GROUP_METADATA_TOPIC_NAME -> Uuid.randomUuid(), - "foo" -> Uuid.randomUuid()) + Topic.GROUP_METADATA_TOPIC_NAME -> Uuid.fromString("JaTH2JYK2ed2GzUapg8tgg"), + "foo" -> Uuid.fromString("d2Gg8tgzJa2JYK2eTHUapg")) val allPartitions = List( offset0, offset1, diff --git a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala index ee8d3c6b6b643..891e708d1ab54 100644 --- a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala @@ -20,7 +20,7 @@ package kafka.server import java.nio.ByteBuffer import java.util.{Collections, Properties} import kafka.utils.TestUtils -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{TopicIdPartition, TopicPartition} import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.message.ProduceRequestData import org.apache.kafka.common.protocol.Errors @@ -91,6 +91,7 @@ class ProduceRequestTest extends BaseRequestTest { topicConfig.setProperty(messageTimeStampConfig, "1000") val partitionToLeader = TestUtils.createTopic(zkClient, topic, 1, 1, servers, topicConfig) val leader = partitionToLeader(partition) + val topicDescription = TestUtils.describeTopic(createAdminClient(), topic) def createRecords(magicValue: Byte, timestamp: Long, codec: CompressionType): MemoryRecords = { val buf = ByteBuffer.allocate(512) @@ -102,11 +103,12 @@ class ProduceRequestTest extends BaseRequestTest { } val records = createRecords(RecordBatch.MAGIC_VALUE_V2, recordTimestamp, CompressionType.GZIP) - val topicPartition = new TopicPartition("topic", partition) + val topicPartition = new TopicIdPartition(topicDescription.topicId(), partition, "topic") val produceResponse = sendProduceRequest(leader, ProduceRequest.forCurrentMagic(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( new ProduceRequestData.TopicProduceData() .setName(topicPartition.topic()) + .setTopicId(topicPartition.topicId()) .setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData() .setIndex(topicPartition.partition()) .setRecords(records)))).iterator)) @@ -118,7 +120,9 @@ class ProduceRequestTest extends BaseRequestTest { val topicProduceResponse = produceResponse.data.responses.asScala.head assertEquals(1, topicProduceResponse.partitionResponses.size) val partitionProduceResponse = topicProduceResponse.partitionResponses.asScala.head - val tp = new TopicPartition(topicProduceResponse.name, partitionProduceResponse.index) + val tp = new TopicIdPartition(topicProduceResponse.topicId(), + partitionProduceResponse.index, + getTopicNames().get(topicProduceResponse.topicId()).getOrElse("")) assertEquals(topicPartition, tp) assertEquals(Errors.INVALID_TIMESTAMP, Errors.forCode(partitionProduceResponse.errorCode)) // there are 3 records with InvalidTimestampException created from inner function createRecords diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 1c47d7b846d8b..708c5d3b2ae12 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -99,9 +99,9 @@ object ReplicaManagerTest { class ReplicaManagerTest { private val topic = "test-topic" - private val topicId = Uuid.fromString("YK2ed2GaTH2JpgzUaJ8tgg"); + private val topicId = Uuid.fromString("YK2ed2GaTH2JpgzUaJ8tgg") private val topicIds = scala.Predef.Map("test-topic" -> topicId) - private val topicNames = scala.Predef.Map(topicId -> "test-topic") + private val topicNames = topicIds.map(_.swap) private val transactionalId = "txn" private val time = new MockTime private val metrics = new Metrics @@ -141,7 +141,16 @@ class ReplicaManagerTest { callback(Map.empty[TopicPartition, Errors].toMap) } // make sure metadataCache can map between topic name and id - topicNames.foreach { case (topicId, topicName) => + setupMetadataCacheWithTopicIds(topicIds, metadataCache) + } + + private def setupMetadataCacheWithTopicIds(topicIds: Map[String, Uuid], metadataCache: MetadataCache): Unit = { + val topicNames = topicIds.map(_.swap) + when(metadataCache.topicNamesToIds()).thenReturn(topicIds.asJava) + when(metadataCache.topicIdsToNames()).thenReturn(topicNames.asJava) + when(metadataCache.topicIdInfo()).thenReturn((topicIds.asJava, topicNames.asJava)) + + topicIds.foreach { case (topicName, topicId) => when(metadataCache.getTopicId(topicName)).thenReturn(topicId) } } @@ -3034,10 +3043,6 @@ class ReplicaManagerTest { result.fire(response.get) } - topicNames.foreach { case (topicId, topicName) => - when(replicaManager.metadataCache.getTopicId(topicName)).thenReturn(topicId) - } - replicaManager.appendRecords( timeout = 1000, requiredAcks = requiredAcks, @@ -3053,7 +3058,6 @@ class ReplicaManagerTest { private def handleProduceAppendToMultipleTopics(replicaManager: ReplicaManager, entriesToAppend: Map[TopicPartition, MemoryRecords], transactionalId: String, - origin: AppendOrigin = AppendOrigin.CLIENT, requiredAcks: Short = -1): CallbackResult[Map[TopicIdPartition, PartitionResponse]] = { val result = new CallbackResult[Map[TopicIdPartition, PartitionResponse]]() def appendCallback(responses: Map[TopicIdPartition, PartitionResponse]): Unit = { @@ -3061,9 +3065,6 @@ class ReplicaManagerTest { result.fire(responses) } - topicNames.foreach { case (topicId, topicName) => - when(replicaManager.metadataCache.getTopicId(topicName)).thenReturn(topicId) - } replicaManager.handleProduceAppend( timeout = 1000, requiredAcks = requiredAcks, @@ -3092,10 +3093,6 @@ class ReplicaManagerTest { result.fire(response.get) } - topicNames.foreach { case (topicId, topicName) => - when(replicaManager.metadataCache.getTopicId(topicName)).thenReturn(topicId) - } - val entriesPerPartition = Map(partition -> records) replicaManager.handleProduceAppend( timeout = 1000, @@ -3333,9 +3330,6 @@ class ReplicaManagerTest { val aliveBrokers = aliveBrokerIds.map(brokerId => new Node(brokerId, s"host$brokerId", brokerId)) brokerTopicStats = new BrokerTopicStats(java.util.Optional.of(KafkaConfig.fromProps(props))) - when(metadataCache.topicIdInfo()).thenReturn((topicIds.asJava, topicNames.asJava)) - when(metadataCache.topicNamesToIds()).thenReturn(topicIds.asJava) - when(metadataCache.topicIdsToNames()).thenReturn(topicNames.asJava) when(metadataCache.metadataVersion()).thenReturn(config.interBrokerProtocolVersion) mockGetAliveBrokerFunctions(metadataCache, aliveBrokers) val mockProducePurgatory = new DelayedOperationPurgatory[DelayedProduce]( From 8c3602ba24c7ad95b058500f5c237a19c1ac3c67 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Thu, 16 May 2024 14:53:49 +0100 Subject: [PATCH 05/40] fix integration tests --- .../kafka/api/AuthorizerIntegrationTest.scala | 29 ++++++++++--------- .../kafka/server/EdgeCaseRequestTest.scala | 8 +++-- 2 files changed, 21 insertions(+), 16 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index f6bc848fc47e0..ad5c95d44f4fb 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -92,10 +92,11 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { val requestKeyToError = (topicNames: Map[Uuid, String], version: Short) => Map[ApiKeys, Nothing => Errors]( ApiKeys.METADATA -> ((resp: requests.MetadataResponse) => resp.errors.asScala.find(_._1 == topic).getOrElse(("test", Errors.NONE))._2), ApiKeys.PRODUCE -> ((resp: requests.ProduceResponse) => { - val topicId: Uuid = topicNames.find(topicName => topicName._2 == topic).map(_._1).getOrElse(Uuid.ZERO_UUID) + val topicId = topicNames.find(topicName => topicName._2 == topic).map(_._1).getOrElse(Uuid.ZERO_UUID) + val topicName = if (version >= 12) "" else topic Errors.forCode( resp.data - .responses.find(topic, topicId) + .responses.find(topicName, topicId) .partitionResponses.asScala.find(_.index == part).get .errorCode ) @@ -250,18 +251,20 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { new requests.MetadataRequest.Builder(List(topic).asJava, allowAutoTopicCreation).build() } - private def createProduceRequest = + private def createProduceRequestWithId(id: Uuid) = { requests.ProduceRequest.forCurrentMagic(new ProduceRequestData() - .setTopicData(new ProduceRequestData.TopicProduceDataCollection( - Collections.singletonList(new ProduceRequestData.TopicProduceData() - .setName(tp.topic).setPartitionData(Collections.singletonList( - new ProduceRequestData.PartitionProduceData() - .setIndex(tp.partition) - .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes)))))) - .iterator)) - .setAcks(1.toShort) - .setTimeoutMs(5000)) + .setTopicData(new ProduceRequestData.TopicProduceDataCollection( + Collections.singletonList(new ProduceRequestData.TopicProduceData() + .setName(tp.topic).setTopicId(id).setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData() + .setIndex(tp.partition) + .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes)))))) + .iterator)) + .setAcks(1.toShort) + .setTimeoutMs(5000)) .build() + } + private def createProduceRequest = createProduceRequestWithId(getTopicIds().getOrElse(tp.topic, Uuid.ZERO_UUID)) private def createFetchRequest = { val partitionMap = new util.LinkedHashMap[TopicPartition, requests.FetchRequest.PartitionData] @@ -708,7 +711,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { val topicNames = Map(id -> "topic") val requestKeyToRequest = mutable.LinkedHashMap[ApiKeys, AbstractRequest]( ApiKeys.METADATA -> createMetadataRequest(allowAutoTopicCreation = false), - ApiKeys.PRODUCE -> createProduceRequest, + ApiKeys.PRODUCE -> createProduceRequestWithId(id), ApiKeys.FETCH -> createFetchRequestWithUnknownTopic(id, ApiKeys.FETCH.latestVersion()), ApiKeys.LIST_OFFSETS -> createListOffsetsRequest, ApiKeys.OFFSET_COMMIT -> createOffsetCommitRequest, diff --git a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala index a018b5d6af48a..129c199e2faf2 100755 --- a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala @@ -21,7 +21,6 @@ import java.io.{DataInputStream, DataOutputStream} import java.net.Socket import java.nio.ByteBuffer import java.util.Collections - import kafka.integration.KafkaServerTestHarness import kafka.network.SocketServer import kafka.utils._ @@ -33,7 +32,7 @@ import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRec import org.apache.kafka.common.requests.{ProduceResponse, ResponseHeader} import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.ByteUtils -import org.apache.kafka.common.{TopicPartition, requests} +import org.apache.kafka.common.{TopicPartition, Uuid, requests} import org.apache.kafka.server.config.ServerLogConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest @@ -129,10 +128,13 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness { val version = ApiKeys.PRODUCE.latestVersion: Short val (serializedBytes, responseHeaderVersion) = { val headerBytes = requestHeaderBytes(ApiKeys.PRODUCE.id, version, "", correlationId) + val topicId = getTopicIds().getOrElse(topicPartition.topic(), Uuid.ZERO_UUID) val request = requests.ProduceRequest.forCurrentMagic(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection( Collections.singletonList(new ProduceRequestData.TopicProduceData() - .setName(topicPartition.topic()).setPartitionData(Collections.singletonList( + .setName(topicPartition.topic()) + .setTopicId(topicId) + .setPartitionData(Collections.singletonList( new ProduceRequestData.PartitionProduceData() .setIndex(topicPartition.partition()) .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("message".getBytes)))))) From 27ed97b53f03e6ac38760f062968ad5053778412 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Wed, 22 May 2024 16:09:35 +0100 Subject: [PATCH 06/40] address part of the feedback --- .../kafka/clients/producer/internals/Sender.java | 14 +++++++++----- core/src/main/scala/kafka/server/KafkaApis.scala | 8 +++++--- 2 files changed, 14 insertions(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 166bc26d2e3eb..2a8e1bc8df8a9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -611,6 +611,7 @@ private void handleProduceResponse(ClientResponse response, Map partitionsWithUpdatedLeaderInfo = new HashMap<>(); produceResponse.data().responses().forEach(r -> r.partitionResponses().forEach(p -> { + // Version 12 drop topic name and add support to topic id. However, metadata can be used to map topic id to topic name. String topicName = (r.name() == null || r.name().isEmpty()) ? metadata.topicNames().get(r.topicId()) : r.name(); TopicPartition tp = new TopicPartition(topicName, p.index()); @@ -897,15 +898,18 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo // which is supporting the new magic version to one which doesn't, then we will need to convert. if (!records.hasMatchingMagic(minUsedMagic)) records = batch.records().downConvert(minUsedMagic, 0, time).records(); - Optional topicProduceData = canUseTopicId ? - Optional.ofNullable(tpd.find(tp.topic(), topicIds.get(tp.topic()))) : - tpd.stream().filter(data -> data.name().equals(tp.topic())).findFirst(); + ProduceRequestData.TopicProduceData tpData = canUseTopicId ? + tpd.find(tp.topic(), topicIds.get(tp.topic())) : + tpd.find(new ProduceRequestData.TopicProduceData().setName(tp.topic())); - ProduceRequestData.TopicProduceData tpData = topicProduceData.orElse(new ProduceRequestData.TopicProduceData().setName(tp.topic())); + if (tpData == null) { + tpData = new ProduceRequestData.TopicProduceData().setName(tp.topic()); + tpd.add(tpData); + } if (canUseTopicId) { tpData.setTopicId(topicIds.get(tp.topic())); } - tpd.add(tpData); + tpData.partitionData().add(new ProduceRequestData.PartitionProduceData() .setIndex(tp.partition()) .setRecords(records)); diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 0dd928607bf3a..848ea13ad9822 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -612,9 +612,11 @@ class KafkaApis(val requestChannel: RequestChannel, produceRequest.data.topicData.forEach { topic => topic.partitionData.forEach { partition => - val topicIdIsMissing = topic.topicId == null || topic.topicId == Uuid.ZERO_UUID - val topicName: String = if (topicIdIsMissing) topic.name else metadataCache.getTopicName(topic.topicId).getOrElse(topic.name) - val topicId: Uuid = if (topicIdIsMissing) metadataCache.getTopicId(topicName) else topic.topicId + val (topicName, topicId) = if (produceRequest.version() >= 12) { + (metadataCache.getTopicName(topic.topicId).getOrElse(topic.name), topic.topicId()) + } else { + (topic.name(), metadataCache.getTopicId(topic.name())) + } val topicPartition = new TopicPartition(topicName, partition.index()) if (topicName == null || topicName.isEmpty) From 35dba4bd550dd269a03f5b1ce1d649346a481512 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Tue, 28 May 2024 16:24:43 +0100 Subject: [PATCH 07/40] Add testing for while recreate the topic and reassignment --- .../api/ProducerSendWhileDeletionTest.scala | 98 ++++++++++++++++++- 1 file changed, 94 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala index 365a872d01b9b..8808be62044cb 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala @@ -17,11 +17,11 @@ package kafka.api import kafka.utils.TestUtils -import org.apache.kafka.clients.admin.NewPartitionReassignment -import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} +import org.apache.kafka.clients.admin.{Admin, NewPartitionReassignment, TopicDescription} +import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord, RecordMetadata} import org.apache.kafka.common.TopicPartition import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} -import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -33,7 +33,7 @@ import scala.jdk.CollectionConverters._ class ProducerSendWhileDeletionTest extends IntegrationTestHarness { val producerCount: Int = 1 - val brokerCount: Int = 2 + val brokerCount: Int = 3 serverConfig.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, 2.toString) serverConfig.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, 2.toString) @@ -84,4 +84,94 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness { assertEquals(topic, producer.send(new ProducerRecord(topic, null, "value".getBytes(StandardCharsets.UTF_8))).get.topic()) } + /** + * Tests that Producer produce to new topic id after recreation. + * + * Producer will attempt to send messages to the partition specified in each record, and should + * succeed as long as the metadata has been updated with new topic id. + */ + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testSendWithRecreatedTopic(quorum: String): Unit = { + val numRecords = 10 + val topic = "topic" + createTopic(topic) + val admin = createAdminClient() + val topicId = getTopicMetadata(admin, topic).topicId() + val producer = createProducer() + + (1 to numRecords).foreach { i => + val resp = producer.send(new ProducerRecord(topic, null, ("value" + i).getBytes(StandardCharsets.UTF_8))).get + assertEquals(topic, resp.topic()) + } + // Start topic deletion + deleteTopic(topic, listenerName) + + // Verify that the topic is deleted when no metadata request comes in + TestUtils.verifyTopicDeletion(zkClientOrNull, topic, 2, brokers) + createTopic(topic) + assertNotEquals(topicId, getTopicMetadata(admin, topic).topicId()) + + // Producer should be able to send messages even after topic gets recreated + val recordMetadata: RecordMetadata = producer.send(new ProducerRecord(topic, null, "value".getBytes(StandardCharsets.UTF_8))).get + assertEquals(topic, recordMetadata.topic()) + assertEquals(0, recordMetadata.offset()) + } + + /** + * Tests that Producer produce to topic during reassignment where topic metadata change on broker side. + * + * Producer will attempt to send messages to the partition specified in each record, and should + * succeed as long as the metadata on the leader has been updated with new topic id. + */ + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testSendWithTopicReassignmentIsMidWay(quorum: String): Unit = { + val numRecords = 10 + val topic = "topic" + val partition0: TopicPartition = new TopicPartition(topic, 0) + val partition1 = new TopicPartition(topic, 1) + val admin: Admin = createAdminClient() + + // Create topic with leader as 0 for the 2 partitions. + createTopicWithAssignment(topic, Map(0 -> Seq(0, 1), 1 -> Seq(0, 1))) + TestUtils.assertLeader(admin, partition1, 0) + + val topicDetails = getTopicMetadata(admin, topic) + assertEquals(0, topicDetails.partitions().get(0).leader().id()) + val producer = createProducer() + + (1 to numRecords).foreach { i => + val resp = producer.send(new ProducerRecord(topic, null, ("value" + i).getBytes(StandardCharsets.UTF_8))).get + assertEquals(topic, resp.topic()) + } + + val reassignment = Map( + partition0 -> Optional.of(new NewPartitionReassignment(util.Arrays.asList(2, 1))), + partition1 -> Optional.of(new NewPartitionReassignment(util.Arrays.asList(2, 1))) + ) + + // Change assignment of one of the replicas from 0 to 2 + admin.alterPartitionReassignments(reassignment.asJava).all().get() + + TestUtils.waitUntilTrue( + () => partitionLeader(admin, partition0) == 2 && partitionLeader(admin, partition1) == 2, + s"Expected preferred leader to become 2, but is ${partitionLeader(admin, partition0)} and ${partitionLeader(admin, partition1)}", + 10000) + TestUtils.assertLeader(admin, partition1, 2) + assertEquals(topicDetails.topicId(), getTopicMetadata(admin, topic).topicId()) + + // Producer should be able to send messages even after topic gets reassigned + assertEquals(topic, producer.send(new ProducerRecord(topic, null, "value".getBytes(StandardCharsets.UTF_8))).get.topic()) + } + + def getTopicMetadata(admin: Admin, topic: String): TopicDescription = { + admin.describeTopics(util.Collections.singletonList(topic)).allTopicNames().get().get(topic) + } + + def partitionLeader(admin: Admin, topicPartition: TopicPartition): Int = { + val partitionMetadata = getTopicMetadata(admin, topicPartition.topic).partitions.get(topicPartition.partition) + val preferredLeaderMetadata = partitionMetadata.leader() + preferredLeaderMetadata.id + } } From 916e471295375182f753419e38dcb86de21db632 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Tue, 11 Jun 2024 22:17:14 +0100 Subject: [PATCH 08/40] Fetch supported produce version from apiVersions. --- .../org/apache/kafka/clients/ApiVersions.java | 14 +++++++++++ .../clients/producer/internals/Sender.java | 8 +++++-- .../producer/internals/SenderTest.java | 23 +++++++++++++++---- 3 files changed, 39 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ApiVersions.java b/clients/src/main/java/org/apache/kafka/clients/ApiVersions.java index a09d58166b369..8b78936422b6c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ApiVersions.java +++ b/clients/src/main/java/org/apache/kafka/clients/ApiVersions.java @@ -34,10 +34,21 @@ public class ApiVersions { private final Map nodeApiVersions = new HashMap<>(); private byte maxUsableProduceMagic = RecordBatch.CURRENT_MAGIC_VALUE; + private short maxProduceSupportedVersion = ApiKeys.PRODUCE.latestVersion(); public synchronized void update(String nodeId, NodeApiVersions nodeApiVersions) { this.nodeApiVersions.put(nodeId, nodeApiVersions); this.maxUsableProduceMagic = computeMaxUsableProduceMagic(); + this.maxProduceSupportedVersion = computeMaxProduceSupportedVersion(); + } + + private short computeMaxProduceSupportedVersion() { + Optional knownBrokerNodesMinSupportedVersionForProduce = this.nodeApiVersions.values().stream() + .filter(versions -> versions.apiVersion(ApiKeys.PRODUCE) != null) // filter out Raft controller nodes + .map(versions -> versions.latestUsableVersion(ApiKeys.PRODUCE)) + .min(Short::compare); + return (short) Math.min(ApiKeys.PRODUCE.latestVersion(), + knownBrokerNodesMinSupportedVersionForProduce.orElse(ApiKeys.PRODUCE.latestVersion())); } public synchronized void remove(String nodeId) { @@ -63,5 +74,8 @@ private byte computeMaxUsableProduceMagic() { public synchronized byte maxUsableProduceMagic() { return maxUsableProduceMagic; } + public synchronized short getMaxSupportedProduceVersion() { + return maxProduceSupportedVersion; + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 2a8e1bc8df8a9..88a5cc42059e4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -27,6 +27,7 @@ import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MetadataSnapshot; import org.apache.kafka.clients.NetworkClientUtils; +import org.apache.kafka.clients.NodeApiVersions; import org.apache.kafka.clients.RequestCompletionHandler; import org.apache.kafka.common.InvalidRecordException; import org.apache.kafka.common.KafkaException; @@ -50,6 +51,7 @@ import org.apache.kafka.common.metrics.stats.Avg; import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.metrics.stats.Meter; +import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.RecordBatch; @@ -882,7 +884,7 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo minUsedMagic = batch.magic(); } Map topicIds = getTopicIdsFromBatches(batches); - boolean canUseTopicId = !topicIds.entrySet().stream().anyMatch(e -> e.getValue() == Uuid.ZERO_UUID); + boolean canUseTopicId = apiVersions.getMaxSupportedProduceVersion() >= 12; ProduceRequestData.TopicProduceDataCollection tpd = new ProduceRequestData.TopicProduceDataCollection(); for (ProducerBatch batch : batches) { @@ -903,11 +905,13 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo tpd.find(new ProduceRequestData.TopicProduceData().setName(tp.topic())); if (tpData == null) { - tpData = new ProduceRequestData.TopicProduceData().setName(tp.topic()); + tpData = new ProduceRequestData.TopicProduceData(); tpd.add(tpData); } if (canUseTopicId) { tpData.setTopicId(topicIds.get(tp.topic())); + } else { + tpData.setName(tp.topic()); } tpData.partitionData().add(new ProduceRequestData.PartitionProduceData() 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 9c7adb34417ed..c012dd15ef31f 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 @@ -175,6 +175,7 @@ public class SenderTest { @BeforeEach public void setup() { setupWithTransactionState(null); + apiVersions.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 12)); } @AfterEach @@ -582,8 +583,11 @@ public void testNodeLatencyStats() throws Exception { new BufferPool(totalSize, batchSize, m, time, "producer-internal-metrics")); SenderMetricsRegistry senderMetrics = new SenderMetricsRegistry(m); + ApiVersions apiVersions1 = new ApiVersions(); + apiVersions1.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 12)); + Sender sender = new Sender(logContext, client, metadata, this.accumulator, false, MAX_REQUEST_SIZE, ACKS_ALL, 1, - senderMetrics, time, REQUEST_TIMEOUT, 1000L, null, new ApiVersions()); + senderMetrics, time, REQUEST_TIMEOUT, 1000L, null, apiVersions1); // Produce and send batch. long time1 = time.milliseconds(); @@ -2407,6 +2411,7 @@ public void testTransactionalSplitBatchAndSend() throws Exception { txnManager.beginTransaction(); txnManager.maybeAddPartition(tp); + apiVersions.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 12)); client.prepareResponse(buildAddPartitionsToTxnResponseData(0, Collections.singletonMap(tp, Errors.NONE))); sender.runOnce(); @@ -2424,16 +2429,21 @@ private void testSplitBatchAndSend(TransactionManager txnManager, String metricGrpName = "producer-metrics"; // Set a good compression ratio. CompressionRatioEstimator.setEstimation(topic, CompressionType.GZIP, 0.2f); + ApiVersions apiVersions = new ApiVersions(); try (Metrics m = new Metrics()) { accumulator = new RecordAccumulator(logContext, batchSize, Compression.gzip().build(), - 0, 0L, 0L, deliveryTimeoutMs, m, metricGrpName, time, new ApiVersions(), txnManager, + 0, 0L, 0L, deliveryTimeoutMs, m, metricGrpName, time, apiVersions, txnManager, new BufferPool(totalSize, batchSize, metrics, time, "producer-internal-metrics")); SenderMetricsRegistry senderMetrics = new SenderMetricsRegistry(m); Sender sender = new Sender(logContext, client, metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries, - senderMetrics, time, REQUEST_TIMEOUT, 1000L, txnManager, new ApiVersions()); + senderMetrics, time, REQUEST_TIMEOUT, 1000L, txnManager, apiVersions); // Create a two broker cluster, with partition 0 on broker 0 and partition 1 on broker 1 MetadataResponse metadataUpdate1 = RequestTestUtils.metadataUpdateWith(2, Collections.singletonMap(topic, 2)); client.prepareMetadataUpdate(metadataUpdate1); + metadataUpdate1.brokers().stream().forEach(node -> + apiVersions.update(node.idString(), NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 12)) + ); + // Send the first message. long nowMs = time.milliseconds(); Cluster cluster = TestUtils.singletonCluster(); @@ -3426,6 +3436,9 @@ public void testWhenProduceResponseReturnsWithALeaderShipChangeErrorAndNewLeader } })); Cluster startingMetadataCluster = metadata.fetch(); + startingMetadataCluster.nodes().forEach(node -> + apiVersions.update(node.idString(), NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 12)) + ); // Produce to tp0/1/2, where NO_LEADER_OR_FOLLOWER with new leader info is returned for tp0/1, and tp2 is returned without errors. Future futureIsProducedTp0 = appendToAccumulator(tp0, 0L, "key", "value"); @@ -3446,7 +3459,9 @@ public void testWhenProduceResponseReturnsWithALeaderShipChangeErrorAndNewLeader responses.put(tp0, new OffsetAndError(-1, Errors.NOT_LEADER_OR_FOLLOWER)); responses.put(tp1, new OffsetAndError(-1, Errors.NOT_LEADER_OR_FOLLOWER)); responses.put(tp2, new OffsetAndError(100, Errors.NONE)); - + newNodes.forEach(node -> + apiVersions.update(node.idString(), NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 12)) + ); Map partitionLeaderInfo = new HashMap<>(); ProduceResponseData.LeaderIdAndEpoch tp0LeaderInfo = new ProduceResponseData.LeaderIdAndEpoch(); tp0LeaderInfo.setLeaderEpoch(tp0LeaderEpoch + 1); From c59b10c9a309011a8ed66b73b3981edd86f4ffaf Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Tue, 11 Jun 2024 22:24:39 +0100 Subject: [PATCH 09/40] Fetch supported produce version from apiVersions. --- .../java/org/apache/kafka/clients/ApiVersions.java | 11 ++++++----- .../kafka/clients/producer/internals/Sender.java | 4 +--- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ApiVersions.java b/clients/src/main/java/org/apache/kafka/clients/ApiVersions.java index 8b78936422b6c..2ca095153c157 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ApiVersions.java +++ b/clients/src/main/java/org/apache/kafka/clients/ApiVersions.java @@ -34,15 +34,15 @@ public class ApiVersions { private final Map nodeApiVersions = new HashMap<>(); private byte maxUsableProduceMagic = RecordBatch.CURRENT_MAGIC_VALUE; - private short maxProduceSupportedVersion = ApiKeys.PRODUCE.latestVersion(); + private short maxSupportedProduceVersion = ApiKeys.PRODUCE.latestVersion(); public synchronized void update(String nodeId, NodeApiVersions nodeApiVersions) { this.nodeApiVersions.put(nodeId, nodeApiVersions); this.maxUsableProduceMagic = computeMaxUsableProduceMagic(); - this.maxProduceSupportedVersion = computeMaxProduceSupportedVersion(); + this.maxSupportedProduceVersion = computeMaxSupportedProduceVersion(); } - private short computeMaxProduceSupportedVersion() { + private short computeMaxSupportedProduceVersion() { Optional knownBrokerNodesMinSupportedVersionForProduce = this.nodeApiVersions.values().stream() .filter(versions -> versions.apiVersion(ApiKeys.PRODUCE) != null) // filter out Raft controller nodes .map(versions -> versions.latestUsableVersion(ApiKeys.PRODUCE)) @@ -54,6 +54,7 @@ private short computeMaxProduceSupportedVersion() { public synchronized void remove(String nodeId) { this.nodeApiVersions.remove(nodeId); this.maxUsableProduceMagic = computeMaxUsableProduceMagic(); + this.maxSupportedProduceVersion = computeMaxSupportedProduceVersion(); } public synchronized NodeApiVersions get(String nodeId) { @@ -74,8 +75,8 @@ private byte computeMaxUsableProduceMagic() { public synchronized byte maxUsableProduceMagic() { return maxUsableProduceMagic; } - public synchronized short getMaxSupportedProduceVersion() { - return maxProduceSupportedVersion; + public synchronized short maxSupportedProduceVersion() { + return maxSupportedProduceVersion; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 88a5cc42059e4..5929e683c33b3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -27,7 +27,6 @@ import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MetadataSnapshot; import org.apache.kafka.clients.NetworkClientUtils; -import org.apache.kafka.clients.NodeApiVersions; import org.apache.kafka.clients.RequestCompletionHandler; import org.apache.kafka.common.InvalidRecordException; import org.apache.kafka.common.KafkaException; @@ -51,7 +50,6 @@ import org.apache.kafka.common.metrics.stats.Avg; import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.metrics.stats.Meter; -import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.RecordBatch; @@ -884,7 +882,7 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo minUsedMagic = batch.magic(); } Map topicIds = getTopicIdsFromBatches(batches); - boolean canUseTopicId = apiVersions.getMaxSupportedProduceVersion() >= 12; + boolean canUseTopicId = apiVersions.maxSupportedProduceVersion() >= 12; ProduceRequestData.TopicProduceDataCollection tpd = new ProduceRequestData.TopicProduceDataCollection(); for (ProducerBatch batch : batches) { From 646839dd72c98eec74122adef9f7ddf9cc1cf063 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Wed, 12 Jun 2024 12:04:33 +0100 Subject: [PATCH 10/40] fix --- .../clients/producer/internals/Sender.java | 4 +- .../producer/internals/SenderTest.java | 48 ++++++++++++------- .../group/GroupMetadataManagerTest.scala | 6 ++- 3 files changed, 38 insertions(+), 20 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 5929e683c33b3..c8b26a0bb5ad5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -882,7 +882,9 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo minUsedMagic = batch.magic(); } Map topicIds = getTopicIdsFromBatches(batches); - boolean canUseTopicId = apiVersions.maxSupportedProduceVersion() >= 12; + // Use topic id if the max supported producer request version >= 12 and metadata has topic ids for all topics + // Otherwise send the request with topic name and the broker + boolean canUseTopicId = apiVersions.maxSupportedProduceVersion() >= 12 && topicIds.values().stream().anyMatch(id -> id != Uuid.ZERO_UUID); ProduceRequestData.TopicProduceDataCollection tpd = new ProduceRequestData.TopicProduceDataCollection(); for (ProducerBatch batch : batches) { 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 c012dd15ef31f..f519b4fb97dad 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 @@ -154,12 +154,10 @@ public class SenderTest { private static final String TOPIC_NAME = "test"; private static final Uuid TOPIC_ID = Uuid.fromString("MKXx1fIkQy2J9jXHhK8m1w"); + private static final Map TOPIC_IDS = Collections.singletonMap(TOPIC_NAME, TOPIC_ID); private TopicPartition tp0 = new TopicPartition(TOPIC_NAME, 0); private TopicPartition tp1 = new TopicPartition(TOPIC_NAME, 1); private TopicPartition tp2 = new TopicPartition(TOPIC_NAME, 2); - private TopicIdPartition topicIdPartition0 = new TopicIdPartition(TOPIC_ID, tp0); - private TopicIdPartition topicIdPartition1 = new TopicIdPartition(TOPIC_ID, tp0); - private TopicIdPartition topicIdPartition2 = new TopicIdPartition(TOPIC_ID, tp0); private MockTime time = new MockTime(); private final int batchSize = 16 * 1024; private final ProducerMetadata metadata = new ProducerMetadata(0, 0, Long.MAX_VALUE, TOPIC_IDLE_MS, @@ -176,6 +174,10 @@ public class SenderTest { public void setup() { setupWithTransactionState(null); apiVersions.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 12)); + this.client.updateMetadata( + RequestTestUtils.metadataUpdateWithIds(1, + Collections.singletonMap(TOPIC_NAME, 3), + TOPIC_IDS)); } @AfterEach @@ -186,7 +188,13 @@ public void tearDown() { private static Map partitionRecords(ProduceRequest request) { Map partitionRecords = new HashMap<>(); request.data().topicData().forEach(tpData -> tpData.partitionData().forEach(p -> { - TopicPartition tp = new TopicPartition(tpData.name(), p.index()); + String topicName = tpData.name(); + + if (request.version() >= 12) { + topicName = TOPIC_IDS.entrySet().stream().filter(e -> e.getValue() == tpData.topicId()).map(Map.Entry::getKey).findFirst().get(); + } + + TopicPartition tp = new TopicPartition(topicName, p.index()); partitionRecords.put(tp, (MemoryRecords) p.records()); })); return Collections.unmodifiableMap(partitionRecords); @@ -442,7 +450,7 @@ public void testSendInOrder() throws Exception { Sender sender = new Sender(logContext, client, metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries, senderMetrics, time, REQUEST_TIMEOUT, RETRY_BACKOFF_MS, null, apiVersions); // Create a two broker cluster, with partition 0 on broker 0 and partition 1 on broker 1 - MetadataResponse metadataUpdate1 = RequestTestUtils.metadataUpdateWith(2, Collections.singletonMap(TOPIC_NAME, 2)); + MetadataResponse metadataUpdate1 = RequestTestUtils.metadataUpdateWithIds(2, Collections.singletonMap(TOPIC_NAME, 2), TOPIC_IDS); client.prepareMetadataUpdate(metadataUpdate1); // Send the first message. @@ -463,7 +471,7 @@ public void testSendInOrder() throws Exception { appendToAccumulator(tp2, 0L, "key2", "value2"); // Update metadata before sender receives response from broker 0. Now partition 2 moves to broker 0 - MetadataResponse metadataUpdate2 = RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(TOPIC_NAME, 2)); + MetadataResponse metadataUpdate2 = RequestTestUtils.metadataUpdateWithIds(1, Collections.singletonMap(TOPIC_NAME, 2), TOPIC_IDS); client.prepareMetadataUpdate(metadataUpdate2); // Sender should not send the second message to node 0. assertEquals(1, sender.inFlightBatches(tp2).size()); @@ -538,12 +546,12 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { @Test public void testMetadataTopicExpiry() throws Exception { long offset = 0; - client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(TOPIC_NAME, 2))); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, Collections.singletonMap(TOPIC_NAME, 2), TOPIC_IDS)); Future future = appendToAccumulator(tp0); sender.runOnce(); assertTrue(metadata.containsTopic(tp0.topic()), "Topic not added to metadata"); - client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(TOPIC_NAME, 2))); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, Collections.singletonMap(TOPIC_NAME, 2), TOPIC_IDS)); sender.runOnce(); // send produce request client.respond(produceResponse(tp0, offset, Errors.NONE, 0)); sender.runOnce(); @@ -555,12 +563,12 @@ public void testMetadataTopicExpiry() throws Exception { assertTrue(metadata.containsTopic(tp0.topic()), "Topic not retained in metadata list"); time.sleep(TOPIC_IDLE_MS); - client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(TOPIC_NAME, 2))); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, Collections.singletonMap(TOPIC_NAME, 2), TOPIC_IDS)); assertFalse(metadata.containsTopic(tp0.topic()), "Unused topic has not been expired"); future = appendToAccumulator(tp0); sender.runOnce(); assertTrue(metadata.containsTopic(tp0.topic()), "Topic not added to metadata"); - client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(TOPIC_NAME, 2))); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, Collections.singletonMap(TOPIC_NAME, 2), TOPIC_IDS)); sender.runOnce(); // send produce request client.respond(produceResponse(tp0, offset + 1, Errors.NONE, 0)); sender.runOnce(); @@ -676,7 +684,7 @@ public void testInitProducerIdWithMaxInFlightOne() { // Process metadata response, prepare FindCoordinator and InitProducerId responses. // Verify producerId after the sender is run to process responses. - MetadataResponse metadataUpdate = RequestTestUtils.metadataUpdateWith(1, Collections.emptyMap()); + MetadataResponse metadataUpdate = RequestTestUtils.metadataUpdateWithIds(1, Collections.emptyMap(), Collections.emptyMap()); client.respond(metadataUpdate); prepareFindCoordinatorResponse(Errors.NONE, "testInitProducerIdWithPendingMetadataRequest"); prepareInitProducerResponse(Errors.NONE, producerIdAndEpoch.producerId, producerIdAndEpoch.epoch); @@ -701,7 +709,7 @@ public void testIdempotentInitProducerIdWithMaxInFlightOne() { // Process metadata and InitProducerId responses. // Verify producerId after the sender is run to process responses. - MetadataResponse metadataUpdate = RequestTestUtils.metadataUpdateWith(1, Collections.emptyMap()); + MetadataResponse metadataUpdate = RequestTestUtils.metadataUpdateWithIds(1, Collections.emptyMap(), Collections.emptyMap()); client.respond(metadataUpdate); sender.runOnce(); sender.runOnce(); @@ -2438,7 +2446,7 @@ private void testSplitBatchAndSend(TransactionManager txnManager, Sender sender = new Sender(logContext, client, metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries, senderMetrics, time, REQUEST_TIMEOUT, 1000L, txnManager, apiVersions); // Create a two broker cluster, with partition 0 on broker 0 and partition 1 on broker 1 - MetadataResponse metadataUpdate1 = RequestTestUtils.metadataUpdateWith(2, Collections.singletonMap(topic, 2)); + MetadataResponse metadataUpdate1 = RequestTestUtils.metadataUpdateWithIds(2, Collections.singletonMap(topic, 2), TOPIC_IDS); client.prepareMetadataUpdate(metadataUpdate1); metadataUpdate1.brokers().stream().forEach(node -> apiVersions.update(node.idString(), NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 12)) @@ -3228,7 +3236,8 @@ public void testProducerBatchRetriesWhenPartitionLeaderChanges() throws Exceptio int tp0LeaderEpoch = 100; int epoch = tp0LeaderEpoch; this.client.updateMetadata( - RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(TOPIC_NAME, 2), + RequestTestUtils.metadataUpdateWithIds(1, new HashSet<>(Arrays.asList(new TopicIdPartition(TOPIC_ID, tp0), + new TopicIdPartition(TOPIC_ID, tp1))), tp -> { if (tp0.equals(tp)) { return epoch; @@ -3255,7 +3264,8 @@ public void testProducerBatchRetriesWhenPartitionLeaderChanges() throws Exceptio // Update leader epoch for tp0 int newEpoch = ++tp0LeaderEpoch; this.client.updateMetadata( - RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(TOPIC_NAME, 2), + RequestTestUtils.metadataUpdateWithIds(1, new HashSet<>(Arrays.asList(new TopicIdPartition(TOPIC_ID, tp0), + new TopicIdPartition(TOPIC_ID, tp1))), tp -> { if (tp0.equals(tp)) { return newEpoch; @@ -3343,7 +3353,8 @@ public void testWhenProduceResponseReturnsWithALeaderShipChangeErrorButNoNewLead int tp1LeaderEpoch = 200; int tp2LeaderEpoch = 300; this.client.updateMetadata( - RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(TOPIC_NAME, 3), + RequestTestUtils.metadataUpdateWithIds(1, new HashSet<>(Arrays.asList(new TopicIdPartition(TOPIC_ID, tp0), + new TopicIdPartition(TOPIC_ID, tp1), new TopicIdPartition(TOPIC_ID, tp2))), tp -> { if (tp0.equals(tp)) { return tp0LeaderEpoch; @@ -3423,7 +3434,8 @@ public void testWhenProduceResponseReturnsWithALeaderShipChangeErrorAndNewLeader int tp1LeaderEpoch = 200; int tp2LeaderEpoch = 300; this.client.updateMetadata( - RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(TOPIC_NAME, 3), + RequestTestUtils.metadataUpdateWithIds(1, new HashSet<>(Arrays.asList(new TopicIdPartition(TOPIC_ID, tp0), + new TopicIdPartition(TOPIC_ID, tp1), new TopicIdPartition(TOPIC_ID, tp2))), tp -> { if (tp0.equals(tp)) { return tp0LeaderEpoch; @@ -3727,7 +3739,7 @@ private void setupWithTransactionState( metadata.add(TOPIC_NAME, time.milliseconds()); if (updateMetadata) - this.client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(TOPIC_NAME, 2))); + this.client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(1, Collections.singletonMap(TOPIC_NAME, 2), TOPIC_IDS)); } private void assertSuccessfulSend() throws InterruptedException { diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala index b1a002d162448..6fb4df4f5f467 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala @@ -1692,7 +1692,7 @@ class GroupMetadataManagerTest { val topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), 0, "foo") val validTopicIdPartition = new TopicIdPartition(topicIdPartition.topicId, 1, "foo") val offset = 37 - val requireStable = true; + val requireStable = true groupMetadataManager.addOwnedPartition(groupPartitionId) val group = new GroupMetadata(groupId, Empty, time) @@ -1704,6 +1704,8 @@ class GroupMetadataManagerTest { validTopicIdPartition -> OffsetAndMetadata(offset, "", time.milliseconds()) ) + when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + expectAppendMessage(Errors.NONE) var commitErrors: Option[immutable.Map[TopicIdPartition, Errors]] = None @@ -1764,6 +1766,8 @@ class GroupMetadataManagerTest { val capturedResponseCallback: ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, PartitionResponse] => Unit]) when(replicaManager.getMagic(any())).thenReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE)) + when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + var commitErrors: Option[immutable.Map[TopicIdPartition, Errors]] = None def callback(errors: immutable.Map[TopicIdPartition, Errors]): Unit = { From 7ed70b1177985ea67e571dc789c24f2634487aeb Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Tue, 18 Jun 2024 16:25:50 +0100 Subject: [PATCH 11/40] address feedback --- .../clients/producer/internals/Sender.java | 14 ++++--- .../kafka/common/requests/ProduceRequest.java | 2 +- .../producer/internals/SenderTest.java | 6 ++- .../kafka/common/message/MessageTest.java | 1 - .../common/requests/ProduceRequestTest.java | 9 ++--- .../common/requests/RequestResponseTest.java | 38 +++++++++++-------- .../main/scala/kafka/server/KafkaApis.scala | 2 +- 7 files changed, 41 insertions(+), 31 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 9b36f3d1407eb..c0672ed8d7e18 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -613,7 +613,7 @@ private void handleProduceResponse(ClientResponse response, Map r.partitionResponses().forEach(p -> { // Version 12 drop topic name and add support to topic id. However, metadata can be used to map topic id to topic name. - String topicName = (r.name() == null || r.name().isEmpty()) ? metadata.topicNames().get(r.topicId()) : r.name(); + String topicName = metadata.topicNames().getOrDefault(r.topicId(), r.name()); TopicPartition tp = new TopicPartition(topicName, p.index()); ProduceResponse.PartitionResponse partResp = new ProduceResponse.PartitionResponse( Errors.forCode(p.errorCode()), @@ -906,13 +906,15 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo if (tpData == null) { tpData = new ProduceRequestData.TopicProduceData(); + + if (canUseTopicId) { + tpData.setTopicId(topicIds.get(tp.topic())); + } else { + tpData.setName(tp.topic()); + } + tpd.add(tpData); } - if (canUseTopicId) { - tpData.setTopicId(topicIds.get(tp.topic())); - } else { - tpData.setName(tp.topic()); - } tpData.partitionData().add(new ProduceRequestData.PartitionProduceData() .setIndex(tp.partition()) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 269815cdbccea..8f5fe714c1a42 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -63,7 +63,7 @@ public static Builder forMagic(byte magic, ProduceRequestData data) { } private static boolean canNotSupportTopicId(ProduceRequestData data) { - return data.topicData().stream().anyMatch(d -> d.topicId() == null || d.topicId() == Uuid.ZERO_UUID); + return data.topicData().stream().anyMatch(d -> d.topicId() == Uuid.ZERO_UUID); } public static Builder forCurrentMagic(ProduceRequestData data) { 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 abf1c34fe3eb8..48c6584a99830 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 @@ -3658,7 +3658,11 @@ private ProduceResponse produceResponse(Map resp TopicPartition topicPartition = entry.getKey(); ProduceResponseData.TopicProduceResponse topicData = data.responses().find(topicPartition.topic(), TOPIC_ID); if (topicData == null) { - topicData = new ProduceResponseData.TopicProduceResponse().setName(topicPartition.topic()).setTopicId(TOPIC_ID); + if (apiVersions.maxSupportedProduceVersion() >= 12) { + topicData = new ProduceResponseData.TopicProduceResponse().setTopicId(TOPIC_ID); + } else { + topicData = new ProduceResponseData.TopicProduceResponse().setName(topicPartition.topic()); + } data.responses().add(topicData); } diff --git a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java index 34df62f9de511..a20e1fc8707eb 100644 --- a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java +++ b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java @@ -973,7 +973,6 @@ public void testProduceResponseVersions() throws Exception { responseData.setThrottleTimeMs(0); } - if (version >= 12) { responseData.responses().iterator().next().setTopicId(topicId); } else { diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java index 544e01b27652b..4b5150e584a05 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java @@ -121,11 +121,10 @@ public void testBuildWithCurrentMessageFormat() { ProduceRequest.Builder requestBuilder = ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE, new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( - new ProduceRequestData.TopicProduceData() - .setName("test") - .setTopicId(Uuid.fromString("H3Emm3vW7AKKO4NTRPaCWt")) - .setPartitionData(Collections.singletonList( - new ProduceRequestData.PartitionProduceData().setIndex(9).setRecords(builder.build())))) + new ProduceRequestData.TopicProduceData() + .setTopicId(Uuid.fromString("H3Emm3vW7AKKO4NTRPaCWt")) + .setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData().setIndex(9).setRecords(builder.build())))) .iterator())) .setAcks((short) 1) .setTimeoutMs(5000)); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 80b5962d4063d..4496e7207f679 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -481,11 +481,8 @@ public void testProduceRequestPartitionSize() { ProduceRequest request = ProduceRequest.forMagic(RecordBatch.MAGIC_VALUE_V2, new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection(asList( - new ProduceRequestData.TopicProduceData().setName(tp0.topic()).setTopicId(topicId).setPartitionData( - singletonList(new ProduceRequestData.PartitionProduceData().setIndex(tp0.partition()).setRecords(records0))), - new ProduceRequestData.TopicProduceData().setName(tp1.topic()).setTopicId(topicId).setPartitionData( - singletonList(new ProduceRequestData.PartitionProduceData().setIndex(tp1.partition()).setRecords(records1)))) - .iterator())) + createTopicProduceData(PRODUCE.latestVersion(), records0, tp0), + createTopicProduceData(PRODUCE.latestVersion(), records1, tp1)).iterator())) .setAcks((short) 1) .setTimeoutMs(5000) .setTransactionalId("transactionalId")) @@ -2556,29 +2553,38 @@ private ProduceRequest createProduceRequest(short version) { .setAcks((short) -1) .setTimeoutMs(123) .setTopicData(new ProduceRequestData.TopicProduceDataCollection(singletonList( - new ProduceRequestData.TopicProduceData() - .setName("topic1") - .setPartitionData(singletonList(new ProduceRequestData.PartitionProduceData() - .setIndex(1) - .setRecords(records)))).iterator())); + createTopicProduceData(version, records, new TopicIdPartition(Uuid.ZERO_UUID, 1, "topic1")) + ).iterator())); return new ProduceRequest.Builder(version, version, data).build(version); } + byte magic = version == 2 ? RecordBatch.MAGIC_VALUE_V1 : RecordBatch.MAGIC_VALUE_V2; MemoryRecords records = MemoryRecords.withRecords(magic, Compression.NONE, new SimpleRecord("woot".getBytes())); + TopicIdPartition topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), 0, "test"); return ProduceRequest.forMagic(magic, new ProduceRequestData() - .setTopicData(new ProduceRequestData.TopicProduceDataCollection(singletonList( - new ProduceRequestData.TopicProduceData() - .setName("test") - .setPartitionData(singletonList(new ProduceRequestData.PartitionProduceData() - .setIndex(0) - .setRecords(records)))).iterator())) + .setTopicData(new ProduceRequestData.TopicProduceDataCollection( + singletonList(createTopicProduceData(version, records, topicIdPartition)).iterator() + )) .setAcks((short) 1) .setTimeoutMs(5000) .setTransactionalId(version >= 3 ? "transactionalId" : null)) .build(version); } + private static ProduceRequestData.TopicProduceData createTopicProduceData(short version, MemoryRecords records, TopicIdPartition tp) { + ProduceRequestData.TopicProduceData topicProduceData = new ProduceRequestData.TopicProduceData() + .setPartitionData(singletonList(new ProduceRequestData.PartitionProduceData() + .setIndex(tp.partition()) + .setRecords(records))); + if (version >= 12) { + topicProduceData.setTopicId(tp.topicId()); + } else { + topicProduceData.setName(tp.topic()); + } + return topicProduceData; + } + @SuppressWarnings("deprecation") private ProduceResponse createProduceResponse() { Map responseData = new HashMap<>(); diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index a23db83432699..e29a7b46d7804 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -628,7 +628,7 @@ class KafkaApis(val requestChannel: RequestChannel, } val topicPartition = new TopicPartition(topicName, partition.index()) - if (topicName == null || topicName.isEmpty) + if (topicName.isEmpty) nonExistingTopicResponses += new TopicIdPartition(topicId, topicPartition) -> new PartitionResponse(Errors.UNKNOWN_TOPIC_ID) else if (!metadataCache.contains(topicPartition)) nonExistingTopicResponses += new TopicIdPartition(topicId, topicPartition) -> new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION) From 1c12e482e89cb709a2dd335dccf70dbeb195ec38 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Wed, 26 Jun 2024 17:36:54 +0100 Subject: [PATCH 12/40] add comment --- .../java/org/apache/kafka/common/requests/ProduceRequest.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 8f5fe714c1a42..ae25bf71dca5b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -142,6 +142,10 @@ Map partitionSizes() { Map tmpPartitionSizes = new HashMap<>(); data.topicData().forEach(topicData -> topicData.partitionData().forEach(partitionData -> + // While topic id and name will never be populated at the same time in the request, to simplify + // initializing `TopicIdPartition` the code will use both topic name and id. + // Topic Id will be Uuid.ZERO_UUID in versions < 12 and topic name will be used as main identifier of topic partition. + // Topic Name will be empty string in versions >= 12 and topic id will be used as the main identifier. tmpPartitionSizes.compute(new TopicIdPartition(topicData.topicId(), partitionData.index(), topicData.name()), (ignored, previousValue) -> partitionData.records().sizeInBytes() + (previousValue == null ? 0 : previousValue)) From d388d8ab6dbfd414d2fe1d5cbf2b0fabe20d8ac1 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Fri, 28 Jun 2024 23:41:48 +0100 Subject: [PATCH 13/40] fix import --- .../org/apache/kafka/common/requests/ProduceResponseTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java index bd7c5efa58e0b..63fb6c89e33b0 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java @@ -17,7 +17,7 @@ package org.apache.kafka.common.requests; -import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.ProduceResponseData; import org.apache.kafka.common.protocol.ApiKeys; From a2e858ce897278fda4376b6f54f11d3643cddc1b Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Sat, 29 Jun 2024 00:03:40 +0100 Subject: [PATCH 14/40] apply spotless --- .../org/apache/kafka/common/requests/ProduceRequestTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java index 4508a31e83110..cb7d46d247c74 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java @@ -18,8 +18,8 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.InvalidRecordException; -import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.errors.UnsupportedCompressionTypeException; import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.protocol.ApiKeys; From 30f0aa08a20e936249bb6c597faa6fdf8700b2cb Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Mon, 1 Jul 2024 14:00:54 +0100 Subject: [PATCH 15/40] fix tests --- .../apache/kafka/common/requests/RequestResponseTest.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 9f49a92e3d90b..ba3e4ff5da829 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -473,8 +473,8 @@ public void cannotUseFindCoordinatorV0ToFindTransactionCoordinator() { @Test public void testProduceRequestPartitionSize() { Uuid topicId = Uuid.fromString("e9TvBGX5JkYAB0AQorYD4w"); - TopicIdPartition tp0 = new TopicIdPartition(topicId, 0, "test"); - TopicIdPartition tp1 = new TopicIdPartition(topicId, 1, "test"); + TopicIdPartition tp0 = createTopicIdPartition(topicId, 0); + TopicIdPartition tp1 = createTopicIdPartition(topicId, 1); MemoryRecords records0 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, Compression.NONE, new SimpleRecord("woot".getBytes())); MemoryRecords records1 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, @@ -2586,6 +2586,10 @@ private static ProduceRequestData.TopicProduceData createTopicProduceData(short return topicProduceData; } + private static TopicIdPartition createTopicIdPartition(Uuid topicId, int partitionIndex) { + return new TopicIdPartition(topicId, partitionIndex, ""); + } + @SuppressWarnings("deprecation") private ProduceResponse createProduceResponse() { Map responseData = new HashMap<>(); From c97212fe00cc5f33d85e0312d3148435036d2710 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Mon, 30 Sep 2024 15:19:04 +0100 Subject: [PATCH 16/40] Address some of the feedback --- .../clients/producer/internals/Sender.java | 4 +-- .../common/requests/ProduceResponse.java | 1 - .../scala/kafka/server/DelayedProduce.scala | 12 +++---- .../scala/kafka/server/ReplicaManager.scala | 36 +++++++++---------- 4 files changed, 26 insertions(+), 27 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 8b3802caa2957..df4646bf33148 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -882,7 +882,7 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo if (batch.magic() < minUsedMagic) minUsedMagic = batch.magic(); } - Map topicIds = getTopicIdsFromBatches(batches); + Map topicIds = getTopicIdsForBatches(batches); // Use topic id if the max supported producer request version >= 12 and metadata has topic ids for all topics // Otherwise send the request with topic name and the broker boolean canUseTopicId = apiVersions.maxSupportedProduceVersion() >= 12 && topicIds.values().stream().anyMatch(id -> id != Uuid.ZERO_UUID); @@ -943,7 +943,7 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo log.trace("Sent produce request to {}: {}", nodeId, requestBuilder); } - private Map getTopicIdsFromBatches(List batches) { + private Map getTopicIdsForBatches(List batches) { return batches.stream() .collect(Collectors.toMap( b -> b.topicPartition.topic(), diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 4241260d77416..818726b41019b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -33,7 +33,6 @@ import java.util.Objects; import java.util.stream.Collectors; - /** * This wrapper supports both v0 and v8 of ProduceResponse. * diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index 9c08c383aafe0..92b071bbf53d4 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -90,11 +90,11 @@ class DelayedProduce(delayMs: Long, */ override def tryComplete(): Boolean = { // check for each partition if it still has pending acks - produceMetadata.produceStatus.forKeyValue { (topicPartition, status) => - trace(s"Checking produce satisfaction for $topicPartition, current status $status") + produceMetadata.produceStatus.forKeyValue { (topicIdPartition, status) => + trace(s"Checking produce satisfaction for $topicIdPartition, current status $status") // skip those partitions that have already been satisfied if (status.acksPending) { - val (hasEnough, error) = replicaManager.getPartitionOrError(topicPartition.topicPartition()) match { + val (hasEnough, error) = replicaManager.getPartitionOrError(topicIdPartition.topicPartition()) match { case Left(err) => // Case A (false, err) @@ -119,10 +119,10 @@ class DelayedProduce(delayMs: Long, } override def onExpiration(): Unit = { - produceMetadata.produceStatus.forKeyValue { (topicPartition, status) => + produceMetadata.produceStatus.forKeyValue { (topicIdPartition, status) => if (status.acksPending) { - debug(s"Expiring produce request for partition $topicPartition with status $status") - DelayedProduceMetrics.recordExpiration(topicPartition.topicPartition()) + debug(s"Expiring produce request for partition $topicIdPartition with status $status") + DelayedProduceMetrics.recordExpiration(topicIdPartition.topicPartition()) } } } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 6b83d460afd6f..ade7ea7b93d40 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -845,11 +845,11 @@ class ReplicaManager(val config: KafkaConfig, val transactionalProducerInfo = mutable.HashSet[(Long, Short)]() val topicPartitionBatchInfo = mutable.Map[TopicPartition, Int]() val topicIds = entriesPerPartition.keys.map(tp => tp.topic() -> tp.topicId()).toMap - entriesPerPartition.forKeyValue { (topicPartition, records) => + entriesPerPartition.forKeyValue { (topicIdPartition, records) => // Produce requests (only requests that require verification) should only have one batch per partition in "batches" but check all just to be safe. val transactionalBatches = records.batches.asScala.filter(batch => batch.hasProducerId && batch.isTransactional) transactionalBatches.foreach(batch => transactionalProducerInfo.add(batch.producerId, batch.producerEpoch)) - if (transactionalBatches.nonEmpty) topicPartitionBatchInfo.put(topicPartition.topicPartition(), records.firstBatch.baseSequence) + if (transactionalBatches.nonEmpty) topicPartitionBatchInfo.put(topicIdPartition.topicPartition(), records.firstBatch.baseSequence) } if (transactionalProducerInfo.size > 1) { throw new InvalidPidMappingException("Transactional records contained more than one producer ID") @@ -1420,34 +1420,34 @@ class ReplicaManager(val config: KafkaConfig, if (traceEnabled) trace(s"Append [$entriesPerPartition] to local log") - entriesPerPartition.map { case (topicPartition, records) => - brokerTopicStats.topicStats(topicPartition.topic).totalProduceRequestRate.mark() + entriesPerPartition.map { case (topicIdPartition, records) => + brokerTopicStats.topicStats(topicIdPartition.topic).totalProduceRequestRate.mark() brokerTopicStats.allTopicsStats.totalProduceRequestRate.mark() // reject appending to internal topics if it is not allowed - if (Topic.isInternal(topicPartition.topic) && !internalTopicsAllowed) { - (topicPartition, LogAppendResult( + if (Topic.isInternal(topicIdPartition.topic) && !internalTopicsAllowed) { + (topicIdPartition, LogAppendResult( LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, - Some(new InvalidTopicException(s"Cannot append to internal topic ${topicPartition.topic}")), + Some(new InvalidTopicException(s"Cannot append to internal topic ${topicIdPartition.topic}")), hasCustomErrorMessage = false)) } else { try { - val partition = getPartitionOrException(topicPartition.topicPartition()) + val partition = getPartitionOrException(topicIdPartition.topicPartition()) val info = partition.appendRecordsToLeader(records, origin, requiredAcks, requestLocal, - verificationGuards.getOrElse(topicPartition.topicPartition(), VerificationGuard.SENTINEL)) + verificationGuards.getOrElse(topicIdPartition.topicPartition(), VerificationGuard.SENTINEL)) val numAppendedMessages = info.numMessages // update stats for successfully appended bytes and messages as bytesInRate and messageInRate - brokerTopicStats.topicStats(topicPartition.topic).bytesInRate.mark(records.sizeInBytes) + brokerTopicStats.topicStats(topicIdPartition.topic).bytesInRate.mark(records.sizeInBytes) brokerTopicStats.allTopicsStats.bytesInRate.mark(records.sizeInBytes) - brokerTopicStats.topicStats(topicPartition.topic).messagesInRate.mark(numAppendedMessages) + brokerTopicStats.topicStats(topicIdPartition.topic).messagesInRate.mark(numAppendedMessages) brokerTopicStats.allTopicsStats.messagesInRate.mark(numAppendedMessages) if (traceEnabled) - trace(s"${records.sizeInBytes} written to log $topicPartition beginning at offset " + + trace(s"${records.sizeInBytes} written to log $topicIdPartition beginning at offset " + s"${info.firstOffset} and ending at offset ${info.lastOffset}") - (topicPartition, LogAppendResult(info, exception = None, hasCustomErrorMessage = false)) + (topicIdPartition, LogAppendResult(info, exception = None, hasCustomErrorMessage = false)) } catch { // NOTE: Failed produce requests metric is not incremented for known exceptions // it is supposed to indicate un-expected failures of a broker in handling a produce request @@ -1457,15 +1457,15 @@ class ReplicaManager(val config: KafkaConfig, _: RecordBatchTooLargeException | _: CorruptRecordException | _: KafkaStorageException) => - (topicPartition, LogAppendResult(LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, Some(e), hasCustomErrorMessage = false)) + (topicIdPartition, LogAppendResult(LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, Some(e), hasCustomErrorMessage = false)) case rve: RecordValidationException => - val logStartOffset = processFailedRecord(topicPartition.topicPartition(), rve.invalidException) + val logStartOffset = processFailedRecord(topicIdPartition.topicPartition(), rve.invalidException) val recordErrors = rve.recordErrors - (topicPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithAdditionalInfo(logStartOffset, recordErrors), + (topicIdPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithAdditionalInfo(logStartOffset, recordErrors), Some(rve.invalidException), hasCustomErrorMessage = true)) case t: Throwable => - val logStartOffset = processFailedRecord(topicPartition.topicPartition(), t) - (topicPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithLogStartOffset(logStartOffset), + val logStartOffset = processFailedRecord(topicIdPartition.topicPartition(), t) + (topicIdPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithLogStartOffset(logStartOffset), Some(t), hasCustomErrorMessage = false)) } } From dc72f92631659528522d06066869c9d3428be4d1 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Mon, 30 Sep 2024 15:37:54 +0100 Subject: [PATCH 17/40] fix filtering `entriesWithoutErrorsPerPartition` based on topicIdPartition to relay on topic partition instead --- core/src/main/scala/kafka/server/ReplicaManager.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index ade7ea7b93d40..7e6270e3e6add 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -858,7 +858,7 @@ class ReplicaManager(val config: KafkaConfig, def postVerificationCallback(newRequestLocal: RequestLocal, results: (Map[TopicPartition, Errors], Map[TopicPartition, VerificationGuard])): Unit = { val (preAppendErrors, verificationGuards) = results - val errorResults = preAppendErrors.map { + val errorResults: Map[TopicIdPartition, LogAppendResult] = preAppendErrors.map { case (topicPartition, error) => // translate transaction coordinator errors to known producer response errors val customException = @@ -882,7 +882,7 @@ class ReplicaManager(val config: KafkaConfig, hasCustomErrorMessage = customException.isDefined ) } - val entriesWithoutErrorsPerPartition = entriesPerPartition.filter { case (key, _) => !errorResults.contains(key) } + val entriesWithoutErrorsPerPartition = entriesPerPartition.filter { case (key, _) => !errorResults.exists(_._1.topicPartition() == key.topicPartition()) } val preAppendPartitionResponses = buildProducePartitionStatus(errorResults).map { case (k, status) => k -> status.responseStatus } From c3d2c2435af41d29e67e8d25f1c4710595611f8d Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Thu, 10 Oct 2024 14:34:12 +0100 Subject: [PATCH 18/40] remove maxSupportedProduceVersion --- .../org/apache/kafka/clients/ApiVersions.java | 16 ---------------- .../clients/producer/internals/Sender.java | 19 ++++--------------- .../main/scala/kafka/server/KafkaApis.scala | 6 +++--- 3 files changed, 7 insertions(+), 34 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ApiVersions.java b/clients/src/main/java/org/apache/kafka/clients/ApiVersions.java index 22c9556b1e8f2..3127be7355aed 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ApiVersions.java +++ b/clients/src/main/java/org/apache/kafka/clients/ApiVersions.java @@ -34,7 +34,6 @@ public class ApiVersions { private final Map nodeApiVersions = new HashMap<>(); private byte maxUsableProduceMagic = RecordBatch.CURRENT_MAGIC_VALUE; - private short maxSupportedProduceVersion = ApiKeys.PRODUCE.latestVersion(); // The maximum finalized feature epoch of all the node api versions. private long maxFinalizedFeaturesEpoch = -1; @@ -52,26 +51,15 @@ public static class FinalizedFeaturesInfo { public synchronized void update(String nodeId, NodeApiVersions nodeApiVersions) { this.nodeApiVersions.put(nodeId, nodeApiVersions); this.maxUsableProduceMagic = computeMaxUsableProduceMagic(); - this.maxSupportedProduceVersion = computeMaxSupportedProduceVersion(); if (maxFinalizedFeaturesEpoch < nodeApiVersions.finalizedFeaturesEpoch()) { this.maxFinalizedFeaturesEpoch = nodeApiVersions.finalizedFeaturesEpoch(); this.finalizedFeatures = nodeApiVersions.finalizedFeatures(); } } - private short computeMaxSupportedProduceVersion() { - Optional knownBrokerNodesMinSupportedVersionForProduce = this.nodeApiVersions.values().stream() - .filter(versions -> versions.apiVersion(ApiKeys.PRODUCE) != null) // filter out Raft controller nodes - .map(versions -> versions.latestUsableVersion(ApiKeys.PRODUCE)) - .min(Short::compare); - return (short) Math.min(ApiKeys.PRODUCE.latestVersion(), - knownBrokerNodesMinSupportedVersionForProduce.orElse(ApiKeys.PRODUCE.latestVersion())); - } - public synchronized void remove(String nodeId) { this.nodeApiVersions.remove(nodeId); this.maxUsableProduceMagic = computeMaxUsableProduceMagic(); - this.maxSupportedProduceVersion = computeMaxSupportedProduceVersion(); } public synchronized NodeApiVersions get(String nodeId) { @@ -100,8 +88,4 @@ private byte computeMaxUsableProduceMagic() { public synchronized byte maxUsableProduceMagic() { return maxUsableProduceMagic; } - public synchronized short maxSupportedProduceVersion() { - return maxSupportedProduceVersion; - } - } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index df4646bf33148..11ee18d80dc60 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -883,10 +883,6 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo minUsedMagic = batch.magic(); } Map topicIds = getTopicIdsForBatches(batches); - // Use topic id if the max supported producer request version >= 12 and metadata has topic ids for all topics - // Otherwise send the request with topic name and the broker - boolean canUseTopicId = apiVersions.maxSupportedProduceVersion() >= 12 && topicIds.values().stream().anyMatch(id -> id != Uuid.ZERO_UUID); - ProduceRequestData.TopicProduceDataCollection tpd = new ProduceRequestData.TopicProduceDataCollection(); for (ProducerBatch batch : batches) { TopicPartition tp = batch.topicPartition; @@ -901,19 +897,12 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo // which is supporting the new magic version to one which doesn't, then we will need to convert. if (!records.hasMatchingMagic(minUsedMagic)) records = batch.records().downConvert(minUsedMagic, 0, time).records(); - ProduceRequestData.TopicProduceData tpData = canUseTopicId ? - tpd.find(tp.topic(), topicIds.get(tp.topic())) : - tpd.find(new ProduceRequestData.TopicProduceData().setName(tp.topic())); + ProduceRequestData.TopicProduceData tpData = tpd.find(tp.topic(), topicIds.get(tp.topic())); if (tpData == null) { - tpData = new ProduceRequestData.TopicProduceData(); - - if (canUseTopicId) { - tpData.setTopicId(topicIds.get(tp.topic())); - } else { - tpData.setName(tp.topic()); - } - + Uuid topicId = metadata.topicIds().getOrDefault(tp.topic(), Uuid.ZERO_UUID); + tpData = new ProduceRequestData.TopicProduceData() + .setTopicId(topicId).setName(tp.topic()); tpd.add(tpData); } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 2aa1c50bfc7d5..908507d72f00a 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -633,10 +633,10 @@ class KafkaApis(val requestChannel: RequestChannel, produceRequest.data.topicData.forEach { topic => topic.partitionData.forEach { partition => - val (topicName, topicId) = if (produceRequest.version() >= 12) { - (metadataCache.getTopicName(topic.topicId).getOrElse(topic.name), topic.topicId()) - } else { + val (topicName, topicId) = if (topic.topicId().equals(Uuid.ZERO_UUID)) { (topic.name(), metadataCache.getTopicId(topic.name())) + } else { + (metadataCache.getTopicName(topic.topicId).getOrElse(topic.name), topic.topicId()) } val topicPartition = new TopicPartition(topicName, partition.index()) From f7f0a0480a8ab308af8a705ab715fadef0c5c4b4 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Tue, 15 Oct 2024 13:46:47 +0100 Subject: [PATCH 19/40] update test --- .../clients/producer/internals/SenderTest.java | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) 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 dc8984c511d74..56d6c5d17171b 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 @@ -158,9 +158,9 @@ public class SenderTest { private static final String TOPIC_NAME = "test"; private static final Uuid TOPIC_ID = Uuid.fromString("MKXx1fIkQy2J9jXHhK8m1w"); private static final Map TOPIC_IDS = Collections.singletonMap(TOPIC_NAME, TOPIC_ID); - private TopicPartition tp0 = new TopicPartition(TOPIC_NAME, 0); - private TopicPartition tp1 = new TopicPartition(TOPIC_NAME, 1); - private TopicPartition tp2 = new TopicPartition(TOPIC_NAME, 2); + private final TopicPartition tp0 = new TopicPartition(TOPIC_NAME, 0); + private final TopicPartition tp1 = new TopicPartition(TOPIC_NAME, 1); + private final TopicPartition tp2 = new TopicPartition(TOPIC_NAME, 2); private MockTime time = new MockTime(); private final int batchSize = 16 * 1024; private final ProducerMetadata metadata = new ProducerMetadata(0, 0, Long.MAX_VALUE, TOPIC_IDLE_MS, @@ -3659,11 +3659,9 @@ private ProduceResponse produceResponse(Map resp TopicPartition topicPartition = entry.getKey(); ProduceResponseData.TopicProduceResponse topicData = data.responses().find(topicPartition.topic(), TOPIC_ID); if (topicData == null) { - if (apiVersions.maxSupportedProduceVersion() >= 12) { - topicData = new ProduceResponseData.TopicProduceResponse().setTopicId(TOPIC_ID); - } else { - topicData = new ProduceResponseData.TopicProduceResponse().setName(topicPartition.topic()); - } + topicData = new ProduceResponseData.TopicProduceResponse() + .setTopicId(TOPIC_ID) + .setName(topicPartition.topic()); data.responses().add(topicData); } From 654a967d0adc3320475129ad42d7c64e8e9e07b4 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Tue, 15 Oct 2024 16:03:12 +0100 Subject: [PATCH 20/40] update FetchResponseStats to use TopicIdPartition --- .../main/scala/kafka/server/KafkaApis.scala | 20 +++++++++---------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 5885e042e2fa3..8b006e179345e 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -121,7 +121,7 @@ class KafkaApis(val requestChannel: RequestChannel, val clientMetricsManager: Option[ClientMetricsManager] ) extends ApiRequestHandler with Logging { - type FetchResponseStats = Map[TopicPartition, RecordValidationStats] + type FetchResponseStats = Map[TopicIdPartition, RecordValidationStats] this.logIdent = "[KafkaApi-%d] ".format(brokerId) val configHelper = new ConfigHelper(metadataCache, config, configRepository) val authHelper = new AuthHelper(authorizer) @@ -746,8 +746,8 @@ class KafkaApis(val requestChannel: RequestChannel, } def processingStatsCallback(processingStats: FetchResponseStats): Unit = { - processingStats.foreachEntry { (tp, info) => - updateRecordConversionStats(request, tp, info) + processingStats.foreachEntry { (topicIdPartition, info) => + updateRecordConversionStats(request, topicIdPartition.topicPartition(), info) } } @@ -764,9 +764,7 @@ class KafkaApis(val requestChannel: RequestChannel, transactionalId = produceRequest.transactionalId, entriesPerPartition = authorizedRequestInfo, responseCallback = sendResponseCallback, - recordValidationStatsCallback = result => processingStatsCallback(result.map { - case (partition, response) => (partition.topicPartition(), response) - }), + recordValidationStatsCallback = processingStatsCallback, requestLocal = requestLocal, transactionSupportedOperation = transactionSupportedOperation) @@ -930,12 +928,12 @@ class KafkaApis(val requestChannel: RequestChannel, val partitions = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData] val reassigningPartitions = mutable.Set[TopicIdPartition]() val nodeEndpoints = new mutable.HashMap[Int, Node] - responsePartitionData.foreach { case (tp, data) => + responsePartitionData.foreach { case (topicIdPartition, data) => val abortedTransactions = data.abortedTransactions.orElse(null) val lastStableOffset: Long = data.lastStableOffset.orElse(FetchResponse.INVALID_LAST_STABLE_OFFSET) - if (data.isReassignmentFetch) reassigningPartitions.add(tp) + if (data.isReassignmentFetch) reassigningPartitions.add(topicIdPartition) val partitionData = new FetchResponseData.PartitionData() - .setPartitionIndex(tp.partition) + .setPartitionIndex(topicIdPartition.partition) .setErrorCode(maybeDownConvertStorageError(data.error).code) .setHighWatermark(data.highWatermark) .setLastStableOffset(lastStableOffset) @@ -947,7 +945,7 @@ class KafkaApis(val requestChannel: RequestChannel, if (versionId >= 16) { data.error match { case Errors.NOT_LEADER_OR_FOLLOWER | Errors.FENCED_LEADER_EPOCH => - val leaderNode = getCurrentLeader(tp.topicPartition(), request.context.listenerName) + val leaderNode = getCurrentLeader(topicIdPartition.topicPartition(), request.context.listenerName) leaderNode.node.foreach { node => nodeEndpoints.put(node.id(), node) } @@ -959,7 +957,7 @@ class KafkaApis(val requestChannel: RequestChannel, } data.divergingEpoch.ifPresent(partitionData.setDivergingEpoch(_)) - partitions.put(tp, partitionData) + partitions.put(topicIdPartition, partitionData) } erroneous.foreach { case (tp, data) => partitions.put(tp, data) } From 315b6a31a26901c86496e5f91127545cf14590c4 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Tue, 15 Oct 2024 16:12:24 +0100 Subject: [PATCH 21/40] address some of the feedback --- .../clients/producer/internals/Sender.java | 4 +-- .../group/CoordinatorPartitionWriter.scala | 4 +-- .../group/GroupMetadataManager.scala | 6 ++-- .../transaction/TransactionStateManager.scala | 4 +-- .../main/scala/kafka/server/KafkaApis.scala | 2 +- .../scala/kafka/server/ReplicaManager.scala | 2 +- .../api/ProducerSendWhileDeletionTest.scala | 20 +++++------ .../CoordinatorPartitionWriterTest.scala | 4 +-- .../group/GroupCoordinatorTest.scala | 6 ++-- .../group/GroupMetadataManagerTest.scala | 35 +++++++++---------- .../TransactionStateManagerTest.scala | 24 ++++++------- .../unit/kafka/server/KafkaApisTest.scala | 2 +- .../ReplicaManagerConcurrencyTest.scala | 2 +- .../kafka/server/ReplicaManagerTest.scala | 4 +-- 14 files changed, 59 insertions(+), 60 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 11ee18d80dc60..c2fcecf93709c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -882,7 +882,7 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo if (batch.magic() < minUsedMagic) minUsedMagic = batch.magic(); } - Map topicIds = getTopicIdsForBatches(batches); + Map topicIds = topicIdsForBatches(batches); ProduceRequestData.TopicProduceDataCollection tpd = new ProduceRequestData.TopicProduceDataCollection(); for (ProducerBatch batch : batches) { TopicPartition tp = batch.topicPartition; @@ -932,7 +932,7 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo log.trace("Sent produce request to {}: {}", nodeId, requestBuilder); } - private Map getTopicIdsForBatches(List batches) { + private Map topicIdsForBatches(List batches) { return batches.stream() .collect(Collectors.toMap( b -> b.topicPartition.topic(), diff --git a/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala b/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala index 182f933b7b41a..c8a41a0f683b3 100644 --- a/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala +++ b/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala @@ -144,7 +144,7 @@ class CoordinatorPartitionWriter( requiredAcks = 1, internalTopicsAllowed = true, origin = AppendOrigin.COORDINATOR, - entriesPerPartition = Map(replicaManager.getTopicIdPartition(tp) -> records), + entriesPerPartition = Map(replicaManager.topicIdPartition(tp) -> records), responseCallback = results => appendResults = results, requestLocal = RequestLocal.noCaching, verificationGuards = Map(tp -> verificationGuard), @@ -154,7 +154,7 @@ class CoordinatorPartitionWriter( actionQueue = directActionQueue ) - val partitionResult = appendResults.getOrElse(replicaManager.getTopicIdPartition(tp), + val partitionResult = appendResults.getOrElse(replicaManager.topicIdPartition(tp), throw new IllegalStateException(s"Append status $appendResults should have partition $tp.")) if (partitionResult.error != Errors.NONE) { diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala index eb83586e7141d..7aa6f7f6c4a0f 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala @@ -257,7 +257,7 @@ class GroupMetadataManager(brokerId: Int, } val groupMetadataPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, partitionFor(group.groupId)) - val groupMetadataTopicIdPartition = replicaManager.getTopicIdPartition(groupMetadataPartition) + val groupMetadataTopicIdPartition = replicaManager.topicIdPartition(groupMetadataPartition) val groupMetadataRecords = Map(groupMetadataTopicIdPartition -> records) val generationId = group.generationId @@ -376,7 +376,7 @@ class GroupMetadataManager(brokerId: Int, producerId: Long, records: Map[TopicPartition, MemoryRecords]): Map[TopicIdPartition, PartitionResponse] => Unit = { val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, partitionFor(group.groupId)) - val offsetTopicIdPartition = replicaManager.getTopicIdPartition(offsetTopicPartition) + val offsetTopicIdPartition = replicaManager.topicIdPartition(offsetTopicPartition) // set the callback function to insert offsets into cache after log append completed def putCacheCallback(responseStatus: Map[TopicIdPartition, PartitionResponse]): Unit = { // the append response should only contain the topics partition @@ -488,7 +488,7 @@ class GroupMetadataManager(brokerId: Int, group.prepareOffsetCommit(filteredOffsetMetadata) } - val topicIdPartitionsToRecords = Utils.convertKeys(records.asJava, replicaManager.getTopicIdPartition).asScala + val topicIdPartitionsToRecords = Utils.convertKeys(records.asJava, replicaManager.topicIdPartition).asScala appendForGroup(group, topicIdPartitionsToRecords, requestLocal, putCacheCallback, verificationGuards) } diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala index 738165111d7f3..90c7624a78162 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala @@ -283,7 +283,7 @@ class TransactionStateManager(brokerId: Int, requiredAcks = TransactionLog.EnforcedRequiredAcks, internalTopicsAllowed = true, origin = AppendOrigin.COORDINATOR, - entriesPerPartition = Map(replicaManager.getTopicIdPartition(transactionPartition) -> tombstoneRecords), + entriesPerPartition = Map(replicaManager.topicIdPartition(transactionPartition) -> tombstoneRecords), responseCallback = removeFromCacheCallback, requestLocal = RequestLocal.noCaching) } @@ -631,7 +631,7 @@ class TransactionStateManager(brokerId: Int, val records = MemoryRecords.withRecords(TransactionLog.EnforcedCompression, new SimpleRecord(timestamp, keyBytes, valueBytes)) val topicPartition = new TopicPartition(Topic.TRANSACTION_STATE_TOPIC_NAME, partitionFor(transactionalId)) - val transactionStateTopicIdPartition = replicaManager.getTopicIdPartition(topicPartition) + val transactionStateTopicIdPartition = replicaManager.topicIdPartition(topicPartition) val recordsPerPartition = Map(transactionStateTopicIdPartition -> records) // set the callback function to update transaction status in cache after log append completed diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 8b006e179345e..cfff031f59355 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -2483,7 +2483,7 @@ class KafkaApis(val requestChannel: RequestChannel, } else { // Otherwise, the regular appendRecords path is used for all the non __consumer_offsets // partitions or for all partitions when the new group coordinator is disabled. - controlRecords += replicaManager.getTopicIdPartition(partition) -> MemoryRecords.withEndTransactionMarker( + controlRecords += replicaManager.topicIdPartition(partition) -> MemoryRecords.withEndTransactionMarker( producerId, marker.producerEpoch, new EndTransactionMarker(controlRecordType, marker.coordinatorEpoch) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 1c7c3f50fbe92..167cb87d71fe5 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -647,7 +647,7 @@ class ReplicaManager(val config: KafkaConfig, errorMap } - def getTopicIdPartition(topicPartition: TopicPartition): TopicIdPartition = { + def topicIdPartition(topicPartition: TopicPartition): TopicIdPartition = { val topicId = metadataCache.getTopicId(topicPartition.topic()) new TopicIdPartition(topicId, topicPartition) } diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala index 8808be62044cb..bd1237ebb8e3f 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala @@ -50,7 +50,7 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness { * succeed as long as the partition is included in the metadata. */ @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) + @ValueSource(strings = Array("kraft")) def testSendWithTopicDeletionMidWay(quorum: String): Unit = { val numRecords = 10 val topic = "topic" @@ -91,13 +91,13 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness { * succeed as long as the metadata has been updated with new topic id. */ @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) + @ValueSource(strings = Array("kraft")) def testSendWithRecreatedTopic(quorum: String): Unit = { val numRecords = 10 val topic = "topic" createTopic(topic) val admin = createAdminClient() - val topicId = getTopicMetadata(admin, topic).topicId() + val topicId = topicMetadata(admin, topic).topicId() val producer = createProducer() (1 to numRecords).foreach { i => @@ -110,7 +110,7 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness { // Verify that the topic is deleted when no metadata request comes in TestUtils.verifyTopicDeletion(zkClientOrNull, topic, 2, brokers) createTopic(topic) - assertNotEquals(topicId, getTopicMetadata(admin, topic).topicId()) + assertNotEquals(topicId, topicMetadata(admin, topic).topicId()) // Producer should be able to send messages even after topic gets recreated val recordMetadata: RecordMetadata = producer.send(new ProducerRecord(topic, null, "value".getBytes(StandardCharsets.UTF_8))).get @@ -125,7 +125,7 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness { * succeed as long as the metadata on the leader has been updated with new topic id. */ @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) + @ValueSource(strings = Array("kraft")) def testSendWithTopicReassignmentIsMidWay(quorum: String): Unit = { val numRecords = 10 val topic = "topic" @@ -137,7 +137,7 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness { createTopicWithAssignment(topic, Map(0 -> Seq(0, 1), 1 -> Seq(0, 1))) TestUtils.assertLeader(admin, partition1, 0) - val topicDetails = getTopicMetadata(admin, topic) + val topicDetails = topicMetadata(admin, topic) assertEquals(0, topicDetails.partitions().get(0).leader().id()) val producer = createProducer() @@ -159,18 +159,18 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness { s"Expected preferred leader to become 2, but is ${partitionLeader(admin, partition0)} and ${partitionLeader(admin, partition1)}", 10000) TestUtils.assertLeader(admin, partition1, 2) - assertEquals(topicDetails.topicId(), getTopicMetadata(admin, topic).topicId()) + assertEquals(topicDetails.topicId(), topicMetadata(admin, topic).topicId()) // Producer should be able to send messages even after topic gets reassigned assertEquals(topic, producer.send(new ProducerRecord(topic, null, "value".getBytes(StandardCharsets.UTF_8))).get.topic()) } - def getTopicMetadata(admin: Admin, topic: String): TopicDescription = { + def topicMetadata(admin: Admin, topic: String): TopicDescription = { admin.describeTopics(util.Collections.singletonList(topic)).allTopicNames().get().get(topic) } - def partitionLeader(admin: Admin, topicPartition: TopicPartition): Int = { - val partitionMetadata = getTopicMetadata(admin, topicPartition.topic).partitions.get(topicPartition.partition) + private def partitionLeader(admin: Admin, topicPartition: TopicPartition): Int = { + val partitionMetadata = topicMetadata(admin, topicPartition.topic).partitions.get(topicPartition.partition) val preferredLeaderMetadata = partitionMetadata.leader() preferredLeaderMetadata.id } diff --git a/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala index feb65addc8f8b..592a59a931983 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala @@ -88,7 +88,7 @@ class CoordinatorPartitionWriterTest { val tp = new TopicPartition("foo", 0) val topicId = Uuid.fromString("TbEp6-A4s3VPT1TwiI5COw") val replicaManager = mock(classOf[ReplicaManager]) - when(replicaManager.getTopicIdPartition(tp)).thenReturn(new TopicIdPartition(topicId, tp)) + when(replicaManager.topicIdPartition(tp)).thenReturn(new TopicIdPartition(topicId, tp)) val partitionRecordWriter = new CoordinatorPartitionWriter( replicaManager @@ -198,7 +198,7 @@ class CoordinatorPartitionWriterTest { val tp = new TopicPartition("foo", 0) val topicId = Uuid.fromString("TbEp6-A4s3VPT1TwiI5COw") val replicaManager = mock(classOf[ReplicaManager]) - when(replicaManager.getTopicIdPartition(tp)).thenReturn(new TopicIdPartition(topicId, tp)) + when(replicaManager.topicIdPartition(tp)).thenReturn(new TopicIdPartition(topicId, tp)) val partitionRecordWriter = new CoordinatorPartitionWriter( replicaManager diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala index 8d220738d5c8e..d599a083c709f 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala @@ -129,7 +129,7 @@ class GroupCoordinatorTest { // add the partition into the owned partition list groupPartitionId = groupCoordinator.partitionFor(groupId) val groupPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId) - when(replicaManager.getTopicIdPartition(groupPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, groupPartition)) + when(replicaManager.topicIdPartition(groupPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, groupPartition)) groupCoordinator.groupManager.addOwnedPartition(groupPartitionId) } @@ -2786,8 +2786,8 @@ class GroupCoordinatorTest { val groupMetadataPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupCoordinator.partitionFor(groupId)) val offsetTopicPartitions = List(groupMetadataPartition, otherGroupMetadataPartition) - when(replicaManager.getTopicIdPartition(groupMetadataPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, groupMetadataPartition)) - when(replicaManager.getTopicIdPartition(otherGroupMetadataPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, otherGroupMetadataPartition)) + when(replicaManager.topicIdPartition(groupMetadataPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, groupMetadataPartition)) + when(replicaManager.topicIdPartition(otherGroupMetadataPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, otherGroupMetadataPartition)) groupCoordinator.groupManager.addOwnedPartition(offsetTopicPartitions(1).partition) val errors = mutable.ArrayBuffer[Errors]() diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala index c5ec7f5c74d51..426c9fa72080d 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala @@ -95,7 +95,6 @@ class GroupMetadataManagerTest { config.groupCoordinatorConfig.offsetCommitTimeoutMs) } - @BeforeEach def setUp(): Unit = { defaultOffsetRetentionMs = offsetConfig.offsetsRetentionMs @@ -1117,7 +1116,7 @@ class GroupMetadataManagerTest { } val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) - when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + when(replicaManager.topicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) groupMetadataManager.storeGroup(group, Map.empty, callback) assertEquals(Some(Errors.NONE), maybeError) @@ -1144,7 +1143,7 @@ class GroupMetadataManagerTest { } val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) - when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + when(replicaManager.topicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) groupMetadataManager.storeGroup(group, Map.empty, callback) assertEquals(Some(Errors.NONE), maybeError) @@ -1178,7 +1177,7 @@ class GroupMetadataManagerTest { groupMetadataManager.addGroup(group) val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) - when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + when(replicaManager.topicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) expectAppendMessage(appendError) var maybeError: Option[Errors] = None @@ -1224,7 +1223,7 @@ class GroupMetadataManagerTest { maybeError = Some(error) } val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) - when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + when(replicaManager.topicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) groupMetadataManager.storeGroup(group, Map(memberId -> Array[Byte]()), callback) assertEquals(Some(Errors.NONE), maybeError) @@ -1281,7 +1280,7 @@ class GroupMetadataManagerTest { groupMetadataManager.addGroup(group) val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) - when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + when(replicaManager.topicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) val offsets = immutable.Map(topicIdPartition -> OffsetAndMetadata(offset, "", time.milliseconds())) @@ -1346,7 +1345,7 @@ class GroupMetadataManagerTest { def callback(errors: immutable.Map[TopicIdPartition, Errors]): Unit = { commitErrors = Some(errors) } - when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + when(replicaManager.topicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) val verificationGuard = new VerificationGuard() @@ -1393,7 +1392,7 @@ class GroupMetadataManagerTest { val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) val offsets = immutable.Map(topicIdPartition -> OffsetAndMetadata(offset, "", time.milliseconds())) - when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + when(replicaManager.topicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) when(replicaManager.getMagic(any())).thenReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE)) var commitErrors: Option[immutable.Map[TopicIdPartition, Errors]] = None @@ -1446,7 +1445,7 @@ class GroupMetadataManagerTest { val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) val offsets = immutable.Map(topicIdPartition -> OffsetAndMetadata(offset, "", time.milliseconds())) - when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + when(replicaManager.topicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) when(replicaManager.getMagic(any())).thenReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE)) var commitErrors: Option[immutable.Map[TopicIdPartition, Errors]] = None @@ -1538,7 +1537,7 @@ class GroupMetadataManagerTest { val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) val offsets = immutable.Map(topicIdPartition -> OffsetAndMetadata(offset, "", time.milliseconds())) - when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + when(replicaManager.topicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) when(replicaManager.getMagic(any())).thenReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE)) var commitErrors: Option[immutable.Map[TopicIdPartition, Errors]] = None @@ -1586,7 +1585,7 @@ class GroupMetadataManagerTest { groupMetadataManager.addGroup(group) val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) - when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + when(replicaManager.topicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) val offsets = immutable.Map( topicIdPartition -> OffsetAndMetadata(offset, "", time.milliseconds()), @@ -1702,7 +1701,7 @@ class GroupMetadataManagerTest { validTopicIdPartition -> OffsetAndMetadata(offset, "", time.milliseconds()) ) - when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + when(replicaManager.topicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) expectAppendMessage(Errors.NONE) @@ -1764,7 +1763,7 @@ class GroupMetadataManagerTest { val capturedResponseCallback: ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, PartitionResponse] => Unit]) when(replicaManager.getMagic(any())).thenReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE)) - when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + when(replicaManager.topicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) var commitErrors: Option[immutable.Map[TopicIdPartition, Errors]] = None @@ -1829,7 +1828,7 @@ class GroupMetadataManagerTest { groupMetadataManager.addGroup(group) val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) - when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + when(replicaManager.topicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) // expire the offset after 1 millisecond val startMs = time.milliseconds val offsets = immutable.Map( @@ -1986,7 +1985,7 @@ class GroupMetadataManagerTest { groupMetadataManager.addGroup(group) val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) - when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + when(replicaManager.topicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) // expire the offset after 1 millisecond val startMs = time.milliseconds val offsets = immutable.Map( @@ -2071,7 +2070,7 @@ class GroupMetadataManagerTest { group.initNextGeneration() val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) - when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + when(replicaManager.topicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) val startMs = time.milliseconds // old clients, expiry timestamp is explicitly set val tp1OffsetAndMetadata = OffsetAndMetadata(offset, "", startMs, startMs + 1) @@ -2251,7 +2250,7 @@ class GroupMetadataManagerTest { groupMetadataManager.addGroup(group) val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) - when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + when(replicaManager.topicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) // expire the offset after 1 and 3 milliseconds (old clients) and after default retention (new clients) val startMs = time.milliseconds // old clients, expiry timestamp is explicitly set @@ -2361,7 +2360,7 @@ class GroupMetadataManagerTest { group.transitionTo(Stable) val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) - when(replicaManager.getTopicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) + when(replicaManager.topicIdPartition(offsetTopicPartition)).thenReturn(new TopicIdPartition(groupMetadataTopicId, offsetTopicPartition)) val startMs = time.milliseconds diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala index 527f9ecbc53a5..14e69e344ad6f 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala @@ -103,8 +103,8 @@ class TransactionStateManagerTest { // make sure the transactional id hashes to the assigning partition id assertEquals(partitionId, transactionManager.partitionFor(transactionalId1)) assertEquals(partitionId, transactionManager.partitionFor(transactionalId2)) - when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 0))).thenReturn(new TopicIdPartition(transactionTopicId, 0, TRANSACTION_STATE_TOPIC_NAME)) - when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 1))).thenReturn(new TopicIdPartition(transactionTopicId, 1, TRANSACTION_STATE_TOPIC_NAME)) + when(replicaManager.topicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 0))).thenReturn(new TopicIdPartition(transactionTopicId, 0, TRANSACTION_STATE_TOPIC_NAME)) + when(replicaManager.topicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 1))).thenReturn(new TopicIdPartition(transactionTopicId, 1, TRANSACTION_STATE_TOPIC_NAME)) } @AfterEach @@ -674,8 +674,8 @@ class TransactionStateManagerTest { expectTransactionalIdExpiration(Errors.MESSAGE_TOO_LARGE, attemptedAppends) assertEquals(allTransactionalIds, listExpirableTransactionalIds()) - when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 0))).thenReturn(new TopicIdPartition(transactionTopicId, 0, TRANSACTION_STATE_TOPIC_NAME)) - when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 1))).thenReturn(new TopicIdPartition(transactionTopicId, 1, TRANSACTION_STATE_TOPIC_NAME)) + when(replicaManager.topicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 0))).thenReturn(new TopicIdPartition(transactionTopicId, 0, TRANSACTION_STATE_TOPIC_NAME)) + when(replicaManager.topicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 1))).thenReturn(new TopicIdPartition(transactionTopicId, 1, TRANSACTION_STATE_TOPIC_NAME)) transactionManager.removeExpiredTransactionalIds() verify(replicaManager, atLeastOnce()).appendRecords( anyLong(), @@ -716,8 +716,8 @@ class TransactionStateManagerTest { // No log config returned for partition 0 since it is offline when(replicaManager.getLogConfig(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, offlinePartitionId))) .thenReturn(None) - when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 0))).thenReturn(new TopicIdPartition(transactionTopicId, 0, TRANSACTION_STATE_TOPIC_NAME)) - when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 1))).thenReturn(new TopicIdPartition(transactionTopicId, 1, TRANSACTION_STATE_TOPIC_NAME)) + when(replicaManager.topicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 0))).thenReturn(new TopicIdPartition(transactionTopicId, 0, TRANSACTION_STATE_TOPIC_NAME)) + when(replicaManager.topicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 1))).thenReturn(new TopicIdPartition(transactionTopicId, 1, TRANSACTION_STATE_TOPIC_NAME)) val appendedRecords = mutable.Map.empty[TopicIdPartition, mutable.Buffer[MemoryRecords]] expectTransactionalIdExpiration(Errors.NONE, appendedRecords) @@ -764,8 +764,8 @@ class TransactionStateManagerTest { expectTransactionalIdExpiration(Errors.NONE, appendedRecords) assertEquals(allTransactionalIds, listExpirableTransactionalIds()) - when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 0))).thenReturn(new TopicIdPartition(transactionTopicId, 0, TRANSACTION_STATE_TOPIC_NAME)) - when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 1))).thenReturn(new TopicIdPartition(transactionTopicId, 1, TRANSACTION_STATE_TOPIC_NAME)) + when(replicaManager.topicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 0))).thenReturn(new TopicIdPartition(transactionTopicId, 0, TRANSACTION_STATE_TOPIC_NAME)) + when(replicaManager.topicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 1))).thenReturn(new TopicIdPartition(transactionTopicId, 1, TRANSACTION_STATE_TOPIC_NAME)) transactionManager.removeExpiredTransactionalIds() verify(replicaManager, atLeastOnce()).appendRecords( @@ -815,8 +815,8 @@ class TransactionStateManagerTest { time.sleep(txnConfig.transactionalIdExpirationMs + 1) reset(replicaManager) - when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 0))).thenReturn(new TopicIdPartition(transactionTopicId, 0, TRANSACTION_STATE_TOPIC_NAME)) - when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 1))).thenReturn(new TopicIdPartition(transactionTopicId, 1, TRANSACTION_STATE_TOPIC_NAME)) + when(replicaManager.topicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 0))).thenReturn(new TopicIdPartition(transactionTopicId, 0, TRANSACTION_STATE_TOPIC_NAME)) + when(replicaManager.topicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 1))).thenReturn(new TopicIdPartition(transactionTopicId, 1, TRANSACTION_STATE_TOPIC_NAME)) expectLogConfig(partitionIds, maxBatchSize) val appendedRecords = mutable.Map.empty[TopicIdPartition, mutable.Buffer[MemoryRecords]] @@ -1145,8 +1145,8 @@ class TransactionStateManagerTest { ) when(replicaManager.getMagic(any())) .thenReturn(Some(RecordBatch.MAGIC_VALUE_V1)) - when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 0))).thenReturn(new TopicIdPartition(transactionTopicId, 0, TRANSACTION_STATE_TOPIC_NAME)) - when(replicaManager.getTopicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 1))).thenReturn(new TopicIdPartition(transactionTopicId, 1, TRANSACTION_STATE_TOPIC_NAME)) + when(replicaManager.topicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 0))).thenReturn(new TopicIdPartition(transactionTopicId, 0, TRANSACTION_STATE_TOPIC_NAME)) + when(replicaManager.topicIdPartition(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, 1))).thenReturn(new TopicIdPartition(transactionTopicId, 1, TRANSACTION_STATE_TOPIC_NAME)) } @Test diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 20c43ac2a79a5..59e061e17442e 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -3352,7 +3352,7 @@ class KafkaApisTest extends Logging { allPartitions.foreach { tp => when(replicaManager.getMagic(tp)).thenReturn(Some(RecordBatch.MAGIC_VALUE_V2)) - when(replicaManager.getTopicIdPartition(tp)).thenReturn(new TopicIdPartition(topicIds.get(tp.topic()).getOrElse(Uuid.ZERO_UUID), tp)) + when(replicaManager.topicIdPartition(tp)).thenReturn(new TopicIdPartition(topicIds.get(tp.topic()).getOrElse(Uuid.ZERO_UUID), tp)) } when(groupCoordinator.completeTransaction( diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala index c9becf9aba1c6..405da56bdf8d5 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala @@ -290,7 +290,7 @@ class ReplicaManagerConcurrencyTest extends Logging { } val future = new CompletableFuture[ProduceResponse.PartitionResponse]() - val topicIdPartition = replicaManager.getTopicIdPartition(topicPartition) + val topicIdPartition = replicaManager.topicIdPartition(topicPartition) def produceCallback(results: collection.Map[TopicIdPartition, ProduceResponse.PartitionResponse]): Unit = { try { diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 54891d495f3aa..105b5cedaf611 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -3164,7 +3164,7 @@ class ReplicaManagerTest { requiredAcks = requiredAcks, internalTopicsAllowed = false, transactionalId = transactionalId, - entriesPerPartition = entriesToAppend.map(e => replicaManager.getTopicIdPartition(e._1) -> e._2), + entriesPerPartition = entriesToAppend.map(e => replicaManager.topicIdPartition(e._1) -> e._2), responseCallback = appendCallback, transactionSupportedOperation = transactionSupportedOperation ) @@ -3193,7 +3193,7 @@ class ReplicaManagerTest { requiredAcks = requiredAcks, internalTopicsAllowed = false, transactionalId = transactionalId, - entriesPerPartition = entriesPerPartition.map(e => replicaManager.getTopicIdPartition(e._1) -> e._2), + entriesPerPartition = entriesPerPartition.map(e => replicaManager.topicIdPartition(e._1) -> e._2), responseCallback = appendCallback, transactionSupportedOperation = transactionSupportedOperation ) From ce85e4fd4740d9de579c57349a72dcbe6e6eff2f Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Tue, 15 Oct 2024 16:32:07 +0100 Subject: [PATCH 22/40] address feedback --- .../api/ProducerSendWhileDeletionTest.scala | 19 ++++--------------- 1 file changed, 4 insertions(+), 15 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala index bd1237ebb8e3f..7c2e6ce8178e6 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala @@ -147,18 +147,13 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness { } val reassignment = Map( - partition0 -> Optional.of(new NewPartitionReassignment(util.Arrays.asList(2, 1))), - partition1 -> Optional.of(new NewPartitionReassignment(util.Arrays.asList(2, 1))) + partition0 -> Optional.of(new NewPartitionReassignment(util.Arrays.asList(1, 2))), + partition1 -> Optional.of(new NewPartitionReassignment(util.Arrays.asList(1, 2))) ) - // Change assignment of one of the replicas from 0 to 2 + // Change assignment of one of the replicas from 0 to 2. Leadership moves be 1. admin.alterPartitionReassignments(reassignment.asJava).all().get() - - TestUtils.waitUntilTrue( - () => partitionLeader(admin, partition0) == 2 && partitionLeader(admin, partition1) == 2, - s"Expected preferred leader to become 2, but is ${partitionLeader(admin, partition0)} and ${partitionLeader(admin, partition1)}", - 10000) - TestUtils.assertLeader(admin, partition1, 2) + TestUtils.assertLeader(admin, partition1, 1) assertEquals(topicDetails.topicId(), topicMetadata(admin, topic).topicId()) // Producer should be able to send messages even after topic gets reassigned @@ -168,10 +163,4 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness { def topicMetadata(admin: Admin, topic: String): TopicDescription = { admin.describeTopics(util.Collections.singletonList(topic)).allTopicNames().get().get(topic) } - - private def partitionLeader(admin: Admin, topicPartition: TopicPartition): Int = { - val partitionMetadata = topicMetadata(admin, topicPartition.topic).partitions.get(topicPartition.partition) - val preferredLeaderMetadata = partitionMetadata.leader() - preferredLeaderMetadata.id - } } From 67e7bd24f17f0c464c7f81a8498339549ff13c96 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Tue, 29 Oct 2024 13:59:54 +0000 Subject: [PATCH 23/40] feedback --- .../group/CoordinatorPartitionWriter.scala | 5 +++-- core/src/main/scala/kafka/server/KafkaApis.scala | 4 ++-- .../main/scala/kafka/server/ReplicaManager.scala | 16 ++++++++-------- 3 files changed, 13 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala b/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala index 6b01cf4e9d320..e49d344c46bc6 100644 --- a/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala +++ b/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala @@ -140,12 +140,13 @@ class CoordinatorPartitionWriter( records: MemoryRecords ): Long = { var appendResults: Map[TopicIdPartition, PartitionResponse] = Map.empty + val topicIdPartition = replicaManager.topicIdPartition(tp) replicaManager.appendRecords( timeout = 0L, requiredAcks = 1, internalTopicsAllowed = true, origin = AppendOrigin.COORDINATOR, - entriesPerPartition = Map(replicaManager.topicIdPartition(tp) -> records), + entriesPerPartition = Map(topicIdPartition -> records), responseCallback = results => appendResults = results, requestLocal = RequestLocal.noCaching, verificationGuards = Map(tp -> verificationGuard), @@ -155,7 +156,7 @@ class CoordinatorPartitionWriter( actionQueue = directActionQueue ) - val partitionResult = appendResults.getOrElse(replicaManager.topicIdPartition(tp), + val partitionResult = appendResults.getOrElse(topicIdPartition, throw new IllegalStateException(s"Append status $appendResults should have partition $tp.")) if (partitionResult.error != Errors.NONE) { diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index cfff031f59355..b831d3e6cc817 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -2499,8 +2499,8 @@ class KafkaApis(val requestChannel: RequestChannel, entriesPerPartition = controlRecords, requestLocal = requestLocal, responseCallback = errors => { - errors.foreachEntry { (tp, partitionResponse) => - markerResults.put(tp.topicPartition(), partitionResponse.error) + errors.foreachEntry { (topicIdPartition, partitionResponse) => + markerResults.put(topicIdPartition.topicPartition(), partitionResponse.error) } maybeComplete() } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 310434ed1b258..5c4744ac03ae2 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -915,7 +915,7 @@ class ReplicaManager(val config: KafkaConfig, hasCustomErrorMessage = customException.isDefined ) } - val entriesWithoutErrorsPerPartition = entriesPerPartition.filter { case (key, _) => !errorResults.exists(_._1.topicPartition() == key.topicPartition()) } + val entriesWithoutErrorsPerPartition = entriesPerPartition.filter { case (key, _) => !errorResults.contains(key) } val preAppendPartitionResponses = buildProducePartitionStatus(errorResults).map { case (k, status) => k -> status.responseStatus } @@ -1439,15 +1439,15 @@ class ReplicaManager(val config: KafkaConfig, verificationGuards: Map[TopicPartition, VerificationGuard]): Map[TopicOptionalIdPartition, LogAppendResult] = { val traceEnabled = isTraceEnabled - def processFailedRecord(topicPartition: TopicPartition, t: Throwable) = { - val logStartOffset = onlinePartition(topicPartition).map(_.logStartOffset).getOrElse(-1L) - brokerTopicStats.topicStats(topicPartition.topic).failedProduceRequestRate.mark() + def processFailedRecord(topicIdPartition: TopicIdPartition, t: Throwable) = { + val logStartOffset = onlinePartition(topicIdPartition.topicPartition()).map(_.logStartOffset).getOrElse(-1L) + brokerTopicStats.topicStats(topicIdPartition.topic).failedProduceRequestRate.mark() brokerTopicStats.allTopicsStats.failedProduceRequestRate.mark() t match { case _: InvalidProducerEpochException => - info(s"Error processing append operation on partition $topicPartition", t) + info(s"Error processing append operation on partition $topicIdPartition", t) case _ => - error(s"Error processing append operation on partition $topicPartition", t) + error(s"Error processing append operation on partition $topicIdPartition", t) } logStartOffset @@ -1498,12 +1498,12 @@ class ReplicaManager(val config: KafkaConfig, _: KafkaStorageException) => (new TopicOptionalIdPartition(Optional.empty(), topicIdPartition.topicPartition()), LogAppendResult(LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, Some(e), hasCustomErrorMessage = false)) case rve: RecordValidationException => - val logStartOffset = processFailedRecord(topicIdPartition.topicPartition(), rve.invalidException) + val logStartOffset = processFailedRecord(topicIdPartition, rve.invalidException) val recordErrors = rve.recordErrors (new TopicOptionalIdPartition(Optional.empty(), topicIdPartition.topicPartition()), LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithAdditionalInfo(logStartOffset, recordErrors), Some(rve.invalidException), hasCustomErrorMessage = true)) case t: Throwable => - val logStartOffset = processFailedRecord(topicIdPartition.topicPartition(), t) + val logStartOffset = processFailedRecord(topicIdPartition, t) (new TopicOptionalIdPartition(Optional.empty(), topicIdPartition.topicPartition()), LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithLogStartOffset(logStartOffset), Some(t), hasCustomErrorMessage = false)) } From 270ab275d7fa2f50ae0ccec78a72e0ce3a295967 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Wed, 6 Nov 2024 13:21:45 +0000 Subject: [PATCH 24/40] fix tests --- .../scala/kafka/server/ReplicaManager.scala | 15 ++++----- .../kafka/server/ReplicaManagerTest.scala | 32 +++++++------------ 2 files changed, 18 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 5c4744ac03ae2..ab1e39c36ad7c 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -1462,13 +1462,13 @@ class ReplicaManager(val config: KafkaConfig, // reject appending to internal topics if it is not allowed if (Topic.isInternal(topicIdPartition.topic) && !internalTopicsAllowed) { - (new TopicOptionalIdPartition(Optional.empty(), topicIdPartition.topicPartition()), LogAppendResult( + (new TopicOptionalIdPartition(Optional.ofNullable(topicIdPartition.topicId()), topicIdPartition.topicPartition()), LogAppendResult( LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, Some(new InvalidTopicException(s"Cannot append to internal topic ${topicIdPartition.topic}")), hasCustomErrorMessage = false)) } else { try { - val partition = getPartitionOrException(topicIdPartition.topicPartition()) + val partition: Partition = getPartitionOrException(topicIdPartition.topicPartition()) val info = partition.appendRecordsToLeader(records, origin, requiredAcks, requestLocal, verificationGuards.getOrElse(topicIdPartition.topicPartition(), VerificationGuard.SENTINEL)) val numAppendedMessages = info.numMessages @@ -1483,10 +1483,9 @@ class ReplicaManager(val config: KafkaConfig, trace(s"${records.sizeInBytes} written to log $topicIdPartition beginning at offset " + s"${info.firstOffset} and ending at offset ${info.lastOffset}") - var topicId: Optional[Uuid] = Optional.empty() - if (partition.topicId.isDefined) topicId = Optional.of(partition.topicId.get) - + val topicId: Optional[Uuid] = partition.topicId.toJava (new TopicOptionalIdPartition(topicId, topicIdPartition.topicPartition()), LogAppendResult(info, exception = None, hasCustomErrorMessage = false)) + } catch { // NOTE: Failed produce requests metric is not incremented for known exceptions // it is supposed to indicate un-expected failures of a broker in handling a produce request @@ -1496,15 +1495,15 @@ class ReplicaManager(val config: KafkaConfig, _: RecordBatchTooLargeException | _: CorruptRecordException | _: KafkaStorageException) => - (new TopicOptionalIdPartition(Optional.empty(), topicIdPartition.topicPartition()), LogAppendResult(LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, Some(e), hasCustomErrorMessage = false)) + (new TopicOptionalIdPartition(Optional.ofNullable(topicIdPartition.topicId()), topicIdPartition.topicPartition()), LogAppendResult(LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, Some(e), hasCustomErrorMessage = false)) case rve: RecordValidationException => val logStartOffset = processFailedRecord(topicIdPartition, rve.invalidException) val recordErrors = rve.recordErrors - (new TopicOptionalIdPartition(Optional.empty(), topicIdPartition.topicPartition()), LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithAdditionalInfo(logStartOffset, recordErrors), + (new TopicOptionalIdPartition(Optional.ofNullable(topicIdPartition.topicId()), topicIdPartition.topicPartition()), LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithAdditionalInfo(logStartOffset, recordErrors), Some(rve.invalidException), hasCustomErrorMessage = true)) case t: Throwable => val logStartOffset = processFailedRecord(topicIdPartition, t) - (new TopicOptionalIdPartition(Optional.empty(), topicIdPartition.topicPartition()), LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithLogStartOffset(logStartOffset), + (new TopicOptionalIdPartition(Optional.ofNullable(topicIdPartition.topicId()), topicIdPartition.topicPartition()), LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithLogStartOffset(logStartOffset), Some(t), hasCustomErrorMessage = false)) } } diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index a622fec9aba94..aa6920723085c 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -318,7 +318,7 @@ class ReplicaManagerTest { .setPartitionEpoch(0) .setReplicas(Seq[Integer](0).asJava) .setIsNew(false)).asJava, - Collections.singletonMap(topic, Uuid.randomUuid()), + Collections.singletonMap(topic, topicId), Set(new Node(0, "host1", 0)).asJava).build(), (_, _) => ()) appendRecords(rm, new TopicPartition(topic, 0), MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("first message".getBytes()), new SimpleRecord("second message".getBytes()))) @@ -356,7 +356,6 @@ class ReplicaManagerTest { mockGetAliveBrokerFunctions(metadataCache, Seq(new Node(0, "host0", 0))) when(metadataCache.metadataVersion()).thenReturn(config.interBrokerProtocolVersion) val tp0 = new TopicPartition(topic, 0) - val uuid = Uuid.randomUuid() val rm = new ReplicaManager( metrics = metrics, config = config, @@ -371,7 +370,7 @@ class ReplicaManagerTest { try { val partition = rm.createPartition(tp0) partition.createLogIfNotExists(isNew = false, isFutureReplica = false, - new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints.asJava), Option.apply(uuid)) + new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints.asJava), Option.apply(topicId)) val response = rm.becomeLeaderOrFollower(0, new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, Seq(new LeaderAndIsrPartitionState() @@ -384,7 +383,7 @@ class ReplicaManagerTest { .setPartitionEpoch(0) .setReplicas(Seq[Integer](0).asJava) .setIsNew(false)).asJava, - Collections.singletonMap(topic, uuid), + Collections.singletonMap(topic, topicId), Set(new Node(0, "host1", 0)).asJava).build(), (_, _) => ()) // expect the errorCounts only has 1 entry with Errors.NONE val errorCounts = response.errorCounts() @@ -399,11 +398,11 @@ class ReplicaManagerTest { } else { val mockLog = mock(classOf[UnifiedLog]) when(spyLogManager.getLog(tp0, isFuture = true)).thenReturn(Option.apply(mockLog)) - when(mockLog.topicId).thenReturn(Option.apply(uuid)) + when(mockLog.topicId).thenReturn(Option.apply(topicId)) when(mockLog.parentDir).thenReturn(dir2.getAbsolutePath) } - val topicIdMap: Map[String, Option[Uuid]] = Map(topic -> Option.apply(uuid)) + val topicIdMap: Map[String, Option[Uuid]] = Map(topic -> Option.apply(topicId)) rm.maybeAddLogDirFetchers(Set(partition), new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints.asJava), topicIdMap) if (futureLogCreated) { // since the futureLog is already created, we don't have to abort and pause the cleaning @@ -440,7 +439,7 @@ class ReplicaManagerTest { try { val brokerList = Seq[Integer](0, 1).asJava - val topicIds = Collections.singletonMap(topic, Uuid.randomUuid()) + val topicIds = Collections.singletonMap(topic, topicId) val partition = rm.createPartition(new TopicPartition(topic, 0)) partition.createLogIfNotExists(isNew = false, isFutureReplica = false, @@ -617,7 +616,7 @@ class ReplicaManagerTest { .setPartitionEpoch(0) .setReplicas(brokerList) .setIsNew(true)).asJava, - Collections.singletonMap(topic, Uuid.randomUuid()), + Collections.singletonMap(topic, topicId), Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ()) replicaManager.getPartitionOrException(new TopicPartition(topic, 0)) @@ -682,7 +681,7 @@ class ReplicaManagerTest { .setPartitionEpoch(0) .setReplicas(brokerList) .setIsNew(true)).asJava, - Collections.singletonMap(topic, Uuid.randomUuid()), + Collections.singletonMap(topic, topicId), Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava, false, AbstractControlRequest.Type.UNKNOWN @@ -1304,7 +1303,6 @@ class ReplicaManagerTest { // Create 2 partitions, assign replica 0 as the leader for both a different follower (1 and 2) for each val tp0 = new TopicPartition(topic, 0) val tp1 = new TopicPartition(topic, 1) - val topicId = Uuid.randomUuid() val tidp0 = new TopicIdPartition(topicId, tp0) val tidp1 = new TopicIdPartition(topicId, tp1) val offsetCheckpoints = new LazyOffsetCheckpoints(replicaManager.highWatermarkCheckpoints.asJava) @@ -1417,7 +1415,6 @@ class ReplicaManagerTest { private def verifyBecomeFollowerWhenLeaderIsUnchangedButMissedLeaderUpdate(extraProps: Properties, expectTruncation: Boolean): Unit = { val topicPartition = 0 - val topicId = Uuid.randomUuid() val followerBrokerId = 0 val leaderBrokerId = 1 val controllerId = 0 @@ -1509,7 +1506,6 @@ class ReplicaManagerTest { @Test def testPreferredReplicaAsFollower(): Unit = { val topicPartition = 0 - val topicId = Uuid.randomUuid() val followerBrokerId = 0 val leaderBrokerId = 1 val leaderEpoch = 1 @@ -1563,7 +1559,6 @@ class ReplicaManagerTest { @Test def testPreferredReplicaAsLeader(): Unit = { val topicPartition = 0 - val topicId = Uuid.randomUuid() val followerBrokerId = 0 val leaderBrokerId = 1 val leaderEpoch = 1 @@ -1625,7 +1620,6 @@ class ReplicaManagerTest { val leaderNode = new Node(leaderBrokerId, "host1", 0, "rack-a") val followerNode = new Node(followerBrokerId, "host2", 1, "rack-b") val brokerList = Seq[Integer](leaderBrokerId, followerBrokerId).asJava - val topicId = Uuid.randomUuid() val tp0 = new TopicPartition(topic, 0) val tidp0 = new TopicIdPartition(topicId, tp0) @@ -1702,7 +1696,6 @@ class ReplicaManagerTest { val leaderBrokerId = 0 val followerBrokerId = 1 val brokerList = Seq[Integer](leaderBrokerId, followerBrokerId).asJava - val topicId = Uuid.randomUuid() val tp0 = new TopicPartition(topic, 0) val tidp0 = new TopicIdPartition(topicId, tp0) @@ -1752,7 +1745,6 @@ class ReplicaManagerTest { val leaderBrokerId = 0 val followerBrokerId = 1 val brokerList = Seq[Integer](leaderBrokerId, followerBrokerId).asJava - val topicId = Uuid.randomUuid() val tp0 = new TopicPartition(topic, 0) val tidp0 = new TopicIdPartition(topicId, tp0) @@ -5249,16 +5241,14 @@ class ReplicaManagerTest { val localId = 1 val otherId = localId + 1 val numOfRecords = 3 - val topicIdPartition = new TopicIdPartition(Uuid.fromString("zUaJeJpgd2GaTH28tggYK2"), 0, "foo") - val topicPartition = topicIdPartition.topicPartition() + val topicPartition = new TopicPartition("foo", 0) val replicaManager = setupReplicaManagerWithMockedPurgatories(new MockTimer(time), localId, enableRemoteStorage = enableRemoteStorage) try { // Make the local replica the leader val leaderTopicsDelta = topicsCreateDelta(localId, true) val leaderMetadataImage = imageFromTopics(leaderTopicsDelta.apply()) - val topicId = leaderMetadataImage.topics().topicsByName.get("foo").id - val topicIdPartition = new TopicIdPartition(topicId, topicPartition) + val topicIdPartition = new TopicIdPartition(FOO_UUID, topicPartition) replicaManager.applyDelta(leaderTopicsDelta, leaderMetadataImage) @@ -5619,7 +5609,7 @@ class ReplicaManagerTest { val localId = 1 val otherId = localId + 1 val numOfRecords = 3 - val topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), 0, "foo") + val topicIdPartition = new TopicIdPartition(FOO_UUID, 0, "foo") val topicPartition = topicIdPartition.topicPartition() val replicaManager = setupReplicaManagerWithMockedPurgatories(new MockTimer(time), localId, enableRemoteStorage = enableRemoteStorage) From bed9c8e4385c2c90b14e9163784756359da235e4 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Wed, 6 Nov 2024 16:16:30 +0000 Subject: [PATCH 25/40] add partitionOrException(topicIdPartition) --- .../scala/kafka/server/ReplicaManager.scala | 23 +++++++++++++++++-- .../share/SharePartitionManagerTest.java | 6 ++--- 2 files changed, 24 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index ab1e39c36ad7c..5a8d1a4d270be 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -735,6 +735,24 @@ class ReplicaManager(val config: KafkaConfig, } } + def getPartitionOrException(topicIdPartition: TopicIdPartition): Partition = { + getPartitionOrError(topicIdPartition.topicPartition()) match { + case Left(Errors.KAFKA_STORAGE_ERROR) => + throw new KafkaStorageException(s"Partition ${topicIdPartition.topicPartition()} is in an offline log directory") + + case Left(error) => + throw error.exception(s"Error while fetching partition state for ${topicIdPartition.topicPartition()}") + + case Right(partition) => + val topicId = partition.topicId + if (topicId.contains(topicIdPartition.topicId())) { + partition + } else { + throw new UnknownTopicIdException(s"Partition $topicIdPartition's topic id doesn't match the one on disk $topicId.'") + } + } + } + def getPartitionOrError(topicPartition: TopicPartition): Either[Errors, Partition] = { getPartition(topicPartition) match { case HostedPartition.Online(partition) => @@ -1468,7 +1486,7 @@ class ReplicaManager(val config: KafkaConfig, hasCustomErrorMessage = false)) } else { try { - val partition: Partition = getPartitionOrException(topicIdPartition.topicPartition()) + val partition = getPartitionOrException(topicIdPartition) val info = partition.appendRecordsToLeader(records, origin, requiredAcks, requestLocal, verificationGuards.getOrElse(topicIdPartition.topicPartition(), VerificationGuard.SENTINEL)) val numAppendedMessages = info.numMessages @@ -1494,7 +1512,8 @@ class ReplicaManager(val config: KafkaConfig, _: RecordTooLargeException | _: RecordBatchTooLargeException | _: CorruptRecordException | - _: KafkaStorageException) => + _: KafkaStorageException | + _: UnknownTopicIdException) => (new TopicOptionalIdPartition(Optional.ofNullable(topicIdPartition.topicId()), topicIdPartition.topicPartition()), LogAppendResult(LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, Some(e), hasCustomErrorMessage = false)) case rve: RecordValidationException => val logStartOffset = processFailedRecord(topicIdPartition, rve.invalidException) diff --git a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java index 4e461d3df3b24..a051fa6fb4da5 100644 --- a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java +++ b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java @@ -155,7 +155,7 @@ public void setUp() { new SystemTimer("sharePartitionManagerTestTimer")); mockReplicaManager = mock(ReplicaManager.class); Partition partition = mockPartition(); - when(mockReplicaManager.getPartitionOrException(Mockito.any())).thenReturn(partition); + when(mockReplicaManager.getPartitionOrException(Mockito.any(TopicPartition.class))).thenReturn(partition); } @AfterEach @@ -2171,7 +2171,7 @@ public void testSharePartitionInitializationFailure() throws Exception { ReplicaManager replicaManager = mock(ReplicaManager.class); // First check should throw KafkaStorageException, second check should return partition which // is not leader. - when(replicaManager.getPartitionOrException(any())) + when(replicaManager.getPartitionOrException(any(TopicPartition.class))) .thenThrow(new KafkaStorageException("Exception")) .thenReturn(partition); SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder() @@ -2212,7 +2212,7 @@ public void testSharePartitionPartialInitializationFailure() throws Exception { when(partition1.isLeader()).thenReturn(false); ReplicaManager replicaManager = mock(ReplicaManager.class); - when(replicaManager.getPartitionOrException(any())) + when(replicaManager.getPartitionOrException(any(TopicPartition.class))) .thenReturn(partition1); SharePartition sp1 = mock(SharePartition.class); From f8830b16281859fea333d0517aadf73832b896da Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Wed, 6 Nov 2024 17:14:06 +0000 Subject: [PATCH 26/40] cleanup --- .../main/scala/kafka/server/ReplicaManager.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 5a8d1a4d270be..24df40c6d1376 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -1479,8 +1479,10 @@ class ReplicaManager(val config: KafkaConfig, brokerTopicStats.allTopicsStats.totalProduceRequestRate.mark() // reject appending to internal topics if it is not allowed + val topicOptionalIdPartition = new TopicOptionalIdPartition(Optional.ofNullable(topicIdPartition.topicId()), + topicIdPartition.topicPartition()) if (Topic.isInternal(topicIdPartition.topic) && !internalTopicsAllowed) { - (new TopicOptionalIdPartition(Optional.ofNullable(topicIdPartition.topicId()), topicIdPartition.topicPartition()), LogAppendResult( + (topicOptionalIdPartition, LogAppendResult( LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, Some(new InvalidTopicException(s"Cannot append to internal topic ${topicIdPartition.topic}")), hasCustomErrorMessage = false)) @@ -1501,8 +1503,7 @@ class ReplicaManager(val config: KafkaConfig, trace(s"${records.sizeInBytes} written to log $topicIdPartition beginning at offset " + s"${info.firstOffset} and ending at offset ${info.lastOffset}") - val topicId: Optional[Uuid] = partition.topicId.toJava - (new TopicOptionalIdPartition(topicId, topicIdPartition.topicPartition()), LogAppendResult(info, exception = None, hasCustomErrorMessage = false)) + (topicOptionalIdPartition, LogAppendResult(info, exception = None, hasCustomErrorMessage = false)) } catch { // NOTE: Failed produce requests metric is not incremented for known exceptions @@ -1514,21 +1515,22 @@ class ReplicaManager(val config: KafkaConfig, _: CorruptRecordException | _: KafkaStorageException | _: UnknownTopicIdException) => - (new TopicOptionalIdPartition(Optional.ofNullable(topicIdPartition.topicId()), topicIdPartition.topicPartition()), LogAppendResult(LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, Some(e), hasCustomErrorMessage = false)) + (topicOptionalIdPartition, LogAppendResult(LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, Some(e), hasCustomErrorMessage = false)) case rve: RecordValidationException => val logStartOffset = processFailedRecord(topicIdPartition, rve.invalidException) val recordErrors = rve.recordErrors - (new TopicOptionalIdPartition(Optional.ofNullable(topicIdPartition.topicId()), topicIdPartition.topicPartition()), LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithAdditionalInfo(logStartOffset, recordErrors), + (topicOptionalIdPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithAdditionalInfo(logStartOffset, recordErrors), Some(rve.invalidException), hasCustomErrorMessage = true)) case t: Throwable => val logStartOffset = processFailedRecord(topicIdPartition, t) - (new TopicOptionalIdPartition(Optional.ofNullable(topicIdPartition.topicId()), topicIdPartition.topicPartition()), LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithLogStartOffset(logStartOffset), + (topicOptionalIdPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithLogStartOffset(logStartOffset), Some(t), hasCustomErrorMessage = false)) } } } } + def fetchOffset(topics: Seq[ListOffsetsTopic], duplicatePartitions: Set[TopicPartition], isolationLevel: IsolationLevel, From d90580e182cf254e5b82be69174c71bf56d5fb97 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Wed, 6 Nov 2024 17:32:50 +0000 Subject: [PATCH 27/40] cleanup --- .../src/main/java/org/apache/kafka/clients/ApiVersions.java | 3 ++- .../org/apache/kafka/clients/producer/internals/Sender.java | 1 - 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ApiVersions.java b/clients/src/main/java/org/apache/kafka/clients/ApiVersions.java index 3127be7355aed..35b504ded26ea 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ApiVersions.java +++ b/clients/src/main/java/org/apache/kafka/clients/ApiVersions.java @@ -88,4 +88,5 @@ private byte computeMaxUsableProduceMagic() { public synchronized byte maxUsableProduceMagic() { return maxUsableProduceMagic; } -} + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index c2fcecf93709c..f878964af2348 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -612,7 +612,6 @@ private void handleProduceResponse(ClientResponse response, Map partitionsWithUpdatedLeaderInfo = new HashMap<>(); produceResponse.data().responses().forEach(r -> r.partitionResponses().forEach(p -> { - // Version 12 drop topic name and add support to topic id. However, metadata can be used to map topic id to topic name. String topicName = metadata.topicNames().getOrDefault(r.topicId(), r.name()); TopicPartition tp = new TopicPartition(topicName, p.index()); From 33ba2b090732304117a377b0b423580745a89bc3 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Wed, 6 Nov 2024 17:42:52 +0000 Subject: [PATCH 28/40] handle topicid zero or null when get partition --- core/src/main/scala/kafka/server/ReplicaManager.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 24df40c6d1376..9f162c725ffd6 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -745,7 +745,9 @@ class ReplicaManager(val config: KafkaConfig, case Right(partition) => val topicId = partition.topicId - if (topicId.contains(topicIdPartition.topicId())) { + // If topic id is set to zero fall back to non topic id aware behaviour + val topicIdNotProvided = topicIdPartition.topicId() == Uuid.ZERO_UUID || topicIdPartition.topicId() == null + if (topicIdNotProvided || topicId.contains(topicIdPartition.topicId())) { partition } else { throw new UnknownTopicIdException(s"Partition $topicIdPartition's topic id doesn't match the one on disk $topicId.'") From 92bad703e58377abea894a465667aba83c3943e1 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Wed, 6 Nov 2024 17:56:04 +0000 Subject: [PATCH 29/40] refactor --- .../org/apache/kafka/clients/producer/internals/Sender.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index f878964af2348..5be0a1ceb6365 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -896,10 +896,10 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo // which is supporting the new magic version to one which doesn't, then we will need to convert. if (!records.hasMatchingMagic(minUsedMagic)) records = batch.records().downConvert(minUsedMagic, 0, time).records(); - ProduceRequestData.TopicProduceData tpData = tpd.find(tp.topic(), topicIds.get(tp.topic())); + Uuid topicId = topicIds.get(tp.topic()); + ProduceRequestData.TopicProduceData tpData = tpd.find(tp.topic(), topicId); if (tpData == null) { - Uuid topicId = metadata.topicIds().getOrDefault(tp.topic(), Uuid.ZERO_UUID); tpData = new ProduceRequestData.TopicProduceData() .setTopicId(topicId).setName(tp.topic()); tpd.add(tpData); From c57cc634562adbb55e0ff498d7141230a26098ec Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Thu, 7 Nov 2024 14:46:29 +0000 Subject: [PATCH 30/40] fix test --- core/src/main/scala/kafka/server/ReplicaManager.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 9f162c725ffd6..73cb0e5bfe5e2 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -744,10 +744,11 @@ class ReplicaManager(val config: KafkaConfig, throw error.exception(s"Error while fetching partition state for ${topicIdPartition.topicPartition()}") case Right(partition) => - val topicId = partition.topicId - // If topic id is set to zero fall back to non topic id aware behaviour + // Get topic id for an existing partition from disk if topicId is none get it from the metadata cache + val topicId = partition.topicId.getOrElse(metadataCache.getTopicId(topicIdPartition.topic())) + // If topic id is set to zero or null fall back to non topic id aware behaviour val topicIdNotProvided = topicIdPartition.topicId() == Uuid.ZERO_UUID || topicIdPartition.topicId() == null - if (topicIdNotProvided || topicId.contains(topicIdPartition.topicId())) { + if (topicIdNotProvided || topicId == topicIdPartition.topicId()) { partition } else { throw new UnknownTopicIdException(s"Partition $topicIdPartition's topic id doesn't match the one on disk $topicId.'") From 44da5ba52e0bd38bf2e84964253c7e3283b8d2f0 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Tue, 3 Dec 2024 06:54:19 +0000 Subject: [PATCH 31/40] address some of the feedback --- .../org/apache/kafka/clients/ApiVersions.java | 1 - .../org/apache/kafka/common/utils/Utils.java | 8 ++-- .../producer/internals/SenderTest.java | 6 +-- .../common/requests/RequestResponseTest.java | 9 +++-- .../transaction/TransactionStateManager.scala | 2 +- .../main/scala/kafka/server/KafkaApis.scala | 6 +-- .../scala/kafka/server/ReplicaManager.scala | 39 +++++++++---------- .../kafka/api/AuthorizerIntegrationTest.scala | 5 ++- .../api/ProducerSendWhileDeletionTest.scala | 8 ++-- .../kafka/server/EdgeCaseRequestTest.scala | 1 - .../unit/kafka/server/KafkaApisTest.scala | 1 - .../kafka/server/ReplicaManagerTest.scala | 4 +- 12 files changed, 45 insertions(+), 45 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ApiVersions.java b/clients/src/main/java/org/apache/kafka/clients/ApiVersions.java index 35b504ded26ea..6363070d8a52d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ApiVersions.java +++ b/clients/src/main/java/org/apache/kafka/clients/ApiVersions.java @@ -88,5 +88,4 @@ private byte computeMaxUsableProduceMagic() { public synchronized byte maxUsableProduceMagic() { return maxUsableProduceMagic; } - } \ No newline at end of file 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 43cd66177c99c..1bd81006d7023 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 @@ -1689,9 +1689,11 @@ public static ConfigDef mergeConfigs(List configDefs) { * Convert a map's keys to another type. */ public static Map convertKeys(Map originalMap, Function converter) { - Map newMap = new HashMap<>(); - originalMap.forEach((key, value) -> newMap.put(converter.apply(key), value)); - return newMap; + return originalMap.entrySet().stream() + .collect(Collectors.toMap( + entry -> converter.apply(entry.getKey()), + Map.Entry::getValue + )); } /** 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 56d6c5d17171b..dd5f9cb45c797 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 @@ -595,11 +595,10 @@ public void testNodeLatencyStats() throws Exception { new BufferPool(totalSize, batchSize, m, time, "producer-internal-metrics")); SenderMetricsRegistry senderMetrics = new SenderMetricsRegistry(m); - ApiVersions apiVersions1 = new ApiVersions(); - apiVersions1.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 12)); + apiVersions.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 12)); Sender sender = new Sender(logContext, client, metadata, this.accumulator, false, MAX_REQUEST_SIZE, ACKS_ALL, 1, - senderMetrics, time, REQUEST_TIMEOUT, 1000L, null, apiVersions1); + senderMetrics, time, REQUEST_TIMEOUT, 1000L, null, apiVersions); // Produce and send batch. long time1 = time.milliseconds(); @@ -2441,7 +2440,6 @@ private void testSplitBatchAndSend(TransactionManager txnManager, String metricGrpName = "producer-metrics"; // Set a good compression ratio. CompressionRatioEstimator.setEstimation(topic, CompressionType.GZIP, 0.2f); - ApiVersions apiVersions = new ApiVersions(); try (Metrics m = new Metrics()) { accumulator = new RecordAccumulator(logContext, batchSize, Compression.gzip().build(), 0, 0L, 0L, deliveryTimeoutMs, m, metricGrpName, time, apiVersions, txnManager, diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 6d5a246344a4e..9911e30958210 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -473,8 +473,9 @@ public void cannotUseFindCoordinatorV0ToFindTransactionCoordinator() { @Test public void testProduceRequestPartitionSize() { Uuid topicId = Uuid.fromString("e9TvBGX5JkYAB0AQorYD4w"); - TopicIdPartition tp0 = createTopicIdPartition(topicId, 0); - TopicIdPartition tp1 = createTopicIdPartition(topicId, 1); + String topicName = "foo"; + TopicIdPartition tp0 = createTopicIdPartition(topicId, 0, topicName); + TopicIdPartition tp1 = createTopicIdPartition(topicId, 1, topicName); MemoryRecords records0 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, Compression.NONE, new SimpleRecord("woot".getBytes())); MemoryRecords records1 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, @@ -2594,8 +2595,8 @@ private static ProduceRequestData.TopicProduceData createTopicProduceData(short return topicProduceData; } - private static TopicIdPartition createTopicIdPartition(Uuid topicId, int partitionIndex) { - return new TopicIdPartition(topicId, partitionIndex, ""); + private static TopicIdPartition createTopicIdPartition(Uuid topicId, int partitionIndex, String topicName) { + return new TopicIdPartition(topicId, partitionIndex, topicName); } @SuppressWarnings("deprecation") diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala index 28058cc93f8ef..f0c019bb1087c 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala @@ -49,7 +49,7 @@ import scala.collection.mutable /** * Transaction state manager is part of the transaction coordinator, it manages: * - * 1. the transaction log, which is a special internal topic.Ëš + * 1. the transaction log, which is a special internal topic. * 2. the transaction metadata including its ongoing transaction status. * 3. the background expiration of the transaction as well as the transactional id. * diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index e22504628f2b1..3cb5ba0f6fb12 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -677,19 +677,19 @@ class KafkaApis(val requestChannel: RequestChannel, var errorInResponse = false val nodeEndpoints = new mutable.HashMap[Int, Node] - mergedResponseStatus.foreachEntry { (topicPartition, status) => + mergedResponseStatus.foreachEntry { (topicIdPartition, status) => if (status.error != Errors.NONE) { errorInResponse = true debug("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( request.header.correlationId, request.header.clientId, - topicPartition, + topicIdPartition, status.error.exceptionName)) if (request.header.apiVersion >= 10) { status.error match { case Errors.NOT_LEADER_OR_FOLLOWER => - val leaderNode = getCurrentLeader(topicPartition.topicPartition(), request.context.listenerName) + val leaderNode = getCurrentLeader(topicIdPartition.topicPartition(), request.context.listenerName) leaderNode.node.foreach { node => nodeEndpoints.put(node.id(), node) } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 0b16f23df9077..2e11eac811b86 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -56,7 +56,7 @@ import org.apache.kafka.image.{LocalReplicaChanges, MetadataImage, TopicsDelta} import org.apache.kafka.metadata.LeaderAndIsr import org.apache.kafka.metadata.LeaderConstants.NO_LEADER import org.apache.kafka.server.{ActionQueue, DelayedActionQueue, common} -import org.apache.kafka.server.common.{DirectoryEventHandler, RequestLocal, StopPartition, TopicOptionalIdPartition} +import org.apache.kafka.server.common.{DirectoryEventHandler, RequestLocal, StopPartition} import org.apache.kafka.server.common.MetadataVersion._ import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.network.BrokerEndPoint @@ -840,15 +840,13 @@ class ReplicaManager(val config: KafkaConfig, } val sTime = time.milliseconds - val localProduceResultsWithTopicId = appendToLocalLog(internalTopicsAllowed = internalTopicsAllowed, + val localProduceResults = appendToLocalLog(internalTopicsAllowed = internalTopicsAllowed, origin, entriesPerPartition, requiredAcks, requestLocal, verificationGuards.toMap) debug("Produce to local log in %d ms".format(time.milliseconds - sTime)) - val localProduceResults : Map[TopicIdPartition, LogAppendResult] = localProduceResultsWithTopicId.map { - case(k, v) => (new TopicIdPartition(k.topicId().orElse(Uuid.ZERO_UUID), k.topicPartition()), v)} val produceStatus = buildProducePartitionStatus(localProduceResults) - addCompletePurgatoryAction(actionQueue, localProduceResultsWithTopicId) + addCompletePurgatoryAction(actionQueue, localProduceResults) recordValidationStatsCallback(localProduceResults.map { case (k, v) => k -> v.info.recordValidationStats }) @@ -1000,19 +998,19 @@ class ReplicaManager(val config: KafkaConfig, private def addCompletePurgatoryAction( actionQueue: ActionQueue, - appendResults: Map[TopicOptionalIdPartition, LogAppendResult] + appendResults: Map[TopicIdPartition, LogAppendResult] ): Unit = { actionQueue.add { - () => appendResults.foreach { case (topicOptionalIdPartition, result) => - val requestKey = new TopicPartitionOperationKey(topicOptionalIdPartition.topicPartition) + () => appendResults.foreach { case (topicIdPartition, result) => + val requestKey = new TopicPartitionOperationKey(topicIdPartition.topicPartition) result.info.leaderHwChange match { case LeaderHwChange.INCREASED => // some delayed operations may be unblocked after HW changed delayedProducePurgatory.checkAndComplete(requestKey) delayedFetchPurgatory.checkAndComplete(requestKey) delayedDeleteRecordsPurgatory.checkAndComplete(requestKey) - if (topicOptionalIdPartition.topicId.isPresent) delayedShareFetchPurgatory.checkAndComplete(new DelayedShareFetchPartitionKey( - topicOptionalIdPartition.topicId.get, topicOptionalIdPartition.partition)) + if (topicIdPartition.topicId != Uuid.ZERO_UUID) delayedShareFetchPurgatory.checkAndComplete(new DelayedShareFetchPartitionKey( + topicIdPartition.topicId, topicIdPartition.partition)) case LeaderHwChange.SAME => // probably unblock some follower fetch requests since log end offset has been updated delayedFetchPurgatory.checkAndComplete(requestKey) @@ -1051,8 +1049,9 @@ class ReplicaManager(val config: KafkaConfig, } } - private def sendInvalidRequiredAcksResponse(entries: Map[TopicIdPartition, MemoryRecords], - responseCallback: Map[TopicIdPartition, PartitionResponse] => Unit): Unit = { + private def sendInvalidRequiredAcksResponse( + entries: Map[TopicIdPartition, MemoryRecords], + responseCallback: Map[TopicIdPartition, PartitionResponse] => Unit): Unit = { // If required.acks is outside accepted range, something is wrong with the client // Just return an error and don't handle the request at all val responseStatus = entries.map { case (topicIdPartition, _) => @@ -1455,7 +1454,7 @@ class ReplicaManager(val config: KafkaConfig, requiredAcks: Short, requestLocal: RequestLocal, verificationGuards: Map[TopicPartition, VerificationGuard]): - Map[TopicOptionalIdPartition, LogAppendResult] = { + Map[TopicIdPartition, LogAppendResult] = { val traceEnabled = isTraceEnabled def processFailedRecord(topicIdPartition: TopicIdPartition, t: Throwable) = { val logStartOffset = onlinePartition(topicIdPartition.topicPartition()).map(_.logStartOffset).getOrElse(-1L) @@ -1479,10 +1478,9 @@ class ReplicaManager(val config: KafkaConfig, brokerTopicStats.allTopicsStats.totalProduceRequestRate.mark() // reject appending to internal topics if it is not allowed - val topicOptionalIdPartition = new TopicOptionalIdPartition(Optional.ofNullable(topicIdPartition.topicId()), - topicIdPartition.topicPartition()) + if (Topic.isInternal(topicIdPartition.topic) && !internalTopicsAllowed) { - (topicOptionalIdPartition, LogAppendResult( + (topicIdPartition, LogAppendResult( LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, Some(new InvalidTopicException(s"Cannot append to internal topic ${topicIdPartition.topic}")), hasCustomErrorMessage = false)) @@ -1503,7 +1501,7 @@ class ReplicaManager(val config: KafkaConfig, trace(s"${records.sizeInBytes} written to log $topicIdPartition beginning at offset " + s"${info.firstOffset} and ending at offset ${info.lastOffset}") - (topicOptionalIdPartition, LogAppendResult(info, exception = None, hasCustomErrorMessage = false)) + (topicIdPartition, LogAppendResult(info, exception = None, hasCustomErrorMessage = false)) } catch { // NOTE: Failed produce requests metric is not incremented for known exceptions @@ -1515,22 +1513,21 @@ class ReplicaManager(val config: KafkaConfig, _: CorruptRecordException | _: KafkaStorageException | _: UnknownTopicIdException) => - (topicOptionalIdPartition, LogAppendResult(LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, Some(e), hasCustomErrorMessage = false)) + (topicIdPartition, LogAppendResult(LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, Some(e), hasCustomErrorMessage = false)) case rve: RecordValidationException => val logStartOffset = processFailedRecord(topicIdPartition, rve.invalidException) val recordErrors = rve.recordErrors - (topicOptionalIdPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithAdditionalInfo(logStartOffset, recordErrors), + (topicIdPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithAdditionalInfo(logStartOffset, recordErrors), Some(rve.invalidException), hasCustomErrorMessage = true)) case t: Throwable => val logStartOffset = processFailedRecord(topicIdPartition, t) - (topicOptionalIdPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithLogStartOffset(logStartOffset), + (topicIdPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithLogStartOffset(logStartOffset), Some(t), hasCustomErrorMessage = false)) } } } } - def fetchOffset(topics: Seq[ListOffsetsTopic], duplicatePartitions: Set[TopicPartition], isolationLevel: IsolationLevel, diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 6a4947e45d080..9abf5352f9ce9 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -92,7 +92,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { val requestKeyToError = (topicNames: Map[Uuid, String], version: Short) => Map[ApiKeys, Nothing => Errors]( ApiKeys.METADATA -> ((resp: requests.MetadataResponse) => resp.errors.asScala.find(_._1 == topic).getOrElse(("test", Errors.NONE))._2), ApiKeys.PRODUCE -> ((resp: requests.ProduceResponse) => { - val topicId = topicNames.find(topicName => topicName._2 == topic).map(_._1).getOrElse(Uuid.ZERO_UUID) + + val topicId = topicNames.find { case (topicId, topicName) => topicName == topic}.map(_._1).getOrElse(Uuid.ZERO_UUID) val topicName = if (version >= 12) "" else topic Errors.forCode( resp.data @@ -269,7 +270,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { requests.ProduceRequest.forCurrentMagic(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection( Collections.singletonList(new ProduceRequestData.TopicProduceData() - .setName(tp.topic).setTopicId(id).setPartitionData(Collections.singletonList( + .setTopicId(id).setPartitionData(Collections.singletonList( new ProduceRequestData.PartitionProduceData() .setIndex(tp.partition) .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes)))))) diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala index 7c2e6ce8178e6..491cff57ab03d 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala @@ -122,7 +122,7 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness { * Tests that Producer produce to topic during reassignment where topic metadata change on broker side. * * Producer will attempt to send messages to the partition specified in each record, and should - * succeed as long as the metadata on the leader has been updated with new topic id. + * succeed as long as the metadata cache on the leader includes the partition topic id. */ @ParameterizedTest @ValueSource(strings = Array("kraft")) @@ -138,7 +138,8 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness { TestUtils.assertLeader(admin, partition1, 0) val topicDetails = topicMetadata(admin, topic) - assertEquals(0, topicDetails.partitions().get(0).leader().id()) + TestUtils.assertLeader(admin, partition1, 0) + val producer = createProducer() (1 to numRecords).foreach { i => @@ -154,13 +155,14 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness { // Change assignment of one of the replicas from 0 to 2. Leadership moves be 1. admin.alterPartitionReassignments(reassignment.asJava).all().get() TestUtils.assertLeader(admin, partition1, 1) + TestUtils.assertLeader(admin, partition0, 1) assertEquals(topicDetails.topicId(), topicMetadata(admin, topic).topicId()) // Producer should be able to send messages even after topic gets reassigned assertEquals(topic, producer.send(new ProducerRecord(topic, null, "value".getBytes(StandardCharsets.UTF_8))).get.topic()) } - def topicMetadata(admin: Admin, topic: String): TopicDescription = { + private def topicMetadata(admin: Admin, topic: String): TopicDescription = { admin.describeTopics(util.Collections.singletonList(topic)).allTopicNames().get().get(topic) } } diff --git a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala index cf6314592b1a6..5b494e1a5cb6f 100755 --- a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala @@ -133,7 +133,6 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness { val request = requests.ProduceRequest.forCurrentMagic(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection( Collections.singletonList(new ProduceRequestData.TopicProduceData() - .setName(topicPartition.topic()) .setTopicId(topicId) .setPartitionData(Collections.singletonList( new ProduceRequestData.PartitionProduceData() diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 08cbcec4ab7ee..dd67e49b09c04 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -2922,7 +2922,6 @@ class KafkaApisTest extends Logging { reset(replicaManager, clientQuotaManager, clientRequestQuotaManager, requestChannel, txnCoordinator) - val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection( Collections.singletonList(new ProduceRequestData.TopicProduceData() diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index fb778eb0a1cbf..e5c1657cfae48 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -3191,7 +3191,9 @@ class ReplicaManagerTest { requiredAcks = requiredAcks, internalTopicsAllowed = false, transactionalId = transactionalId, - entriesPerPartition = entriesPerPartition.map(e => replicaManager.topicIdPartition(e._1) -> e._2), + entriesPerPartition = entriesPerPartition.map { + case (topicPartition, records) => replicaManager.topicIdPartition(topicPartition) -> records + }, responseCallback = appendCallback, transactionSupportedOperation = transactionSupportedOperation ) From 410fd1fcef4be80208b7ccedec8b0f31c116d555 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Tue, 3 Dec 2024 07:05:42 +0000 Subject: [PATCH 32/40] add doc --- core/src/main/scala/kafka/server/KafkaApis.scala | 1 + core/src/main/scala/kafka/server/ReplicaManager.scala | 4 +++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 3cb5ba0f6fb12..2b139b85bf3d0 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -2493,6 +2493,7 @@ class KafkaApis(val requestChannel: RequestChannel, } else { // Otherwise, the regular appendRecords path is used for all the non __consumer_offsets // partitions or for all partitions when the new group coordinator is disabled. + // If topicIdPartition contains Uuid.ZERO_UUid or null all functionality will fall back on topic name. controlRecords += replicaManager.topicIdPartition(partition) -> MemoryRecords.withEndTransactionMarker( producerId, marker.producerEpoch, diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 2e11eac811b86..18973b15f38bb 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -814,7 +814,9 @@ class ReplicaManager(val config: KafkaConfig, * @param requiredAcks number of replicas who must acknowledge the append before sending the response * @param internalTopicsAllowed boolean indicating whether internal topics can be appended to * @param origin source of the append request (ie, client, replication, coordinator) - * @param entriesPerPartition the records per partition to be appended + * @param entriesPerPartition the records per topic partition to be appended. + * If topic partition contains Uuid.ZERO_UUID or null as topicId the method + * will fall back to the old behaviour and relay on topic name. * @param responseCallback callback for sending the response * @param delayedProduceLock lock for the delayed actions * @param recordValidationStatsCallback callback for updating stats on record conversions From 6e281fdd779d1e34609a012bc9768e04c0b56927 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Fri, 13 Dec 2024 15:54:21 +0000 Subject: [PATCH 33/40] upgrade version to 13 --- .../kafka/clients/producer/internals/Sender.java | 2 +- .../kafka/common/requests/ProduceRequest.java | 4 ++-- .../resources/common/message/ProduceRequest.json | 2 +- .../kafka/clients/producer/KafkaProducerTest.java | 8 +++++--- .../clients/producer/internals/SenderTest.java | 14 +++++++------- .../apache/kafka/common/message/MessageTest.java | 2 +- .../kafka/common/requests/ProduceRequestTest.java | 3 ++- 7 files changed, 19 insertions(+), 16 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index e35b9e7df0ce2..df1a6e4f2d7eb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -612,7 +612,7 @@ private void handleProduceResponse(ClientResponse response, Map partitionsWithUpdatedLeaderInfo = new HashMap<>(); produceResponse.data().responses().forEach(r -> r.partitionResponses().forEach(p -> { - // Version 12 drop topic name and add support to topic id. However, metadata can be used to map topic id to topic name. + // Version 13 drop topic name and add support to topic id. However, metadata can be used to map topic id to topic name. String topicName = metadata.topicNames().getOrDefault(r.topicId(), r.name()); TopicPartition tp = new TopicPartition(topicName, p.index()); ProduceResponse.PartitionResponse partResp = new ProduceResponse.PartitionResponse( diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 7fcce67a6a975..9170a08b72ace 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -147,8 +147,8 @@ Map partitionSizes() { topicData.partitionData().forEach(partitionData -> // While topic id and name will never be populated at the same time in the request, to simplify // initializing `TopicIdPartition` the code will use both topic name and id. - // Topic Id will be Uuid.ZERO_UUID in versions < 12 and topic name will be used as main identifier of topic partition. - // Topic Name will be empty string in versions >= 12 and topic id will be used as the main identifier. + // Topic Id will be Uuid.ZERO_UUID in versions < 13 and topic name will be used as main identifier of topic partition. + // Topic Name will be empty string in versions >= 13 and topic id will be used as the main identifier. tmpPartitionSizes.compute(new TopicIdPartition(topicData.topicId(), partitionData.index(), topicData.name()), (ignored, previousValue) -> partitionData.records().sizeInBytes() + (previousValue == null ? 0 : previousValue)) diff --git a/clients/src/main/resources/common/message/ProduceRequest.json b/clients/src/main/resources/common/message/ProduceRequest.json index 323887de2bf6d..feb79ce0a4e7a 100644 --- a/clients/src/main/resources/common/message/ProduceRequest.json +++ b/clients/src/main/resources/common/message/ProduceRequest.json @@ -55,7 +55,7 @@ "about": "The timeout to await a response in milliseconds." }, { "name": "TopicData", "type": "[]TopicProduceData", "versions": "0+", "about": "Each topic to produce to.", "fields": [ - { "name": "Name", "type": "string", "versions": "0-13", "entityType": "topicName", "mapKey": true, "ignorable": true, + { "name": "Name", "type": "string", "versions": "0-12", "entityType": "topicName", "mapKey": true, "ignorable": true, "about": "The topic name." }, { "name": "TopicId", "type": "uuid", "versions": "13+", "mapKey": true, "ignorable": true, "about": "The unique topic ID" }, { "name": "PartitionData", "type": "[]PartitionProduceData", "versions": "0+", 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 d72e576a78d9b..aea160255942b 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 @@ -38,6 +38,7 @@ import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.config.ConfigException; @@ -1675,6 +1676,7 @@ public void testTransactionV2Produce() throws Exception { KafkaProducerTestContext ctx = new KafkaProducerTestContext<>(testInfo, serializer); String topic = "foo"; + Uuid topicId = Uuid.fromString("klZ9sa2rSvig6QpgGXzALT"); TopicPartition topicPartition = new TopicPartition(topic, 0); Cluster cluster = TestUtils.singletonCluster(topic, 1); @@ -1715,7 +1717,7 @@ public void testTransactionV2Produce() throws Exception { client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "some-txn", NODE)); client.prepareResponse(initProducerIdResponse(1L, (short) 5, Errors.NONE)); - client.prepareResponse(produceResponse(topicPartition, 1L, Errors.NONE, 0, 1)); + client.prepareResponse(produceResponse(new TopicIdPartition(topicId, topicPartition), 1L, Errors.NONE, 0, 1)); client.prepareResponse(endTxnResponse(Errors.NONE)); try (KafkaProducer producer = new KafkaProducer<>( @@ -2669,9 +2671,9 @@ void testDeliveryTimeoutAndLingerMsConfig() { } @SuppressWarnings("deprecation") - private ProduceResponse produceResponse(TopicPartition tp, long offset, Errors error, int throttleTimeMs, int logStartOffset) { + private ProduceResponse produceResponse(TopicIdPartition topicIdPartition, long offset, Errors error, int throttleTimeMs, int logStartOffset) { ProduceResponse.PartitionResponse resp = new ProduceResponse.PartitionResponse(error, offset, RecordBatch.NO_TIMESTAMP, logStartOffset); - Map partResp = singletonMap(tp, resp); + Map partResp = singletonMap(topicIdPartition, resp); return new ProduceResponse(partResp, throttleTimeMs); } 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 54e992a693559..b05621991ff90 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 @@ -176,7 +176,7 @@ public class SenderTest { @BeforeEach public void setup() { setupWithTransactionState(null); - apiVersions.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 12)); + apiVersions.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 13)); this.client.updateMetadata( RequestTestUtils.metadataUpdateWithIds(1, Collections.singletonMap(TOPIC_NAME, 3), @@ -193,7 +193,7 @@ private static Map partitionRecords(ProduceReques request.data().topicData().forEach(tpData -> tpData.partitionData().forEach(p -> { String topicName = tpData.name(); - if (request.version() >= 12) { + if (request.version() >= 13) { topicName = TOPIC_IDS.entrySet().stream().filter(e -> e.getValue() == tpData.topicId()).map(Map.Entry::getKey).findFirst().get(); } @@ -595,7 +595,7 @@ public void testNodeLatencyStats() throws Exception { new BufferPool(totalSize, batchSize, m, time, "producer-internal-metrics")); SenderMetricsRegistry senderMetrics = new SenderMetricsRegistry(m); - apiVersions.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 12)); + apiVersions.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 13)); Sender sender = new Sender(logContext, client, metadata, this.accumulator, false, MAX_REQUEST_SIZE, ACKS_ALL, 1, senderMetrics, time, REQUEST_TIMEOUT, 1000L, null, apiVersions); @@ -2422,7 +2422,7 @@ public void testTransactionalSplitBatchAndSend() throws Exception { txnManager.beginTransaction(); txnManager.maybeAddPartition(tp); - apiVersions.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 12)); + apiVersions.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 13)); client.prepareResponse(buildAddPartitionsToTxnResponseData(0, Collections.singletonMap(tp, Errors.NONE))); sender.runOnce(); @@ -2451,7 +2451,7 @@ private void testSplitBatchAndSend(TransactionManager txnManager, MetadataResponse metadataUpdate1 = RequestTestUtils.metadataUpdateWithIds(2, Collections.singletonMap(topic, 2), TOPIC_IDS); client.prepareMetadataUpdate(metadataUpdate1); metadataUpdate1.brokers().stream().forEach(node -> - apiVersions.update(node.idString(), NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 12)) + apiVersions.update(node.idString(), NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 13)) ); // Send the first message. @@ -3451,7 +3451,7 @@ public void testWhenProduceResponseReturnsWithALeaderShipChangeErrorAndNewLeader })); Cluster startingMetadataCluster = metadata.fetch(); startingMetadataCluster.nodes().forEach(node -> - apiVersions.update(node.idString(), NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 12)) + apiVersions.update(node.idString(), NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 13)) ); // Produce to tp0/1/2, where NO_LEADER_OR_FOLLOWER with new leader info is returned for tp0/1, and tp2 is returned without errors. @@ -3474,7 +3474,7 @@ public void testWhenProduceResponseReturnsWithALeaderShipChangeErrorAndNewLeader responses.put(tp1, new OffsetAndError(-1, Errors.NOT_LEADER_OR_FOLLOWER)); responses.put(tp2, new OffsetAndError(100, Errors.NONE)); newNodes.forEach(node -> - apiVersions.update(node.idString(), NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 12)) + apiVersions.update(node.idString(), NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 13)) ); Map partitionLeaderInfo = new HashMap<>(); ProduceResponseData.LeaderIdAndEpoch tp0LeaderInfo = new ProduceResponseData.LeaderIdAndEpoch(); diff --git a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java index 7db4dc1a449e7..0120beb4bc96a 100644 --- a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java +++ b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java @@ -974,7 +974,7 @@ public void testProduceResponseVersions() throws Exception { responseData.setThrottleTimeMs(0); } - if (version >= 12) { + if (version >= 13) { responseData.responses().iterator().next().setTopicId(topicId); } else { responseData.responses().iterator().next().setName(topicName); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java index 3aa7d9a3af912..b9017dfdeb3cc 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java @@ -150,7 +150,8 @@ public void testBuildWithCurrentMessageFormatWithoutTopicId() { new ProduceRequestData.PartitionProduceData().setIndex(9).setRecords(builder.build())))) .iterator())) .setAcks((short) 1) - .setTimeoutMs(5000)); + .setTimeoutMs(5000), + false); assertEquals(3, requestBuilder.oldestAllowedVersion()); assertEquals(11, requestBuilder.latestAllowedVersion()); } From 900c4600957ca1ef840906669c9c99225dc2de4f Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Fri, 13 Dec 2024 16:21:47 +0000 Subject: [PATCH 34/40] addressing some of @junrao feedback --- .../common/message/ProduceResponse.json | 2 +- .../main/scala/kafka/server/KafkaApis.scala | 2 +- .../scala/kafka/server/ReplicaManager.scala | 9 +-- .../share/SharePartitionManagerTest.java | 4 +- .../kafka/api/AuthorizerIntegrationTest.scala | 4 +- .../CoordinatorPartitionWriterTest.scala | 2 +- .../unit/kafka/server/KafkaApisTest.scala | 81 ++++++++++++------- .../kafka/server/ReplicaManagerTest.scala | 2 +- 8 files changed, 63 insertions(+), 43 deletions(-) diff --git a/clients/src/main/resources/common/message/ProduceResponse.json b/clients/src/main/resources/common/message/ProduceResponse.json index 885975a2affd7..20a81d8c864b9 100644 --- a/clients/src/main/resources/common/message/ProduceResponse.json +++ b/clients/src/main/resources/common/message/ProduceResponse.json @@ -39,7 +39,7 @@ // // Version 12 is the same as version 10 (KIP-890). // Version 13 replaces topic names with topic IDs (KIP-516). May return UNKNOWN_TOPIC_ID error code. - "validVersions": "0-12", + "validVersions": "0-13", "flexibleVersions": "9+", "fields": [ { "name": "Responses", "type": "[]TopicProduceResponse", "versions": "0+", diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 30e0b83088141..8434c5b0e3630 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -2490,7 +2490,7 @@ class KafkaApis(val requestChannel: RequestChannel, } else { // Otherwise, the regular appendRecords path is used for all the non __consumer_offsets // partitions or for all partitions when the new group coordinator is disabled. - // If topicIdPartition contains Uuid.ZERO_UUid or null all functionality will fall back on topic name. + // If topicIdPartition contains Uuid.ZERO_UUid all functionality will fall back on topic name. controlRecords += replicaManager.topicIdPartition(partition) -> MemoryRecords.withEndTransactionMarker( producerId, marker.producerEpoch, diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 5eefbe05dcd46..cbcd300cb9ff2 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -815,8 +815,8 @@ class ReplicaManager(val config: KafkaConfig, * @param internalTopicsAllowed boolean indicating whether internal topics can be appended to * @param origin source of the append request (ie, client, replication, coordinator) * @param entriesPerPartition the records per topic partition to be appended. - * If topic partition contains Uuid.ZERO_UUID or null as topicId the method - * will fall back to the old behaviour and relay on topic name. + * If topic partition contains Uuid.ZERO_UUID as topicId the method + * will fall back to the old behaviour and rely on topic name. * @param responseCallback callback for sending the response * @param delayedProduceLock lock for the delayed actions * @param recordValidationStatsCallback callback for updating stats on record conversions @@ -982,8 +982,8 @@ class ReplicaManager(val config: KafkaConfig, private def buildProducePartitionStatus( results: Map[TopicIdPartition, LogAppendResult] ): Map[TopicIdPartition, ProducePartitionStatus] = { - results.map { case (topicPartition, result) => - topicPartition -> ProducePartitionStatus( + results.map { case (topicIdPartition, result) => + topicIdPartition -> ProducePartitionStatus( result.info.lastOffset + 1, // required offset new PartitionResponse( result.error, @@ -1481,7 +1481,6 @@ class ReplicaManager(val config: KafkaConfig, brokerTopicStats.allTopicsStats.totalProduceRequestRate.mark() // reject appending to internal topics if it is not allowed - if (Topic.isInternal(topicIdPartition.topic) && !internalTopicsAllowed) { (topicIdPartition, LogAppendResult( LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, diff --git a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java index d02ccda8fd3a3..c318fc44bab7f 100644 --- a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java +++ b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java @@ -164,7 +164,7 @@ public void setUp() { new SystemTimer("sharePartitionManagerTestTimer")); mockReplicaManager = mock(ReplicaManager.class); Partition partition = mockPartition(); - when(mockReplicaManager.getPartitionOrException(Mockito.any(TopicPartition.class))).thenReturn(partition); + when(mockReplicaManager.getPartitionOrException((TopicPartition) any())).thenReturn(partition); } @AfterEach @@ -2516,7 +2516,7 @@ public void testListenerRegistration() { ReplicaManager mockReplicaManager = mock(ReplicaManager.class); Partition partition = mockPartition(); - when(mockReplicaManager.getPartitionOrException(Mockito.any())).thenReturn(partition); + when(mockReplicaManager.getPartitionOrException((TopicPartition) Mockito.any())).thenReturn(partition); SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder() .withReplicaManager(mockReplicaManager) diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 0f105b06f249e..50dc0377c11b2 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -93,7 +93,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { ApiKeys.METADATA -> ((resp: requests.MetadataResponse) => resp.errors.asScala.find(_._1 == topic).getOrElse(("test", Errors.NONE))._2), ApiKeys.PRODUCE -> ((resp: requests.ProduceResponse) => { - val topicId = topicNames.find { case (topicId, topicName) => topicName == topic}.map(_._1).getOrElse(Uuid.ZERO_UUID) + val topicId = topicNames.find { case (_, topicName) => topicName == topic} + .map { case (topicId, _) => topicId } + .getOrElse(Uuid.ZERO_UUID) val topicName = if (version >= 12) "" else topic Errors.forCode( resp.data diff --git a/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala index b450fdb35909b..255caddda8c8e 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala @@ -141,7 +141,7 @@ class CoordinatorPartitionWriterTest { )) assertEquals( batch, - recordsCapture.getValue.find(_._1 == new TopicIdPartition(topicId, tp)).getOrElse( + recordsCapture.getValue.find { case (topicIdPartition, _) => topicIdPartition == new TopicIdPartition(topicId, tp) }.getOrElse( throw new AssertionError(s"No records for $tp"))._2 ) } diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index adb0fb7c312cf..6a51798cf24f7 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -2651,15 +2651,21 @@ class KafkaApisTest extends Logging { val responseCallback: ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, PartitionResponse] => Unit]) + val produceData = new ProduceRequestData.TopicProduceData() + .setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData() + .setIndex(tp.partition) + .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes))))) + + if (version >= 13 ) { + produceData.setTopicId(topicId) + } else { + produceData.setName(tp.topic) + } + val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection( - Collections.singletonList(new ProduceRequestData.TopicProduceData() - .setName(tp.topic) - .setTopicId(topicId) - .setPartitionData(Collections.singletonList( - new ProduceRequestData.PartitionProduceData() - .setIndex(tp.partition) - .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes)))))) + Collections.singletonList(produceData) .iterator)) .setAcks(1.toShort) .setTimeoutMs(5000)) @@ -2716,16 +2722,20 @@ class KafkaApisTest extends Logging { val newLeaderId = 2 val newLeaderEpoch = 5 + val produceData = new ProduceRequestData.TopicProduceData() + .setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData() + .setIndex(tp.partition) + .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes))))) + + if (version >= 13 ) { + produceData.setTopicId(topicId) + } else { + produceData.setName(tp.topic) + } val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection( - Collections.singletonList(new ProduceRequestData.TopicProduceData() - .setName(tp.topic) - .setTopicId(topicId) - .setPartitionData(Collections.singletonList( - new ProduceRequestData.PartitionProduceData() - .setIndex(tp.partition) - .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes)))))) - .iterator)) + Collections.singletonList(produceData).iterator)) .setAcks(1.toShort) .setTimeoutMs(5000)) .build(version.toShort) @@ -2784,16 +2794,20 @@ class KafkaApisTest extends Logging { val tp = new TopicIdPartition(topicId, 0, topic) + val produceData = new ProduceRequestData.TopicProduceData() + .setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData() + .setIndex(tp.partition) + .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes))))) + + if (version >= 13 ) { + produceData.setTopicId(topicId) + } else { + produceData.setName(tp.topic) + } val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection( - Collections.singletonList(new ProduceRequestData.TopicProduceData() - .setName(tp.topic) - .setTopicId(topicId) - .setPartitionData(Collections.singletonList( - new ProduceRequestData.PartitionProduceData() - .setIndex(tp.partition) - .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes)))))) - .iterator)) + Collections.singletonList(produceData).iterator)) .setAcks(1.toShort) .setTimeoutMs(5000)) .build(version.toShort) @@ -2853,7 +2867,7 @@ class KafkaApisTest extends Logging { val topicProduceData = new ProduceRequestData.TopicProduceData() - if (version >= 12 ) { + if (version >= 13 ) { topicProduceData.setTopicId(topicId) } else { topicProduceData.setName(tp.topic) @@ -2927,15 +2941,20 @@ class KafkaApisTest extends Logging { reset(replicaManager, clientQuotaManager, clientRequestQuotaManager, requestChannel, txnCoordinator) + val produceData = new ProduceRequestData.TopicProduceData() + .setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData() + .setIndex(tp.partition) + .setRecords(MemoryRecords.withTransactionalRecords(Compression.NONE, 0, 0, 0, new SimpleRecord("test".getBytes))))) + + if (version >= 13 ) { + produceData.setTopicId(topicId) + } else { + produceData.setName(tp.topic) + } val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection( - Collections.singletonList(new ProduceRequestData.TopicProduceData() - .setName(tp.topic) - .setTopicId(tp.topicId()) - .setPartitionData(Collections.singletonList( - new ProduceRequestData.PartitionProduceData() - .setIndex(tp.partition) - .setRecords(MemoryRecords.withTransactionalRecords(Compression.NONE, 0, 0, 0, new SimpleRecord("test".getBytes)))))) + Collections.singletonList(produceData) .iterator)) .setAcks(1.toShort) .setTransactionalId(transactionalId) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index fcc3f1763c36f..277f91dd80da8 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -3145,7 +3145,7 @@ class ReplicaManagerTest { requiredAcks = requiredAcks, internalTopicsAllowed = false, transactionalId = transactionalId, - entriesPerPartition = entriesToAppend.map(e => replicaManager.topicIdPartition(e._1) -> e._2), + entriesPerPartition = entriesToAppend.map { case(tp, memoryRecords) => replicaManager.topicIdPartition(tp) -> memoryRecords }, responseCallback = appendCallback, transactionSupportedOperation = transactionSupportedOperation ) From 0ba7431833afb3fc937b004d89d39d89c61aa8ea Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Wed, 2 Apr 2025 18:13:57 +0200 Subject: [PATCH 35/40] fix merge --- .../clients/producer/KafkaProducerTest.java | 6 +++--- .../common/requests/ProduceRequestTest.java | 2 +- .../common/requests/RequestResponseTest.java | 1 + .../group/CoordinatorPartitionWriter.scala | 2 +- .../main/scala/kafka/server/KafkaApis.scala | 2 +- .../scala/kafka/server/ReplicaManager.scala | 8 +++----- .../server/LocalLeaderEndPointTest.scala | 2 +- .../AbstractCoordinatorConcurrencyTest.scala | 1 + .../CoordinatorPartitionWriterTest.scala | 2 +- ...ransactionCoordinatorConcurrencyTest.scala | 2 +- .../TransactionStateManagerTest.scala | 1 - .../unit/kafka/server/KafkaApisTest.scala | 6 +++--- .../kafka/server/ProduceRequestTest.scala | 11 +++++----- .../ReplicaManagerConcurrencyTest.scala | 5 +++-- .../kafka/server/ReplicaManagerTest.scala | 20 ++++++++++--------- 15 files changed, 37 insertions(+), 34 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 6e140edc89c0e..ca0f90d713970 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 @@ -1425,7 +1425,7 @@ public void testTransactionV2ProduceWithConcurrentTransactionError() throws Exce KafkaProducerTestContext ctx = new KafkaProducerTestContext<>(testInfo, serializer); String topic = "foo"; - TopicPartition topicPartition = new TopicPartition(topic, 0); + TopicIdPartition topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), 0, topic); Cluster cluster = TestUtils.singletonCluster(topic, 1); when(ctx.sender.isRunning()).thenReturn(true); @@ -1464,8 +1464,8 @@ public void testTransactionV2ProduceWithConcurrentTransactionError() throws Exce client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "some-txn", NODE)); client.prepareResponse(initProducerIdResponse(1L, (short) 5, Errors.NONE)); - client.prepareResponse(produceResponse(topicPartition, 1L, Errors.CONCURRENT_TRANSACTIONS, 0, 1)); - client.prepareResponse(produceResponse(topicPartition, 1L, Errors.NONE, 0, 1)); + client.prepareResponse(produceResponse(topicIdPartition, 1L, Errors.CONCURRENT_TRANSACTIONS, 0, 1)); + client.prepareResponse(produceResponse(topicIdPartition, 1L, Errors.NONE, 0, 1)); client.prepareResponse(endTxnResponse(Errors.NONE)); try (KafkaProducer producer = new KafkaProducer<>( diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java index ded1933d6f064..e81c532a961fe 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java @@ -121,7 +121,7 @@ public void testBuildWithCurrentMessageFormatWithoutTopicId() { MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE, TimestampType.CREATE_TIME, 0L); builder.append(10L, null, "a".getBytes()); - ProduceRequest.Builder requestBuilder = ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE, + ProduceRequest.Builder requestBuilder = ProduceRequest.builder( new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( new ProduceRequestData.TopicProduceData() diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 46b57e8e1f9e5..6906dafff11c2 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -2470,6 +2470,7 @@ private OffsetFetchResponse createOffsetFetchResponse(short version) { } private ProduceRequest createProduceRequest(short version) { + TopicIdPartition topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), 0, "test"); MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, Compression.NONE, new SimpleRecord("woot".getBytes())); return ProduceRequest.builder( diff --git a/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala b/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala index 5c673da5c0163..8755ed7e17789 100644 --- a/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala +++ b/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala @@ -139,7 +139,7 @@ class CoordinatorPartitionWriter( records: MemoryRecords ): Long = { // We write synchronously to the leader replica without waiting on replication. - val topicIdPartition = replicaManager.topicIdPartition(tp) + val topicIdPartition: TopicIdPartition = replicaManager.topicIdPartition(tp) val appendResults = replicaManager.appendRecordsToLeader( requiredAcks = 1, internalTopicsAllowed = true, diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 5f925f65ad184..9a237237be570 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -401,7 +401,7 @@ class KafkaApis(val requestChannel: RequestChannel, val (topicName, topicId) = if (topic.topicId().equals(Uuid.ZERO_UUID)) { (topic.name(), metadataCache.getTopicId(topic.name())) } else { - (metadataCache.getTopicName(topic.topicId).getOrElse(topic.name), topic.topicId()) + (metadataCache.getTopicName(topic.topicId).orElse(topic.name), topic.topicId()) } val topicPartition = new TopicPartition(topicName, partition.index()) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 0cdcb8a70234c..ae12d741216ac 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -51,7 +51,7 @@ import org.apache.kafka.image.{LocalReplicaChanges, MetadataImage, TopicsDelta} import org.apache.kafka.metadata.LeaderConstants.NO_LEADER import org.apache.kafka.metadata.MetadataCache import org.apache.kafka.server.{ActionQueue, DelayedActionQueue, ListOffsetsPartitionStatus, common} -import org.apache.kafka.server.common.{DirectoryEventHandler, RequestLocal, StopPartition, TopicOptionalIdPartition} +import org.apache.kafka.server.common.{DirectoryEventHandler, RequestLocal, StopPartition} import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.network.BrokerEndPoint import org.apache.kafka.server.purgatory.{DelayedOperationPurgatory, TopicPartitionOperationKey} @@ -676,7 +676,7 @@ class ReplicaManager(val config: KafkaConfig, requestLocal: RequestLocal = RequestLocal.noCaching, actionQueue: ActionQueue = this.defaultActionQueue, verificationGuards: Map[TopicPartition, VerificationGuard] = Map.empty - ): Map[TopicPartition, LogAppendResult] = { + ): Map[TopicIdPartition, LogAppendResult] = { val startTimeMs = time.milliseconds val localProduceResultsWithTopicId = appendToLocalLog( internalTopicsAllowed = internalTopicsAllowed, @@ -690,9 +690,7 @@ class ReplicaManager(val config: KafkaConfig, addCompletePurgatoryAction(actionQueue, localProduceResultsWithTopicId) - localProduceResultsWithTopicId.map { - case (k, v) => (k.topicPartition, v) - } + localProduceResultsWithTopicId } /** diff --git a/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala b/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala index 6535f1ae1be5c..fd6f66326d11e 100644 --- a/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala +++ b/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala @@ -20,7 +20,7 @@ package kafka.server import kafka.server.QuotaFactory.QuotaManagers import kafka.server.metadata.KRaftMetadataCache import kafka.utils.{CoreUtils, Logging, TestUtils} -import org.apache.kafka.common.{Node, TopicIdPartition, Uuid} +import org.apache.kafka.common.{TopicIdPartition, Uuid} import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset diff --git a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala index c40ef0470114d..d8aa14e6e37f5 100644 --- a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala @@ -32,6 +32,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordValidationStats} import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.utils.{Time, Utils} +import org.apache.kafka.metadata.MetadataCache import org.apache.kafka.server.common.RequestLocal import org.apache.kafka.server.purgatory.{DelayedOperationPurgatory, TopicPartitionOperationKey} import org.apache.kafka.server.util.timer.{MockTimer, Timer} diff --git a/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala index 2087480aa934a..1149ff814171b 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala @@ -138,7 +138,7 @@ class CoordinatorPartitionWriterTest { )) assertEquals( batch, - recordsCapture.getValue.getOrElse(topicIdPartition, throw new AssertionError(s"No records for $tp")) + recordsCapture.getValue.getOrElse(new TopicIdPartition(topicId, tp), throw new AssertionError(s"No records for $tp")) ) } diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala index 89cd2c6a4f7bd..5f2291ed04a06 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala @@ -115,7 +115,7 @@ class TransactionCoordinatorConcurrencyTest extends AbstractCoordinatorConcurren time) val transactionStateTopicId = Uuid.randomUuid() - when(replicaManager.metadataCache.getTopicName(transactionStateTopicId)).thenReturn(Some(Topic.TRANSACTION_STATE_TOPIC_NAME)) + when(replicaManager.metadataCache.getTopicName(transactionStateTopicId)).thenReturn(Optional.of(Topic.TRANSACTION_STATE_TOPIC_NAME)) when(replicaManager.metadataCache.getTopicId(Topic.TRANSACTION_STATE_TOPIC_NAME)).thenReturn(transactionStateTopicId) transactionCoordinator = new TransactionCoordinator( diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala index 668016ac91d78..e982fc492bf9b 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala @@ -23,7 +23,6 @@ import java.util.concurrent.locks.ReentrantLock import javax.management.ObjectName import kafka.server.ReplicaManager import kafka.utils.{Pool, TestUtils} -import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.internals.Topic.TRANSACTION_STATE_TOPIC_NAME diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 27e36264d5a52..e586097236c86 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -2150,7 +2150,7 @@ class KafkaApisTest extends Logging { } else { produceData.setName(tp.topic) } - val produceRequest = ProduceRequest.builder(new ProduceReque + val produceRequest = ProduceRequest.builder(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection( Collections.singletonList(produceData).iterator)) .setAcks(1.toShort) @@ -2322,11 +2322,11 @@ class KafkaApisTest extends Logging { when(metadataCache.getLeaderAndIsr(tp.topic(), tp.partition())).thenAnswer(_ => Optional.empty()) when(metadataCache.getAliveBrokerNode(any(), any())).thenReturn(Optional.empty()) if (version >= 13) { - when(metadataCache.getTopicName(tp.topicId())).thenReturn(Some(tp.topic())) + when(metadataCache.getTopicName(tp.topicId())).thenReturn(Optional.of(tp.topic())) } else { when(metadataCache.getTopicId(tp.topic())).thenReturn(tp.topicId()) } - aApis = createKafkaApis() + val kafkaApis = createKafkaApis() kafkaApis.handleProduceRequest(request, RequestLocal.withThreadConfinedCaching) val response = verifyNoThrottling[ProduceResponse](request) diff --git a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala index eaab8047023e5..3409c0903ee0c 100644 --- a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import java.util.{Collections, Properties} import kafka.utils.TestUtils import org.apache.kafka.clients.admin.{Admin, TopicDescription} -import org.apache.kafka.common.{TopicIdPartition, TopicPartition} +import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.message.ProduceRequestData @@ -53,13 +53,14 @@ class ProduceRequestTest extends BaseRequestTest { val (partition, leader) = createTopicAndFindPartitionWithLeader("topic") def sendAndCheck(memoryRecords: MemoryRecords, expectedOffset: Long): Unit = { - val topicPartition = new TopicPartition("topic", partition) + val topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), partition, "topic") val produceRequest = ProduceRequest.builder(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( new ProduceRequestData.TopicProduceData() - .setName(topicPartition.topic()) + .setName(topicIdPartition.topic()) + .setTopicId(topicIdPartition.topicId()) .setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData() - .setIndex(topicPartition.partition()) + .setIndex(topicIdPartition.partition()) .setRecords(memoryRecords)))).iterator)) .setAcks((-1).toShort) .setTimeoutMs(3000) @@ -71,7 +72,7 @@ class ProduceRequestTest extends BaseRequestTest { assertEquals(1, topicProduceResponse.partitionResponses.size) val partitionProduceResponse = topicProduceResponse.partitionResponses.asScala.head val tp = new TopicPartition(topicProduceResponse.name, partitionProduceResponse.index) - assertEquals(topicPartition, tp) + assertEquals(topicIdPartition, tp) assertEquals(Errors.NONE, Errors.forCode(partitionProduceResponse.errorCode)) assertEquals(expectedOffset, partitionProduceResponse.baseOffset) assertEquals(-1, partitionProduceResponse.logAppendTimeMs) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala index d2d42dd97d41e..0ab9a698f40ad 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala @@ -24,6 +24,7 @@ import kafka.server.QuotaFactory.QuotaManagers import kafka.server.metadata.KRaftMetadataCache import kafka.utils.TestUtils.waitUntilTrue import kafka.utils.{CoreUtils, Logging, TestUtils} +import org.apache.kafka.common import org.apache.kafka.common.metadata.{FeatureLevelRecord, PartitionChangeRecord, PartitionRecord, RegisterBrokerRecord, TopicRecord} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors @@ -292,9 +293,9 @@ class ReplicaManagerConcurrencyTest extends Logging { } val future = new CompletableFuture[ProduceResponse.PartitionResponse]() - val topicIdPartition = replicaManager.topicIdPartition(topicPartition) + val topicIdPartition: common.TopicIdPartition = replicaManager.topicIdPartition(topicPartition) - def produceCallback(results: collection.Map[TopicIdPartition, ProduceResponse.PartitionResponse]): Unit = { + def produceCallback(results: collection.Map[common.TopicIdPartition, ProduceResponse.PartitionResponse]): Unit = { try { assertEquals(1, results.size) val (topicPartition, result) = results.head diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index eac95be0750cc..6a03cd283b167 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -122,8 +122,7 @@ class ReplicaManagerTest { private var mockRemoteLogManager: RemoteLogManager = _ private var addPartitionsToTxnManager: AddPartitionsToTxnManager = _ private var brokerTopicStats: BrokerTopicStats = _ - private val transactionSupportedOperation = genericErrorSupported - private val metadataCache: MetadataCache = mock(classOf[MetadataCache]) + private val metadataCache: KRaftMetadataCache = mock(classOf[KRaftMetadataCache]) private val quotaExceededThrottleTime = 1000 private val quotaAvailableThrottleTime = 0 @@ -161,11 +160,14 @@ class ReplicaManagerTest { setupMetadataCacheWithTopicIds(topicIds, metadataCache) } - private def setupMetadataCacheWithTopicIds(topicIds: Map[String, Uuid], metadataCache: MetadataCache): Unit = { + private def setupMetadataCacheWithTopicIds(topicIds: Map[String, Uuid], metadataCache: KRaftMetadataCache): Unit = { val topicNames = topicIds.map(_.swap) - when(metadataCache.topicNamesToIds()).thenReturn(topicIds.asJava) + topicNames.foreach { + case (id, name) => + when(metadataCache.getTopicName(id)).thenReturn(Optional.of(name)) + when(metadataCache.getTopicId(name)).thenReturn(id) + } when(metadataCache.topicIdsToNames()).thenReturn(topicNames.asJava) - when(metadataCache.topicIdInfo()).thenReturn((topicIds.asJava, topicNames.asJava)) topicIds.foreach { case (topicName, topicId) => when(metadataCache.getTopicId(topicName)).thenReturn(topicId) @@ -6183,8 +6185,8 @@ class ReplicaManagerTest { internalTopicsAllowed = true, origin = AppendOrigin.CLIENT, entriesPerPartition = Map( - foo.topicPartition -> records, - bar.topicPartition -> records + foo -> records, + bar -> records ), requestLocal = RequestLocal.noCaching ) @@ -6192,13 +6194,13 @@ class ReplicaManagerTest { assertNotNull(result) assertEquals(2, result.size) - val fooResult = result(foo.topicPartition) + val fooResult = result(foo) assertEquals(Errors.NONE, fooResult.error) assertEquals(0, fooResult.info.logStartOffset) assertEquals(0, fooResult.info.firstOffset) assertEquals(0, fooResult.info.lastOffset) - val barResult = result(bar.topicPartition) + val barResult = result(bar) assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, barResult.error) assertEquals(LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, barResult.info) } finally { From 2f37078260c17ad74674c38f4bfc67039c30c9ad Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Wed, 9 Apr 2025 12:34:31 +0100 Subject: [PATCH 36/40] address feedback --- .../kafka/common/requests/ProduceRequest.java | 22 +++---- .../common/requests/ProduceResponse.java | 1 + .../org/apache/kafka/common/utils/Utils.java | 11 ---- .../producer/internals/SenderTest.java | 61 ++++++++++--------- .../kafka/common/message/MessageTest.java | 2 +- .../common/requests/ProduceRequestTest.java | 2 +- .../common/requests/RequestResponseTest.java | 22 ++++--- .../connect/mirror/MirrorCheckpointTask.java | 3 - .../connect/mirror/MirrorSourceConnector.java | 3 - .../transaction/TransactionStateManager.scala | 6 +- .../main/scala/kafka/server/KafkaApis.scala | 4 +- .../scala/kafka/server/ReplicaManager.scala | 2 +- .../kafka/api/AuthorizerIntegrationTest.scala | 3 +- .../api/ProducerSendWhileDeletionTest.scala | 17 +++--- .../kafka/server/ProduceRequestTest.scala | 31 +++++----- .../kafka/server/ReplicaManagerTest.scala | 6 +- 16 files changed, 96 insertions(+), 100 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 41783529bd2b3..a39ea157d72fa 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -18,7 +18,6 @@ import org.apache.kafka.common.InvalidRecordException; import org.apache.kafka.common.TopicIdPartition; -import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.UnsupportedCompressionTypeException; import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.message.ProduceResponseData; @@ -55,10 +54,6 @@ public static Builder builder(ProduceRequestData data) { return builder(data, false); } - private static boolean canNotSupportTopicId(ProduceRequestData data) { - return data.topicData().stream().anyMatch(d -> d.topicId() == Uuid.ZERO_UUID); - } - public static class Builder extends AbstractRequest.Builder { private final ProduceRequestData data; @@ -120,10 +115,11 @@ Map partitionSizes() { Map tmpPartitionSizes = new HashMap<>(); data.topicData().forEach(topicData -> topicData.partitionData().forEach(partitionData -> - // While topic id and name will never be populated at the same time in the request, to simplify - // initializing `TopicIdPartition` the code will use both topic name and id. - // Topic Id will be Uuid.ZERO_UUID in versions < 13 and topic name will be used as main identifier of topic partition. - // Topic Name will be empty string in versions >= 13 and topic id will be used as the main identifier. + // While topic id and name might not be populated at the same time in the request all the time; + // for example on server side they will never be populated together while in produce client they will be, + // to simplify initializing `TopicIdPartition` the code will use both topic name and id. + // TopicId will be Uuid.ZERO_UUID in versions < 13 and topic name will be used as main identifier of topic partition. + // TopicName will be empty string in versions >= 13 and topic id will be used as the main identifier. tmpPartitionSizes.compute(new TopicIdPartition(topicData.topicId(), partitionData.index(), topicData.name()), (ignored, previousValue) -> partitionData.records().sizeInBytes() + (previousValue == null ? 0 : previousValue)) @@ -170,14 +166,14 @@ public ProduceResponse getErrorResponse(int throttleTimeMs, Throwable e) { if (acks == 0) return null; ApiError apiError = ApiError.fromThrowable(e); ProduceResponseData data = new ProduceResponseData().setThrottleTimeMs(throttleTimeMs); - partitionSizes().forEach((tp, ignored) -> { - ProduceResponseData.TopicProduceResponse tpr = data.responses().find(tp.topic(), tp.topicId()); + partitionSizes().forEach((tpId, ignored) -> { + ProduceResponseData.TopicProduceResponse tpr = data.responses().find(tpId.topic(), tpId.topicId()); if (tpr == null) { - tpr = new ProduceResponseData.TopicProduceResponse().setName(tp.topic()).setTopicId(tp.topicId()); + tpr = new ProduceResponseData.TopicProduceResponse().setName(tpId.topic()).setTopicId(tpId.topicId()); data.responses().add(tpr); } tpr.partitionResponses().add(new ProduceResponseData.PartitionProduceResponse() - .setIndex(tp.partition()) + .setIndex(tpId.partition()) .setRecordErrors(Collections.emptyList()) .setBaseOffset(INVALID_OFFSET) .setLogAppendTimeMs(RecordBatch.NO_TIMESTAMP) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index fc843276d79d3..f7175b7d75fa8 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -56,6 +56,7 @@ * {@link Errors#INVALID_TXN_STATE} * {@link Errors#INVALID_PRODUCER_ID_MAPPING} * {@link Errors#CONCURRENT_TRANSACTIONS} + * {@link Errors#UNKNOWN_TOPIC_ID} */ public class ProduceResponse extends AbstractResponse { public static final long INVALID_OFFSET = -1L; 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 070ced5ddc9b2..8aa21f7377f8d 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 @@ -1691,17 +1691,6 @@ public static ConfigDef mergeConfigs(List configDefs) { return all; } - /** - * Convert a map's keys to another type. - */ - public static Map convertKeys(Map originalMap, Function converter) { - return originalMap.entrySet().stream() - .collect(Collectors.toMap( - entry -> converter.apply(entry.getKey()), - Map.Entry::getValue - )); - } - /** * A runnable that can throw checked exception. */ 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 6f1629ee2f506..bf6f7713e6e49 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 @@ -157,7 +157,7 @@ public class SenderTest { private static final String TOPIC_NAME = "test"; private static final Uuid TOPIC_ID = Uuid.fromString("MKXx1fIkQy2J9jXHhK8m1w"); - private static final Map TOPIC_IDS = Collections.singletonMap(TOPIC_NAME, TOPIC_ID); + private static final Map TOPIC_IDS = new HashMap<>(Map.of(TOPIC_NAME, TOPIC_ID)); private final TopicPartition tp0 = new TopicPartition(TOPIC_NAME, 0); private final TopicPartition tp1 = new TopicPartition(TOPIC_NAME, 1); private final TopicPartition tp2 = new TopicPartition(TOPIC_NAME, 2); @@ -176,7 +176,7 @@ public class SenderTest { @BeforeEach public void setup() { setupWithTransactionState(null); - apiVersions.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 13)); + apiVersions.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, ApiKeys.PRODUCE.oldestVersion(), ApiKeys.PRODUCE.latestVersion())); this.client.updateMetadata( RequestTestUtils.metadataUpdateWithIds(1, Collections.singletonMap(TOPIC_NAME, 3), @@ -193,7 +193,7 @@ private static Map partitionRecords(ProduceReques request.data().topicData().forEach(tpData -> tpData.partitionData().forEach(p -> { String topicName = tpData.name(); - if (request.version() >= 13) { + if (request.version() >= 13 && tpData.topicId() != Uuid.ZERO_UUID) { topicName = TOPIC_IDS.entrySet().stream().filter(e -> e.getValue() == tpData.topicId()).map(Map.Entry::getKey).findFirst().get(); } @@ -550,7 +550,7 @@ public void testNodeLatencyStats() throws Exception { new BufferPool(totalSize, batchSize, m, time, "producer-internal-metrics")); SenderMetricsRegistry senderMetrics = new SenderMetricsRegistry(m); - apiVersions.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 13)); + apiVersions.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, ApiKeys.PRODUCE.oldestVersion(), ApiKeys.PRODUCE.latestVersion())); Sender sender = new Sender(logContext, client, metadata, this.accumulator, false, MAX_REQUEST_SIZE, ACKS_ALL, 1, senderMetrics, time, REQUEST_TIMEOUT, 1000L, null, apiVersions); @@ -2357,39 +2357,44 @@ public void testBumpEpochWhenOutOfOrderSequenceReceived() throws InterruptedExce @Test public void testIdempotentSplitBatchAndSend() throws Exception { - TopicPartition tp = new TopicPartition("testSplitBatchAndSend", 1); + Uuid topicId = Uuid.fromString("2J9hK8m1wHMKjXfIkQyXx1"); + TOPIC_IDS.put("testSplitBatchAndSend", topicId); + TopicIdPartition tpId = new TopicIdPartition(topicId, new TopicPartition("testSplitBatchAndSend", 1)); TransactionManager txnManager = createTransactionManager(); ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(123456L, (short) 0); setupWithTransactionState(txnManager); prepareAndReceiveInitProducerId(123456L, Errors.NONE); assertTrue(txnManager.hasProducerId()); - testSplitBatchAndSend(txnManager, producerIdAndEpoch, tp); + testSplitBatchAndSend(txnManager, producerIdAndEpoch, tpId); } @Test public void testTransactionalSplitBatchAndSend() throws Exception { + Uuid topicId = Uuid.fromString("2J9hK8m1wHMKjXfIkQyXx1"); + TOPIC_IDS.put("testSplitBatchAndSend", topicId); ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(123456L, (short) 0); - TopicPartition tp = new TopicPartition("testSplitBatchAndSend", 1); + TopicIdPartition tpId = new TopicIdPartition(topicId, new TopicPartition("testSplitBatchAndSend", 1)); + TransactionManager txnManager = new TransactionManager(logContext, "testSplitBatchAndSend", 60000, 100, apiVersions); setupWithTransactionState(txnManager); doInitTransactions(txnManager, producerIdAndEpoch); txnManager.beginTransaction(); - txnManager.maybeAddPartition(tp); + txnManager.maybeAddPartition(tpId.topicPartition()); apiVersions.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 13)); - client.prepareResponse(buildAddPartitionsToTxnResponseData(0, Collections.singletonMap(tp, Errors.NONE))); + client.prepareResponse(buildAddPartitionsToTxnResponseData(0, Collections.singletonMap(tpId.topicPartition(), Errors.NONE))); sender.runOnce(); - testSplitBatchAndSend(txnManager, producerIdAndEpoch, tp); + testSplitBatchAndSend(txnManager, producerIdAndEpoch, tpId); } @SuppressWarnings("deprecation") private void testSplitBatchAndSend(TransactionManager txnManager, ProducerIdAndEpoch producerIdAndEpoch, - TopicPartition tp) throws Exception { + TopicIdPartition tpId) throws Exception { int maxRetries = 1; - String topic = tp.topic(); + String topic = tpId.topic(); int deliveryTimeoutMs = 3000; long totalSize = 1024 * 1024; String metricGrpName = "producer-metrics"; @@ -2406,20 +2411,20 @@ private void testSplitBatchAndSend(TransactionManager txnManager, MetadataResponse metadataUpdate1 = RequestTestUtils.metadataUpdateWithIds(2, Collections.singletonMap(topic, 2), TOPIC_IDS); client.prepareMetadataUpdate(metadataUpdate1); metadataUpdate1.brokers().stream().forEach(node -> - apiVersions.update(node.idString(), NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 13)) + apiVersions.update(node.idString(), NodeApiVersions.create(ApiKeys.PRODUCE.id, ApiKeys.PRODUCE.oldestVersion(), ApiKeys.PRODUCE.latestVersion())) ); // Send the first message. long nowMs = time.milliseconds(); Cluster cluster = TestUtils.singletonCluster(); Future f1 = - accumulator.append(tp.topic(), tp.partition(), 0L, "key1".getBytes(), new byte[batchSize / 2], null, null, MAX_BLOCK_TIMEOUT, nowMs, cluster).future; + accumulator.append(tpId.topic(), tpId.partition(), 0L, "key1".getBytes(), new byte[batchSize / 2], null, null, MAX_BLOCK_TIMEOUT, nowMs, cluster).future; Future f2 = - accumulator.append(tp.topic(), tp.partition(), 0L, "key2".getBytes(), new byte[batchSize / 2], null, null, MAX_BLOCK_TIMEOUT, nowMs, cluster).future; + accumulator.append(tpId.topic(), tpId.partition(), 0L, "key2".getBytes(), new byte[batchSize / 2], null, null, MAX_BLOCK_TIMEOUT, nowMs, cluster).future; sender.runOnce(); // connect sender.runOnce(); // send produce request - assertEquals(2, txnManager.sequenceNumber(tp), "The next sequence should be 2"); + assertEquals(2, txnManager.sequenceNumber(tpId.topicPartition()), "The next sequence should be 2"); String id = client.requests().peek().destination(); assertEquals(ApiKeys.PRODUCE, client.requests().peek().requestBuilder().apiKey()); Node node = new Node(Integer.parseInt(id), "localhost", 0); @@ -2427,15 +2432,15 @@ private void testSplitBatchAndSend(TransactionManager txnManager, assertTrue(client.isReady(node, time.milliseconds()), "Client ready status should be true"); Map responseMap = new HashMap<>(); - responseMap.put(new TopicIdPartition(TOPIC_ID, tp), new ProduceResponse.PartitionResponse(Errors.MESSAGE_TOO_LARGE)); + responseMap.put(tpId, new ProduceResponse.PartitionResponse(Errors.MESSAGE_TOO_LARGE)); client.respond(new ProduceResponse(responseMap)); sender.runOnce(); // split and reenqueue - assertEquals(2, txnManager.sequenceNumber(tp), "The next sequence should be 2"); + assertEquals(2, txnManager.sequenceNumber(tpId.topicPartition()), "The next sequence should be 2"); // The compression ratio should have been improved once. assertEquals(CompressionType.GZIP.rate - CompressionRatioEstimator.COMPRESSION_RATIO_IMPROVING_STEP, CompressionRatioEstimator.estimation(topic, CompressionType.GZIP), 0.01); sender.runOnce(); // send the first produce request - assertEquals(2, txnManager.sequenceNumber(tp), "The next sequence number should be 2"); + assertEquals(2, txnManager.sequenceNumber(tpId.topicPartition()), "The next sequence number should be 2"); assertFalse(f1.isDone(), "The future shouldn't have been done."); assertFalse(f2.isDone(), "The future shouldn't have been done."); id = client.requests().peek().destination(); @@ -2444,14 +2449,14 @@ private void testSplitBatchAndSend(TransactionManager txnManager, assertEquals(1, client.inFlightRequestCount()); assertTrue(client.isReady(node, time.milliseconds()), "Client ready status should be true"); - responseMap.put(new TopicIdPartition(TOPIC_ID, tp), new ProduceResponse.PartitionResponse(Errors.NONE, 0L, 0L, 0L)); - client.respond(produceRequestMatcher(tp, producerIdAndEpoch, 0, txnManager.isTransactional()), + responseMap.put(tpId, new ProduceResponse.PartitionResponse(Errors.NONE, 0L, 0L, 0L)); + client.respond(produceRequestMatcher(tpId.topicPartition(), producerIdAndEpoch, 0, txnManager.isTransactional()), new ProduceResponse(responseMap)); sender.runOnce(); // receive assertTrue(f1.isDone(), "The future should have been done."); - assertEquals(2, txnManager.sequenceNumber(tp), "The next sequence number should still be 2"); - assertEquals(OptionalInt.of(0), txnManager.lastAckedSequence(tp), "The last ack'd sequence number should be 0"); + assertEquals(2, txnManager.sequenceNumber(tpId.topicPartition()), "The next sequence number should still be 2"); + assertEquals(OptionalInt.of(0), txnManager.lastAckedSequence(tpId.topicPartition()), "The last ack'd sequence number should be 0"); assertFalse(f2.isDone(), "The future shouldn't have been done."); assertEquals(0L, f1.get().offset(), "Offset of the first message should be 0"); sender.runOnce(); // send the second produce request @@ -2461,16 +2466,16 @@ private void testSplitBatchAndSend(TransactionManager txnManager, assertEquals(1, client.inFlightRequestCount()); assertTrue(client.isReady(node, time.milliseconds()), "Client ready status should be true"); - responseMap.put(new TopicIdPartition(TOPIC_ID, tp), new ProduceResponse.PartitionResponse(Errors.NONE, 1L, 0L, 0L)); - client.respond(produceRequestMatcher(tp, producerIdAndEpoch, 1, txnManager.isTransactional()), + responseMap.put(tpId, new ProduceResponse.PartitionResponse(Errors.NONE, 1L, 0L, 0L)); + client.respond(produceRequestMatcher(tpId.topicPartition(), producerIdAndEpoch, 1, txnManager.isTransactional()), new ProduceResponse(responseMap)); sender.runOnce(); // receive assertTrue(f2.isDone(), "The future should have been done."); - assertEquals(2, txnManager.sequenceNumber(tp), "The next sequence number should be 2"); - assertEquals(OptionalInt.of(1), txnManager.lastAckedSequence(tp), "The last ack'd sequence number should be 1"); + assertEquals(2, txnManager.sequenceNumber(tpId.topicPartition()), "The next sequence number should be 2"); + assertEquals(OptionalInt.of(1), txnManager.lastAckedSequence(tpId.topicPartition()), "The last ack'd sequence number should be 1"); assertEquals(1L, f2.get().offset(), "Offset of the first message should be 1"); - assertTrue(accumulator.getDeque(tp).isEmpty(), "There should be no batch in the accumulator"); + assertTrue(accumulator.getDeque(tpId.topicPartition()).isEmpty(), "There should be no batch in the accumulator"); assertTrue((Double) (m.metrics().get(senderMetrics.batchSplitRate).metricValue()) > 0, "There should be a split"); } } diff --git a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java index 91aa96ac6e1b3..0d5dbbc3ae841 100644 --- a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java +++ b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java @@ -941,7 +941,7 @@ public void testProduceResponseVersions() throws Exception { testAllMessageRoundTripsBetweenVersions(version, (short) 5, responseData, responseData); } else if (version >= 6 && version <= 7) { testAllMessageRoundTripsBetweenVersions(version, (short) 8, responseData, responseData); - } else if (version < 12) { + } else if (version <= 12) { testAllMessageRoundTripsBetweenVersions(version, (short) 12, responseData, responseData); } else { testEquivalentMessageRoundTrip(version, responseData); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java index e81c532a961fe..34a92587ce7da 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java @@ -133,7 +133,7 @@ public void testBuildWithCurrentMessageFormatWithoutTopicId() { .setTimeoutMs(5000), false); assertEquals(3, requestBuilder.oldestAllowedVersion()); - assertEquals(11, requestBuilder.latestAllowedVersion()); + assertEquals(13, requestBuilder.latestAllowedVersion()); } @Test diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 6906dafff11c2..c97b4c5f3d8e3 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -455,8 +455,8 @@ public void cannotUseFindCoordinatorV0ToFindTransactionCoordinator() { public void testProduceRequestPartitionSize() { Uuid topicId = Uuid.fromString("e9TvBGX5JkYAB0AQorYD4w"); String topicName = "foo"; - TopicIdPartition tp0 = createTopicIdPartition(topicId, 0, topicName); - TopicIdPartition tp1 = createTopicIdPartition(topicId, 1, topicName); + TopicIdPartition tpId0 = createTopicIdPartition(topicId, 0, topicName); + TopicIdPartition tpId1 = createTopicIdPartition(topicId, 1, topicName); MemoryRecords records0 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, Compression.NONE, new SimpleRecord("woot".getBytes())); MemoryRecords records1 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, @@ -464,16 +464,24 @@ public void testProduceRequestPartitionSize() { ProduceRequest request = ProduceRequest.builder( new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection(asList( - createTopicProduceData(PRODUCE.latestVersion(), records0, tp0), - createTopicProduceData(PRODUCE.latestVersion(), records1, tp1)).iterator())) + createTopicProduceData(PRODUCE.latestVersion(), records0, tpId0), + createTopicProduceData(PRODUCE.latestVersion(), records1, tpId1)).iterator())) .setAcks((short) 1) .setTimeoutMs(5000) .setTransactionalId("transactionalId"), true) .build((short) 7); assertEquals(2, request.partitionSizes().size()); - assertEquals(records0.sizeInBytes(), (int) request.partitionSizes().get(tp0)); - assertEquals(records1.sizeInBytes(), (int) request.partitionSizes().get(tp1)); + + assertEquals(records0.sizeInBytes(), partitionSize(request.partitionSizes(), tpId0)); + assertEquals(records1.sizeInBytes(), partitionSize(request.partitionSizes(), tpId1)); + } + + private int partitionSize(Map partitionSizes, TopicIdPartition topicIdPartition) { + return partitionSizes.entrySet().stream() + .filter(tpId -> tpId.getKey().topicId() == topicIdPartition.topicId() && + tpId.getKey().partition() == topicIdPartition.partition()).map(Map.Entry::getValue) + .findFirst().get(); } @Test @@ -2490,7 +2498,7 @@ private static ProduceRequestData.TopicProduceData createTopicProduceData(short .setPartitionData(singletonList(new ProduceRequestData.PartitionProduceData() .setIndex(tp.partition()) .setRecords(records))); - if (version >= 12) { + if (version >= 13) { topicProduceData.setTopicId(tp.topicId()); } else { topicProduceData.setName(tp.topic()); diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java index 96306936c66ee..254e2bf8ca649 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java @@ -88,9 +88,6 @@ public MirrorCheckpointTask() {} this.topicFilter = topic -> true; this.interval = Duration.ofNanos(1); this.pollTimeout = Duration.ofNanos(1); - // read __offset-sync-topic - // update __checkpoint_topic has consumer group offset on source/destination - // admincliuent -> consumergroupX 100 on source and consumergroupX 20 on destination } @Override diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java index dfc5ea375505e..a129390b39785 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java @@ -169,9 +169,6 @@ public void start(Map props) { "refreshing topics"); log.info("Started {} with {} topic-partitions.", connectorName, knownSourceTopicPartitions.size()); log.info("Starting {} took {} ms.", connectorName, System.currentTimeMillis() - start); - - // your topics -> destination - // __offset-sync-topic } @Override diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala index f408c62802061..62295b0b50644 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala @@ -648,8 +648,8 @@ class TransactionStateManager(brokerId: Int, val timestamp = time.milliseconds() val records = MemoryRecords.withRecords(TransactionLog.EnforcedCompression, new SimpleRecord(timestamp, keyBytes, valueBytes)) - val topicPartition = new TopicPartition(Topic.TRANSACTION_STATE_TOPIC_NAME, partitionFor(transactionalId)) - val transactionStateTopicIdPartition = replicaManager.topicIdPartition(topicPartition) + val transactionStateTopicPartition = new TopicPartition(Topic.TRANSACTION_STATE_TOPIC_NAME, partitionFor(transactionalId)) + val transactionStateTopicIdPartition = replicaManager.topicIdPartition(transactionStateTopicPartition) val recordsPerPartition = Map(transactionStateTopicIdPartition -> records) // set the callback function to update transaction status in cache after log append completed @@ -657,7 +657,7 @@ class TransactionStateManager(brokerId: Int, // the append response should only contain the topics partition if (responseStatus.size != 1 || !responseStatus.contains(transactionStateTopicIdPartition)) throw new IllegalStateException("Append status %s should only have one partition %s" - .format(responseStatus, topicPartition)) + .format(responseStatus, transactionStateTopicPartition)) val status = responseStatus(transactionStateTopicIdPartition) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 859ee38b787b3..d2fb9b72163c0 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -108,7 +108,7 @@ class KafkaApis(val requestChannel: RequestChannel, val groupConfigManager: GroupConfigManager ) extends ApiRequestHandler with Logging { - type FetchResponseStats = Map[TopicIdPartition, RecordValidationStats] + type ProduceResponseStats = Map[TopicIdPartition, RecordValidationStats] this.logIdent = "[KafkaApi-%d] ".format(brokerId) val configHelper = new ConfigHelper(metadataCache, config, configRepository) val authHelper = new AuthHelper(authorizer) @@ -512,7 +512,7 @@ class KafkaApis(val requestChannel: RequestChannel, } } - def processingStatsCallback(processingStats: FetchResponseStats): Unit = { + def processingStatsCallback(processingStats: ProduceResponseStats): Unit = { processingStats.foreachEntry { (topicIdPartition, info) => updateRecordConversionStats(request, topicIdPartition.topicPartition(), info) } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index c6a10b202e414..6a90f4d86485a 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -595,7 +595,7 @@ class ReplicaManager(val config: KafkaConfig, // Get topic id for an existing partition from disk if topicId is none get it from the metadata cache val topicId = partition.topicId.getOrElse(metadataCache.getTopicId(topicIdPartition.topic())) // If topic id is set to zero or null fall back to non topic id aware behaviour - val topicIdNotProvided = topicIdPartition.topicId() == Uuid.ZERO_UUID || topicIdPartition.topicId() == null + val topicIdNotProvided = topicIdPartition.topicId() == Uuid.ZERO_UUID if (topicIdNotProvided || topicId == topicIdPartition.topicId()) { partition } else { diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 4d0ce86bc442b..d5b1b5a055c83 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -98,7 +98,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { val topicId = topicNames.find { case (_, topicName) => topicName == topic} .map { case (topicId, _) => topicId } .getOrElse(Uuid.ZERO_UUID) - val topicName = if (version >= 12) "" else topic + val topicName = if (version >= 13) "" else topic Errors.forCode( resp.data .responses.find(topicName, topicId) @@ -276,6 +276,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { .setTimeoutMs(5000)) .build() } + private def createProduceRequest = createProduceRequestWithId(getTopicIds().getOrElse(tp.topic, Uuid.ZERO_UUID)) private def createFetchRequest = { diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala index 9be106a1aa021..82c5f7ce0151d 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala @@ -34,7 +34,7 @@ import scala.jdk.CollectionConverters._ class ProducerSendWhileDeletionTest extends IntegrationTestHarness { val producerCount: Int = 1 val brokerCount: Int = 3 - val defaultLingerMs: Int = 5; + val defaultLingerMs: Int = 5 serverConfig.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, 2.toString) serverConfig.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, 2.toString) @@ -125,20 +125,19 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness { * Producer will attempt to send messages to the partition specified in each record, and should * succeed as long as the metadata cache on the leader includes the partition topic id. */ - def testSendWithTopicReassignmentIsMidWay(): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testSendWithTopicReassignmentIsMidWay(quorum: String): Unit = { val numRecords = 10 val topic = "topic" val partition0: TopicPartition = new TopicPartition(topic, 0) - val partition1 = new TopicPartition(topic, 1) val admin: Admin = createAdminClient() - // Create topic with leader as 0 for the 2 partitions. - createTopicWithAssignment(topic, Map(0 -> Seq(0, 1), 1 -> Seq(0, 1))) - TestUtils.assertLeader(admin, partition1, 0) + // Create topic with leader as 0 for the 1 partition. + createTopicWithAssignment(topic, Map(0 -> Seq(0))) + TestUtils.assertLeader(admin, partition0, 0) val topicDetails = topicMetadata(admin, topic) - TestUtils.assertLeader(admin, partition1, 0) - val producer = createProducer() (1 to numRecords).foreach { i => @@ -148,12 +147,10 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness { val reassignment = Map( partition0 -> Optional.of(new NewPartitionReassignment(util.Arrays.asList(1, 2))), - partition1 -> Optional.of(new NewPartitionReassignment(util.Arrays.asList(1, 2))) ) // Change assignment of one of the replicas from 0 to 2. Leadership moves be 1. admin.alterPartitionReassignments(reassignment.asJava).all().get() - TestUtils.assertLeader(admin, partition1, 1) TestUtils.assertLeader(admin, partition0, 1) assertEquals(topicDetails.topicId(), topicMetadata(admin, topic).topicId()) diff --git a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala index 3409c0903ee0c..e057e9b0e6531 100644 --- a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import java.util.{Collections, Properties} import kafka.utils.TestUtils import org.apache.kafka.clients.admin.{Admin, TopicDescription} -import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} +import org.apache.kafka.common.{TopicIdPartition, TopicPartition} import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.message.ProduceRequestData @@ -53,14 +53,14 @@ class ProduceRequestTest extends BaseRequestTest { val (partition, leader) = createTopicAndFindPartitionWithLeader("topic") def sendAndCheck(memoryRecords: MemoryRecords, expectedOffset: Long): Unit = { - val topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), partition, "topic") + val topicId = getTopicIds().get("topic").get val produceRequest = ProduceRequest.builder(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( new ProduceRequestData.TopicProduceData() - .setName(topicIdPartition.topic()) - .setTopicId(topicIdPartition.topicId()) + .setName("topic") + .setTopicId(topicId) .setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData() - .setIndex(topicIdPartition.partition()) + .setIndex(partition) .setRecords(memoryRecords)))).iterator)) .setAcks((-1).toShort) .setTimeoutMs(3000) @@ -71,8 +71,8 @@ class ProduceRequestTest extends BaseRequestTest { val topicProduceResponse = produceResponse.data.responses.asScala.head assertEquals(1, topicProduceResponse.partitionResponses.size) val partitionProduceResponse = topicProduceResponse.partitionResponses.asScala.head - val tp = new TopicPartition(topicProduceResponse.name, partitionProduceResponse.index) - assertEquals(topicIdPartition, tp) + assertEquals(topicId, topicProduceResponse.topicId()) + assertEquals(partition, partitionProduceResponse.index()) assertEquals(Errors.NONE, Errors.forCode(partitionProduceResponse.errorCode)) assertEquals(expectedOffset, partitionProduceResponse.baseOffset) assertEquals(-1, partitionProduceResponse.logAppendTimeMs) @@ -187,13 +187,13 @@ class ProduceRequestTest extends BaseRequestTest { // Send the produce request to the non-replica val records = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("key".getBytes, "value".getBytes)) - val topicPartition = new TopicPartition("topic", partition) val produceRequest = ProduceRequest.builder(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( new ProduceRequestData.TopicProduceData() - .setName(topicPartition.topic()) + .setName("topic") + .setTopicId(getTopicIds().get("topic").get) .setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData() - .setIndex(topicPartition.partition()) + .setIndex(partition) .setRecords(records)))).iterator)) .setAcks((-1).toShort) .setTimeoutMs(3000) @@ -219,19 +219,20 @@ class ProduceRequestTest extends BaseRequestTest { @ValueSource(strings = Array("kraft")) def testCorruptLz4ProduceRequest(quorum: String): Unit = { val (partition, leader) = createTopicAndFindPartitionWithLeader("topic") + val topicId = getTopicIds().get("topic").get val timestamp = 1000000 val memoryRecords = MemoryRecords.withRecords(Compression.lz4().build(), new SimpleRecord(timestamp, "key".getBytes, "value".getBytes)) // Change the lz4 checksum value (not the kafka record crc) so that it doesn't match the contents val lz4ChecksumOffset = 6 memoryRecords.buffer.array.update(DefaultRecordBatch.RECORD_BATCH_OVERHEAD + lz4ChecksumOffset, 0) - val topicPartition = new TopicPartition("topic", partition) val produceResponse = sendProduceRequest(leader, ProduceRequest.builder(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( new ProduceRequestData.TopicProduceData() - .setName(topicPartition.topic()) + .setName("topic") + .setTopicId(topicId) .setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData() - .setIndex(topicPartition.partition()) + .setIndex(partition) .setRecords(memoryRecords)))).iterator)) .setAcks((-1).toShort) .setTimeoutMs(3000) @@ -241,8 +242,8 @@ class ProduceRequestTest extends BaseRequestTest { val topicProduceResponse = produceResponse.data.responses.asScala.head assertEquals(1, topicProduceResponse.partitionResponses.size) val partitionProduceResponse = topicProduceResponse.partitionResponses.asScala.head - val tp = new TopicPartition(topicProduceResponse.name, partitionProduceResponse.index) - assertEquals(topicPartition, tp) + assertEquals(topicId, topicProduceResponse.topicId()) + assertEquals(partition, partitionProduceResponse.index()) assertEquals(Errors.CORRUPT_MESSAGE, Errors.forCode(partitionProduceResponse.errorCode)) assertEquals(-1, partitionProduceResponse.baseOffset) assertEquals(-1, partitionProduceResponse.logAppendTimeMs) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index f1176e29e0076..77d1fa080d9aa 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -160,7 +160,7 @@ class ReplicaManagerTest { setupMetadataCacheWithTopicIds(topicIds, metadataCache) } - private def setupMetadataCacheWithTopicIds(topicIds: Map[String, Uuid], metadataCache: KRaftMetadataCache): Unit = { + private def setupMetadataCacheWithTopicIds(topicIds: Map[String, Uuid], metadataCache:MetadataCache): Unit = { val topicNames = topicIds.map(_.swap) topicNames.foreach { case (id, name) => @@ -743,6 +743,7 @@ class ReplicaManagerTest { val timer = new MockTimer(time) val replicaManager = setupReplicaManagerWithMockedPurgatories(timer) val topicPartition = new TopicPartition(topic, 0) + setupMetadataCacheWithTopicIds(topicIds, replicaManager.metadataCache) def assertLateTransactionCount(expectedCount: Option[Int]): Unit = { assertEquals(expectedCount, yammerGaugeValue[Int]("PartitionsWithLateTransactionsCount")) @@ -808,6 +809,7 @@ class ReplicaManagerTest { def testReadCommittedFetchLimitedAtLSO(): Unit = { val timer = new MockTimer(time) val replicaManager = setupReplicaManagerWithMockedPurgatories(timer) + setupMetadataCacheWithTopicIds(topicIds, replicaManager.metadataCache) try { val brokerList = Seq[Integer](0, 1).asJava @@ -930,6 +932,7 @@ class ReplicaManagerTest { def testDelayedFetchIncludesAbortedTransactions(): Unit = { val timer = new MockTimer(time) val replicaManager = setupReplicaManagerWithMockedPurgatories(timer) + setupMetadataCacheWithTopicIds(topicIds, replicaManager.metadataCache) try { val brokerList = Seq[Integer](0, 1).asJava @@ -2411,6 +2414,7 @@ class ReplicaManagerTest { val sequence = 0 val replicaManager = setupReplicaManagerWithMockedPurgatories(mockTimer) + setupMetadataCacheWithTopicIds(topicIds, replicaManager.metadataCache) try { replicaManager.becomeLeaderOrFollower(1, makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), new LeaderAndIsr(0, List(0, 1).map(Int.box).asJava)), From aa811f183dfe25ebf052ade2745059ee74ecbbae Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Wed, 16 Apr 2025 09:02:28 +0100 Subject: [PATCH 37/40] address feedback --- .../apache/kafka/clients/producer/internals/SenderTest.java | 6 +++--- .../apache/kafka/common/requests/ProduceRequestTest.java | 6 +++--- core/src/main/scala/kafka/server/ReplicaManager.scala | 2 +- .../integration/kafka/api/AuthorizerIntegrationTest.scala | 1 - .../test/scala/unit/kafka/server/ProduceRequestTest.scala | 1 - 5 files changed, 7 insertions(+), 9 deletions(-) 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 bf6f7713e6e49..9a6d1acb6e1e3 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 @@ -2382,7 +2382,7 @@ public void testTransactionalSplitBatchAndSend() throws Exception { txnManager.beginTransaction(); txnManager.maybeAddPartition(tpId.topicPartition()); - apiVersions.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 13)); + apiVersions.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, ApiKeys.PRODUCE.oldestVersion(), ApiKeys.PRODUCE.latestVersion())); client.prepareResponse(buildAddPartitionsToTxnResponseData(0, Collections.singletonMap(tpId.topicPartition(), Errors.NONE))); sender.runOnce(); @@ -3413,7 +3413,7 @@ public void testWhenProduceResponseReturnsWithALeaderShipChangeErrorAndNewLeader })); Cluster startingMetadataCluster = metadata.fetch(); startingMetadataCluster.nodes().forEach(node -> - apiVersions.update(node.idString(), NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 13)) + apiVersions.update(node.idString(), NodeApiVersions.create(ApiKeys.PRODUCE.id, ApiKeys.PRODUCE.oldestVersion(), ApiKeys.PRODUCE.latestVersion())) ); // Produce to tp0/1/2, where NO_LEADER_OR_FOLLOWER with new leader info is returned for tp0/1, and tp2 is returned without errors. @@ -3436,7 +3436,7 @@ public void testWhenProduceResponseReturnsWithALeaderShipChangeErrorAndNewLeader responses.put(tp1, new OffsetAndError(-1, Errors.NOT_LEADER_OR_FOLLOWER)); responses.put(tp2, new OffsetAndError(100, Errors.NONE)); newNodes.forEach(node -> - apiVersions.update(node.idString(), NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 13)) + apiVersions.update(node.idString(), NodeApiVersions.create(ApiKeys.PRODUCE.id, ApiKeys.PRODUCE.oldestVersion(), ApiKeys.PRODUCE.latestVersion())) ); Map partitionLeaderInfo = new HashMap<>(); ProduceResponseData.LeaderIdAndEpoch tp0LeaderInfo = new ProduceResponseData.LeaderIdAndEpoch(); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java index 34a92587ce7da..5a7467fee2bb8 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java @@ -111,7 +111,7 @@ public void testBuildWithCurrentMessageFormat() { .setAcks((short) 1) .setTimeoutMs(5000), false); - assertEquals(3, requestBuilder.oldestAllowedVersion()); + assertEquals(ApiKeys.PRODUCE.oldestVersion(), requestBuilder.oldestAllowedVersion()); assertEquals(ApiKeys.PRODUCE.latestVersion(), requestBuilder.latestAllowedVersion()); } @@ -132,8 +132,8 @@ public void testBuildWithCurrentMessageFormatWithoutTopicId() { .setAcks((short) 1) .setTimeoutMs(5000), false); - assertEquals(3, requestBuilder.oldestAllowedVersion()); - assertEquals(13, requestBuilder.latestAllowedVersion()); + assertEquals(ApiKeys.PRODUCE.oldestVersion(), requestBuilder.oldestAllowedVersion()); + assertEquals(ApiKeys.PRODUCE.latestVersion(), requestBuilder.latestAllowedVersion()); } @Test diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 6a90f4d86485a..d04171cdbaca5 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -594,7 +594,7 @@ class ReplicaManager(val config: KafkaConfig, case Right(partition) => // Get topic id for an existing partition from disk if topicId is none get it from the metadata cache val topicId = partition.topicId.getOrElse(metadataCache.getTopicId(topicIdPartition.topic())) - // If topic id is set to zero or null fall back to non topic id aware behaviour + // If topic id is set to zero_uuid fall back to non topic id aware behaviour val topicIdNotProvided = topicIdPartition.topicId() == Uuid.ZERO_UUID if (topicIdNotProvided || topicId == topicIdPartition.topicId()) { partition diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index d5b1b5a055c83..57bdcac0bbb88 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -94,7 +94,6 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { val requestKeyToError = (topicNames: Map[Uuid, String], version: Short) => Map[ApiKeys, Nothing => Errors]( ApiKeys.METADATA -> ((resp: requests.MetadataResponse) => resp.errors.asScala.find(_._1 == topic).getOrElse(("test", Errors.NONE))._2), ApiKeys.PRODUCE -> ((resp: requests.ProduceResponse) => { - val topicId = topicNames.find { case (_, topicName) => topicName == topic} .map { case (topicId, _) => topicId } .getOrElse(Uuid.ZERO_UUID) diff --git a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala index e057e9b0e6531..6d6313d27c8b0 100644 --- a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala @@ -57,7 +57,6 @@ class ProduceRequestTest extends BaseRequestTest { val produceRequest = ProduceRequest.builder(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( new ProduceRequestData.TopicProduceData() - .setName("topic") .setTopicId(topicId) .setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData() .setIndex(partition) From cb66c851fb9962dbcc32642674b429718190330d Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Wed, 16 Apr 2025 10:58:08 +0100 Subject: [PATCH 38/40] fix test --- .../kafka/network/DynamicConnectionQuotaTest.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala b/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala index 4f5cd7f4a2803..b9136e9a75b09 100644 --- a/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord} import org.apache.kafka.common.requests.{ProduceRequest, ProduceResponse} import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.test.api.Flaky -import org.apache.kafka.common.{KafkaException, requests} +import org.apache.kafka.common.{KafkaException, Uuid, requests} import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.server.config.QuotaConfig import org.junit.jupiter.api.Assertions._ @@ -56,7 +56,7 @@ class DynamicConnectionQuotaTest extends BaseRequestTest { val plaintextListenerDefaultQuota = 30 var executor: ExecutorService = _ var admin: Admin = _ - + var topicId: Uuid = _ override def brokerPropertyOverrides(properties: Properties): Unit = { properties.put(QuotaConfig.NUM_QUOTA_SAMPLES_CONFIG, "2") properties.put("listener.name.plaintext.max.connection.creation.rate", plaintextListenerDefaultQuota.toString) @@ -67,6 +67,7 @@ class DynamicConnectionQuotaTest extends BaseRequestTest { super.setUp(testInfo) admin = createAdminClient(listener) TestUtils.createTopicWithAdmin(admin, topic, brokers, controllerServers) + topicId = TestUtils.describeTopic(admin, topic).topicId() } @AfterEach @@ -308,7 +309,7 @@ class DynamicConnectionQuotaTest extends BaseRequestTest { requests.ProduceRequest.builder(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection( Collections.singletonList(new ProduceRequestData.TopicProduceData() - .setName(topic) + .setTopicId(topicId) .setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData() .setIndex(0) .setRecords(MemoryRecords.withRecords(Compression.NONE, From 6932bd1a8ba7fa84bc08ca9e26aa7552c0807ece Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Wed, 16 Apr 2025 11:19:47 +0100 Subject: [PATCH 39/40] address feedback --- .../integration/kafka/api/AuthorizerIntegrationTest.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index de46083482508..4cf3c485f6424 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -97,10 +97,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { val topicId = topicNames.find { case (_, topicName) => topicName == topic} .map { case (topicId, _) => topicId } .getOrElse(Uuid.ZERO_UUID) - val topicName = if (version >= 13) "" else topic Errors.forCode( resp.data - .responses.find(topicName, topicId) + .responses.find("", topicId) // version is always >= 13 no need to use topic name .partitionResponses.asScala.find(_.index == part).get .errorCode ) From 54ecdf5828584b23001a253aea44315b617ecd48 Mon Sep 17 00:00:00 2001 From: OmniaGM Date: Wed, 16 Apr 2025 16:10:37 +0100 Subject: [PATCH 40/40] address feedback --- .../clients/producer/internals/SenderTest.java | 17 ++++++++++------- .../api/ProducerSendWhileDeletionTest.scala | 6 +++--- 2 files changed, 13 insertions(+), 10 deletions(-) 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 b40e12e00a3b8..50cfced97c699 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 @@ -157,7 +157,10 @@ public class SenderTest { private static final String TOPIC_NAME = "test"; private static final Uuid TOPIC_ID = Uuid.fromString("MKXx1fIkQy2J9jXHhK8m1w"); - private static final Map TOPIC_IDS = new HashMap<>(Map.of(TOPIC_NAME, TOPIC_ID)); + private static final Map TOPIC_IDS = Map.of( + TOPIC_NAME, TOPIC_ID, + "testSplitBatchAndSend", Uuid.fromString("2J9hK8m1wHMKjXfIkQyXx1") + ); private final TopicPartition tp0 = new TopicPartition(TOPIC_NAME, 0); private final TopicPartition tp1 = new TopicPartition(TOPIC_NAME, 1); private final TopicPartition tp2 = new TopicPartition(TOPIC_NAME, 2); @@ -2357,9 +2360,9 @@ public void testBumpEpochWhenOutOfOrderSequenceReceived() throws InterruptedExce @Test public void testIdempotentSplitBatchAndSend() throws Exception { - Uuid topicId = Uuid.fromString("2J9hK8m1wHMKjXfIkQyXx1"); - TOPIC_IDS.put("testSplitBatchAndSend", topicId); - TopicIdPartition tpId = new TopicIdPartition(topicId, new TopicPartition("testSplitBatchAndSend", 1)); + TopicIdPartition tpId = new TopicIdPartition( + TOPIC_IDS.getOrDefault("testSplitBatchAndSend", Uuid.ZERO_UUID), + new TopicPartition("testSplitBatchAndSend", 1)); TransactionManager txnManager = createTransactionManager(); ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(123456L, (short) 0); setupWithTransactionState(txnManager); @@ -2370,10 +2373,10 @@ public void testIdempotentSplitBatchAndSend() throws Exception { @Test public void testTransactionalSplitBatchAndSend() throws Exception { - Uuid topicId = Uuid.fromString("2J9hK8m1wHMKjXfIkQyXx1"); - TOPIC_IDS.put("testSplitBatchAndSend", topicId); ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(123456L, (short) 0); - TopicIdPartition tpId = new TopicIdPartition(topicId, new TopicPartition("testSplitBatchAndSend", 1)); + TopicIdPartition tpId = new TopicIdPartition( + TOPIC_IDS.getOrDefault("testSplitBatchAndSend", Uuid.ZERO_UUID), + new TopicPartition("testSplitBatchAndSend", 1)); TransactionManager txnManager = new TransactionManager(logContext, "testSplitBatchAndSend", 60000, 100, apiVersions); diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala index 82c5f7ce0151d..aaedbf0962090 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala @@ -146,12 +146,12 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness { } val reassignment = Map( - partition0 -> Optional.of(new NewPartitionReassignment(util.Arrays.asList(1, 2))), + partition0 -> Optional.of(new NewPartitionReassignment(util.Arrays.asList(2))), ) - // Change assignment of one of the replicas from 0 to 2. Leadership moves be 1. + // Change assignment of one of the replicas from 0 to 2. Leadership moves be . admin.alterPartitionReassignments(reassignment.asJava).all().get() - TestUtils.assertLeader(admin, partition0, 1) + TestUtils.assertLeader(admin, partition0, 2) assertEquals(topicDetails.topicId(), topicMetadata(admin, topic).topicId()) // Producer should be able to send messages even after topic gets reassigned