chia7712 commented on code in PR #19964: URL: https://github.com/apache/kafka/pull/19964#discussion_r2152340372
########## clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java: ########## @@ -609,7 +607,14 @@ private void handleProduceResponse(ClientResponse response, Map<TopicPartition, .collect(Collectors.toList()), p.errorMessage(), p.currentLeader()); - ProducerBatch batch = batches.get(tp); + // 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 might be empty. + ProducerBatch batch = batches.entrySet().stream() + .filter(entry -> + entry.getKey().same(new TopicIdPartition(r.topicId(), p.index(), r.name())) + ).map(Map.Entry::getValue).findFirst().orElse(null); Review Comment: It is possible to have null batch, right? For example, the topic is recreated after the batch is generated ########## clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java: ########## @@ -609,7 +607,14 @@ private void handleProduceResponse(ClientResponse response, Map<TopicPartition, .collect(Collectors.toList()), p.errorMessage(), p.currentLeader()); - ProducerBatch batch = batches.get(tp); + // 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 might be empty. + ProducerBatch batch = batches.entrySet().stream() + .filter(entry -> + entry.getKey().same(new TopicIdPartition(r.topicId(), p.index(), r.name())) Review Comment: Should we create the `TopicIdPartition` outside the stream to avoid creating many temporary objects? ########## clients/src/main/java/org/apache/kafka/common/TopicIdPartition.java: ########## @@ -78,6 +78,21 @@ public TopicPartition topicPartition() { return topicPartition; } + /** + * Checking if TopicIdPartition meant to be the same reference to same this object but doesn't have all the data. + * If topic name is empty and topic id is persisted then the method will relay on topic id only + * otherwise the method will relay on topic name. + * @return true if topic has same topicId and partition index as topic names some time might be empty. + */ + public boolean same(TopicIdPartition tpId) { + if (tpId.topic().isEmpty() && !tpId.topicId.equals(Uuid.ZERO_UUID)) { Review Comment: `TopicIdPartition#topic` is nullable, so the null check is necessary. ########## clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java: ########## @@ -855,7 +860,7 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo if (batches.isEmpty()) return; - final Map<TopicPartition, ProducerBatch> recordsByPartition = new HashMap<>(batches.size()); + final Map<TopicIdPartition, ProducerBatch> recordsByPartition = new HashMap<>(batches.size()); Map<String, Uuid> topicIds = topicIdsForBatches(batches); Review Comment: one question: what happens if the topic id is unknown (`Uuid.ZERO_UUID`)? -- 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