lianetm commented on code in PR #19577:
URL: https://github.com/apache/kafka/pull/19577#discussion_r2116058907


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java:
##########
@@ -698,15 +699,22 @@ class OffsetCommitRequestState extends 
RetriableRequestState {
         }
 
         public NetworkClientDelegate.UnsentRequest toUnsentRequest() {
+            Map<String, Uuid> topicIds = metadata.topicIds();
+            boolean canUseTopicIds = true;
             Map<String, OffsetCommitRequestData.OffsetCommitRequestTopic> 
requestTopicDataMap = new HashMap<>();
             for (Map.Entry<TopicPartition, OffsetAndMetadata> entry : 
offsets.entrySet()) {
                 TopicPartition topicPartition = entry.getKey();
                 OffsetAndMetadata offsetAndMetadata = entry.getValue();
+                Uuid topicId = topicIds.getOrDefault(topicPartition.topic(), 
Uuid.ZERO_UUID);
+                if (topicId.equals(Uuid.ZERO_UUID)) {
+                    canUseTopicIds = false;
+                }
 
                 OffsetCommitRequestData.OffsetCommitRequestTopic topic = 
requestTopicDataMap
                     .getOrDefault(topicPartition.topic(),
                         new OffsetCommitRequestData.OffsetCommitRequestTopic()
                             .setName(topicPartition.topic())
+                            .setTopicId(topicId)

Review Comment:
   This pair is an interesting bit. It surfaces the gap that here we could be 
wrongly committing positions for a re-created topic (same topic name, but 
different topic ID). That is covered with 
[KAFKA-13447](https://issues.apache.org/jira/browse/KAFKA-13447) still open, 
but raising it to make sure we're considering it. 
   
   I expect we need to ensure that the topic ID we have here (from metadata) is 
the same one we got the position from. If it's not the same (topic re-created), 
then we need to :
   - ensure we don't commit the position
   - surface error to the user (if commit sync or in callback if async. 
Swallow/log for auto-commit on the interval or before revocation?)
   Makes sense? I guess we want to handle this on the other jira, but closely 
related. 
   



-- 
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