Skip to content

KAFKA-19397: Ensure consistent metadata usage in produce request and response #19964

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 15 commits into
base: trunk
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 13 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.TopicPartitionInfo;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.common.test.ClusterInstance;
Expand All @@ -37,14 +38,19 @@
import org.apache.kafka.storage.internals.log.UnifiedLog;
import org.apache.kafka.test.TestUtils;

import org.junit.jupiter.api.Timeout;

import java.io.File;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicInteger;

import static org.apache.kafka.clients.producer.ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG;
import static org.apache.kafka.clients.producer.ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG;
Expand Down Expand Up @@ -157,6 +163,47 @@ public void testSendWithRecreatedTopic() throws Exception {
}
}

@Timeout(90)
@ClusterTest
public void testSendWhileTopicGetRecreated() {
int maxNumTopicRecreationAttempts = 10;
var recreateTopicFuture = CompletableFuture.supplyAsync(() -> {
var topicIds = new HashSet<Uuid>();
while (topicIds.size() < maxNumTopicRecreationAttempts) {
try (var admin = cluster.admin()) {
if (admin.listTopics().names().get().contains(topic)) {
admin.deleteTopics(List.of(topic)).all().get();
}
topicIds.add(admin.createTopics(List.of(new NewTopic(topic, 1, (short) 1))).topicId(topic).get());
} catch (Exception e) {
// ignore
}
}
return topicIds;
});

AtomicInteger numSuccess = new AtomicInteger(0);
var producerFuture = CompletableFuture.runAsync(() -> {
try (var producer = cluster.producer()) {
for (int i = 1; i <= numRecords; i++) {
var resp = producer.send(new ProducerRecord<>(topic, null, ("value" + i).getBytes()),
(metadata, exception) -> {
if (metadata != null) {
numSuccess.incrementAndGet();
}
}).get();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's possible for get() to throw an exception like TopicNotExist, right? If that happens, the assertion on numRecords will fail.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes you are right this might get UnknownTopicOrPartitionException if we kept retrying and failed. Updated this to use MockProducerInterceptor which count success and wrapped this in TestUtils.waitForCondition

assertEquals(topic, resp.topic());
}
} catch (Exception e) {
// ignore
}
});
var topicIds = recreateTopicFuture.join();
producerFuture.join();
assertEquals(maxNumTopicRecreationAttempts, topicIds.size());
assertEquals(numRecords, numSuccess.intValue());
}

@ClusterTest
public void testSendWithTopicReassignmentIsMidWay() throws Exception {
var partition0 = new TopicPartition(topic, 0);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -565,7 +565,7 @@ private boolean awaitNodeReady(Node node, FindCoordinatorRequest.CoordinatorType
/**
* Handle a produce response
*/
private void handleProduceResponse(ClientResponse response, Map<TopicPartition, ProducerBatch> batches, long now) {
private void handleProduceResponse(ClientResponse response, Map<TopicPartition, ProducerBatch> batches, Map<Uuid, String> topicNames, long now) {
RequestHeader requestHeader = response.requestHeader();
int correlationId = requestHeader.correlationId();
if (response.wasTimedOut()) {
Expand Down Expand Up @@ -595,9 +595,6 @@ private void handleProduceResponse(ClientResponse response, Map<TopicPartition,
// This will be set by completeBatch.
Map<TopicPartition, Metadata.LeaderIdAndEpoch> partitionsWithUpdatedLeaderInfo = new HashMap<>();
produceResponse.data().responses().forEach(r -> r.partitionResponses().forEach(p -> {
// 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(
Errors.forCode(p.errorCode()),
p.baseOffset(),
Expand All @@ -609,7 +606,19 @@ private void handleProduceResponse(ClientResponse response, Map<TopicPartition,
.collect(Collectors.toList()),
p.errorMessage(),
p.currentLeader());

// Version 13 drop topic name and add support to topic id.
// We need to find batch based on topic id and partition index only as
// topic name in the response will be empty.
// For older versions, topic id is zero, and we will find the batch based on the topic name.
TopicPartition tp = (!r.topicId().equals(Uuid.ZERO_UUID) && topicNames.containsKey(r.topicId())) ?
new TopicPartition(topicNames.get(r.topicId()), p.index()) :
new TopicPartition(r.name(), p.index());

ProducerBatch batch = batches.get(tp);
if (batch == null) {
throw new IllegalStateException("batch created for " + tp + " can't be found.");
}
completeBatch(batch, partResp, correlationId, now, partitionsWithUpdatedLeaderInfo);
}));

Expand Down Expand Up @@ -892,7 +901,7 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo
.setTopicData(tpd),
useTransactionV1Version
);
RequestCompletionHandler callback = response -> handleProduceResponse(response, recordsByPartition, time.milliseconds());
RequestCompletionHandler callback = response -> handleProduceResponse(response, recordsByPartition, metadata.topicNames(), time.milliseconds());

String nodeId = Integer.toString(destination);
ClientRequest clientRequest = client.newClientRequest(nodeId, requestBuilder, now, acks != 0,
Expand Down