junrao commented on code in PR #19964:
URL: https://github.com/apache/kafka/pull/19964#discussion_r2178023461


##########
clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java:
##########
@@ -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());

Review Comment:
   @lucasbru : Thanks for reporting the issue. I think this is the problem. 
This callback is called when the response completes, at which point, the 
metadata could have changed. Calling `metadata.topicNames()` at that point will 
potentially return metadata inconsistent with what's being used when generating 
the produce request. So, we need to call `metadata.topicNames()` outside the 
callback, save it as a final and pass it to the callback.



##########
clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java:
##########
@@ -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.");

Review Comment:
   It would be useful to include `topicNames`, `r.topicId()` and `r.name()` too.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to