You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Hangleton (via GitHub)" <gi...@apache.org> on 2023/02/13 16:30:35 UTC

[GitHub] [kafka] Hangleton opened a new pull request, #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Hangleton opened a new pull request, #13240:
URL: https://github.com/apache/kafka/pull/13240

   WIP - writing tests.
   
   *More detailed description of your change,
   if necessary. The PR title and PR message become
   the squashed commit message, so use a separate
   comment to ping reviewers.*
   
   *Summary of testing strategy (including rationale)
   for the feature or bug fix. Unit and/or integration
   tests are expected for any behaviour change and
   system tests should be considered for larger changes.*
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1105987099


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -425,35 +425,72 @@ class KafkaApis(val requestChannel: RequestChannel,
       requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
       CompletableFuture.completedFuture[Unit](())
     } else {
+      val topicNames =
+        if (offsetCommitRequest.version() >= 9)
+          metadataCache.topicIdsToNames()
+        else
+          Collections.emptyMap[Uuid, String]()
+
+      // For version < 9, lookup from topicNames fails and the topic name (which cannot be null) is returned.
+      // For version >= 9, if lookup from topicNames fails, there are two possibilities:
+      //
+      // a) The topic ID was left to default and the topic name should have been populated as a fallback instead.
+      //    If none was provided, null is returned.
+      //
+      // b) The topic ID was not default but is not present in the local topic IDs cache. In this case, because
+      //    clients should make exclusive use of topic name or topic ID, the topic name should be null. If however
+      //    the client provided a topic name, we do not want to use it, because any topic with the same name
+      //    present locally would then have a topic ID which does not match the topic ID in the request.
+      def resolveTopicName(topic: OffsetCommitRequestData.OffsetCommitRequestTopic): String = {
+          val resolvedFromId = topicNames.get(topic.topicId())
+          if (resolvedFromId != null)
+            resolvedFromId
+          else if (offsetCommitRequest.version() < 9 || Uuid.ZERO_UUID.equals(topic.topicId)) {
+            topic.name()
+          } else {
+            null
+          }
+      }
+
       val authorizedTopics = authHelper.filterByAuthorized(
         request.context,
         READ,
         TOPIC,
-        offsetCommitRequest.data.topics.asScala
-      )(_.name)
+        offsetCommitRequest.data.topics.asScala.filter(topic => resolveTopicName(topic) != null)
+      )(resolveTopicName)
 
       val responseBuilder = new OffsetCommitResponse.Builder()
       val authorizedTopicsRequest = new mutable.ArrayBuffer[OffsetCommitRequestData.OffsetCommitRequestTopic]()
       offsetCommitRequest.data.topics.forEach { topic =>
-        if (!authorizedTopics.contains(topic.name)) {
+        val topicName = resolveTopicName(topic)
+        if (topicName == null) {
+          // Topic name cannot be null for version < 9. From version >= 9, topicName is null iff it cannot
+          // be resolved from the local topic IDs cache or topic ID was left to default but no fallback topic
+          // name was provided.
+          responseBuilder.addPartitions[OffsetCommitRequestData.OffsetCommitRequestPartition](
+            topic.name, topic.topicId, topic.partitions, _.partitionIndex, Errors.UNKNOWN_TOPIC_ID)
+        } else if (!authorizedTopics.contains(topicName)) {
           // If the topic is not authorized, we add the topic and all its partitions
           // to the response with TOPIC_AUTHORIZATION_FAILED.
           responseBuilder.addPartitions[OffsetCommitRequestData.OffsetCommitRequestPartition](
-            topic.name, topic.partitions, _.partitionIndex, Errors.TOPIC_AUTHORIZATION_FAILED)
-        } else if (!metadataCache.contains(topic.name)) {
+            topic.name, topic.topicId, topic.partitions, _.partitionIndex, Errors.TOPIC_AUTHORIZATION_FAILED)
+        } else if (!metadataCache.contains(topicName)) {

Review Comment:
   You are right, yes. Let's make this explicit in the code.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1447864999

   Thanks for review David. Updating the PR to enforce the exclusive use of topic IDs from version 9 and adding the integration test you mentioned. Thanks for the guidance!


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1125687294


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java:
##########
@@ -59,7 +63,36 @@ public OffsetCommitRequest build(short version) {
                 throw new UnsupportedVersionException("The broker offset commit protocol version " +
                         version + " does not support usage of config group.instance.id.");
             }
-            return new OffsetCommitRequest(data, version);
+
+            // Copy since we can mutate it.
+            OffsetCommitRequestData requestData = data.duplicate();

Review Comment:
   Sure! Added the tests. Thanks.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1454803355

   I think that we have to do 2) anyway because topic ids may not be enabled on the server side.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1129311477


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1352,8 +1370,22 @@ public void handle(OffsetCommitResponse commitResponse, RequestFuture<Void> futu
             Set<String> unauthorizedTopics = new HashSet<>();
 
             for (OffsetCommitResponseData.OffsetCommitResponseTopic topic : commitResponse.data().topics()) {
+                String topicName = topic.name();
+
+                if (commitResponse.version() >= 9) {

Review Comment:
   Yes, this works. Thanks for the call-out.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1129167871


##########
clients/src/main/java/org/apache/kafka/common/TopicResolver.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common;
+
+import org.apache.kafka.common.errors.InvalidTopicException;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Encapsulates the resolution of a topic name from its ID, or its ID from its name, using a local
+ * bidirectional mapping. This resolver assumes there is a bijection between topic IDs and topic names.
+ * <p></p>
+ * Note that this class intends to be used for the (reverse) lookup of topic IDs/names, but
+ * not to characterize the set of topics which are known by a client. Use the
+ * {@link org.apache.kafka.clients.MetadataCache} for that purpose.
+ */
+//@Immutable, @ThreadSafe
+public class TopicResolver {

Review Comment:
   I agree with you and am not satisfied either with `TopicResolver` but could not find a better name. `TopicIdResolver` would be misleading because this class treats topic ids and names symmetrically. One of the closest entity with similar purposes as this is in [`MetadataCache#L93`](https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/MetadataCache.scala#L93) where `topicIdInfo` is used to refer to the bidirectional mapping. The suffix `Info` could be used here as well although it is not strictly aligned with other use of that suffix such as in [`TopicPartitionInfo`](https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/TopicPartitionInfo.java).  Interestingly another entity for which may have had to be assigned a generic name is [`TopicCollection`](https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/TopicCollection.java).
   
   Using another name to refer to the dual name/id reference such as `TopicRefResolver` introduces yet another noun (_reference_) not used elsewhere in the codebase and which can be confusing.
   
   So, I am not sure about what could be a better name but maybe `TopicInfoResolver` or `TopicIdInfoResolver` or `TopicIdInfo` or `TopicIdResolver` may sound better albeit still ambiguous and partially incorrect?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1131114261


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -119,28 +131,52 @@ public boolean shouldClientThrottle(short version) {
         return version >= 4;
     }
 
+    public short version() {
+        return version;
+    }
+
     public static class Builder {
         OffsetCommitResponseData data = new OffsetCommitResponseData();
         HashMap<String, OffsetCommitResponseTopic> byTopicName = new HashMap<>();
+        private final TopicResolver topicResolver;
+        private final short version;
+
+        public Builder(TopicResolver topicResolver, short version) {

Review Comment:
   How about doing the following?
   
   We change the signature of `GroupCoordinator.handleCommitOffsets` to the following:
   
   ```
     def handleCommitOffsets(groupId: String,
                             memberId: String,
                             groupInstanceId: Option[String],
                             generationId: Int,
                             offsetMetadata: immutable.Map[TopicIdPartition, OffsetAndMetadata],
                             responseCallback: immutable.Map[TopicIdPartition, Errors] => Unit,
                             requestLocal: RequestLocal = RequestLocal.NoCaching): Unit = {
   ```
   
   Note the change from `TopicPartition` to `TopicIdPartition` for `offsetMetadata` and `responseCallback`.
   
   Then, we have to adapt the implementation of `handleCommitOffsets` to get the `TopicPartition` from the `TopicIdPartition` where required. We can keep `pendingOffsetCommits` and `offsets` keyed by `TopicPartition` for now in `GroupMetadataManager`.
   
   This allows the preservation of the topic ids provided to the GroupCoordinator but it does not provide any stronger guarantee for the offsets yet (as you pointed out). With this approach, we don't depend on the resolver at all.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1485037330

   Hello David, this PR has been updated and is ready for review. Thanks!


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1121487731


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -425,35 +425,63 @@ class KafkaApis(val requestChannel: RequestChannel,
       requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
       CompletableFuture.completedFuture[Unit](())
     } else {
+      val topicNames =
+        if (offsetCommitRequest.version() >= 9)
+          metadataCache.topicIdsToNames()
+        else
+          Collections.emptyMap[Uuid, String]()
+
+      val resolvedTopics = new ArrayBuffer[OffsetCommitRequestData.OffsetCommitRequestTopic]()
+      offsetCommitRequest.data.topics.forEach { topic =>
+        var topicName = topic.name()
+        if (Utils.isBlank(topicName)) {

Review Comment:
   It would be better to rely on the version of the request instead of the topic name here.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1121496085


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java:
##########
@@ -101,18 +103,18 @@ public static List<OffsetCommitResponseTopic> getErrorResponseTopics(
                                            .setPartitionIndex(requestPartition.partitionIndex())
                                            .setErrorCode(e.code()));
             }
-            responseTopicData.add(new OffsetCommitResponseTopic()
-                    .setName(entry.name())
-                    .setPartitions(responsePartitions)
-            );
+            OffsetCommitResponseTopic responseTopic = new OffsetCommitResponseTopic()
+                .setTopicId(version >= 9 ? entry.topicId() : Uuid.ZERO_UUID)
+                .setName(version < 9 ? entry.name() : null);

Review Comment:
   I think that we could just set both the name and the id all the time as the fields are ignorable. The serialization framework will do the right thing based on the version. We could also remove `version` from the arguments.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1428304503

   Hello David (@dajac), still working on this but opening a draft if you wish to start reviewing at your convenience.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1105846605


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -425,35 +425,72 @@ class KafkaApis(val requestChannel: RequestChannel,
       requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
       CompletableFuture.completedFuture[Unit](())
     } else {
+      val topicNames =
+        if (offsetCommitRequest.version() >= 9)
+          metadataCache.topicIdsToNames()
+        else
+          Collections.emptyMap[Uuid, String]()
+
+      // For version < 9, lookup from topicNames fails and the topic name (which cannot be null) is returned.
+      // For version >= 9, if lookup from topicNames fails, there are two possibilities:
+      //
+      // a) The topic ID was left to default and the topic name should have been populated as a fallback instead.
+      //    If none was provided, null is returned.
+      //
+      // b) The topic ID was not default but is not present in the local topic IDs cache. In this case, because
+      //    clients should make exclusive use of topic name or topic ID, the topic name should be null. If however
+      //    the client provided a topic name, we do not want to use it, because any topic with the same name
+      //    present locally would then have a topic ID which does not match the topic ID in the request.
+      def resolveTopicName(topic: OffsetCommitRequestData.OffsetCommitRequestTopic): String = {
+          val resolvedFromId = topicNames.get(topic.topicId())
+          if (resolvedFromId != null)
+            resolvedFromId
+          else if (offsetCommitRequest.version() < 9 || Uuid.ZERO_UUID.equals(topic.topicId)) {
+            topic.name()
+          } else {
+            null
+          }
+      }
+
       val authorizedTopics = authHelper.filterByAuthorized(
         request.context,
         READ,
         TOPIC,
-        offsetCommitRequest.data.topics.asScala
-      )(_.name)
+        offsetCommitRequest.data.topics.asScala.filter(topic => resolveTopicName(topic) != null)
+      )(resolveTopicName)
 
       val responseBuilder = new OffsetCommitResponse.Builder()
       val authorizedTopicsRequest = new mutable.ArrayBuffer[OffsetCommitRequestData.OffsetCommitRequestTopic]()
       offsetCommitRequest.data.topics.forEach { topic =>
-        if (!authorizedTopics.contains(topic.name)) {
+        val topicName = resolveTopicName(topic)
+        if (topicName == null) {
+          // Topic name cannot be null for version < 9. From version >= 9, topicName is null iff it cannot
+          // be resolved from the local topic IDs cache or topic ID was left to default but no fallback topic
+          // name was provided.
+          responseBuilder.addPartitions[OffsetCommitRequestData.OffsetCommitRequestPartition](

Review Comment:
   There is very likely a bug here. In this case, `topic.name` is `null` and the response builder uses a HashMap keyed by topic name. Therefore, all the topics with an unknown topic id will end up together.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1104738082


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java:
##########
@@ -59,6 +62,14 @@ public OffsetCommitRequest build(short version) {
                 throw new UnsupportedVersionException("The broker offset commit protocol version " +
                         version + " does not support usage of config group.instance.id.");
             }
+            if (version >= 9) {
+                data.topics().forEach(topic -> {
+                    // Set the topic name to null if a topic ID for the topic is present.
+                    if (!Uuid.ZERO_UUID.equals(topic.topicId())) {

Review Comment:
   Note: topic ID must not be null for a request/response version >= 9 to be serialized. `ZERO_UUID` means no topic ID specified.



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -425,35 +425,72 @@ class KafkaApis(val requestChannel: RequestChannel,
       requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
       CompletableFuture.completedFuture[Unit](())
     } else {
+      val topicNames =
+        if (offsetCommitRequest.version() >= 9)
+          metadataCache.topicIdsToNames()
+        else
+          Collections.emptyMap[Uuid, String]()
+
+      // For version < 9, lookup from topicNames fails and the topic name (which cannot be null) is returned.
+      // For version >= 9, if lookup from topicNames fails, there are two possibilities:
+      //
+      // a) The topic ID was left to default and the topic name should have been populated as a fallback instead.
+      //    If none was provided, null is returned.
+      //
+      // b) The topic ID was not default but is not present in the local topic IDs cache. In this case, because
+      //    clients should make exclusive use of topic name or topic ID, the topic name should be null. If however
+      //    the client provided a topic name, we do not want to use it, because any topic with the same name
+      //    present locally would then have a topic ID which does not match the topic ID in the request.
+      def resolveTopicName(topic: OffsetCommitRequestData.OffsetCommitRequestTopic): String = {
+          val resolvedFromId = topicNames.get(topic.topicId())
+          if (resolvedFromId != null)
+            resolvedFromId
+          else if (offsetCommitRequest.version() < 9 || Uuid.ZERO_UUID.equals(topic.topicId)) {
+            topic.name()
+          } else {
+            null
+          }
+      }
+
       val authorizedTopics = authHelper.filterByAuthorized(
         request.context,
         READ,
         TOPIC,
-        offsetCommitRequest.data.topics.asScala
-      )(_.name)
+        offsetCommitRequest.data.topics.asScala.filter(topic => resolveTopicName(topic) != null)
+      )(resolveTopicName)
 
       val responseBuilder = new OffsetCommitResponse.Builder()
       val authorizedTopicsRequest = new mutable.ArrayBuffer[OffsetCommitRequestData.OffsetCommitRequestTopic]()
       offsetCommitRequest.data.topics.forEach { topic =>
-        if (!authorizedTopics.contains(topic.name)) {
+        val topicName = resolveTopicName(topic)
+        if (topicName == null) {

Review Comment:
   The sequence of validation chosen here reflects what is used on the fetch request path:
   
   - If topic IDs are used and the given topic ID cannot be resolved (and no fallback name is provided), send back `UNKNOWN_TOPIC_ID`;
   - If the topic name is valid but that name is not authorized, send `TOPIC_AUTHORIZATION_FAILED`;
   - If the topic name is authorized but not present in the metadata cache (in which case, that topic will not have been resolved via its ID because in this case, we expect it to be in the cache), send `UNKNOWN_TOPIC_OR_PARTITION`.



##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -192,7 +196,15 @@ public Builder merge(
             return this;
         }
 
-        public OffsetCommitResponse build() {
+        public OffsetCommitResponse build(short version) {
+            if (version >= 9) {
+                data.topics().forEach(topic -> {
+                    // Set the topic name to null if a topic ID for the topic is present.
+                    if (!Uuid.ZERO_UUID.equals(topic.topicId())) {

Review Comment:
   Note: topic ID must not be null for a request/response version >= 9 to be serialized. `ZERO_UUID` means no topic ID specified.



##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java:
##########
@@ -78,11 +89,18 @@ public OffsetCommitRequestData data() {
         return data;
     }
 
-    public Map<TopicPartition, Long> offsets() {
+    public Map<TopicPartition, Long> offsets(TopicResolver topicResolver) {
         Map<TopicPartition, Long> offsets = new HashMap<>();
         for (OffsetCommitRequestTopic topic : data.topics()) {
+            String topicName = topic.name();
+
+            if (version() >= 9 && topicName == null) {

Review Comment:
   Note that if the `topicName` is not null, we should also check if it resolves to the same UUID as we have cached locally.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1204162396


##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.KafkaApisTest.{NameAndId, newOffsetCommitRequestData, newOffsetCommitResponseData}
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetCommitRequestTest.assertResponseEquals
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
+
+import java.util.Optional.empty
+import java.util.Properties
+import scala.collection.{Map, Seq}
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava}
+
+class OffsetCommitRequestTest extends BaseRequestTest {
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val groupId = "groupId"
+
+  var consumer: KafkaConsumer[_, _] = _
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    consumer = createConsumer()
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    if (consumer != null)
+      Utils.closeQuietly(consumer, "KafkaConsumer")
+    super.tearDown()
+  }
+
+  def createTopics(topicNames: String*): Seq[NameAndId] = {
+    topicNames.map(topic => {
+      createTopic(topic)
+      val topicId: Uuid = getTopicIds().get(topic) match {
+        case Some(x) => x
+        case _ => throw new AssertionError("Topic ID not found for " + topic)
+      }
+      NameAndId(topic, topicId)
+    })
+  }
+
+
+  @Test
+  def testTopicIdsArePopulatedInOffsetCommitResponses(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+    consumer.subscribe(topics.map(_.name).asJava)
+
+    sendOffsetCommitRequest(
+      topics.map((_, ListMap(0 -> offset))),
+      topics.map((_, Map(0 -> Errors.NONE))),
+      ApiKeys.OFFSET_COMMIT.allVersions().asScala

Review Comment:
   @clolov Yes, I think the idea of parameterizing the test by version of request is it is faster to identify version-specific failures.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1205089112


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2707,11 +2742,271 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_MEMBER_ID);
         assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testCommitOffsetUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+            prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_TOPIC_ID));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertFalse(coordinator.commitOffsetsSync(offsets, time.timer(timeoutMs)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testRetryCommitUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.UNKNOWN_TOPIC_ID)));
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.NONE)));
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertTrue(coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                // Unlike the commit offset sync API, the async API does not retry.
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    static Stream<Arguments> commitOffsetTestArgs() {
+        Map<TopicIdPartition, Long> byTopicIdOffsets = new HashMap<>();
+        byTopicIdOffsets.put(ti1p, 100L);
+        byTopicIdOffsets.put(ti2p, 200L);
+
+        TopicIdPartition unknownTopicIdPartition =
+            new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("topic3", 5));
+
+        Map<TopicIdPartition, Long> byTopicNameOffsets = new HashMap<>();
+        byTopicNameOffsets.put(ti1p, 100L);
+        byTopicNameOffsets.put(ti2p, 200L);
+        byTopicNameOffsets.put(unknownTopicIdPartition, 300L);
+
+        OffsetCommitRequestData byTopicIdData = new OffsetCommitRequestData()
+            .setGroupId(groupId)
+            .setGenerationId(OffsetCommitRequest.DEFAULT_GENERATION_ID)
+            .setTopics(Arrays.asList(
+                new OffsetCommitRequestTopic()
+                    .setTopicId(ti1p.topicId())
+                    .setName(topic1)
+                    .setPartitions(singletonList(new OffsetCommitRequestPartition()
+                        .setPartitionIndex(t1p.partition())
+                        .setCommittedOffset(100L)
+                        .setCommittedMetadata("metadata"))),
+                new OffsetCommitRequestTopic()
+                    .setTopicId(ti2p.topicId())
+                    .setName(topic2)
+                    .setPartitions(singletonList(new OffsetCommitRequestPartition()
+                        .setPartitionIndex(t2p.partition())
+                        .setCommittedOffset(200L)
+                        .setCommittedMetadata("metadata")))
+            ));
+
+        OffsetCommitRequestData byTopicNameData = byTopicIdData.duplicate();
+        byTopicNameData.topics().add(new OffsetCommitRequestTopic()
+            .setName(unknownTopicIdPartition.topic())
+            .setPartitions(singletonList(new OffsetCommitRequestPartition()
+                .setPartitionIndex(5)
+                .setCommittedOffset(300L)
+                .setCommittedMetadata("metadata")))
+        );
+
+        return Stream.of(
+            Arguments.of(true, byTopicIdOffsets, byTopicIdData, (short) 9),
+            Arguments.of(false, byTopicIdOffsets, byTopicIdData, (short) 9),
+            Arguments.of(true, byTopicNameOffsets, byTopicNameData, (short) 8),
+            Arguments.of(false, byTopicNameOffsets, byTopicNameData, (short) 8)
+        );
+    }
+
+    private static Map<TopicPartition, OffsetAndMetadata> offsetAndMetadata(Map<TopicIdPartition, Long> offsets) {
+        return offsets.entrySet().stream()
+            .map(e -> new SimpleEntry<>(e.getKey().topicPartition(), new OffsetAndMetadata(e.getValue(), "metadata")))
+            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+    }
+
+    @ParameterizedTest
+    @MethodSource("commitOffsetTestArgs")
+    public void testTopicIdsArePopulatedByTheConsumerCoordinator(
+            boolean commitSync,
+            Map<TopicIdPartition, Long> offsets,
+            OffsetCommitRequestData expectedRequestData,
+            short expectedRequestVersion) {
+
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        OffsetCommitRequestCaptor captor = new OffsetCommitRequestCaptor();
+        prepareOffsetCommitRequest(offsets, Errors.NONE, false, captor);
+
+        Map<TopicPartition, OffsetAndMetadata> input = offsetAndMetadata(offsets);
+
+        if (commitSync) {
+            assertTrue(coordinator.commitOffsetsSync(input, time.timer(Long.MAX_VALUE)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(input, (inputOffsets, exception) -> {
+                // Notes:
+                // 1) The offsets passed to the callback are the same object provided to the offset commit method.
+                //    The validation on the offsets is not required but defensive.
+                // 2) We validate that the commit was successful, which is the case if the exception is null.
+                // 3) We validate this callback was invoked, which is not necessary but defensive.
+                assertSame(inputOffsets, input);
+                assertNull(exception);
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+
+        // The consumer does not provide a guarantee on the order of occurrence of topics and partitions in the
+        // OffsetCommit request, since a map of offsets is provided to the consumer API. Here, both requests
+        // are asserted to be identical irrespective of the order in which topic and partitions appear in the requests.
+        assertRequestEquals(
+            new OffsetCommitRequest(expectedRequestData, expectedRequestVersion),
+            captor.request
+        );
+    }
+
+    @ParameterizedTest
+    @NullSource
+    @ValueSource(strings = { "", "test1" })
+    public void testInvalidTopicIdReturnedByBrokerWhenCommittingOffsetSync(String topicName) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(t1p, new OffsetAndMetadata(100L, "m"));
+
+        // The following offset commit response is valid and the authorization failure results in failing the
+        // offset commit invocation.
+        client.prepareResponse(offsetCommitResponse(topicName, ti1p.topicId(), Errors.GROUP_AUTHORIZATION_FAILED));
+        assertThrows(GroupAuthorizationException.class,
+            () -> coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));
+
+        // The following offset commit response defines a topic incorrectly. The coordinator ignores the topic,
+        // and the group authorization failure is therefore not propagated.
+        client.prepareResponse(offsetCommitResponse(topicName, Uuid.ZERO_UUID, Errors.GROUP_AUTHORIZATION_FAILED));
+        assertTrue(coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));

Review Comment:
   Just to clarify, do you mean the commit offsets method should return false when at least 1 over n > 1 could not be committed due to topic id mismatch, or when n == 1 could not be committed for the same reason?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1129266598


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1379,7 +1411,8 @@ public void handle(OffsetCommitResponse commitResponse, RequestFuture<Void> futu
                             future.raise(error);
                             return;
                         } else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS
-                                || error == Errors.UNKNOWN_TOPIC_OR_PARTITION) {
+                                || error == Errors.UNKNOWN_TOPIC_OR_PARTITION
+                                || error == Errors.UNKNOWN_TOPIC_ID) {

Review Comment:
   With the synchronous API in the consumer, the error is not surfaced (only `true`/`false`). I added the missing tests to exercise the asynchronous API for this use case, and it did expose the `UnknownTopicIdException` to the user. Since it violates the API contract which exclusively relies on topic names, I raised the error `UNKNOWN_TOPIC_OR_PARTITION` when an `UNKNOWN_TOPIC_ID` is returned in the offset commit response. Do you think this is sensible?
   
   I added the corresponding unit tests for the consumer coordinator.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1453619279

   Used v8 for AlterConsumerGroupOffsets in the admin client and added corresponding unit and integration tests. `MirrorConnectorsIntegrationBaseTest` is now successful.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1124559198


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -167,8 +213,24 @@ public <P> Builder addPartitions(
         }
 
         public Builder merge(
-            OffsetCommitResponseData newData
+            OffsetCommitResponseData newData,
+            Logger logger
         ) {
+            if (version >= 9) {

Review Comment:
   Not strictly needed. We can remove it and the logger as well.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1130866578


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -119,28 +131,52 @@ public boolean shouldClientThrottle(short version) {
         return version >= 4;
     }
 
+    public short version() {
+        return version;
+    }
+
     public static class Builder {
         OffsetCommitResponseData data = new OffsetCommitResponseData();
         HashMap<String, OffsetCommitResponseTopic> byTopicName = new HashMap<>();
+        private final TopicResolver topicResolver;
+        private final short version;
+
+        public Builder(TopicResolver topicResolver, short version) {

Review Comment:
   Hi David, thanks for the insight. I think you are right that implementing support of topic ids in the functional layer before exposing it in the API makes sense as it provides the guarantee that offsets and metadata belong to the partitions of the right topic in case of homonyms.
   
   Now, one question is how deep we go in the integration of ids in this layer. Would you consider changing the data model authored by the group coordinator down to the `OffsetCommitValue ` as prescribed by KIP 848?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1119248386


##########
clients/src/main/resources/common/message/OffsetCommitRequest.json:
##########
@@ -47,8 +49,10 @@
       "about": "The time period in ms to retain the offset." },
     { "name": "Topics", "type": "[]OffsetCommitRequestTopic", "versions": "0+",
       "about": "The topics to commit offsets for.",  "fields": [
-      { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName",
+      { "name": "Name", "type": "string", "versions": "0+", "nullableVersions": "9+", "entityType": "topicName",
         "about": "The topic name." },
+      { "name": "TopicId", "type": "uuid", "versions": "9+",
+        "about": "The unique topic ID." },

Review Comment:
   @Hangleton I discussed offline with a few committers and the consensus is that having both the topic name and the topic id in the same version is not the right way. They share the same concerns that we discussed last week. Could you update the PR to only have TopicId from version 9? We can also remove the nullableVersions for the Name and set the versions to 0-8. I suppose that both fields could be ignorable.
   
   Regarding the admin client, which does not support topic ids, it cannot use version 9 at the moment. We need to handle this in the Builder (we can set the maximum allowed version).
   
   Sorry for this late change.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1449935049

   > Hello David, thanks for the fast review. Apologies for being slow, I hadn't finished the previous revision. Will include your comments. Working on it right now. Thanks!
   
   No worries. You are not slow. I just 


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1129272424


##########
clients/src/main/java/org/apache/kafka/common/TopicResolver.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common;
+
+import org.apache.kafka.common.errors.InvalidTopicException;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Encapsulates the resolution of a topic name from its ID, or its ID from its name, using a local
+ * bidirectional mapping. This resolver assumes there is a bijection between topic IDs and topic names.
+ * <p></p>
+ * Note that this class intends to be used for the (reverse) lookup of topic IDs/names, but
+ * not to characterize the set of topics which are known by a client. Use the
+ * {@link org.apache.kafka.clients.MetadataCache} for that purpose.
+ */
+//@Immutable, @ThreadSafe
+public class TopicResolver {

Review Comment:
   I would tend to have a preference for a business type which conveys semantics versus a generic data structure, but that is not very important here especially since the entity exposing the bidirectional mapping is relatively short-lived when used in the code. One advantage of a generic DS is that it can be reused for other purposes. Very happy to expose it as a bimap. I could not find an existing implementation in the codebase or its dependencies, although there is a bidirectional multimap defined within restricted scope [here](https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java#L459-L489).



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1130942769


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -119,28 +131,52 @@ public boolean shouldClientThrottle(short version) {
         return version >= 4;
     }
 
+    public short version() {
+        return version;
+    }
+
     public static class Builder {
         OffsetCommitResponseData data = new OffsetCommitResponseData();
         HashMap<String, OffsetCommitResponseTopic> byTopicName = new HashMap<>();
+        private final TopicResolver topicResolver;
+        private final short version;
+
+        public Builder(TopicResolver topicResolver, short version) {

Review Comment:
   The OffsetCommitValue part is not possible at the moment because we don’t have a way to downgrade. My colleague @jeffkbkim works on a proposal for this. We could start by either migrating from using TopicPartition to using TopicIdPartition or handling this in the GroupCoordinatorAdaptor layer. The former is likely simpler.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1129710408


##########
clients/src/main/java/org/apache/kafka/common/TopicIdAndNameBiMapping.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common;
+
+import org.apache.kafka.common.errors.InvalidTopicException;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Encapsulates the mapping between topic names and ids assuming a 1:1 relationship between
+ * a name and an id.
+ * <p></p>
+ * Note that this class intends to be used for the (reverse) lookup of topic IDs/names, but
+ * not to characterize the set of topics which are known by a client. Use the
+ * {@link org.apache.kafka.clients.MetadataCache} for that purpose.
+ */
+public class TopicIdAndNameBiMapping {
+    private final Map<String, Uuid> topicIds;
+    private final Map<Uuid, String> topicNames;
+
+    /**
+     * A mapping which universe of topic ids and names is captured from the input map. The reverse association
+     * between a topic ID and a topic name is computed by this method. If there are more than one topic name
+     * resolving to the same topic ID, an {@link InvalidTopicException} is thrown.
+     */
+    public static TopicIdAndNameBiMapping fromTopicIds(Map<String, Uuid> topicIds) {

Review Comment:
   Maybe `TopicIdAndNameBiMap`?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1128110858


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java:
##########
@@ -199,7 +199,8 @@ public NetworkClientDelegate.UnsentRequest toUnsentRequest() {
                             .setGenerationId(generation.generationId)
                             .setMemberId(generation.memberId)
                             .setGroupInstanceId(groupInstanceId)
-                            .setTopics(new ArrayList<>(requestTopicDataMap.values())));
+                            .setTopics(new ArrayList<>(requestTopicDataMap.values())),
+                    false /* Support of topic ids will be added with KAFKA-14777 */);

Review Comment:
   nit: We usually don't leave such comment in our code base.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1352,8 +1370,22 @@ public void handle(OffsetCommitResponse commitResponse, RequestFuture<Void> futu
             Set<String> unauthorizedTopics = new HashSet<>();
 
             for (OffsetCommitResponseData.OffsetCommitResponseTopic topic : commitResponse.data().topics()) {
+                String topicName = topic.name();
+
+                if (commitResponse.version() >= 9) {
+                    topicName = topicResolver.getTopicName(topic.topicId()).orElse(null);
+
+                    if (topicName == null) {
+                        // OffsetCommit responses version 9 must use topic IDs. The topic's ID must have been
+                        // known by the client which sent the OffsetCommitRequest but was removed from the metadata
+                        // before the response was received.

Review Comment:
   Is this really true? As we keep the `TopicResolver` used to construct the request, all topics should be there. This case could happen if the server returns an unexpected topic id that was not in the request and that is not in the `TopicResolver`. Do I get this right?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1323,27 +1335,33 @@ RequestFuture<Void> sendOffsetCommitRequest(final Map<TopicPartition, OffsetAndM
             groupInstanceId = null;
         }
 
+        boolean canUseTopicIds = topicPartitionsWithoutTopicId == 0;
+
         OffsetCommitRequest.Builder builder = new OffsetCommitRequest.Builder(
                 new OffsetCommitRequestData()
                         .setGroupId(this.rebalanceConfig.groupId)
                         .setGenerationId(generation.generationId)
                         .setMemberId(generation.memberId)
                         .setGroupInstanceId(groupInstanceId)
-                        .setTopics(new ArrayList<>(requestTopicDataMap.values()))
+                        .setTopics(new ArrayList<>(requestTopicDataMap.values())),
+                canUseTopicIds
         );
 
         log.trace("Sending OffsetCommit request with {} to coordinator {}", offsets, coordinator);
 
         return client.send(coordinator, builder)
-                .compose(new OffsetCommitResponseHandler(offsets, generation));
+                .compose(new OffsetCommitResponseHandler(offsets, generation, topicResolver));
     }
 
     private class OffsetCommitResponseHandler extends CoordinatorResponseHandler<OffsetCommitResponse, Void> {
         private final Map<TopicPartition, OffsetAndMetadata> offsets;
+        private final TopicResolver topicResolver;
 
-        private OffsetCommitResponseHandler(Map<TopicPartition, OffsetAndMetadata> offsets, Generation generation) {
+        private OffsetCommitResponseHandler(
+                Map<TopicPartition, OffsetAndMetadata> offsets, Generation generation, TopicResolver topicResolver) {

Review Comment:
   nit: We usually don't break long lines like this. I personally prefer the following:
   
   ```
   private OffsetCommitResponseHandler(
        Map<TopicPartition, OffsetAndMetadata> offsets,
        Generation generation,
        TopicResolver topicResolver
   ) {
   ```
   
   You can find other ways in the code base.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1272,6 +1275,9 @@ RequestFuture<Void> sendOffsetCommitRequest(final Map<TopicPartition, OffsetAndM
         if (coordinator == null)
             return RequestFuture.coordinatorNotAvailable();
 
+        TopicResolver topicResolver = metadata.topicResolver();
+        int topicPartitionsWithoutTopicId = 0;

Review Comment:
   nit: Should we use a boolean?



##########
clients/src/main/java/org/apache/kafka/common/TopicResolver.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common;
+
+import org.apache.kafka.common.errors.InvalidTopicException;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Encapsulates the resolution of a topic name from its ID, or its ID from its name, using a local
+ * bidirectional mapping. This resolver assumes there is a bijection between topic IDs and topic names.
+ * <p></p>
+ * Note that this class intends to be used for the (reverse) lookup of topic IDs/names, but
+ * not to characterize the set of topics which are known by a client. Use the
+ * {@link org.apache.kafka.clients.MetadataCache} for that purpose.
+ */
+//@Immutable, @ThreadSafe
+public class TopicResolver {

Review Comment:
   I am not really happy with this name but I could not find a better one yet. My concern is that this class is really about resolving topic ids/names and not really topics per say. Have you considered any alternatives?



##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java:
##########
@@ -49,7 +46,14 @@ public static class Builder extends AbstractRequest.Builder<OffsetCommitRequest>
         private final OffsetCommitRequestData data;
 
         public Builder(OffsetCommitRequestData data) {

Review Comment:
   Is this constructor still used?



##########
clients/src/main/java/org/apache/kafka/common/TopicResolver.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common;
+
+import org.apache.kafka.common.errors.InvalidTopicException;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Encapsulates the resolution of a topic name from its ID, or its ID from its name, using a local
+ * bidirectional mapping. This resolver assumes there is a bijection between topic IDs and topic names.
+ * <p></p>
+ * Note that this class intends to be used for the (reverse) lookup of topic IDs/names, but
+ * not to characterize the set of topics which are known by a client. Use the
+ * {@link org.apache.kafka.clients.MetadataCache} for that purpose.
+ */
+//@Immutable, @ThreadSafe
+public class TopicResolver {
+    private final Map<String, Uuid> topicIds;
+    private final Map<Uuid, String> topicNames;
+
+    /**
+     * A resolver which universe of topic ids and names is captured from the input map. The reverse association
+     * between a topic ID and a topic name is computed by this method. If there are more than one topic name
+     * resolving to the same topic ID, an {@link InvalidTopicException} is thrown.
+     */
+    public static TopicResolver fromTopicIds(Map<String, Uuid> topicIds) {
+        Map<Uuid, String> topicNames = new HashMap<>(topicIds.size());
+
+        for (Map.Entry<String, Uuid> e: topicIds.entrySet()) {
+            String conflicting = topicNames.putIfAbsent(e.getValue(), e.getKey());
+            if (conflicting != null) {
+                throw new InvalidTopicException(

Review Comment:
   Should this be an InvalidStateException?



##########
clients/src/test/java/org/apache/kafka/test/MoreAssertions.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.test;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.message.OffsetCommitRequestData;
+import org.apache.kafka.common.message.OffsetCommitRequestData.OffsetCommitRequestPartition;
+import org.apache.kafka.common.message.OffsetCommitRequestData.OffsetCommitRequestTopic;
+import org.apache.kafka.common.requests.OffsetCommitRequest;
+
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static java.util.function.Function.identity;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public final class MoreAssertions {

Review Comment:
   This does not look good. It would be better to place those helpers in `OffsetCommitRequestTest` for instance or to keep them where they are used.



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -425,35 +426,59 @@ class KafkaApis(val requestChannel: RequestChannel,
       requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
       CompletableFuture.completedFuture[Unit](())
     } else {
+      val topicResolver = metadataCache.topicResolver()
+      val responseBuilder = new OffsetCommitResponse.Builder(topicResolver, offsetCommitRequest.version())
+
+      val resolvedTopics =
+        if (offsetCommitRequest.version() < 9)
+          offsetCommitRequest.data.topics().asScala

Review Comment:
   nit: You can omit the `()` after `topics` as we usually don't put them for getters in Scala. There are a few other cases in the PR.



##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -167,8 +213,24 @@ public <P> Builder addPartitions(
         }
 
         public Builder merge(
-            OffsetCommitResponseData newData
+            OffsetCommitResponseData newData,
+            Logger logger
         ) {
+            if (version >= 9) {
+                // This method is called after the group coordinator committed the offsets. The group coordinator
+                // provides the OffsetCommitResponseData it built in the process. As of now, this data does
+                // not contain topic ids, so we resolve them here.
+                newData.topics().forEach(topic -> {
+                    Uuid topicId = topicResolver.getTopicId(topic.name()).orElse(Uuid.ZERO_UUID);
+                    if (Uuid.ZERO_UUID.equals(topicId)) {

Review Comment:
   Should we just throw an illegale state exception if we end up having a topic without id? Ignoring it seems to be risky.



##########
core/src/main/scala/kafka/server/MetadataCache.scala:
##########
@@ -90,6 +90,10 @@ trait MetadataCache {
 
   def topicIdsToNames(): util.Map[Uuid, String]
 
+  def topicResolver(): TopicResolver = {
+    TopicResolver.wrap(topicNamesToIds(), topicIdsToNames())

Review Comment:
   I think that there is a race condition here. You have no guarantee that both maps are consistent with each others.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1281,10 +1287,16 @@ RequestFuture<Void> sendOffsetCommitRequest(final Map<TopicPartition, OffsetAndM
                 return RequestFuture.failure(new IllegalArgumentException("Invalid offset: " + offsetAndMetadata.offset()));
             }
 
+            Uuid topicId = topicResolver.getTopicId(topicPartition.topic()).orElse(ZERO_UUID);
+            if (topicId.equals(ZERO_UUID)) {

Review Comment:
   nit: I usually prefer to use `ZERO_UUID.equals(...` as it is safe for null values.



##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java:
##########
@@ -49,7 +46,14 @@ public static class Builder extends AbstractRequest.Builder<OffsetCommitRequest>
         private final OffsetCommitRequestData data;
 
         public Builder(OffsetCommitRequestData data) {
-            super(ApiKeys.OFFSET_COMMIT);
+            this(data, true);
+        }
+
+        public Builder(OffsetCommitRequestData data, boolean canUseTopicIds) {
+            // Version 8 is the maximum version that can be used without topic IDs.
+            super(ApiKeys.OFFSET_COMMIT,
+                ApiKeys.OFFSET_COMMIT.oldestVersion(),
+                canUseTopicIds ? ApiKeys.OFFSET_COMMIT.latestVersion() : (short) 8);

Review Comment:
   nit: When we break the line like this, we usually align the arguments on the first one. Otherwise, you can use the style that I mentioned earlier.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1379,7 +1411,8 @@ public void handle(OffsetCommitResponse commitResponse, RequestFuture<Void> futu
                             future.raise(error);
                             return;
                         } else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS
-                                || error == Errors.UNKNOWN_TOPIC_OR_PARTITION) {
+                                || error == Errors.UNKNOWN_TOPIC_OR_PARTITION
+                                || error == Errors.UNKNOWN_TOPIC_ID) {

Review Comment:
   For my understanding, are we going to propagate this error back to the end user?



##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -119,28 +131,52 @@ public boolean shouldClientThrottle(short version) {
         return version >= 4;
     }
 
+    public short version() {
+        return version;
+    }
+
     public static class Builder {
         OffsetCommitResponseData data = new OffsetCommitResponseData();
         HashMap<String, OffsetCommitResponseTopic> byTopicName = new HashMap<>();
+        private final TopicResolver topicResolver;
+        private final short version;
+
+        public Builder(TopicResolver topicResolver, short version) {

Review Comment:
   I am not sure about passing the `TopicResolver` here. My understanding is that we are doing this because topic ids are lost when we call the group coordinator. Wouldn't it better to update the group coordinator to preserve those topic ids? We may be able to handle this in the GroupCoordinatorAdaptor or we could switch to using TopicIdPartitions. We could also consider doing this in a separate PR as this one is already quite large.



##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -50,14 +52,21 @@
 public class OffsetCommitResponse extends AbstractResponse {
 
     private final OffsetCommitResponseData data;
+    private final short version;

Review Comment:
   Did you check how we did this for the FetchRequest?



##########
clients/src/main/java/org/apache/kafka/common/TopicResolver.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common;
+
+import org.apache.kafka.common.errors.InvalidTopicException;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Encapsulates the resolution of a topic name from its ID, or its ID from its name, using a local
+ * bidirectional mapping. This resolver assumes there is a bijection between topic IDs and topic names.
+ * <p></p>
+ * Note that this class intends to be used for the (reverse) lookup of topic IDs/names, but
+ * not to characterize the set of topics which are known by a client. Use the
+ * {@link org.apache.kafka.clients.MetadataCache} for that purpose.
+ */
+//@Immutable, @ThreadSafe

Review Comment:
   We don't really use those in our code base at the moment. We usually just mention those characteristics in the java doc.



##########
clients/src/main/java/org/apache/kafka/common/TopicResolver.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common;
+
+import org.apache.kafka.common.errors.InvalidTopicException;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Encapsulates the resolution of a topic name from its ID, or its ID from its name, using a local
+ * bidirectional mapping. This resolver assumes there is a bijection between topic IDs and topic names.
+ * <p></p>
+ * Note that this class intends to be used for the (reverse) lookup of topic IDs/names, but
+ * not to characterize the set of topics which are known by a client. Use the
+ * {@link org.apache.kafka.clients.MetadataCache} for that purpose.
+ */
+//@Immutable, @ThreadSafe
+public class TopicResolver {
+    private final Map<String, Uuid> topicIds;
+    private final Map<Uuid, String> topicNames;
+
+    /**
+     * A resolver which universe of topic ids and names is captured from the input map. The reverse association
+     * between a topic ID and a topic name is computed by this method. If there are more than one topic name
+     * resolving to the same topic ID, an {@link InvalidTopicException} is thrown.
+     */
+    public static TopicResolver fromTopicIds(Map<String, Uuid> topicIds) {
+        Map<Uuid, String> topicNames = new HashMap<>(topicIds.size());
+
+        for (Map.Entry<String, Uuid> e: topicIds.entrySet()) {
+            String conflicting = topicNames.putIfAbsent(e.getValue(), e.getKey());
+            if (conflicting != null) {
+                throw new InvalidTopicException(
+                        "Topic " + e.getKey() + " shares the same ID " + e.getValue() + " as topic " + conflicting);
+            }
+        }
+
+        return new TopicResolver(topicIds, topicNames);
+    }
+
+    /**
+     * A resolver which acts as a wrapper around the input mapping of topic ids from/to topic names.
+     * No validation is performed about the consistency of the mapping. This method is to be preferred
+     * when the copy of the input maps needs to be avoided.
+     */
+    public static TopicResolver wrap(Map<String, Uuid> topicIds, Map<Uuid, String> topicNames) {
+        return new TopicResolver(topicIds, topicNames);
+    }
+
+    /**
+     * A resolver with no existing mapping between any topic name and id.
+     */
+    public static TopicResolver emptyResolver() {
+        return fromTopicIds(Collections.emptyMap());
+    }
+
+    private TopicResolver(Map<String, Uuid> topicIds, Map<Uuid, String> topicNames) {
+        this.topicIds = Collections.unmodifiableMap(topicIds);
+        this.topicNames = Collections.unmodifiableMap(topicNames);
+    }
+
+    /**
+     * Returns the ID of the topic with the given name, if that association exists.
+     */
+    public Optional<Uuid> getTopicId(String name) {

Review Comment:
   I wonder if using Optional is necessary here given that we always use `orNull` and `orDefault`. What do you think?



##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -35,6 +39,7 @@
  * Possible error codes:
  *
  *   - {@link Errors#UNKNOWN_TOPIC_OR_PARTITION}
+ *   - {@link Errors#UNKNOWN_TOPIC_ID}

Review Comment:
   nit: Should we also add the other ones?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] clolov commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "clolov (via GitHub)" <gi...@apache.org>.
clolov commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1185861417


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2707,11 +2742,271 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_MEMBER_ID);
         assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testCommitOffsetUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+            prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_TOPIC_ID));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertFalse(coordinator.commitOffsetsSync(offsets, time.timer(timeoutMs)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testRetryCommitUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.UNKNOWN_TOPIC_ID)));
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.NONE)));
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertTrue(coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                // Unlike the commit offset sync API, the async API does not retry.

Review Comment:
   Yup, I will get to this today



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] clolov commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "clolov (via GitHub)" <gi...@apache.org>.
clolov commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1185002319


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -129,16 +150,19 @@
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
 
 public abstract class ConsumerCoordinatorTest {
-    private final String topic1 = "test1";
-    private final String topic2 = "test2";
-    private final TopicPartition t1p = new TopicPartition(topic1, 0);
-    private final TopicPartition t2p = new TopicPartition(topic2, 0);
-    private final String groupId = "test-group";
+    private static String topic1 = "test1";
+    private static String topic2 = "test2";
+    private static TopicPartition t1p = new TopicPartition(topic1, 0);
+    private static TopicIdPartition ti1p = new TopicIdPartition(Uuid.randomUuid(), t1p);

Review Comment:
   I believe t1p is abstracted because it is being used in 175 other places in this test class for test setup and assertions.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1226632779


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2707,11 +2734,287 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
-        assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
+        prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(ti1p.topicPartition(),
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testCommitOffsetUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+            prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_TOPIC_ID));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            ti1p.topicPartition(),
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertFalse(coordinator.commitOffsetsSync(offsets, time.timer(timeoutMs)));
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {

Review Comment:
   My understanding is that we don't retry when `commitOffsetsAsync` is used. Is it correct? If it is, it may be better to split the test in two. It is really misleading otherwise.



##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -46,6 +49,7 @@
  *   - {@link Errors#INVALID_COMMIT_OFFSET_SIZE}
  *   - {@link Errors#TOPIC_AUTHORIZATION_FAILED}
  *   - {@link Errors#GROUP_AUTHORIZATION_FAILED}
+ *   - {@link Errors#STALE_MEMBER_EPOCH}

Review Comment:
   Should we remove this one for now as it is not implemented yet?



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2707,11 +2734,287 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
-        assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
+        prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(ti1p.topicPartition(),
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testCommitOffsetUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+            prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_TOPIC_ID));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            ti1p.topicPartition(),
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertFalse(coordinator.commitOffsetsSync(offsets, time.timer(timeoutMs)));
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    private Map<TopicPartition, OffsetAndMetadata> testRetryCommitWithUnknownTopicIdSetup() {

Review Comment:
   nit: It may be better to name this one `prepare....`.



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2707,11 +2734,287 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
-        assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
+        prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(ti1p.topicPartition(),
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testCommitOffsetUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+            prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_TOPIC_ID));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            ti1p.topicPartition(),
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertFalse(coordinator.commitOffsetsSync(offsets, time.timer(timeoutMs)));
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    private Map<TopicPartition, OffsetAndMetadata> testRetryCommitWithUnknownTopicIdSetup() {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.UNKNOWN_TOPIC_ID)));
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.NONE)));
+
+        return singletonMap(
+            ti1p.topicPartition(),
+            new OffsetAndMetadata(100L, "metadata")
+        );
+    }
+
+    @Test
+    public void testRetryCommitAsyncUnknownTopicId() {
+        Map<TopicPartition, OffsetAndMetadata> offsets = testRetryCommitWithUnknownTopicIdSetup();
+
+        AtomicBoolean callbackInvoked = new AtomicBoolean();
+        coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+            // Unlike the commit offset sync API, the async API does not retry.
+            assertSame(inputOffsets, offsets);
+            assertEquals(RetriableCommitFailedException.class, exception.getClass());
+            assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+            callbackInvoked.set(true);
+        });
+
+        coordinator.invokeCompletedOffsetCommitCallbacks();
+        assertTrue(callbackInvoked.get());
+    }
+
+    @Test
+    public void testRetryCommitSyncUnknownTopicId() {
+        Map<TopicPartition, OffsetAndMetadata> offsets = testRetryCommitWithUnknownTopicIdSetup();
+
+        assertTrue(coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));
+    }
+
+    private static Map<TopicPartition, OffsetAndMetadata> offsetAndMetadata(Map<TopicIdPartition, Long> offsets) {
+        return offsets.entrySet().stream()
+            .map(e -> new SimpleEntry<>(e.getKey().topicPartition(), new OffsetAndMetadata(e.getValue(), "metadata")))
+            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+    }
+
+    @Test
+    public void testTopicIdsArePopulatedByTheConsumerCoordinatorInV9() {
+        Map<TopicIdPartition, Long> byTopicIdOffsets = new HashMap<>();
+        byTopicIdOffsets.put(ti1p, 100L);
+        byTopicIdOffsets.put(ti2p, 200L);
+
+        TopicIdPartition unknownTopicIdPartition =
+            new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("topic3", 5));
+
+        Map<TopicIdPartition, Long> byTopicNameOffsets = new HashMap<>();
+        byTopicNameOffsets.put(ti1p, 100L);
+        byTopicNameOffsets.put(ti2p, 200L);
+        byTopicNameOffsets.put(unknownTopicIdPartition, 300L);

Review Comment:
   Is this used anywhere?



##########
clients/src/test/java/org/apache/kafka/common/requests/OffsetCommitResponseTest.java:
##########
@@ -85,19 +96,186 @@ public void testParse() {
             ))
             .setThrottleTimeMs(throttleTimeMs);
 
-        for (short version : ApiKeys.OFFSET_COMMIT.allVersions()) {
-            ByteBuffer buffer = MessageUtil.toByteBuffer(data, version);
-            OffsetCommitResponse response = OffsetCommitResponse.parse(buffer, version);
-            assertEquals(expectedErrorCounts, response.errorCounts());
+        ByteBuffer buffer = MessageUtil.toByteBuffer(data, version);
+        OffsetCommitResponse response = OffsetCommitResponse.parse(buffer, version);
+        assertEquals(expectedErrorCounts, response.errorCounts());
 
-            if (version >= 3) {
-                assertEquals(throttleTimeMs, response.throttleTimeMs());
-            } else {
-                assertEquals(DEFAULT_THROTTLE_TIME, response.throttleTimeMs());
-            }
+        if (version >= 3) {
+            assertEquals(throttleTimeMs, response.throttleTimeMs());
+        } else {
+            assertEquals(DEFAULT_THROTTLE_TIME, response.throttleTimeMs());
+        }
+
+        assertEquals(version >= 4, response.shouldClientThrottle(version));
+    }
+
+    @ParameterizedTest
+    @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_COMMIT)
+    public void testOffsetCommitResponseBuilder(short version) {
+        NameAndId topic3 = new NameAndId("topic3");
+        NameAndId topic4 = new NameAndId("topic4");
+        NameAndId topic5 = new NameAndId("topic5");
+        NameAndId topic6 = new NameAndId("topic6");
+
+        Map<String, Uuid> topicIds = new HashMap<>();
+        topicIds.put(topicOne, topic1Id);
+        asList(topic3, topic4, topic5, topic6).forEach(nai -> topicIds.put(nai.name, nai.id));
+
+        OffsetCommitResponse.Builder builder = new OffsetCommitResponse.Builder()
+            .addPartition(topicOne, topic1Id, partitionOne, Errors.NONE)
+            .addPartition(topicOne, topic1Id, partitionTwo, Errors.NONE)
+            .addPartitions(topic6.name, topic6.id, asList(11, 12), identity(), Errors.NONE);
+
+        List<OffsetCommitResponseTopic> expectedTopics = new ArrayList<>();
+
+        if (version < 9) {
+            builder.addPartition(topicTwo, Uuid.ZERO_UUID, 3, Errors.NONE)
+                .addPartition(topicTwo, Uuid.ZERO_UUID, 4, Errors.NONE)
+                .addPartition(topic3.name, Uuid.ZERO_UUID, 5, Errors.NONE)
+                .addPartition(topic3.name, Uuid.ZERO_UUID, 6, Errors.NONE);
+
+            expectedTopics.addAll(asList(
+                createResponseTopic(topicOne, topic1Id, partitionOne, partitionTwo, Errors.NONE),
+                createResponseTopic(topic6.name, topic6.id, 11, 12, Errors.NONE),
+                createResponseTopic(topicTwo, Uuid.ZERO_UUID, 3, 4, Errors.NONE),
+                createResponseTopic(topic3.name, Uuid.ZERO_UUID, 5, 6, Errors.NONE)
+            ));
 
-            assertEquals(version >= 4, response.shouldClientThrottle(version));
+        } else {
+            builder.addPartition(topic4.name, topic4.id, 7, Errors.NONE)
+                .addPartition(topic4.name, topic4.id, 8, Errors.NONE)
+                .addPartition(topic5.name, topic5.id, 9, Errors.NONE)
+                .addPartition(topic5.name, topic5.id, 10, Errors.NONE)
+                .addPartition(topicTwo, Uuid.ZERO_UUID, 3, Errors.NONE);
+
+            expectedTopics.addAll(asList(
+                createResponseTopic(topicOne, topic1Id, partitionOne, partitionTwo, Errors.NONE),
+                createResponseTopic(topic6.name, topic6.id, 11, 12, Errors.NONE),
+                createResponseTopic(topic4.name, topic4.id, 7, 8, Errors.NONE),
+                createResponseTopic(topic5.name, topic5.id, 9, 10, Errors.NONE),
+                createResponseTopic(topicTwo, Uuid.ZERO_UUID, 3, Errors.NONE)
+            ));
         }
+
+        assertEquals(new OffsetCommitResponseData().setTopics(expectedTopics), builder.build().data());
+    }
+
+    @Test
+    public void testAddPartitionRequiresAValidTopicName() {
+        assertThrows(IllegalArgumentException.class,
+            () -> new OffsetCommitResponse.Builder()
+                .addPartition("", Uuid.randomUuid(), 0, Errors.NONE));
+
+        assertThrows(IllegalArgumentException.class,
+            () -> new OffsetCommitResponse.Builder()
+                .addPartition(null, Uuid.randomUuid(), 0, Errors.NONE));
+    }
+
+    @Test
+    public void testMergeOffsetCommitRequestData() {
+        NameAndId topic3 = new NameAndId("topic3");
+        NameAndId topic4 = new NameAndId("topic4");
+        NameAndId topic5 = new NameAndId("topic5");
+        NameAndId topic6 = new NameAndId("topic6");
+
+        Map<String, Uuid> topicIds = new HashMap<>();
+        topicIds.put(topicOne, topic1Id);
+        asList(topic3, topic4, topic5, topic6).forEach(nai -> topicIds.put(nai.name, nai.id));
+
+        OffsetCommitResponse.Builder builder = new OffsetCommitResponse.Builder()
+            .addPartition(topicOne, topic1Id, partitionOne, Errors.NONE)
+            .addPartition(topicOne, topic1Id, partitionTwo, Errors.NONE)
+            .addPartitions(topic6.name, topic6.id, asList(11, 12), identity(), Errors.NONE);
+
+        OffsetCommitResponseData coordinatorResults = new OffsetCommitResponseData()
+            .setTopics(Arrays.asList(
+                createResponseTopic(topicTwo, Uuid.ZERO_UUID, 3, 4, Errors.NONE),
+                createResponseTopic(topic3.name, topic3.id, 5, 6, Errors.NONE),
+                createResponseTopic(topic4.name, topic4.id, 7, 8, Errors.NONE),
+                createResponseTopic(topic5.name, topic5.id, 9, 10, Errors.NONE)
+            ));
+
+        List<OffsetCommitResponseTopic> expectedTopics = new ArrayList<>();
+        expectedTopics.addAll(asList(
+            createResponseTopic(topicOne, topic1Id, partitionOne, partitionTwo, Errors.NONE),
+            createResponseTopic(topic6.name, topic6.id, 11, 12, Errors.NONE),
+            createResponseTopic(topicTwo, Uuid.ZERO_UUID, 3, 4, Errors.NONE),
+            createResponseTopic(topic3.name, topic3.id, 5, 6, Errors.NONE),
+            createResponseTopic(topic4.name, topic4.id, 7, 8, Errors.NONE),
+            createResponseTopic(topic5.name, topic5.id, 9, 10, Errors.NONE)
+        ));
+
+        OffsetCommitResponse response = builder.merge(coordinatorResults).build();
+        assertEquals(new OffsetCommitResponseData().setTopics(expectedTopics), response.data());
+    }
+
+    private static OffsetCommitResponseTopic createResponseTopic(
+        String topicName,
+        Uuid topicId,
+        int partition,
+        Errors error
+    ) {
+        return new OffsetCommitResponseTopic()
+            .setTopicId(topicId)
+            .setName(topicName)
+            .setPartitions(new ArrayList<>(asList(
+                new OffsetCommitResponsePartition()
+                    .setPartitionIndex(partition)
+                    .setErrorCode(error.code())
+            )));
+    }
+
+    private static OffsetCommitResponseTopic createResponseTopic(
+        String topicName,
+        Uuid topicId,
+        int firstPartition,
+        int secondPartition,
+        Errors error
+    ) {
+        OffsetCommitResponseTopic topic = createResponseTopic(topicName, topicId, firstPartition, error);
+        topic.partitions().add(new OffsetCommitResponsePartition()
+            .setPartitionIndex(secondPartition)
+            .setErrorCode(error.code()));
+
+        return topic;
     }
 
+    public static final class NameAndId {

Review Comment:
   It is a bit weird to have this class defined here but I cannot think of a better place for now. Thoughts?



##########
clients/src/main/resources/common/message/OffsetCommitResponse.json:
##########
@@ -28,15 +28,20 @@
   // Version 7 offsetCommitRequest supports a new field called groupInstanceId to indicate member identity across restarts.
   //
   // Version 8 is the first flexible version.
-  "validVersions": "0-8",
+  //
+  // Version 9 adds TopicId field and can return STALE_MEMBER_EPOCH, UNKNOWN_MEMBER_ID
+  // and UNKNOWN_TOPIC_ID errors (KIP-848).

Review Comment:
   Should we remove STALE_MEMBER_EPOCH and UNKNOWN_MEMBER_ID for now?



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2707,11 +2734,287 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
-        assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
+        prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(ti1p.topicPartition(),
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testCommitOffsetUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+            prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_TOPIC_ID));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            ti1p.topicPartition(),
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertFalse(coordinator.commitOffsetsSync(offsets, time.timer(timeoutMs)));
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    private Map<TopicPartition, OffsetAndMetadata> testRetryCommitWithUnknownTopicIdSetup() {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.UNKNOWN_TOPIC_ID)));
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.NONE)));
+
+        return singletonMap(
+            ti1p.topicPartition(),
+            new OffsetAndMetadata(100L, "metadata")
+        );
+    }
+
+    @Test
+    public void testRetryCommitAsyncUnknownTopicId() {
+        Map<TopicPartition, OffsetAndMetadata> offsets = testRetryCommitWithUnknownTopicIdSetup();
+
+        AtomicBoolean callbackInvoked = new AtomicBoolean();
+        coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+            // Unlike the commit offset sync API, the async API does not retry.
+            assertSame(inputOffsets, offsets);
+            assertEquals(RetriableCommitFailedException.class, exception.getClass());
+            assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+            callbackInvoked.set(true);
+        });
+
+        coordinator.invokeCompletedOffsetCommitCallbacks();
+        assertTrue(callbackInvoked.get());
+    }
+
+    @Test
+    public void testRetryCommitSyncUnknownTopicId() {
+        Map<TopicPartition, OffsetAndMetadata> offsets = testRetryCommitWithUnknownTopicIdSetup();
+
+        assertTrue(coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));
+    }
+
+    private static Map<TopicPartition, OffsetAndMetadata> offsetAndMetadata(Map<TopicIdPartition, Long> offsets) {
+        return offsets.entrySet().stream()
+            .map(e -> new SimpleEntry<>(e.getKey().topicPartition(), new OffsetAndMetadata(e.getValue(), "metadata")))
+            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+    }
+
+    @Test
+    public void testTopicIdsArePopulatedByTheConsumerCoordinatorInV9() {
+        Map<TopicIdPartition, Long> byTopicIdOffsets = new HashMap<>();
+        byTopicIdOffsets.put(ti1p, 100L);
+        byTopicIdOffsets.put(ti2p, 200L);
+
+        TopicIdPartition unknownTopicIdPartition =
+            new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("topic3", 5));
+
+        Map<TopicIdPartition, Long> byTopicNameOffsets = new HashMap<>();
+        byTopicNameOffsets.put(ti1p, 100L);
+        byTopicNameOffsets.put(ti2p, 200L);
+        byTopicNameOffsets.put(unknownTopicIdPartition, 300L);

Review Comment:
   Is this used anywhere?



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2707,11 +2742,271 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_MEMBER_ID);
         assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testCommitOffsetUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+            prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_TOPIC_ID));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertFalse(coordinator.commitOffsetsSync(offsets, time.timer(timeoutMs)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testRetryCommitUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.UNKNOWN_TOPIC_ID)));
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.NONE)));
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertTrue(coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                // Unlike the commit offset sync API, the async API does not retry.
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    static Stream<Arguments> commitOffsetTestArgs() {
+        Map<TopicIdPartition, Long> byTopicIdOffsets = new HashMap<>();
+        byTopicIdOffsets.put(ti1p, 100L);
+        byTopicIdOffsets.put(ti2p, 200L);
+
+        TopicIdPartition unknownTopicIdPartition =
+            new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("topic3", 5));
+
+        Map<TopicIdPartition, Long> byTopicNameOffsets = new HashMap<>();
+        byTopicNameOffsets.put(ti1p, 100L);
+        byTopicNameOffsets.put(ti2p, 200L);
+        byTopicNameOffsets.put(unknownTopicIdPartition, 300L);
+
+        OffsetCommitRequestData byTopicIdData = new OffsetCommitRequestData()
+            .setGroupId(groupId)
+            .setGenerationId(OffsetCommitRequest.DEFAULT_GENERATION_ID)
+            .setTopics(Arrays.asList(
+                new OffsetCommitRequestTopic()
+                    .setTopicId(ti1p.topicId())
+                    .setName(topic1)
+                    .setPartitions(singletonList(new OffsetCommitRequestPartition()
+                        .setPartitionIndex(t1p.partition())
+                        .setCommittedOffset(100L)
+                        .setCommittedMetadata("metadata"))),
+                new OffsetCommitRequestTopic()
+                    .setTopicId(ti2p.topicId())
+                    .setName(topic2)
+                    .setPartitions(singletonList(new OffsetCommitRequestPartition()
+                        .setPartitionIndex(t2p.partition())
+                        .setCommittedOffset(200L)
+                        .setCommittedMetadata("metadata")))
+            ));
+
+        OffsetCommitRequestData byTopicNameData = byTopicIdData.duplicate();
+        byTopicNameData.topics().add(new OffsetCommitRequestTopic()
+            .setName(unknownTopicIdPartition.topic())
+            .setPartitions(singletonList(new OffsetCommitRequestPartition()
+                .setPartitionIndex(5)
+                .setCommittedOffset(300L)
+                .setCommittedMetadata("metadata")))
+        );
+
+        return Stream.of(
+            Arguments.of(true, byTopicIdOffsets, byTopicIdData, (short) 9),
+            Arguments.of(false, byTopicIdOffsets, byTopicIdData, (short) 9),
+            Arguments.of(true, byTopicNameOffsets, byTopicNameData, (short) 8),
+            Arguments.of(false, byTopicNameOffsets, byTopicNameData, (short) 8)
+        );
+    }
+
+    private static Map<TopicPartition, OffsetAndMetadata> offsetAndMetadata(Map<TopicIdPartition, Long> offsets) {
+        return offsets.entrySet().stream()
+            .map(e -> new SimpleEntry<>(e.getKey().topicPartition(), new OffsetAndMetadata(e.getValue(), "metadata")))
+            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+    }
+
+    @ParameterizedTest
+    @MethodSource("commitOffsetTestArgs")
+    public void testTopicIdsArePopulatedByTheConsumerCoordinator(
+            boolean commitSync,
+            Map<TopicIdPartition, Long> offsets,
+            OffsetCommitRequestData expectedRequestData,
+            short expectedRequestVersion) {
+
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        OffsetCommitRequestCaptor captor = new OffsetCommitRequestCaptor();
+        prepareOffsetCommitRequest(offsets, Errors.NONE, false, captor);
+
+        Map<TopicPartition, OffsetAndMetadata> input = offsetAndMetadata(offsets);
+
+        if (commitSync) {
+            assertTrue(coordinator.commitOffsetsSync(input, time.timer(Long.MAX_VALUE)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(input, (inputOffsets, exception) -> {
+                // Notes:
+                // 1) The offsets passed to the callback are the same object provided to the offset commit method.
+                //    The validation on the offsets is not required but defensive.
+                // 2) We validate that the commit was successful, which is the case if the exception is null.
+                // 3) We validate this callback was invoked, which is not necessary but defensive.
+                assertSame(inputOffsets, input);
+                assertNull(exception);
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+
+        // The consumer does not provide a guarantee on the order of occurrence of topics and partitions in the
+        // OffsetCommit request, since a map of offsets is provided to the consumer API. Here, both requests
+        // are asserted to be identical irrespective of the order in which topic and partitions appear in the requests.
+        assertRequestEquals(
+            new OffsetCommitRequest(expectedRequestData, expectedRequestVersion),
+            captor.request
+        );
+    }
+
+    @ParameterizedTest
+    @NullSource
+    @ValueSource(strings = { "", "test1" })
+    public void testInvalidTopicIdReturnedByBrokerWhenCommittingOffsetSync(String topicName) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(t1p, new OffsetAndMetadata(100L, "m"));
+
+        // The following offset commit response is valid and the authorization failure results in failing the
+        // offset commit invocation.
+        client.prepareResponse(offsetCommitResponse(topicName, ti1p.topicId(), Errors.GROUP_AUTHORIZATION_FAILED));
+        assertThrows(GroupAuthorizationException.class,
+            () -> coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));
+
+        // The following offset commit response defines a topic incorrectly. The coordinator ignores the topic,
+        // and the group authorization failure is therefore not propagated.
+        client.prepareResponse(offsetCommitResponse(topicName, Uuid.ZERO_UUID, Errors.GROUP_AUTHORIZATION_FAILED));
+        assertTrue(coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));

Review Comment:
   I think that the method should return false if any mismatched topic id. If I commit foo-topic-id and bar-topic-id, the method should not succeed if we don't get a response for any of them, right?



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2707,11 +2734,287 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
-        assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
+        prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(ti1p.topicPartition(),
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testCommitOffsetUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+            prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_TOPIC_ID));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            ti1p.topicPartition(),
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertFalse(coordinator.commitOffsetsSync(offsets, time.timer(timeoutMs)));
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    private Map<TopicPartition, OffsetAndMetadata> testRetryCommitWithUnknownTopicIdSetup() {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.UNKNOWN_TOPIC_ID)));
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.NONE)));
+
+        return singletonMap(
+            ti1p.topicPartition(),
+            new OffsetAndMetadata(100L, "metadata")
+        );

Review Comment:
   nit: I would inline this in the respective tests because it seems not related to what this method does.



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2707,11 +2734,287 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
-        assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
+        prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(ti1p.topicPartition(),
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testCommitOffsetUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+            prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_TOPIC_ID));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            ti1p.topicPartition(),
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertFalse(coordinator.commitOffsetsSync(offsets, time.timer(timeoutMs)));
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    private Map<TopicPartition, OffsetAndMetadata> testRetryCommitWithUnknownTopicIdSetup() {

Review Comment:
   nit: It may be better to name this one `prepare....`.



##########
clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java:
##########
@@ -2052,9 +2052,10 @@ private OffsetCommitRequest createOffsetCommitRequest(short version) {
                                                 .setCommittedOffset(200)
                                                 .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
                                                 .setCommittedMetadata(null)
-                                ))
-                ))
-        ).build(version);
+                                )))
+            ),
+                true

Review Comment:
   nit: this seems to be misaligned.



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2707,11 +2734,287 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
-        assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
+        prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(ti1p.topicPartition(),
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testCommitOffsetUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+            prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_TOPIC_ID));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            ti1p.topicPartition(),
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertFalse(coordinator.commitOffsetsSync(offsets, time.timer(timeoutMs)));
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    private Map<TopicPartition, OffsetAndMetadata> testRetryCommitWithUnknownTopicIdSetup() {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.UNKNOWN_TOPIC_ID)));
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.NONE)));
+
+        return singletonMap(
+            ti1p.topicPartition(),
+            new OffsetAndMetadata(100L, "metadata")
+        );
+    }
+
+    @Test
+    public void testRetryCommitAsyncUnknownTopicId() {
+        Map<TopicPartition, OffsetAndMetadata> offsets = testRetryCommitWithUnknownTopicIdSetup();
+
+        AtomicBoolean callbackInvoked = new AtomicBoolean();
+        coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+            // Unlike the commit offset sync API, the async API does not retry.
+            assertSame(inputOffsets, offsets);
+            assertEquals(RetriableCommitFailedException.class, exception.getClass());
+            assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+            callbackInvoked.set(true);
+        });
+
+        coordinator.invokeCompletedOffsetCommitCallbacks();
+        assertTrue(callbackInvoked.get());
+    }
+
+    @Test
+    public void testRetryCommitSyncUnknownTopicId() {
+        Map<TopicPartition, OffsetAndMetadata> offsets = testRetryCommitWithUnknownTopicIdSetup();
+
+        assertTrue(coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));
+    }
+
+    private static Map<TopicPartition, OffsetAndMetadata> offsetAndMetadata(Map<TopicIdPartition, Long> offsets) {
+        return offsets.entrySet().stream()
+            .map(e -> new SimpleEntry<>(e.getKey().topicPartition(), new OffsetAndMetadata(e.getValue(), "metadata")))
+            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+    }
+
+    @Test
+    public void testTopicIdsArePopulatedByTheConsumerCoordinatorInV9() {
+        Map<TopicIdPartition, Long> byTopicIdOffsets = new HashMap<>();
+        byTopicIdOffsets.put(ti1p, 100L);
+        byTopicIdOffsets.put(ti2p, 200L);
+
+        TopicIdPartition unknownTopicIdPartition =
+            new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("topic3", 5));
+
+        Map<TopicIdPartition, Long> byTopicNameOffsets = new HashMap<>();
+        byTopicNameOffsets.put(ti1p, 100L);
+        byTopicNameOffsets.put(ti2p, 200L);
+        byTopicNameOffsets.put(unknownTopicIdPartition, 300L);
+
+        OffsetCommitRequestData byTopicIdData = new OffsetCommitRequestData()
+            .setGroupId(groupId)
+            .setGenerationId(OffsetCommitRequest.DEFAULT_GENERATION_ID)
+            .setTopics(Arrays.asList(
+                new OffsetCommitRequestTopic()
+                    .setTopicId(ti1p.topicId())
+                    .setName(topic1)
+                    .setPartitions(singletonList(new OffsetCommitRequestPartition()
+                        .setPartitionIndex(ti1p.partition())
+                        .setCommittedOffset(100L)
+                        .setCommittedMetadata("metadata"))),
+                new OffsetCommitRequestTopic()
+                    .setTopicId(ti2p.topicId())
+                    .setName(topic2)
+                    .setPartitions(singletonList(new OffsetCommitRequestPartition()
+                        .setPartitionIndex(ti2p.partition())
+                        .setCommittedOffset(200L)
+                        .setCommittedMetadata("metadata")))
+        ));
+
+        OffsetCommitRequestData byTopicNameData = byTopicIdData.duplicate();
+        byTopicNameData.topics().add(new OffsetCommitRequestTopic()
+                .setName(unknownTopicIdPartition.topic())
+                .setPartitions(singletonList(new OffsetCommitRequestPartition()
+                        .setPartitionIndex(5)
+                        .setCommittedOffset(300L)
+                        .setCommittedMetadata("metadata")))
+        );
+
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        MockClient.RequestMatcher requestMatcher = request -> assertRequestEquals(
+            new OffsetCommitRequest(byTopicIdData, (short) 9),
+            (OffsetCommitRequest) request
+        );
+
+        prepareOffsetCommitRequest(byTopicIdOffsets, Errors.NONE, false, requestMatcher);
+
+        Map<TopicPartition, OffsetAndMetadata> input = offsetAndMetadata(byTopicIdOffsets);
+        assertTrue(coordinator.commitOffsetsSync(input, time.timer(Long.MAX_VALUE)));
+    }
+
+    @Test
+    public void testUseOffsetCommitRequestV8IfATopicIdIsMissing() {
+        TopicIdPartition unknownTopicIdPartition =
+                new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("topic3", 5));
+
+        Map<TopicIdPartition, Long> byTopicNameOffsets = new HashMap<>();
+        byTopicNameOffsets.put(ti1p, 100L);
+        byTopicNameOffsets.put(ti2p, 200L);
+        byTopicNameOffsets.put(unknownTopicIdPartition, 300L);
+
+        OffsetCommitRequestData byTopicNameData = new OffsetCommitRequestData()
+            .setGroupId(groupId)
+            .setGenerationId(OffsetCommitRequest.DEFAULT_GENERATION_ID)
+            .setTopics(Arrays.asList(
+                new OffsetCommitRequestTopic()
+                    .setTopicId(ti1p.topicId())
+                    .setName(topic1)
+                    .setPartitions(singletonList(new OffsetCommitRequestPartition()
+                        .setPartitionIndex(ti1p.partition())
+                        .setCommittedOffset(100L)
+                        .setCommittedMetadata("metadata"))),
+                new OffsetCommitRequestTopic()
+                    .setTopicId(ti2p.topicId())
+                    .setName(topic2)
+                    .setPartitions(singletonList(new OffsetCommitRequestPartition()
+                        .setPartitionIndex(ti2p.partition())
+                        .setCommittedOffset(200L)
+                        .setCommittedMetadata("metadata"))),
+                new OffsetCommitRequestTopic()
+                    .setName(unknownTopicIdPartition.topic())
+                    .setPartitions(singletonList(new OffsetCommitRequestPartition()
+                        .setPartitionIndex(5)
+                        .setCommittedOffset(300L)
+                        .setCommittedMetadata("metadata")))
+            ));
+
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        MockClient.RequestMatcher requestMatcher = request -> assertRequestEquals(
+            new OffsetCommitRequest(byTopicNameData, (short) 8),
+            (OffsetCommitRequest) request
+        );
+
+        prepareOffsetCommitRequest(byTopicNameOffsets, Errors.NONE, false, requestMatcher);
+
+        Map<TopicPartition, OffsetAndMetadata> input = offsetAndMetadata(byTopicNameOffsets);
+        assertTrue(coordinator.commitOffsetsSync(input, time.timer(Long.MAX_VALUE)));
+    }
+
+    @ParameterizedTest
+    @NullSource
+    @ValueSource(strings = { "", "test1" })
+    public void testInvalidTopicIdReturnedByBrokerWhenCommittingOffsetSync(String topicName) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(ti1p.topicPartition(), new OffsetAndMetadata(100L, "m"));
+
+        // The following offset commit response is valid and the authorization failure results in failing the
+        // offset commit invocation.
+        client.prepareResponse(offsetCommitResponse(topicName, ti1p.topicId(), Errors.GROUP_AUTHORIZATION_FAILED));
+        assertThrows(GroupAuthorizationException.class,
+            () -> coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));
+
+        // The following offset commit response defines a topic incorrectly. The coordinator ignores the topic,
+        // and the group authorization failure is therefore not propagated.
+        client.prepareResponse(offsetCommitResponse(topicName, Uuid.ZERO_UUID, Errors.GROUP_AUTHORIZATION_FAILED));
+        assertTrue(coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));
+
+        client.prepareResponse(offsetCommitResponse(topicName, Uuid.randomUuid(), Errors.GROUP_AUTHORIZATION_FAILED));
+        assertTrue(coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));
+    }
+
+    @ParameterizedTest
+    @NullSource
+    @ValueSource(strings = { "", "test1" })
+    public void testInvalidTopicIdReturnedByBrokerWhenCommittingOffsetAsync(String topicName) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>();
+        offsets.put(ti1p.topicPartition(), new OffsetAndMetadata(100L, "metadata1"));
+        offsets.put(ti2p.topicPartition(), new OffsetAndMetadata(200L, "metadata2"));
+
+        // Response data which makes the common part of the responses exercised in the use cases below.
+        OffsetCommitResponseData commonData = new OffsetCommitResponseData()
+            .setTopics(Arrays.asList(
+                new OffsetCommitResponseTopic()
+                    .setName(null)
+                    .setTopicId(ti2p.topicId())
+                    .setPartitions(singletonList(new OffsetCommitResponsePartition().setPartitionIndex(0)))
+            ));
+
+        BiConsumer<OffsetCommitResponse, Class<? extends Exception>> asserter = (response, exceptionType) -> {
+            OffsetCommitResponseTopic topic = response.data().topics().get(0);
+            OffsetCommitResponseData data = commonData.duplicate();
+            data.topics().add(topic);
+            client.prepareResponse(new OffsetCommitResponse(data));
+
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (__, exception) -> {
+                if (exceptionType == null)
+                    assertNull(exception);
+                else
+                    assertEquals(exceptionType, exception.getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        };
+
+        // The following offset commit responses are valid and the authorization failure results in failing the
+        // offset commit invocation.
+        asserter.accept(
+            offsetCommitResponse(topicName, ti1p.topicId(), Errors.GROUP_AUTHORIZATION_FAILED),
+            GroupAuthorizationException.class);
+
+        // The following offset commit responses defines a topic incorrectly. The coordinator ignores the topic,
+        // and the group authorization failure is therefore not propagated.
+        asserter.accept(
+            offsetCommitResponse(topicName, Uuid.ZERO_UUID, Errors.GROUP_AUTHORIZATION_FAILED),
+            null);

Review Comment:
   This case is not correct as well in my opinion. The caller should get an exception in this case.



##########
clients/src/test/java/org/apache/kafka/common/requests/OffsetCommitResponseTest.java:
##########
@@ -85,19 +96,186 @@ public void testParse() {
             ))
             .setThrottleTimeMs(throttleTimeMs);
 
-        for (short version : ApiKeys.OFFSET_COMMIT.allVersions()) {
-            ByteBuffer buffer = MessageUtil.toByteBuffer(data, version);
-            OffsetCommitResponse response = OffsetCommitResponse.parse(buffer, version);
-            assertEquals(expectedErrorCounts, response.errorCounts());
+        ByteBuffer buffer = MessageUtil.toByteBuffer(data, version);
+        OffsetCommitResponse response = OffsetCommitResponse.parse(buffer, version);
+        assertEquals(expectedErrorCounts, response.errorCounts());
 
-            if (version >= 3) {
-                assertEquals(throttleTimeMs, response.throttleTimeMs());
-            } else {
-                assertEquals(DEFAULT_THROTTLE_TIME, response.throttleTimeMs());
-            }
+        if (version >= 3) {
+            assertEquals(throttleTimeMs, response.throttleTimeMs());
+        } else {
+            assertEquals(DEFAULT_THROTTLE_TIME, response.throttleTimeMs());
+        }
+
+        assertEquals(version >= 4, response.shouldClientThrottle(version));
+    }
+
+    @ParameterizedTest
+    @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_COMMIT)
+    public void testOffsetCommitResponseBuilder(short version) {
+        NameAndId topic3 = new NameAndId("topic3");
+        NameAndId topic4 = new NameAndId("topic4");
+        NameAndId topic5 = new NameAndId("topic5");
+        NameAndId topic6 = new NameAndId("topic6");
+
+        Map<String, Uuid> topicIds = new HashMap<>();
+        topicIds.put(topicOne, topic1Id);
+        asList(topic3, topic4, topic5, topic6).forEach(nai -> topicIds.put(nai.name, nai.id));
+
+        OffsetCommitResponse.Builder builder = new OffsetCommitResponse.Builder()
+            .addPartition(topicOne, topic1Id, partitionOne, Errors.NONE)
+            .addPartition(topicOne, topic1Id, partitionTwo, Errors.NONE)
+            .addPartitions(topic6.name, topic6.id, asList(11, 12), identity(), Errors.NONE);
+
+        List<OffsetCommitResponseTopic> expectedTopics = new ArrayList<>();
+
+        if (version < 9) {
+            builder.addPartition(topicTwo, Uuid.ZERO_UUID, 3, Errors.NONE)
+                .addPartition(topicTwo, Uuid.ZERO_UUID, 4, Errors.NONE)
+                .addPartition(topic3.name, Uuid.ZERO_UUID, 5, Errors.NONE)
+                .addPartition(topic3.name, Uuid.ZERO_UUID, 6, Errors.NONE);
+
+            expectedTopics.addAll(asList(
+                createResponseTopic(topicOne, topic1Id, partitionOne, partitionTwo, Errors.NONE),
+                createResponseTopic(topic6.name, topic6.id, 11, 12, Errors.NONE),
+                createResponseTopic(topicTwo, Uuid.ZERO_UUID, 3, 4, Errors.NONE),
+                createResponseTopic(topic3.name, Uuid.ZERO_UUID, 5, 6, Errors.NONE)
+            ));
 
-            assertEquals(version >= 4, response.shouldClientThrottle(version));
+        } else {
+            builder.addPartition(topic4.name, topic4.id, 7, Errors.NONE)
+                .addPartition(topic4.name, topic4.id, 8, Errors.NONE)
+                .addPartition(topic5.name, topic5.id, 9, Errors.NONE)
+                .addPartition(topic5.name, topic5.id, 10, Errors.NONE)
+                .addPartition(topicTwo, Uuid.ZERO_UUID, 3, Errors.NONE);
+
+            expectedTopics.addAll(asList(
+                createResponseTopic(topicOne, topic1Id, partitionOne, partitionTwo, Errors.NONE),
+                createResponseTopic(topic6.name, topic6.id, 11, 12, Errors.NONE),
+                createResponseTopic(topic4.name, topic4.id, 7, 8, Errors.NONE),
+                createResponseTopic(topic5.name, topic5.id, 9, 10, Errors.NONE),
+                createResponseTopic(topicTwo, Uuid.ZERO_UUID, 3, Errors.NONE)
+            ));
         }
+
+        assertEquals(new OffsetCommitResponseData().setTopics(expectedTopics), builder.build().data());
+    }
+
+    @Test
+    public void testAddPartitionRequiresAValidTopicName() {
+        assertThrows(IllegalArgumentException.class,
+            () -> new OffsetCommitResponse.Builder()
+                .addPartition("", Uuid.randomUuid(), 0, Errors.NONE));
+
+        assertThrows(IllegalArgumentException.class,
+            () -> new OffsetCommitResponse.Builder()
+                .addPartition(null, Uuid.randomUuid(), 0, Errors.NONE));
+    }
+
+    @Test
+    public void testMergeOffsetCommitRequestData() {
+        NameAndId topic3 = new NameAndId("topic3");
+        NameAndId topic4 = new NameAndId("topic4");
+        NameAndId topic5 = new NameAndId("topic5");
+        NameAndId topic6 = new NameAndId("topic6");
+
+        Map<String, Uuid> topicIds = new HashMap<>();
+        topicIds.put(topicOne, topic1Id);
+        asList(topic3, topic4, topic5, topic6).forEach(nai -> topicIds.put(nai.name, nai.id));
+
+        OffsetCommitResponse.Builder builder = new OffsetCommitResponse.Builder()
+            .addPartition(topicOne, topic1Id, partitionOne, Errors.NONE)
+            .addPartition(topicOne, topic1Id, partitionTwo, Errors.NONE)
+            .addPartitions(topic6.name, topic6.id, asList(11, 12), identity(), Errors.NONE);
+
+        OffsetCommitResponseData coordinatorResults = new OffsetCommitResponseData()
+            .setTopics(Arrays.asList(
+                createResponseTopic(topicTwo, Uuid.ZERO_UUID, 3, 4, Errors.NONE),
+                createResponseTopic(topic3.name, topic3.id, 5, 6, Errors.NONE),
+                createResponseTopic(topic4.name, topic4.id, 7, 8, Errors.NONE),
+                createResponseTopic(topic5.name, topic5.id, 9, 10, Errors.NONE)
+            ));
+
+        List<OffsetCommitResponseTopic> expectedTopics = new ArrayList<>();
+        expectedTopics.addAll(asList(
+            createResponseTopic(topicOne, topic1Id, partitionOne, partitionTwo, Errors.NONE),
+            createResponseTopic(topic6.name, topic6.id, 11, 12, Errors.NONE),
+            createResponseTopic(topicTwo, Uuid.ZERO_UUID, 3, 4, Errors.NONE),
+            createResponseTopic(topic3.name, topic3.id, 5, 6, Errors.NONE),
+            createResponseTopic(topic4.name, topic4.id, 7, 8, Errors.NONE),
+            createResponseTopic(topic5.name, topic5.id, 9, 10, Errors.NONE)
+        ));
+
+        OffsetCommitResponse response = builder.merge(coordinatorResults).build();
+        assertEquals(new OffsetCommitResponseData().setTopics(expectedTopics), response.data());
+    }
+
+    private static OffsetCommitResponseTopic createResponseTopic(
+        String topicName,
+        Uuid topicId,
+        int partition,
+        Errors error
+    ) {
+        return new OffsetCommitResponseTopic()
+            .setTopicId(topicId)
+            .setName(topicName)
+            .setPartitions(new ArrayList<>(asList(
+                new OffsetCommitResponsePartition()
+                    .setPartitionIndex(partition)
+                    .setErrorCode(error.code())
+            )));
+    }
+
+    private static OffsetCommitResponseTopic createResponseTopic(
+        String topicName,
+        Uuid topicId,
+        int firstPartition,
+        int secondPartition,
+        Errors error
+    ) {
+        OffsetCommitResponseTopic topic = createResponseTopic(topicName, topicId, firstPartition, error);
+        topic.partitions().add(new OffsetCommitResponsePartition()
+            .setPartitionIndex(secondPartition)
+            .setErrorCode(error.code()));
+
+        return topic;
     }
 
+    public static final class NameAndId {

Review Comment:
   nit: TopicNameAndId?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1113132735


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java:
##########
@@ -59,7 +63,36 @@ public OffsetCommitRequest build(short version) {
                 throw new UnsupportedVersionException("The broker offset commit protocol version " +
                         version + " does not support usage of config group.instance.id.");
             }
-            return new OffsetCommitRequest(data, version);
+
+            // Copy since we can mutate it.
+            OffsetCommitRequestData requestData = data.duplicate();

Review Comment:
   nit: We probably don't need to duplicate `data` here. I understand why you are doing it but in practice we assume that `data` is owned by the builder once it is given to it.



##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java:
##########
@@ -78,11 +111,18 @@ public OffsetCommitRequestData data() {
         return data;
     }
 
-    public Map<TopicPartition, Long> offsets() {
+    public Map<TopicPartition, Long> offsets(TopicResolver topicResolver) {

Review Comment:
   I just realized that this is only used in tests. I wonder if we should just get rid of it and use the auto-generated classes in tests as well.



##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java:
##########
@@ -59,7 +63,36 @@ public OffsetCommitRequest build(short version) {
                 throw new UnsupportedVersionException("The broker offset commit protocol version " +
                         version + " does not support usage of config group.instance.id.");
             }
-            return new OffsetCommitRequest(data, version);
+
+            // Copy since we can mutate it.
+            OffsetCommitRequestData requestData = data.duplicate();
+
+            if (version >= 9) {
+                requestData.topics().forEach(topic -> {
+                    // Set the topic name to null if a topic ID for the topic is present. If no topic ID is
+                    // provided (i.e. its value is ZERO_UUID), the client should provide a topic name as a
+                    // fallback. This allows the OffsetCommit API to support both topic IDs and topic names
+                    // inside the same request or response.
+                    if (!Uuid.ZERO_UUID.equals(topic.topicId())) {
+                        topic.setName(null);
+                    } else if (topic.name() == null || "".equals(topic.name())) {
+                        // Fail-fast the entire request. This means that a single invalid topic in a multi-topic
+                        // request will make it fail. We may want to relax the constraint to allow the request
+                        // with valid topics (i.e. for which a valid ID or name was provided) exist in the request.
+                        throw new UnknownTopicOrPartitionException(

Review Comment:
   nit: InvalidRequestException would be more appropriate.



##########
clients/src/test/java/org/apache/kafka/common/requests/OffsetCommitRequestTest.java:
##########
@@ -139,4 +151,65 @@ public void testVersionSupportForGroupInstanceId() {
             }
         }
     }
+
+    @Test
+    public void testHandlingOfTopicIdInAllVersions() {
+        for (short version : ApiKeys.OFFSET_COMMIT.allVersions()) {
+            OffsetCommitRequest request = new OffsetCommitRequest.Builder(data).build(version);
+            List<OffsetCommitRequestTopic> requestTopics = request.data().topics();
+
+            if (version >= 9) {
+                // Version >= 9:
+                //   Topic ID may be present or not. Both are valid cases. If no topic ID is provided (null or
+                //   set to ZERO_UUID), a topic name must be provided and will be used. If a topic ID is provided,
+                //   the name will be nullified.
+                assertNull(requestTopics.get(0).name());
+                assertEquals(topicOneId, requestTopics.get(0).topicId());
+
+                assertEquals(topicTwo, requestTopics.get(1).name());
+                assertEquals(Uuid.ZERO_UUID, requestTopics.get(1).topicId());
+
+            } else {
+                // Version < 9:
+                //   Topic ID may be present or not. They are set to ZERO_UUID in the finalized request. Any other
+                //   value would make serialization of the request fail.
+                assertEquals(topicOne, requestTopics.get(0).name());
+                assertEquals(Uuid.ZERO_UUID, requestTopics.get(0).topicId());
+
+                assertEquals(topicTwo, requestTopics.get(1).name());
+                assertEquals(Uuid.ZERO_UUID, requestTopics.get(1).topicId());
+            }
+        }
+    }
+
+    @Test
+    public void testTopicIdMustBeSetIfNoTopicNameIsProvided() {
+        OffsetCommitRequestTopic topic = new OffsetCommitRequestTopic()
+            .setPartitions(Collections.singletonList(requestPartitionOne));
+        OffsetCommitRequestData data = new OffsetCommitRequestData()
+            .setGroupId(groupId)
+            .setTopics(Collections.singletonList(topic));
+
+        assertThrows(UnknownTopicOrPartitionException.class, () -> new OffsetCommitRequest.Builder(data).build((short) 9));
+    }
+
+    @Test
+    public void testResolvesTopicNameIfRequiredWhenListingOffsets() {
+        for (short version : ApiKeys.OFFSET_COMMIT.allVersions()) {

Review Comment:
   nit: ditto.



##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -151,11 +154,12 @@ public Builder addPartition(
 
         public <P> Builder addPartitions(
             String topicName,
+            Uuid topicId,
             List<P> partitions,
             Function<P, Integer> partitionIndex,
             Errors error
         ) {
-            final OffsetCommitResponseTopic topicResponse = getOrCreateTopic(topicName);
+            final OffsetCommitResponseTopic topicResponse = getOrCreateTopic(topicName, topicId);

Review Comment:
   I think that there is a bug here for the case where multiple topic ids are unknown in a single request. For those, the topic name will be null so they will be aggregated in the same OffsetCommitResponseTopic and that one will have the topic id of the first unknown topic id seen.



##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -192,8 +196,27 @@ public Builder merge(
             return this;
         }
 
-        public OffsetCommitResponse build() {
-            return new OffsetCommitResponse(data);
+        public OffsetCommitResponse build(short version) {
+            // Copy since we can mutate it.
+            OffsetCommitResponseData responseData = data.duplicate();
+
+            if (version >= 9) {
+                responseData.topics().forEach(topic -> {
+                    // Set the topic name to null if a topic ID for the topic is present.
+                    if (!Uuid.ZERO_UUID.equals(topic.topicId())) {
+                        topic.setName(null);
+                    }
+                });
+            } else {
+                responseData.topics().forEach(topic -> {
+                    // Topic must be set to default for version < 9.
+                    if (!Uuid.ZERO_UUID.equals(topic.topicId())) {
+                        topic.setTopicId(Uuid.ZERO_UUID);
+                    }
+                    // Topic name must not be null. Validity will be checked at serialization time.

Review Comment:
   nit: I think that we could remove this comment. It does not bring much.



##########
clients/src/main/resources/common/message/OffsetCommitResponse.json:
##########
@@ -28,15 +28,19 @@
   // Version 7 offsetCommitRequest supports a new field called groupInstanceId to indicate member identity across restarts.
   //
   // Version 8 is the first flexible version.
-  "validVersions": "0-8",
+  //
+  // Version 9 adds TopicId field (KIP-848).

Review Comment:
   The KIP also specifies new errors for this version. Could we mention them here?



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -1369,7 +1387,7 @@ public void testJoinPrepareWithDisableAutoCommit() {
         try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.of("group-id"), true)) {
             coordinator.ensureActiveGroup();
 
-            prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.NONE);
+            prepareOffsetCommitRequest(new OffsetCommitResponseSpec().expectedOffsets(singletonMap(t1p, 100L)));

Review Comment:
   What's the reason for this change? If we refactor this, it may be better to directly go with the auto-generated data structures. 



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -425,35 +425,73 @@ class KafkaApis(val requestChannel: RequestChannel,
       requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
       CompletableFuture.completedFuture[Unit](())
     } else {
+      val topicNames =
+        if (offsetCommitRequest.version() >= 9)
+          metadataCache.topicIdsToNames()
+        else
+          Collections.emptyMap[Uuid, String]()
+
+      // For version < 9, lookup from topicNames fails and the topic name (which cannot be null) is returned.
+      // For version >= 9, if lookup from topicNames fails, there are two possibilities:
+      //
+      // a) The topic ID was left to default and the topic name should have been populated as a fallback instead.
+      //    If none was provided, null is returned.
+      //
+      // b) The topic ID was not default but is not present in the local topic IDs cache. In this case, because
+      //    clients should make exclusive use of topic name or topic ID, the topic name should be null. If however
+      //    the client provided a topic name, we do not want to use it, because any topic with the same name
+      //    present locally would then have a topic ID which does not match the topic ID in the request.
+      def resolveTopicName(topic: OffsetCommitRequestData.OffsetCommitRequestTopic): Option[String] = {
+          val resolvedFromId = topicNames.get(topic.topicId())
+          if (resolvedFromId != null)
+            Some(resolvedFromId)
+          else if (offsetCommitRequest.version() < 9 || Uuid.ZERO_UUID.equals(topic.topicId)) {
+            Option(topic.name())
+          } else {
+            None
+          }
+      }
+
+      offsetCommitRequest.data.topics.forEach { topic => resolveTopicName(topic).foreach(topic.setName _) }

Review Comment:
   I would prefer to inline `resolveTopicName` and avoid allocating an `Option` which does not bring much here.
   
   In the mean time, I would directly construct the list of topic names for the authorizer at L461. This way, we could save re-iterating over the topics and the `filter`. What do you think?
   
   Moreover, the KIP states that an `INVALID_REQUEST` should be return if both a topic id and a topic name are provided. We could also handle this here.



##########
core/src/test/scala/unit/kafka/server/KafkaApisTest.scala:
##########
@@ -1351,8 +1351,11 @@ class KafkaApisTest {
 
   @Test
   def testHandleOffsetCommitRequestTopicsAndPartitionsValidation(): Unit = {

Review Comment:
   It would be great if we could extend the tests here. I think that we need to use multiple unresolvable topic ids in the same request and also check the different versions. I am not sure if we could extend this one or if we should add other ones.



##########
clients/src/main/resources/common/message/OffsetCommitRequest.json:
##########
@@ -47,8 +49,10 @@
       "about": "The time period in ms to retain the offset." },
     { "name": "Topics", "type": "[]OffsetCommitRequestTopic", "versions": "0+",
       "about": "The topics to commit offsets for.",  "fields": [
-      { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName",
+      { "name": "Name", "type": "string", "versions": "0+", "nullableVersions": "9+", "entityType": "topicName",
         "about": "The topic name." },
+      { "name": "TopicId", "type": "uuid", "versions": "9+",
+        "about": "The unique topic ID" },

Review Comment:
   nit: Could we add `.` at the end?



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -425,35 +425,72 @@ class KafkaApis(val requestChannel: RequestChannel,
       requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
       CompletableFuture.completedFuture[Unit](())
     } else {
+      val topicNames =
+        if (offsetCommitRequest.version() >= 9)
+          metadataCache.topicIdsToNames()
+        else
+          Collections.emptyMap[Uuid, String]()
+
+      // For version < 9, lookup from topicNames fails and the topic name (which cannot be null) is returned.
+      // For version >= 9, if lookup from topicNames fails, there are two possibilities:
+      //
+      // a) The topic ID was left to default and the topic name should have been populated as a fallback instead.
+      //    If none was provided, null is returned.
+      //
+      // b) The topic ID was not default but is not present in the local topic IDs cache. In this case, because
+      //    clients should make exclusive use of topic name or topic ID, the topic name should be null. If however
+      //    the client provided a topic name, we do not want to use it, because any topic with the same name
+      //    present locally would then have a topic ID which does not match the topic ID in the request.
+      def resolveTopicName(topic: OffsetCommitRequestData.OffsetCommitRequestTopic): String = {
+          val resolvedFromId = topicNames.get(topic.topicId())
+          if (resolvedFromId != null)
+            resolvedFromId
+          else if (offsetCommitRequest.version() < 9 || Uuid.ZERO_UUID.equals(topic.topicId)) {
+            topic.name()
+          } else {
+            null
+          }
+      }
+
       val authorizedTopics = authHelper.filterByAuthorized(
         request.context,
         READ,
         TOPIC,
-        offsetCommitRequest.data.topics.asScala
-      )(_.name)
+        offsetCommitRequest.data.topics.asScala.filter(topic => resolveTopicName(topic) != null)
+      )(resolveTopicName)
 
       val responseBuilder = new OffsetCommitResponse.Builder()
       val authorizedTopicsRequest = new mutable.ArrayBuffer[OffsetCommitRequestData.OffsetCommitRequestTopic]()
       offsetCommitRequest.data.topics.forEach { topic =>
-        if (!authorizedTopics.contains(topic.name)) {
+        val topicName = resolveTopicName(topic)
+        if (topicName == null) {
+          // Topic name cannot be null for version < 9. From version >= 9, topicName is null iff it cannot
+          // be resolved from the local topic IDs cache or topic ID was left to default but no fallback topic
+          // name was provided.
+          responseBuilder.addPartitions[OffsetCommitRequestData.OffsetCommitRequestPartition](

Review Comment:
   This issue is still present. Yeah, we definitely need to update the response builder to support this. One way would be to change the semantic of `addPartitions` to directly add to the response when it is called and to only put the topic in the HashMap when `addPartition` is used.



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2604,11 +2650,95 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        prepareOffsetCommitRequest(new OffsetCommitResponseSpec()
+                .expectedOffsets(singletonMap(t1p, 100L))
+                .error(Errors.UNKNOWN_MEMBER_ID));
+
         assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @Test
+    public void testCommitOffsetUnknownTopicId() {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+                prepareOffsetCommitRequest(new OffsetCommitResponseSpec()
+                        .expectedOffsets(singletonMap(t1p, 100L))
+                        .error(Errors.UNKNOWN_TOPIC_ID)));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+        assertFalse(coordinator.commitOffsetsSync(singletonMap(t1p,
+                new OffsetAndMetadata(100L, "metadata")), time.timer(timeoutMs)));
+    }
+
+    @Test
+    public void testRetryCommitUnknownTopicId() {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        client.prepareResponse(offsetCommitResponse(singletonMap(t1p, Errors.UNKNOWN_TOPIC_ID)));
+        client.prepareResponse(offsetCommitResponse(singletonMap(t1p, Errors.NONE)));
+
+        assertTrue(coordinator.commitOffsetsSync(singletonMap(t1p,
+                new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
+    }
+
+    @Test
+    public void testTopicIdsArePopulatedByTheConsumerCoordinator() {

Review Comment:
   We also need tests to check if the response is handled correctly. 



##########
clients/src/test/java/org/apache/kafka/common/requests/OffsetCommitRequestTest.java:
##########
@@ -139,4 +151,65 @@ public void testVersionSupportForGroupInstanceId() {
             }
         }
     }
+
+    @Test
+    public void testHandlingOfTopicIdInAllVersions() {
+        for (short version : ApiKeys.OFFSET_COMMIT.allVersions()) {

Review Comment:
   nit: You could replace this by the following:
   ```
       @ParameterizedTest
       @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_COMMIT)
   ```



##########
clients/src/test/java/org/apache/kafka/common/requests/OffsetCommitRequestTest.java:
##########
@@ -139,4 +151,65 @@ public void testVersionSupportForGroupInstanceId() {
             }
         }
     }
+
+    @Test
+    public void testHandlingOfTopicIdInAllVersions() {
+        for (short version : ApiKeys.OFFSET_COMMIT.allVersions()) {
+            OffsetCommitRequest request = new OffsetCommitRequest.Builder(data).build(version);
+            List<OffsetCommitRequestTopic> requestTopics = request.data().topics();
+
+            if (version >= 9) {
+                // Version >= 9:
+                //   Topic ID may be present or not. Both are valid cases. If no topic ID is provided (null or
+                //   set to ZERO_UUID), a topic name must be provided and will be used. If a topic ID is provided,
+                //   the name will be nullified.
+                assertNull(requestTopics.get(0).name());
+                assertEquals(topicOneId, requestTopics.get(0).topicId());
+
+                assertEquals(topicTwo, requestTopics.get(1).name());
+                assertEquals(Uuid.ZERO_UUID, requestTopics.get(1).topicId());
+

Review Comment:
   nit: We could remove this empty line.



##########
clients/src/main/resources/common/message/OffsetCommitResponse.json:
##########
@@ -28,15 +28,19 @@
   // Version 7 offsetCommitRequest supports a new field called groupInstanceId to indicate member identity across restarts.
   //
   // Version 8 is the first flexible version.
-  "validVersions": "0-8",
+  //
+  // Version 9 adds TopicId field (KIP-848).
+  "validVersions": "0-9",
   "flexibleVersions": "8+",
   "fields": [
     { "name": "ThrottleTimeMs", "type": "int32", "versions": "3+", "ignorable": true,
       "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." },
     { "name": "Topics", "type": "[]OffsetCommitResponseTopic", "versions": "0+",
       "about": "The responses for each topic.", "fields": [
-      { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName",
+      { "name": "Name", "type": "string", "versions": "0+", "nullableVersions": "9+", "entityType": "topicName",
         "about": "The topic name." },
+      { "name": "TopicId", "type": "uuid", "versions": "9+",
+        "about": "The unique topic ID" },

Review Comment:
   nit: Could we add `.` at the end?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1374,7 +1379,8 @@ public void handle(OffsetCommitResponse commitResponse, RequestFuture<Void> futu
                         } else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) {
                             unauthorizedTopics.add(tp.topic());
                         } else if (error == Errors.OFFSET_METADATA_TOO_LARGE
-                                || error == Errors.INVALID_COMMIT_OFFSET_SIZE) {
+                                || error == Errors.INVALID_COMMIT_OFFSET_SIZE
+                                || error == Errors.UNKNOWN_TOPIC_ID) {

Review Comment:
   At L1361 in this file, we construct `TopicPartition` based on the response data but we don't resolve the topic id. I think that we should add the resolution there as well, no? We probably need to extend tests to better cover this as well.
   
   Regarding `UNKNOWN_TOPIC_ID`, would it make sense to place it after `UNKNOWN_TOPIC_OR_PARTITION` as they are quite similar?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1116205901


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2604,11 +2650,95 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        prepareOffsetCommitRequest(new OffsetCommitResponseSpec()
+                .expectedOffsets(singletonMap(t1p, 100L))
+                .error(Errors.UNKNOWN_MEMBER_ID));
+
         assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @Test
+    public void testCommitOffsetUnknownTopicId() {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+                prepareOffsetCommitRequest(new OffsetCommitResponseSpec()
+                        .expectedOffsets(singletonMap(t1p, 100L))
+                        .error(Errors.UNKNOWN_TOPIC_ID)));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+        assertFalse(coordinator.commitOffsetsSync(singletonMap(t1p,
+                new OffsetAndMetadata(100L, "metadata")), time.timer(timeoutMs)));
+    }
+
+    @Test
+    public void testRetryCommitUnknownTopicId() {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        client.prepareResponse(offsetCommitResponse(singletonMap(t1p, Errors.UNKNOWN_TOPIC_ID)));
+        client.prepareResponse(offsetCommitResponse(singletonMap(t1p, Errors.NONE)));
+
+        assertTrue(coordinator.commitOffsetsSync(singletonMap(t1p,
+                new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
+    }
+
+    @Test
+    public void testTopicIdsArePopulatedByTheConsumerCoordinator() {

Review Comment:
   That is right, I added tests which invoke the sync and async offset commit method.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac closed pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac closed pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9
URL: https://github.com/apache/kafka/pull/13240


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1204376110


##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.KafkaApisTest.{NameAndId, newOffsetCommitRequestData, newOffsetCommitResponseData}
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetCommitRequestTest.assertResponseEquals
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
+
+import java.util.Optional.empty
+import java.util.Properties
+import scala.collection.{Map, Seq}
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava}
+
+class OffsetCommitRequestTest extends BaseRequestTest {
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val groupId = "groupId"
+
+  var consumer: KafkaConsumer[_, _] = _
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    consumer = createConsumer()
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    if (consumer != null)
+      Utils.closeQuietly(consumer, "KafkaConsumer")
+    super.tearDown()
+  }
+
+  def createTopics(topicNames: String*): Seq[NameAndId] = {
+    topicNames.map(topic => {
+      createTopic(topic)
+      val topicId: Uuid = getTopicIds().get(topic) match {
+        case Some(x) => x
+        case _ => throw new AssertionError("Topic ID not found for " + topic)
+      }
+      NameAndId(topic, topicId)
+    })
+  }
+
+
+  @Test
+  def testTopicIdsArePopulatedInOffsetCommitResponses(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+    consumer.subscribe(topics.map(_.name).asJava)
+
+    sendOffsetCommitRequest(
+      topics.map((_, ListMap(0 -> offset))),
+      topics.map((_, Map(0 -> Errors.NONE))),
+      ApiKeys.OFFSET_COMMIT.allVersions().asScala
+    )
+  }
+
+  @Test
+  def testCommitOffsetFromConsumer(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+    consumer.commitSync(offsetsToCommit(topics, offset))
+  }
+
+  @Test
+  def testOffsetCommitWithUnknownTopicId(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+
+    sendOffsetCommitRequest(
+      topics.map((_, ListMap(0 -> offset))) ++ Seq((NameAndId("unresolvable"), ListMap(0 -> offset))),
+      Seq((NameAndId("unresolvable"), ListMap(0 -> Errors.UNKNOWN_TOPIC_ID))) ++ topics.map((_, Map(0 -> Errors.NONE))),
+      ApiKeys.OFFSET_COMMIT.allVersions().asScala.filter(_ >= 9)
+    )
+  }
+
+  @Test
+  def alterConsumerGroupOffsetsDoNotUseTopicIds(): Unit = {

Review Comment:
   Agreed, I thought to put it there because the underlying RPC is used, but you are right, it is a different client-level API.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1204312762


##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.KafkaApisTest.{NameAndId, newOffsetCommitRequestData, newOffsetCommitResponseData}
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetCommitRequestTest.assertResponseEquals
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
+
+import java.util.Optional.empty
+import java.util.Properties
+import scala.collection.{Map, Seq}
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava}
+
+class OffsetCommitRequestTest extends BaseRequestTest {
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val groupId = "groupId"
+
+  var consumer: KafkaConsumer[_, _] = _
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    consumer = createConsumer()
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    if (consumer != null)
+      Utils.closeQuietly(consumer, "KafkaConsumer")
+    super.tearDown()
+  }
+
+  def createTopics(topicNames: String*): Seq[NameAndId] = {
+    topicNames.map(topic => {
+      createTopic(topic)
+      val topicId: Uuid = getTopicIds().get(topic) match {
+        case Some(x) => x
+        case _ => throw new AssertionError("Topic ID not found for " + topic)
+      }
+      NameAndId(topic, topicId)
+    })
+  }
+
+
+  @Test
+  def testTopicIdsArePopulatedInOffsetCommitResponses(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+    consumer.subscribe(topics.map(_.name).asJava)
+
+    sendOffsetCommitRequest(
+      topics.map((_, ListMap(0 -> offset))),
+      topics.map((_, Map(0 -> Errors.NONE))),

Review Comment:
   Added a commit to build the DTOs directly. This removes the contingency on correctness of the test code which built these DTOs.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1505154651

   > @Hangleton There are a few conflicts. Could you please rebase the PR? I plan to make another pass on it afterwards.
   
   Sure, done. Thanks!


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1131047710


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -119,28 +131,52 @@ public boolean shouldClientThrottle(short version) {
         return version >= 4;
     }
 
+    public short version() {
+        return version;
+    }
+
     public static class Builder {
         OffsetCommitResponseData data = new OffsetCommitResponseData();
         HashMap<String, OffsetCommitResponseTopic> byTopicName = new HashMap<>();
+        private final TopicResolver topicResolver;
+        private final short version;
+
+        public Builder(TopicResolver topicResolver, short version) {

Review Comment:
   Thanks for the answer. If I understand correctly, we would then have a resolution of topic ids from topic-name-based persisted data, so this may not prevent offsets from a topic to be provided as those of another topic with the same name (defined at different point in time in the server)?
   
   The resolution can be done in the group coordinator layer, assuming it has access to the topic id resolved upstream by the request handler. Because we want to preserve the same mapping used when request processing started, we need to ensure the right ids are used within the adaptor's `GroupCoordinator#commitOffsets` method(). Since the mapping returned from the metadata cache depends on the snapshot used at the time the mapping is requested, if the adaptor retrieves it from the metadata cache internally, at a different time from the request handler, there is no guarantee the metadata is the same hence that the topic IDs registered with the broker are the same.
   
   This means that the topic ids need to be propagated from the request handler (`KafkaApis`) to the coordinator adaptor somehow. Without a change in the method and contract implemented by the coordinator, these ids could be transferred via the `OffsetCommitRequestData` DTO directly, which means a change in the API schema would be required prior to the change. Alternatively, we may want to change the interface of the coordinator and change the signature of the offset commit method to allow for the propagation of topic ids.
   
   I may be missing the entire thing though?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1131047710


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -119,28 +131,52 @@ public boolean shouldClientThrottle(short version) {
         return version >= 4;
     }
 
+    public short version() {
+        return version;
+    }
+
     public static class Builder {
         OffsetCommitResponseData data = new OffsetCommitResponseData();
         HashMap<String, OffsetCommitResponseTopic> byTopicName = new HashMap<>();
+        private final TopicResolver topicResolver;
+        private final short version;
+
+        public Builder(TopicResolver topicResolver, short version) {

Review Comment:
   Thanks for the answer. If I understand correctly, we would then have a resolution of topic ids from topic-name-based persisted data, so this may not prevent offsets from a topic to be provided as those of another topic with the same name (defined at different point in time in the server)?
   
   The resolution can be done in the group coordinator layer, assuming it has access to the topic id resolved upstream by the request handler. Because we want to preserve the same mapping used when request processing started, we need to ensure the right ids are used within the adaptor's `GroupCoordinator#commitOffsets` method(). Since the mapping returned from the metadata cache depends on the snapshot used at the time the mapping is requested, if the adaptor retrieves it from the metadata cache internally, there is no guarantee the metadata is the same hence that the topic IDs registered with the broker are the same.
   
   This means that the topic ids need to be propagated from the request handler (`KafkaApis`) to the coordinator adaptor somehow. Without a change in the method and contract implemented by the coordinator, these ids could be transferred via the `OffsetCommitRequestData` DTO directly, which means a change in the API schema would be required prior to the change. Alternatively, we may want to change the interface of the coordinator and change the signature of the offset commit method to allow for the propagation of topic ids.
   
   I may be missing the entire thing though?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1129267206


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1352,8 +1370,22 @@ public void handle(OffsetCommitResponse commitResponse, RequestFuture<Void> futu
             Set<String> unauthorizedTopics = new HashSet<>();
 
             for (OffsetCommitResponseData.OffsetCommitResponseTopic topic : commitResponse.data().topics()) {
+                String topicName = topic.name();
+
+                if (commitResponse.version() >= 9) {
+                    topicName = topicResolver.getTopicName(topic.topicId()).orElse(null);
+
+                    if (topicName == null) {
+                        // OffsetCommit responses version 9 must use topic IDs. The topic's ID must have been
+                        // known by the client which sent the OffsetCommitRequest but was removed from the metadata
+                        // before the response was received.

Review Comment:
   Yes, that is right. Apologies, this is a fundamental misunderstanding/overlook.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1106229439


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -425,35 +425,72 @@ class KafkaApis(val requestChannel: RequestChannel,
       requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
       CompletableFuture.completedFuture[Unit](())
     } else {
+      val topicNames =
+        if (offsetCommitRequest.version() >= 9)
+          metadataCache.topicIdsToNames()
+        else
+          Collections.emptyMap[Uuid, String]()
+
+      // For version < 9, lookup from topicNames fails and the topic name (which cannot be null) is returned.
+      // For version >= 9, if lookup from topicNames fails, there are two possibilities:
+      //
+      // a) The topic ID was left to default and the topic name should have been populated as a fallback instead.
+      //    If none was provided, null is returned.
+      //
+      // b) The topic ID was not default but is not present in the local topic IDs cache. In this case, because
+      //    clients should make exclusive use of topic name or topic ID, the topic name should be null. If however
+      //    the client provided a topic name, we do not want to use it, because any topic with the same name
+      //    present locally would then have a topic ID which does not match the topic ID in the request.
+      def resolveTopicName(topic: OffsetCommitRequestData.OffsetCommitRequestTopic): String = {
+          val resolvedFromId = topicNames.get(topic.topicId())
+          if (resolvedFromId != null)
+            resolvedFromId
+          else if (offsetCommitRequest.version() < 9 || Uuid.ZERO_UUID.equals(topic.topicId)) {
+            topic.name()
+          } else {
+            null
+          }
+      }
+
       val authorizedTopics = authHelper.filterByAuthorized(
         request.context,
         READ,
         TOPIC,
-        offsetCommitRequest.data.topics.asScala
-      )(_.name)
+        offsetCommitRequest.data.topics.asScala.filter(topic => resolveTopicName(topic) != null)

Review Comment:
   I replaced this approach with a single iteration over the list of topic data, resolving and populating the topic name in place (line #455). I am concerned though because this involved mutating the request's body. But I am also concerned about the cost of creating a new ArrayBuffer, Sequence or another data structure to pre-filter. Without falling into premature optimization, what do you think about in-place mutation? 
   
   I think the problem here is that we have the instantiation of the `OffsetCommitRequest` decoupled from the resolution of topic IDs. It makes sense since the former corresponds to the request deserialization while the latter corresponds to added semantics unconveyed by the request itself. In responsibility chains on server request handlers, one pattern sometimes adopted is to decorate a request with extraneous information which fall beyond the scope of ser/de. I wonder if topic id resolution could happen before passing it to the business request handler.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1106229439


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -425,35 +425,72 @@ class KafkaApis(val requestChannel: RequestChannel,
       requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
       CompletableFuture.completedFuture[Unit](())
     } else {
+      val topicNames =
+        if (offsetCommitRequest.version() >= 9)
+          metadataCache.topicIdsToNames()
+        else
+          Collections.emptyMap[Uuid, String]()
+
+      // For version < 9, lookup from topicNames fails and the topic name (which cannot be null) is returned.
+      // For version >= 9, if lookup from topicNames fails, there are two possibilities:
+      //
+      // a) The topic ID was left to default and the topic name should have been populated as a fallback instead.
+      //    If none was provided, null is returned.
+      //
+      // b) The topic ID was not default but is not present in the local topic IDs cache. In this case, because
+      //    clients should make exclusive use of topic name or topic ID, the topic name should be null. If however
+      //    the client provided a topic name, we do not want to use it, because any topic with the same name
+      //    present locally would then have a topic ID which does not match the topic ID in the request.
+      def resolveTopicName(topic: OffsetCommitRequestData.OffsetCommitRequestTopic): String = {
+          val resolvedFromId = topicNames.get(topic.topicId())
+          if (resolvedFromId != null)
+            resolvedFromId
+          else if (offsetCommitRequest.version() < 9 || Uuid.ZERO_UUID.equals(topic.topicId)) {
+            topic.name()
+          } else {
+            null
+          }
+      }
+
       val authorizedTopics = authHelper.filterByAuthorized(
         request.context,
         READ,
         TOPIC,
-        offsetCommitRequest.data.topics.asScala
-      )(_.name)
+        offsetCommitRequest.data.topics.asScala.filter(topic => resolveTopicName(topic) != null)

Review Comment:
   I replaced this approach with a single iteration over the list of topic data, resolving and populating the topic name in place (line 455). I am concerned though because this involved mutating the request's body. But I am also concerned about the cost of creating a new ArrayBuffer, Sequence or another data structure to pre-filter. Without falling into premature optimization, what do you think about in-place mutation? 
   
   I think the problem here is that we have the instantiation of the `OffsetCommitRequest` decoupled from the resolution of topic IDs. It makes sense since the former corresponds to the request deserialization while the latter corresponds to added semantics unconveyed by the request itself. In responsibility chains on server request handlers, one pattern sometimes adopted is to decorate a request with extraneous information which fall beyond the scope of ser/de. I wonder if topic id resolution could happen before passing it to the business request handler.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1116200049


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1374,7 +1379,8 @@ public void handle(OffsetCommitResponse commitResponse, RequestFuture<Void> futu
                         } else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) {
                             unauthorizedTopics.add(tp.topic());
                         } else if (error == Errors.OFFSET_METADATA_TOO_LARGE
-                                || error == Errors.INVALID_COMMIT_OFFSET_SIZE) {
+                                || error == Errors.INVALID_COMMIT_OFFSET_SIZE
+                                || error == Errors.UNKNOWN_TOPIC_ID) {

Review Comment:
   That is right, thanks for pointing out. The resolution of topic name has been added to the response handler. If the topic is not defined, or the response topic is invalid because it contains neither an id or name, or contains both, that topic is ignored. The offset commit invocation is however not failed.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1121724366


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -50,14 +52,21 @@
 public class OffsetCommitResponse extends AbstractResponse {
 
     private final OffsetCommitResponseData data;
+    private final short version;

Review Comment:
   Adding the version to the response seems to be an anti-pattern as I haven't seen any other similar use in other responses. Semantically it should be OK because the response instance is supposed to be built against a given version. If another approach is advisable, I will remove it.



##########
clients/src/test/java/org/apache/kafka/common/message/MessageTest.java:
##########
@@ -435,17 +435,32 @@ public void testOffsetCommitRequestVersions() throws Exception {
         int partition = 2;
         int offset = 100;
 
-        testAllMessageRoundTrips(new OffsetCommitRequestData()
-                                     .setGroupId(groupId)
-                                     .setTopics(Collections.singletonList(
-                                         new OffsetCommitRequestTopic()
-                                             .setName(topicName)
-                                             .setPartitions(Collections.singletonList(
-                                                 new OffsetCommitRequestPartition()
-                                                     .setPartitionIndex(partition)
-                                                     .setCommittedMetadata(metadata)
-                                                     .setCommittedOffset(offset)
-                                             )))));
+        OffsetCommitRequestData byTopicName = new OffsetCommitRequestData()
+            .setGroupId(groupId)
+            .setTopics(Collections.singletonList(
+                new OffsetCommitRequestTopic()
+                    .setName(topicName)
+                    .setPartitions(Collections.singletonList(
+                        new OffsetCommitRequestPartition()
+                            .setPartitionIndex(partition)
+                            .setCommittedMetadata(metadata)
+                            .setCommittedOffset(offset)
+                    ))));
+
+        OffsetCommitRequestData byTopicId = new OffsetCommitRequestData()
+            .setGroupId(groupId)
+            .setTopics(Collections.singletonList(
+                new OffsetCommitRequestTopic()
+                    .setTopicId(Uuid.randomUuid())
+                    .setPartitions(Collections.singletonList(
+                        new OffsetCommitRequestPartition()
+                            .setPartitionIndex(partition)
+                            .setCommittedMetadata(metadata)
+                            .setCommittedOffset(offset)
+                    ))));
+
+        testAllMessageRoundTripsBeforeVersion((short) 9, byTopicName, byTopicName);

Review Comment:
   Note: is this OK to break message round trip between < 9 and >= 9?



##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -188,12 +199,62 @@ public Builder merge(
                     }
                 });
             }
-
             return this;
         }
 
-        public OffsetCommitResponse build() {
-            return new OffsetCommitResponse(data);
+        public final OffsetCommitResponse build() {
+            return new OffsetCommitResponse(data, version);
+        }
+
+        protected abstract void validate(String topicName, Uuid topicId);
+
+        protected abstract T classifer(String topicName, Uuid topicId);
+
+        private OffsetCommitResponseTopic getOrCreateTopic(String topicName, Uuid topicId) {
+            T topicKey = classifer(topicName, topicId);
+            OffsetCommitResponseTopic topic = topics.get(topicKey);
+            if (topic == null) {
+                topic = new OffsetCommitResponseTopic().setName(topicName).setTopicId(topicId);
+                data.topics().add(topic);
+                topics.put(topicKey, topic);
+            }
+            return topic;
+        }
+    }
+
+    public static final class BuilderByTopicId extends Builder<Uuid> {

Review Comment:
   Will add Javadoc.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1122271424


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -119,43 +128,53 @@ public boolean shouldClientThrottle(short version) {
         return version >= 4;
     }
 
-    public static class Builder {
-        OffsetCommitResponseData data = new OffsetCommitResponseData();
-        HashMap<String, OffsetCommitResponseTopic> byTopicName = new HashMap<>();
+    public short version() {
+        return version;
+    }
 
-        private OffsetCommitResponseTopic getOrCreateTopic(
-            String topicName
-        ) {
-            OffsetCommitResponseTopic topic = byTopicName.get(topicName);
-            if (topic == null) {
-                topic = new OffsetCommitResponseTopic().setName(topicName);
-                data.topics().add(topic);
-                byTopicName.put(topicName, topic);
-            }
-            return topic;
+    public static Builder<?> newBuilder(TopicResolver topicResolver, short version) {
+        if (version >= 9) {
+            return new Builder<>(topicResolver, new ByTopicId(), version);

Review Comment:
   Note - this duplicated invocation of the `Builder` constructor is to allow the resolution of the parameter type as either `Uuid` or `String`. Not graceful but...



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1105855843


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -425,35 +425,72 @@ class KafkaApis(val requestChannel: RequestChannel,
       requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
       CompletableFuture.completedFuture[Unit](())
     } else {
+      val topicNames =
+        if (offsetCommitRequest.version() >= 9)
+          metadataCache.topicIdsToNames()
+        else
+          Collections.emptyMap[Uuid, String]()
+
+      // For version < 9, lookup from topicNames fails and the topic name (which cannot be null) is returned.
+      // For version >= 9, if lookup from topicNames fails, there are two possibilities:
+      //
+      // a) The topic ID was left to default and the topic name should have been populated as a fallback instead.
+      //    If none was provided, null is returned.
+      //
+      // b) The topic ID was not default but is not present in the local topic IDs cache. In this case, because
+      //    clients should make exclusive use of topic name or topic ID, the topic name should be null. If however
+      //    the client provided a topic name, we do not want to use it, because any topic with the same name
+      //    present locally would then have a topic ID which does not match the topic ID in the request.
+      def resolveTopicName(topic: OffsetCommitRequestData.OffsetCommitRequestTopic): String = {
+          val resolvedFromId = topicNames.get(topic.topicId())
+          if (resolvedFromId != null)
+            resolvedFromId
+          else if (offsetCommitRequest.version() < 9 || Uuid.ZERO_UUID.equals(topic.topicId)) {
+            topic.name()
+          } else {
+            null
+          }
+      }
+
       val authorizedTopics = authHelper.filterByAuthorized(
         request.context,
         READ,
         TOPIC,
-        offsetCommitRequest.data.topics.asScala
-      )(_.name)
+        offsetCommitRequest.data.topics.asScala.filter(topic => resolveTopicName(topic) != null)
+      )(resolveTopicName)
 
       val responseBuilder = new OffsetCommitResponse.Builder()
       val authorizedTopicsRequest = new mutable.ArrayBuffer[OffsetCommitRequestData.OffsetCommitRequestTopic]()
       offsetCommitRequest.data.topics.forEach { topic =>
-        if (!authorizedTopics.contains(topic.name)) {
+        val topicName = resolveTopicName(topic)
+        if (topicName == null) {
+          // Topic name cannot be null for version < 9. From version >= 9, topicName is null iff it cannot
+          // be resolved from the local topic IDs cache or topic ID was left to default but no fallback topic
+          // name was provided.
+          responseBuilder.addPartitions[OffsetCommitRequestData.OffsetCommitRequestPartition](
+            topic.name, topic.topicId, topic.partitions, _.partitionIndex, Errors.UNKNOWN_TOPIC_ID)
+        } else if (!authorizedTopics.contains(topicName)) {
           // If the topic is not authorized, we add the topic and all its partitions
           // to the response with TOPIC_AUTHORIZATION_FAILED.
           responseBuilder.addPartitions[OffsetCommitRequestData.OffsetCommitRequestPartition](
-            topic.name, topic.partitions, _.partitionIndex, Errors.TOPIC_AUTHORIZATION_FAILED)
-        } else if (!metadataCache.contains(topic.name)) {
+            topic.name, topic.topicId, topic.partitions, _.partitionIndex, Errors.TOPIC_AUTHORIZATION_FAILED)
+        } else if (!metadataCache.contains(topicName)) {

Review Comment:
   I suppose that this check is not necessary if we are using topic ids. We already know that the name resolved based on the topic id is valid.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1122830477


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -119,43 +128,53 @@ public boolean shouldClientThrottle(short version) {
         return version >= 4;
     }
 
-    public static class Builder {
-        OffsetCommitResponseData data = new OffsetCommitResponseData();
-        HashMap<String, OffsetCommitResponseTopic> byTopicName = new HashMap<>();
+    public short version() {
+        return version;
+    }
 
-        private OffsetCommitResponseTopic getOrCreateTopic(
-            String topicName
-        ) {
-            OffsetCommitResponseTopic topic = byTopicName.get(topicName);
-            if (topic == null) {
-                topic = new OffsetCommitResponseTopic().setName(topicName);
-                data.topics().add(topic);
-                byTopicName.put(topicName, topic);
-            }
-            return topic;
+    public static Builder<?> newBuilder(TopicResolver topicResolver, short version) {
+        if (version >= 9) {
+            return new Builder<>(topicResolver, new ByTopicId(), version);
+        } else {
+            return new Builder<>(topicResolver, new ByTopicName(), version);
         }
+    }
 
-        public Builder addPartition(
-            String topicName,
-            int partitionIndex,
-            Errors error
-        ) {
-            final OffsetCommitResponseTopic topicResponse = getOrCreateTopic(topicName);
+    public static final class Builder<T> {
+        private final TopicResolver topicResolver;
+        private final TopicClassifier<T> topicClassifier;

Review Comment:
   Thinking about it, it seems unnecessary to adopt a different classification for v >= 9 since topic names should always be resolved when calling `addPartition`. Will remove all this logic and simplify.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1453452130

   The integration test `MirrorConnectorsIntegrationBaseTest` is failing due to unknown topic id detected from the admin client. I am looking into it.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1168410473


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -427,35 +428,59 @@ class KafkaApis(val requestChannel: RequestChannel,
       requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
       CompletableFuture.completedFuture[Unit](())
     } else {
+      val topicIdAndNames = metadataCache.topicIdAndNames()

Review Comment:
   It looks like `topicIdAndNames` is only used if version >= 9. Should we move it that else branch? Moreover, it seems that we don't need the BiMap anymore here. Should we just get the mapping that we need and revert the BiMap think in the `MetadataCache`?



##########
core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala:
##########
@@ -489,24 +489,23 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
   }
 
   private def createOffsetCommitRequest = {
-    new requests.OffsetCommitRequest.Builder(
-        new OffsetCommitRequestData()
-          .setGroupId(group)
-          .setMemberId(JoinGroupRequest.UNKNOWN_MEMBER_ID)
-          .setGenerationId(1)
-          .setTopics(Collections.singletonList(
-            new OffsetCommitRequestData.OffsetCommitRequestTopic()
-              .setName(topic)
-              .setPartitions(Collections.singletonList(
-                new OffsetCommitRequestData.OffsetCommitRequestPartition()
-                  .setPartitionIndex(part)
-                  .setCommittedOffset(0)
-                  .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
-                  .setCommitTimestamp(OffsetCommitRequest.DEFAULT_TIMESTAMP)
-                  .setCommittedMetadata("metadata")
-              )))
-          )
-    ).build()
+    val data = new OffsetCommitRequestData()
+      .setGroupId(group)
+      .setMemberId(JoinGroupRequest.UNKNOWN_MEMBER_ID)
+      .setGenerationId(1)
+      .setTopics(Collections.singletonList(
+        new OffsetCommitRequestData.OffsetCommitRequestTopic()
+          .setName(topic)
+          .setPartitions(Collections.singletonList(
+            new OffsetCommitRequestData.OffsetCommitRequestPartition()
+              .setPartitionIndex(part)
+              .setCommittedOffset(0)
+              .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
+              .setCommitTimestamp(OffsetCommitRequest.DEFAULT_TIMESTAMP)
+              .setCommittedMetadata("metadata")
+          )))
+      )
+    new requests.OffsetCommitRequest.Builder(data, true).build()

Review Comment:
   Just to be sure. The addition of `true` is the only real change here, right? 



##########
core/src/test/scala/unit/kafka/server/KafkaApisTest.scala:
##########
@@ -1264,22 +1270,25 @@ class KafkaApisTest {
     )
   }
 
-  @Test
-  def testHandleOffsetCommitRequest(): Unit = {
-    addTopicToMetadataCache("foo", numPartitions = 1)
+  @ParameterizedTest
+  @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_COMMIT)
+  def testHandleOffsetCommitRequest(version: Short): Unit = {
+    val fooId = Uuid.randomUuid()
+    addTopicToMetadataCache("foo", numPartitions = 1, topicId = fooId)
 
     val offsetCommitRequest = new OffsetCommitRequestData()
       .setGroupId("group")
       .setMemberId("member")
       .setTopics(List(
         new OffsetCommitRequestData.OffsetCommitRequestTopic()
           .setName("foo")
+          .setTopicId(if (version >= 9) fooId else Uuid.ZERO_UUID)

Review Comment:
   I think that `TopicId` is optional so we could just set it here.



##########
core/src/test/scala/unit/kafka/server/KafkaApisTest.scala:
##########
@@ -1353,73 +1375,29 @@ class KafkaApisTest {
 
   @Test
   def testHandleOffsetCommitRequestTopicsAndPartitionsValidation(): Unit = {

Review Comment:
   Should this test be parameterized as well? With this change, it seems that we don't have any tests exercising the validation with topic names now.



##########
core/src/test/scala/unit/kafka/server/KafkaApisTest.scala:
##########
@@ -3885,32 +3882,33 @@ class KafkaApisTest {
 
   @Test
   def rejectOffsetCommitRequestWhenStaticMembershipNotSupported(): Unit = {
-    val offsetCommitRequest = new OffsetCommitRequest.Builder(
-      new OffsetCommitRequestData()
-        .setGroupId("test")
-        .setMemberId("test")
-        .setGroupInstanceId("instanceId")
-        .setGenerationId(100)
-        .setTopics(Collections.singletonList(
-          new OffsetCommitRequestData.OffsetCommitRequestTopic()
-            .setName("test")
-            .setPartitions(Collections.singletonList(
-              new OffsetCommitRequestData.OffsetCommitRequestPartition()
-                .setPartitionIndex(0)
-                .setCommittedOffset(100)
-                .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
-                .setCommittedMetadata("")
-            ))
-        ))
-    ).build()
+    val topicId = Uuid.randomUuid()
+    val data = new OffsetCommitRequestData()

Review Comment:
   Changing the code structure like this is really annoying during reviews. It explodes the diff for no reasons and distracts the reviewing from the more important changes. It would be better to keep those for separate PRs. In this case, we could just add the `true` and the `TopicId` to the previous code.



##########
core/src/test/scala/unit/kafka/server/KafkaApisTest.scala:
##########
@@ -1353,73 +1375,29 @@ class KafkaApisTest {
 
   @Test
   def testHandleOffsetCommitRequestTopicsAndPartitionsValidation(): Unit = {
-    addTopicToMetadataCache("foo", numPartitions = 2)
-    addTopicToMetadataCache("bar", numPartitions = 2)
-
-    val offsetCommitRequest = new OffsetCommitRequestData()
-      .setGroupId("group")
-      .setMemberId("member")
-      .setTopics(List(
-        // foo exists but only has 2 partitions.
-        new OffsetCommitRequestData.OffsetCommitRequestTopic()
-          .setName("foo")
-          .setPartitions(List(
-            new OffsetCommitRequestData.OffsetCommitRequestPartition()
-              .setPartitionIndex(0)
-              .setCommittedOffset(10),
-            new OffsetCommitRequestData.OffsetCommitRequestPartition()
-              .setPartitionIndex(1)
-              .setCommittedOffset(20),
-            new OffsetCommitRequestData.OffsetCommitRequestPartition()
-              .setPartitionIndex(2)
-              .setCommittedOffset(30)).asJava),
-        // bar exists.
-        new OffsetCommitRequestData.OffsetCommitRequestTopic()
-          .setName("bar")
-          .setPartitions(List(
-            new OffsetCommitRequestData.OffsetCommitRequestPartition()
-              .setPartitionIndex(0)
-              .setCommittedOffset(40),
-            new OffsetCommitRequestData.OffsetCommitRequestPartition()
-              .setPartitionIndex(1)
-              .setCommittedOffset(50)).asJava),
-        // zar does not exist.
-        new OffsetCommitRequestData.OffsetCommitRequestTopic()
-          .setName("zar")
-          .setPartitions(List(
-            new OffsetCommitRequestData.OffsetCommitRequestPartition()
-              .setPartitionIndex(0)
-              .setCommittedOffset(60),
-            new OffsetCommitRequestData.OffsetCommitRequestPartition()
-              .setPartitionIndex(1)
-              .setCommittedOffset(70)).asJava)).asJava)
+    val (fooId, barId, bazId, zarId, quxId) =
+      (Uuid.randomUuid(), Uuid.randomUuid(), Uuid.randomUuid(), Uuid.randomUuid(), Uuid.randomUuid())
+
+    addTopicToMetadataCache("foo", numPartitions = 2, topicId = fooId)
+    addTopicToMetadataCache("bar", numPartitions = 2, topicId = barId)
+    addTopicToMetadataCache("baz", numPartitions = 2, topicId = bazId)
+
+    val offsetCommitRequest = newOffsetCommitRequestData("group", "member", Seq(

Review Comment:
   Would you mind if we keep to keep those code refactoring in the tests for separate PR(s)? This PR is already extremely large and I would like to focus on getting the new code right. All those non-related changes are additional (unnecessary) distractions for now.



##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.KafkaApisTest.{NameAndId, newOffsetCommitRequestData, newOffsetCommitResponseData}
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetCommitRequestTest.assertResponseEquals
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
+
+import java.util.Optional.empty
+import java.util.Properties
+import scala.collection.{Map, Seq}
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava}
+
+class OffsetCommitRequestTest extends BaseRequestTest {
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val groupId = "groupId"
+
+  var consumer: KafkaConsumer[_, _] = _
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    consumer = createConsumer()
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    if (consumer != null)
+      Utils.closeQuietly(consumer, "KafkaConsumer")
+    super.tearDown()
+  }
+
+  def createTopics(topicNames: String*): Seq[NameAndId] = {
+    topicNames.map(topic => {
+      createTopic(topic)
+      val topicId: Uuid = getTopicIds().get(topic) match {
+        case Some(x) => x
+        case _ => throw new AssertionError("Topic ID not found for " + topic)
+      }
+      NameAndId(topic, topicId)

Review Comment:
   Do we really need to use `NameAndId` here? This does not seem necessary.



##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.KafkaApisTest.{NameAndId, newOffsetCommitRequestData, newOffsetCommitResponseData}
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetCommitRequestTest.assertResponseEquals
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
+
+import java.util.Optional.empty
+import java.util.Properties
+import scala.collection.{Map, Seq}
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava}
+
+class OffsetCommitRequestTest extends BaseRequestTest {
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val groupId = "groupId"
+
+  var consumer: KafkaConsumer[_, _] = _
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    consumer = createConsumer()
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    if (consumer != null)
+      Utils.closeQuietly(consumer, "KafkaConsumer")
+    super.tearDown()
+  }
+
+  def createTopics(topicNames: String*): Seq[NameAndId] = {
+    topicNames.map(topic => {
+      createTopic(topic)
+      val topicId: Uuid = getTopicIds().get(topic) match {
+        case Some(x) => x
+        case _ => throw new AssertionError("Topic ID not found for " + topic)
+      }
+      NameAndId(topic, topicId)
+    })
+  }
+
+
+  @Test
+  def testTopicIdsArePopulatedInOffsetCommitResponses(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+    consumer.subscribe(topics.map(_.name).asJava)
+
+    sendOffsetCommitRequest(
+      topics.map((_, ListMap(0 -> offset))),
+      topics.map((_, Map(0 -> Errors.NONE))),

Review Comment:
   I would rather prefer to use the request/response data objects here.



##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.KafkaApisTest.{NameAndId, newOffsetCommitRequestData, newOffsetCommitResponseData}
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetCommitRequestTest.assertResponseEquals
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
+
+import java.util.Optional.empty
+import java.util.Properties
+import scala.collection.{Map, Seq}
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava}
+
+class OffsetCommitRequestTest extends BaseRequestTest {
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val groupId = "groupId"
+
+  var consumer: KafkaConsumer[_, _] = _
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    consumer = createConsumer()
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    if (consumer != null)
+      Utils.closeQuietly(consumer, "KafkaConsumer")
+    super.tearDown()
+  }
+
+  def createTopics(topicNames: String*): Seq[NameAndId] = {
+    topicNames.map(topic => {
+      createTopic(topic)
+      val topicId: Uuid = getTopicIds().get(topic) match {
+        case Some(x) => x
+        case _ => throw new AssertionError("Topic ID not found for " + topic)
+      }
+      NameAndId(topic, topicId)
+    })
+  }
+
+
+  @Test
+  def testTopicIdsArePopulatedInOffsetCommitResponses(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+    consumer.subscribe(topics.map(_.name).asJava)
+
+    sendOffsetCommitRequest(
+      topics.map((_, ListMap(0 -> offset))),
+      topics.map((_, Map(0 -> Errors.NONE))),
+      ApiKeys.OFFSET_COMMIT.allVersions().asScala
+    )
+  }
+
+  @Test
+  def testCommitOffsetFromConsumer(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+    consumer.commitSync(offsetsToCommit(topics, offset))
+  }

Review Comment:
   I wonder if we already have integration tests for the consumer covering this. Do we?



##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.KafkaApisTest.{NameAndId, newOffsetCommitRequestData, newOffsetCommitResponseData}
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetCommitRequestTest.assertResponseEquals
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
+
+import java.util.Optional.empty
+import java.util.Properties
+import scala.collection.{Map, Seq}
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava}
+
+class OffsetCommitRequestTest extends BaseRequestTest {
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val groupId = "groupId"
+
+  var consumer: KafkaConsumer[_, _] = _
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    consumer = createConsumer()
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    if (consumer != null)
+      Utils.closeQuietly(consumer, "KafkaConsumer")
+    super.tearDown()
+  }
+
+  def createTopics(topicNames: String*): Seq[NameAndId] = {
+    topicNames.map(topic => {
+      createTopic(topic)
+      val topicId: Uuid = getTopicIds().get(topic) match {
+        case Some(x) => x
+        case _ => throw new AssertionError("Topic ID not found for " + topic)
+      }
+      NameAndId(topic, topicId)
+    })
+  }
+
+
+  @Test
+  def testTopicIdsArePopulatedInOffsetCommitResponses(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+    consumer.subscribe(topics.map(_.name).asJava)
+
+    sendOffsetCommitRequest(
+      topics.map((_, ListMap(0 -> offset))),
+      topics.map((_, Map(0 -> Errors.NONE))),
+      ApiKeys.OFFSET_COMMIT.allVersions().asScala

Review Comment:
   Could we parameterize the test instead of doing this?



##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.KafkaApisTest.{NameAndId, newOffsetCommitRequestData, newOffsetCommitResponseData}
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetCommitRequestTest.assertResponseEquals
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
+
+import java.util.Optional.empty
+import java.util.Properties
+import scala.collection.{Map, Seq}
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava}
+
+class OffsetCommitRequestTest extends BaseRequestTest {
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val groupId = "groupId"
+
+  var consumer: KafkaConsumer[_, _] = _
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    consumer = createConsumer()
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    if (consumer != null)
+      Utils.closeQuietly(consumer, "KafkaConsumer")
+    super.tearDown()
+  }
+
+  def createTopics(topicNames: String*): Seq[NameAndId] = {
+    topicNames.map(topic => {
+      createTopic(topic)
+      val topicId: Uuid = getTopicIds().get(topic) match {
+        case Some(x) => x
+        case _ => throw new AssertionError("Topic ID not found for " + topic)
+      }
+      NameAndId(topic, topicId)
+    })
+  }
+
+
+  @Test
+  def testTopicIdsArePopulatedInOffsetCommitResponses(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")

Review Comment:
   nit: You could get `topicIds` with `getTopicIds("topic1", "topic2", "topic3")`.



##########
core/src/test/scala/unit/kafka/server/KafkaApisTest.scala:
##########
@@ -1264,22 +1270,25 @@ class KafkaApisTest {
     )
   }
 
-  @Test
-  def testHandleOffsetCommitRequest(): Unit = {
-    addTopicToMetadataCache("foo", numPartitions = 1)
+  @ParameterizedTest
+  @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_COMMIT)
+  def testHandleOffsetCommitRequest(version: Short): Unit = {
+    val fooId = Uuid.randomUuid()
+    addTopicToMetadataCache("foo", numPartitions = 1, topicId = fooId)
 
     val offsetCommitRequest = new OffsetCommitRequestData()
       .setGroupId("group")
       .setMemberId("member")
       .setTopics(List(
         new OffsetCommitRequestData.OffsetCommitRequestTopic()
           .setName("foo")
+          .setTopicId(if (version >= 9) fooId else Uuid.ZERO_UUID)
           .setPartitions(List(
             new OffsetCommitRequestData.OffsetCommitRequestPartition()
               .setPartitionIndex(0)
               .setCommittedOffset(10)).asJava)).asJava)
 
-    val requestChannelRequest = buildRequest(new OffsetCommitRequest.Builder(offsetCommitRequest).build())
+    val requestChannelRequest = buildRequest(new OffsetCommitRequest.Builder(offsetCommitRequest, true).build(version))

Review Comment:
   Is using `true` all the time correct here? I suppose that it should be `false` if `version` < 9, no?



##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.KafkaApisTest.{NameAndId, newOffsetCommitRequestData, newOffsetCommitResponseData}
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetCommitRequestTest.assertResponseEquals
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
+
+import java.util.Optional.empty
+import java.util.Properties
+import scala.collection.{Map, Seq}
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava}
+
+class OffsetCommitRequestTest extends BaseRequestTest {
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val groupId = "groupId"
+
+  var consumer: KafkaConsumer[_, _] = _
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    consumer = createConsumer()
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    if (consumer != null)
+      Utils.closeQuietly(consumer, "KafkaConsumer")

Review Comment:
   I think that consumers created with `createConsumer` are closed automatically by the super class.



##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.KafkaApisTest.{NameAndId, newOffsetCommitRequestData, newOffsetCommitResponseData}
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetCommitRequestTest.assertResponseEquals
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
+
+import java.util.Optional.empty
+import java.util.Properties
+import scala.collection.{Map, Seq}
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava}
+
+class OffsetCommitRequestTest extends BaseRequestTest {
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val groupId = "groupId"
+
+  var consumer: KafkaConsumer[_, _] = _
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    consumer = createConsumer()
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    if (consumer != null)
+      Utils.closeQuietly(consumer, "KafkaConsumer")
+    super.tearDown()
+  }
+
+  def createTopics(topicNames: String*): Seq[NameAndId] = {

Review Comment:
   nit: This could be private.



##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.KafkaApisTest.{NameAndId, newOffsetCommitRequestData, newOffsetCommitResponseData}
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetCommitRequestTest.assertResponseEquals
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
+
+import java.util.Optional.empty
+import java.util.Properties
+import scala.collection.{Map, Seq}
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava}
+
+class OffsetCommitRequestTest extends BaseRequestTest {
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val groupId = "groupId"
+
+  var consumer: KafkaConsumer[_, _] = _
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    consumer = createConsumer()
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    if (consumer != null)
+      Utils.closeQuietly(consumer, "KafkaConsumer")
+    super.tearDown()
+  }
+
+  def createTopics(topicNames: String*): Seq[NameAndId] = {
+    topicNames.map(topic => {
+      createTopic(topic)
+      val topicId: Uuid = getTopicIds().get(topic) match {
+        case Some(x) => x
+        case _ => throw new AssertionError("Topic ID not found for " + topic)
+      }
+      NameAndId(topic, topicId)
+    })
+  }
+
+
+  @Test
+  def testTopicIdsArePopulatedInOffsetCommitResponses(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+    consumer.subscribe(topics.map(_.name).asJava)
+
+    sendOffsetCommitRequest(
+      topics.map((_, ListMap(0 -> offset))),
+      topics.map((_, Map(0 -> Errors.NONE))),
+      ApiKeys.OFFSET_COMMIT.allVersions().asScala
+    )
+  }
+
+  @Test
+  def testCommitOffsetFromConsumer(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+    consumer.commitSync(offsetsToCommit(topics, offset))
+  }
+
+  @Test
+  def testOffsetCommitWithUnknownTopicId(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+
+    sendOffsetCommitRequest(
+      topics.map((_, ListMap(0 -> offset))) ++ Seq((NameAndId("unresolvable"), ListMap(0 -> offset))),
+      Seq((NameAndId("unresolvable"), ListMap(0 -> Errors.UNKNOWN_TOPIC_ID))) ++ topics.map((_, Map(0 -> Errors.NONE))),
+      ApiKeys.OFFSET_COMMIT.allVersions().asScala.filter(_ >= 9)
+    )
+  }
+
+  @Test
+  def alterConsumerGroupOffsetsDoNotUseTopicIds(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+    val admin = createAdminClient()
+
+    try {
+      // Would throw an UnknownTopicId exception if the OffsetCommitRequest was set to version 9 or higher.
+      admin.alterConsumerGroupOffsets(groupId, offsetsToCommit(topics, offset)).all.get()
+
+    } finally {
+      Utils.closeQuietly(admin, "AdminClient")
+    }
+  }
+
+  def sendOffsetCommitRequest(offsets: Seq[(NameAndId, Map[Int, Long])],
+                              responses: Seq[(NameAndId, Map[Int, Errors])],
+                              versions: Seq[java.lang.Short]): Unit = {
+
+    val requestData = newOffsetCommitRequestData(
+      groupId = "group",
+      offsets = offsets
+    )
+
+    versions.foreach { version =>
+      val expectedResponse = newOffsetCommitResponseData(
+        version,
+        topicPartitions = responses,
+      )
+      val response = connectAndReceive[OffsetCommitResponse](
+        new OffsetCommitRequest.Builder(requestData, true).build(version)
+      )
+      assertResponseEquals(new OffsetCommitResponse(expectedResponse), response)
+    }
+  }
+
+  def offsetsToCommit(topics: Seq[NameAndId], offset: Long): java.util.Map[TopicPartition, OffsetAndMetadata] = {

Review Comment:
   nit: Let's make all the private methods private.



##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.KafkaApisTest.{NameAndId, newOffsetCommitRequestData, newOffsetCommitResponseData}
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetCommitRequestTest.assertResponseEquals
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
+
+import java.util.Optional.empty
+import java.util.Properties
+import scala.collection.{Map, Seq}
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava}
+
+class OffsetCommitRequestTest extends BaseRequestTest {
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val groupId = "groupId"
+
+  var consumer: KafkaConsumer[_, _] = _
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    consumer = createConsumer()

Review Comment:
   I think that you could pass config overrides to `createConsumer` directly.



##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.KafkaApisTest.{NameAndId, newOffsetCommitRequestData, newOffsetCommitResponseData}
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetCommitRequestTest.assertResponseEquals
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
+
+import java.util.Optional.empty
+import java.util.Properties
+import scala.collection.{Map, Seq}
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava}
+
+class OffsetCommitRequestTest extends BaseRequestTest {
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val groupId = "groupId"
+
+  var consumer: KafkaConsumer[_, _] = _
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    consumer = createConsumer()
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    if (consumer != null)
+      Utils.closeQuietly(consumer, "KafkaConsumer")
+    super.tearDown()
+  }
+
+  def createTopics(topicNames: String*): Seq[NameAndId] = {
+    topicNames.map(topic => {

Review Comment:
   nit: `topicNames.map(topic => {` -> `topicNames.map { topic => `



##########
core/src/test/scala/unit/kafka/server/KafkaApisTest.scala:
##########
@@ -1294,36 +1303,49 @@ class KafkaApisTest {
     )
 
     // This is the response returned by the group coordinator.
-    val offsetCommitResponse = new OffsetCommitResponseData()
+    val coordinatorResponse = new OffsetCommitResponseData()
       .setTopics(List(
         new OffsetCommitResponseData.OffsetCommitResponseTopic()
           .setName("foo")
+          .setTopicId(if (version >= 9) fooId else Uuid.ZERO_UUID)

Review Comment:
   nit: I think that we could set it all the time here as well.



##########
core/src/test/scala/unit/kafka/server/KafkaApisTest.scala:
##########
@@ -1434,91 +1412,110 @@ class KafkaApisTest {
     )
 
     // This is the response returned by the group coordinator.
-    val offsetCommitResponse = new OffsetCommitResponseData()
-      .setTopics(List(
-        new OffsetCommitResponseData.OffsetCommitResponseTopic()
-          .setName("foo")
-          .setPartitions(List(
-            new OffsetCommitResponseData.OffsetCommitResponsePartition()
-              .setPartitionIndex(0)
-              .setErrorCode(Errors.NONE.code),
-            new OffsetCommitResponseData.OffsetCommitResponsePartition()
-              .setPartitionIndex(1)
-              .setErrorCode(Errors.NONE.code)).asJava),
-        new OffsetCommitResponseData.OffsetCommitResponseTopic()
-          .setName("bar")
-          .setPartitions(List(
-            new OffsetCommitResponseData.OffsetCommitResponsePartition()
-              .setPartitionIndex(0)
-              .setErrorCode(Errors.NONE.code),
-            new OffsetCommitResponseData.OffsetCommitResponsePartition()
-              .setPartitionIndex(1)
-              .setErrorCode(Errors.NONE.code)).asJava)).asJava)
+    val coordinatorResponse = newOffsetCommitResponseData(Seq(
+      (NameAndId("foo", fooId), ListMap(0 -> NONE, 1 -> NONE)),
+      (NameAndId("bar", barId), ListMap(0 -> NONE, 1 -> NONE)),
+      (NameAndId("baz", bazId), ListMap(0 -> NONE, 1 -> NONE))
+    ))
 
-    val expectedOffsetCommitResponse = new OffsetCommitResponseData()
-      .setTopics(List(
-        new OffsetCommitResponseData.OffsetCommitResponseTopic()
-          .setName("foo")
-          .setPartitions(List(
-            // foo-2 is first because partitions failing the validation
-            // are put in the response first.
-            new OffsetCommitResponseData.OffsetCommitResponsePartition()
-              .setPartitionIndex(2)
-              .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code),
-            new OffsetCommitResponseData.OffsetCommitResponsePartition()
-              .setPartitionIndex(0)
-              .setErrorCode(Errors.NONE.code),
-            new OffsetCommitResponseData.OffsetCommitResponsePartition()
-              .setPartitionIndex(1)
-              .setErrorCode(Errors.NONE.code)).asJava),
-        // zar is before bar because topics failing the validation are
-        // put in the response first.
-        new OffsetCommitResponseData.OffsetCommitResponseTopic()
-          .setName("zar")
-          .setPartitions(List(
-            new OffsetCommitResponseData.OffsetCommitResponsePartition()
-              .setPartitionIndex(0)
-              .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code),
-            new OffsetCommitResponseData.OffsetCommitResponsePartition()
-              .setPartitionIndex(1)
-              .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code)).asJava),
-        new OffsetCommitResponseData.OffsetCommitResponseTopic()
-          .setName("bar")
-          .setPartitions(List(
-            new OffsetCommitResponseData.OffsetCommitResponsePartition()
-              .setPartitionIndex(0)
-              .setErrorCode(Errors.NONE.code),
-            new OffsetCommitResponseData.OffsetCommitResponsePartition()
-              .setPartitionIndex(1)
-              .setErrorCode(Errors.NONE.code)).asJava)).asJava)
+    val expectedOffsetCommitResponse = newOffsetCommitResponseData(Seq(
+      // zar is before bar because topics failing the validation are put in the response first.
+      (NameAndId(id = zarId), ListMap(0 -> UNKNOWN_TOPIC_ID, 1 -> UNKNOWN_TOPIC_ID)),
+
+      // qux is before bar because topics failing the validation are put in the response first.
+      (NameAndId(id = quxId), ListMap(0 -> UNKNOWN_TOPIC_ID)),
 
-    future.complete(offsetCommitResponse)
+      // foo-2 is first because partitions failing the validation are put in the response first.
+      (NameAndId(id = fooId), ListMap(2 -> UNKNOWN_TOPIC_OR_PARTITION, 0 -> NONE, 1 -> NONE)),
+
+      // Valid topics bar and baz.
+      (NameAndId(id = barId), ListMap(0 -> NONE, 1 -> NONE)),
+      (NameAndId(id = bazId), ListMap(0 -> NONE, 1 -> NONE))
+    ))
+
+    future.complete(coordinatorResponse)
     val response = verifyNoThrottling[OffsetCommitResponse](requestChannelRequest)
     assertEquals(expectedOffsetCommitResponse, response.data)
   }
 
-  @Test
-  def testOffsetCommitWithInvalidPartition(): Unit = {
-    val topic = "topic"
-    addTopicToMetadataCache(topic, numPartitions = 1)
+  @ParameterizedTest
+  @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_COMMIT)
+  def testHandleOffsetCommitRequestTopicsAndPartitionsAllValid(version: Short): Unit = {
+    val (fooId, barId, bazId) = (Uuid.randomUuid(), Uuid.randomUuid(), Uuid.randomUuid())
+
+    addTopicToMetadataCache("foo", numPartitions = 2, topicId = fooId)
+    addTopicToMetadataCache("bar", numPartitions = 2, topicId = barId)
+    addTopicToMetadataCache("baz", numPartitions = 2, topicId = bazId)
+
+    val offsetCommitRequest = newOffsetCommitRequestData("group", "member", Seq(
+      (if (version < 9) NameAndId("foo") else NameAndId(id = fooId), ListMap(0 -> 10, 1 -> 20)),
+      (if (version < 9) NameAndId("bar") else NameAndId(id = barId), ListMap(0 -> 40, 1 -> 50)),
+      (if (version < 9) NameAndId("baz") else NameAndId(id = bazId), ListMap(0 -> 60, 1 -> 80))
+    ))
+
+    val requestChannelRequest = buildRequest(new OffsetCommitRequest.Builder(offsetCommitRequest, true).build(version))

Review Comment:
   nit: `true` should be derived from the `version`.



##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.KafkaApisTest.{NameAndId, newOffsetCommitRequestData, newOffsetCommitResponseData}
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetCommitRequestTest.assertResponseEquals
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
+
+import java.util.Optional.empty
+import java.util.Properties
+import scala.collection.{Map, Seq}
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava}
+
+class OffsetCommitRequestTest extends BaseRequestTest {
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val groupId = "groupId"
+
+  var consumer: KafkaConsumer[_, _] = _
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    consumer = createConsumer()
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    if (consumer != null)
+      Utils.closeQuietly(consumer, "KafkaConsumer")
+    super.tearDown()
+  }
+
+  def createTopics(topicNames: String*): Seq[NameAndId] = {
+    topicNames.map(topic => {
+      createTopic(topic)
+      val topicId: Uuid = getTopicIds().get(topic) match {
+        case Some(x) => x
+        case _ => throw new AssertionError("Topic ID not found for " + topic)
+      }
+      NameAndId(topic, topicId)
+    })
+  }
+
+
+  @Test
+  def testTopicIdsArePopulatedInOffsetCommitResponses(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+    consumer.subscribe(topics.map(_.name).asJava)
+
+    sendOffsetCommitRequest(
+      topics.map((_, ListMap(0 -> offset))),
+      topics.map((_, Map(0 -> Errors.NONE))),
+      ApiKeys.OFFSET_COMMIT.allVersions().asScala
+    )
+  }
+
+  @Test
+  def testCommitOffsetFromConsumer(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+    consumer.commitSync(offsetsToCommit(topics, offset))
+  }
+
+  @Test
+  def testOffsetCommitWithUnknownTopicId(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+
+    sendOffsetCommitRequest(
+      topics.map((_, ListMap(0 -> offset))) ++ Seq((NameAndId("unresolvable"), ListMap(0 -> offset))),
+      Seq((NameAndId("unresolvable"), ListMap(0 -> Errors.UNKNOWN_TOPIC_ID))) ++ topics.map((_, Map(0 -> Errors.NONE))),
+      ApiKeys.OFFSET_COMMIT.allVersions().asScala.filter(_ >= 9)
+    )
+  }
+
+  @Test
+  def alterConsumerGroupOffsetsDoNotUseTopicIds(): Unit = {

Review Comment:
   This test does not seem to be at the right place. It seems to me that `OffsetCommitRequestTest` is more focused on testing the OffsetCommitRequest API.



##########
core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala:
##########
@@ -299,27 +299,27 @@ class RequestQuotaTest extends BaseRequestTest {
               ApiKeys.CONTROLLED_SHUTDOWN.latestVersion)
 
         case ApiKeys.OFFSET_COMMIT =>
-          new OffsetCommitRequest.Builder(
-            new OffsetCommitRequestData()
-              .setGroupId("test-group")
-              .setGenerationId(1)
-              .setMemberId(JoinGroupRequest.UNKNOWN_MEMBER_ID)
-              .setTopics(
-                Collections.singletonList(
-                  new OffsetCommitRequestData.OffsetCommitRequestTopic()
-                    .setName(topic)
-                    .setPartitions(
-                      Collections.singletonList(
-                        new OffsetCommitRequestData.OffsetCommitRequestPartition()
-                          .setPartitionIndex(0)
-                          .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
-                          .setCommittedOffset(0)
-                          .setCommittedMetadata("metadata")
-                      )
+          val data = new OffsetCommitRequestData()
+            .setGroupId("test-group")
+            .setGenerationId(1)
+            .setMemberId(JoinGroupRequest.UNKNOWN_MEMBER_ID)
+            .setTopics(
+              Collections.singletonList(
+                new OffsetCommitRequestData.OffsetCommitRequestTopic()
+                  .setName(topic)
+                  .setPartitions(
+                    Collections.singletonList(
+                      new OffsetCommitRequestData.OffsetCommitRequestPartition()
+                        .setPartitionIndex(0)
+                        .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
+                        .setCommittedOffset(0)
+                        .setCommittedMetadata("metadata")
                     )
-                )
+                  )
               )
-          )
+            )
+          new OffsetCommitRequest.Builder(data, true)

Review Comment:
   nit: Could we revert this change and just add the boolean?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1469956438

   Many thanks David. I will try to get to this in the next couple of days. Apologies for the delay, I wish i could get to this sooner.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] clolov commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "clolov (via GitHub)" <gi...@apache.org>.
clolov commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1185137613


##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.KafkaApisTest.{NameAndId, newOffsetCommitRequestData, newOffsetCommitResponseData}
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetCommitRequestTest.assertResponseEquals
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
+
+import java.util.Optional.empty
+import java.util.Properties
+import scala.collection.{Map, Seq}
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava}
+
+class OffsetCommitRequestTest extends BaseRequestTest {
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val groupId = "groupId"
+
+  var consumer: KafkaConsumer[_, _] = _
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    consumer = createConsumer()
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    if (consumer != null)
+      Utils.closeQuietly(consumer, "KafkaConsumer")
+    super.tearDown()
+  }
+
+  def createTopics(topicNames: String*): Seq[NameAndId] = {
+    topicNames.map(topic => {
+      createTopic(topic)
+      val topicId: Uuid = getTopicIds().get(topic) match {
+        case Some(x) => x
+        case _ => throw new AssertionError("Topic ID not found for " + topic)
+      }
+      NameAndId(topic, topicId)
+    })
+  }
+
+
+  @Test
+  def testTopicIdsArePopulatedInOffsetCommitResponses(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")

Review Comment:
   Sorry, could you elaborate, because I am not certain I follow? `getTopicIds(...)` will return a map but only if the topics requested have been created first. Are you suggesting that since all tests create these three topics we move the creation to the setup method and then we use `getTopicIds` everywhere else?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] clolov commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "clolov (via GitHub)" <gi...@apache.org>.
clolov commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1197974839


##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.KafkaApisTest.{NameAndId, newOffsetCommitRequestData, newOffsetCommitResponseData}
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetCommitRequestTest.assertResponseEquals
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
+
+import java.util.Optional.empty
+import java.util.Properties
+import scala.collection.{Map, Seq}
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava}
+
+class OffsetCommitRequestTest extends BaseRequestTest {
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val groupId = "groupId"
+
+  var consumer: KafkaConsumer[_, _] = _
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    consumer = createConsumer()
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    if (consumer != null)
+      Utils.closeQuietly(consumer, "KafkaConsumer")
+    super.tearDown()
+  }
+
+  def createTopics(topicNames: String*): Seq[NameAndId] = {
+    topicNames.map(topic => {
+      createTopic(topic)
+      val topicId: Uuid = getTopicIds().get(topic) match {
+        case Some(x) => x
+        case _ => throw new AssertionError("Topic ID not found for " + topic)
+      }
+      NameAndId(topic, topicId)
+    })
+  }
+
+
+  @Test
+  def testTopicIdsArePopulatedInOffsetCommitResponses(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+    consumer.subscribe(topics.map(_.name).asJava)
+
+    sendOffsetCommitRequest(
+      topics.map((_, ListMap(0 -> offset))),
+      topics.map((_, Map(0 -> Errors.NONE))),
+      ApiKeys.OFFSET_COMMIT.allVersions().asScala

Review Comment:
   To be honest I would prefer if we leave it like this. If we parameterise it, this means we have to change `sendOffsetCommitRequest`. If we change `sendOffsetCommitRequest` then we need to come up with a different source for `testOffsetCommitWithUnknownTopicId`. Alternatively I can parameterise this test, but I would end up wrapping a single version in a Seq. Is the reason you want it parameterised here so that it breaks it down when running the tests in IntelliJ?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1553290938

   Thanks Christo (@clolov) for your help on the PR, I will take a look at the changes tomorrow. Thanks!


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1204403475


##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.KafkaApisTest.{NameAndId, newOffsetCommitRequestData, newOffsetCommitResponseData}
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetCommitRequestTest.assertResponseEquals
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
+
+import java.util.Optional.empty
+import java.util.Properties
+import scala.collection.{Map, Seq}
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava}
+
+class OffsetCommitRequestTest extends BaseRequestTest {
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val groupId = "groupId"
+
+  var consumer: KafkaConsumer[_, _] = _
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    consumer = createConsumer()
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    if (consumer != null)
+      Utils.closeQuietly(consumer, "KafkaConsumer")
+    super.tearDown()
+  }
+
+  def createTopics(topicNames: String*): Seq[NameAndId] = {
+    topicNames.map(topic => {
+      createTopic(topic)
+      val topicId: Uuid = getTopicIds().get(topic) match {
+        case Some(x) => x
+        case _ => throw new AssertionError("Topic ID not found for " + topic)
+      }
+      NameAndId(topic, topicId)
+    })
+  }
+
+
+  @Test
+  def testTopicIdsArePopulatedInOffsetCommitResponses(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+    consumer.subscribe(topics.map(_.name).asJava)
+
+    sendOffsetCommitRequest(
+      topics.map((_, ListMap(0 -> offset))),
+      topics.map((_, Map(0 -> Errors.NONE))),
+      ApiKeys.OFFSET_COMMIT.allVersions().asScala
+    )
+  }
+
+  @Test
+  def testCommitOffsetFromConsumer(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+    consumer.commitSync(offsetsToCommit(topics, offset))
+  }
+
+  @Test
+  def testOffsetCommitWithUnknownTopicId(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+
+    sendOffsetCommitRequest(
+      topics.map((_, ListMap(0 -> offset))) ++ Seq((NameAndId("unresolvable"), ListMap(0 -> offset))),
+      Seq((NameAndId("unresolvable"), ListMap(0 -> Errors.UNKNOWN_TOPIC_ID))) ++ topics.map((_, Map(0 -> Errors.NONE))),
+      ApiKeys.OFFSET_COMMIT.allVersions().asScala.filter(_ >= 9)
+    )
+  }
+
+  @Test
+  def alterConsumerGroupOffsetsDoNotUseTopicIds(): Unit = {

Review Comment:
   I removed the test since this method is already exercised in `org.apache.kafka.clients.admin.KafkaAdminClientTest#testOffsetCommitNumRetries`.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1205119564


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -3981,23 +4348,13 @@ private void prepareJoinAndSyncResponse(String consumerId, int generation, List<
         }, syncGroupResponse(assignment, Errors.NONE));
     }
 
-    private Map<TopicPartition, Errors> partitionErrors(Collection<TopicPartition> partitions, Errors error) {
-        final Map<TopicPartition, Errors> errors = new HashMap<>();
-        for (TopicPartition partition : partitions) {
-            errors.put(partition, error);
-        }
-        return errors;
-    }
-
-    private void respondToOffsetCommitRequest(final Map<TopicPartition, Long> expectedOffsets, Errors error) {
-        Map<TopicPartition, Errors> errors = partitionErrors(expectedOffsets.keySet(), error);
-        client.respond(offsetCommitRequestMatcher(expectedOffsets), offsetCommitResponse(errors));
-    }
-
-    private MockClient.RequestMatcher offsetCommitRequestMatcher(final Map<TopicPartition, Long> expectedOffsets) {
+    private MockClient.RequestMatcher offsetCommitRequestMatcher(
+        Map<TopicPartition, Long> expectedOffsets,
+        Consumer<OffsetCommitRequest> requestConsumer

Review Comment:
   @dajac Sure, that makes sense to get rid of the consumer, since it is mixing test design pattern and overlaps the responsibilities of the matcher as you pointed out.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1104740902


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java:
##########
@@ -78,11 +89,18 @@ public OffsetCommitRequestData data() {
         return data;
     }
 
-    public Map<TopicPartition, Long> offsets() {
+    public Map<TopicPartition, Long> offsets(TopicResolver topicResolver) {
         Map<TopicPartition, Long> offsets = new HashMap<>();
         for (OffsetCommitRequestTopic topic : data.topics()) {
+            String topicName = topic.name();
+
+            if (version() >= 9 && topicName == null) {

Review Comment:
   Note 1: if the `topicName` is not null, we should also check if it resolves to the same UUID as we have cached locally.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1121486904


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1352,8 +1361,42 @@ public void handle(OffsetCommitResponse commitResponse, RequestFuture<Void> futu
             Set<String> unauthorizedTopics = new HashSet<>();
 
             for (OffsetCommitResponseData.OffsetCommitResponseTopic topic : commitResponse.data().topics()) {
+                String topicName = topic.name();

Review Comment:
   Now that we can rely on the version, we should use it here and simplify all this logic.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1121510354


##########
checkstyle/suppressions.xml:
##########
@@ -93,7 +93,7 @@
               files="(AbstractRequest|AbstractResponse|KerberosLogin|WorkerSinkTaskTest|TransactionManagerTest|SenderTest|KafkaAdminClient|ConsumerCoordinatorTest|KafkaAdminClientTest|KafkaRaftClientTest).java"/>
 
     <suppress checks="NPathComplexity"
-              files="(ConsumerCoordinator|BufferPool|Fetcher|MetricName|Node|ConfigDef|RecordBatch|SslFactory|SslTransportLayer|MetadataResponse|KerberosLogin|Selector|Sender|Serdes|TokenInformation|Agent|Values|PluginUtils|MiniTrogdorCluster|TasksRequest|KafkaProducer|AbstractStickyAssignor|KafkaRaftClient|Authorizer|FetchSessionHandler).java"/>
+              files="(ConsumerCoordinator|BufferPool|Fetcher|MetricName|Node|ConfigDef|RecordBatch|SslFactory|SslTransportLayer|MetadataResponse|KerberosLogin|Selector|Sender|Serdes|TokenInformation|Agent|Values|PluginUtils|MiniTrogdorCluster|TasksRequest|KafkaProducer|AbstractStickyAssignor|KafkaRaftClient|Authorizer|FetchSessionHandler|OffsetCommitResponseTest).java"/>

Review Comment:
   Is this really needed?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1122188628


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -166,20 +187,27 @@ public <P> Builder addPartitions(
             return this;
         }
 
-        public Builder merge(
-            OffsetCommitResponseData newData
-        ) {
+        public Builder<T> merge(OffsetCommitResponseData newData) {
+            if (version >= 9) {
+                // This method is called after the group coordinator committed the offsets. The group coordinator
+                // provides the OffsetCommitResponseData it built in the process. As of now, this data does
+                // not contain topic ids, so we resolve them here.
+                newData.topics().forEach(
+                        topic -> topic.setTopicId(topicResolver.getTopicId(topic.name()).orElse(Uuid.ZERO_UUID)));

Review Comment:
   At this point, topic ids should be always resolvable. However if some aren't, we should fallback to adding the topic "as is" to the response to avoid caching `ZERO_UUID` with risk of overwrites.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1446539881

   @Hangleton Thanks for the update. I will take a look shortly. In the meantime, could you already add a few integration test for the OffsetCommitRequest like we have for the OffsetFetchRequest in kafka.server.OffsetFetchRequestTest? I believe that topic ids are actually lost when they are passed to the group coordinator. Hence, I request with topic ids will very likely return a response with topic names instead of topic ids. The issue is that we don't catch those kind of issues with the existing tests because the logic in the consumer supports both ways.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1511110416

   > Hey @Hangleton. I just got back to this PR. I made a pass over the files in `core` and I left some comments. As a general ask, it would be great if we could keep avoid large refactoring in tests in this PR as they are very distracting. I am not against refactoring but I would do them in separate PRs.
   
   Hi David, thanks for the review. Understand about refactoring, I will try to see if I can revert some of them if possible.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1226607855


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1354,8 +1373,21 @@ public void handle(OffsetCommitResponse commitResponse, RequestFuture<Void> futu
             Set<String> unauthorizedTopics = new HashSet<>();
 
             for (OffsetCommitResponseData.OffsetCommitResponseTopic topic : commitResponse.data().topics()) {
+                String topicName = topic.name();
+
+                if (this.response.requestHeader().apiVersion() >= 9) {
+                    topicName = topicIdAndNames.topicNameOrNull(topic.topicId());
+
+                    if (topicName == null) {
+                        // Could only happen if the broker replied with an ID which was not in the request and
+                        // unknown by this client. This would be a bug.
+                        log.warn("Ignoring invalid topic ID found in OffsetCommit response: " + topic.topicId());
+                        continue;
+                    }

Review Comment:
   As a second thought, I wonder if we should complete the future with an exception here. Being defensive would help us to catch bugs early one. What do you think?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1125687390


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1328,13 +1335,16 @@ RequestFuture<Void> sendOffsetCommitRequest(final Map<TopicPartition, OffsetAndM
             groupInstanceId = null;
         }
 
+        boolean canUseTopicIds = topicPartitionsWithoutTopicId == 0;

Review Comment:
   Sure! Added the tests. Thanks.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1126511918


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -167,8 +213,24 @@ public <P> Builder addPartitions(
         }
 
         public Builder merge(
-            OffsetCommitResponseData newData
+            OffsetCommitResponseData newData,
+            Logger logger
         ) {
+            if (version >= 9) {
+                // This method is called after the group coordinator committed the offsets. The group coordinator
+                // provides the OffsetCommitResponseData it built in the process. As of now, this data does
+                // not contain topic ids, so we resolve them here.
+                newData.topics().forEach(topic -> {
+                    Uuid topicId = topicResolver.getTopicId(topic.name()).orElse(Uuid.ZERO_UUID);
+                    if (Uuid.ZERO_UUID.equals(topicId)) {
+                        // This should not happen because topic names returned by the group coordinator should
+                        // always be resolvable.
+                        logger.debug("Unresolvable topic id for topic {} while preparing " +

Review Comment:
   Hmm, we probably don't want to include a topic without id in the response version 9 here.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1129066723


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1352,8 +1370,22 @@ public void handle(OffsetCommitResponse commitResponse, RequestFuture<Void> futu
             Set<String> unauthorizedTopics = new HashSet<>();
 
             for (OffsetCommitResponseData.OffsetCommitResponseTopic topic : commitResponse.data().topics()) {
+                String topicName = topic.name();
+
+                if (commitResponse.version() >= 9) {

Review Comment:
   @Hangleton I had a deeper look into this and it seems that we could get the version with `this.response.requestHeader().apiVersion()`. Could you check if this would work?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1459774076

   Many thanks, David, for the review. Working on fixing the PR now. I will send the corrective commits by EOD. Thanks.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1429942230

   Thanks for the review, David. I am working on adding unit tests for `OffsetCommitResponse` and the server-side handling of the request/response, and fix the bugs you have identified.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1205119564


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -3981,23 +4348,13 @@ private void prepareJoinAndSyncResponse(String consumerId, int generation, List<
         }, syncGroupResponse(assignment, Errors.NONE));
     }
 
-    private Map<TopicPartition, Errors> partitionErrors(Collection<TopicPartition> partitions, Errors error) {
-        final Map<TopicPartition, Errors> errors = new HashMap<>();
-        for (TopicPartition partition : partitions) {
-            errors.put(partition, error);
-        }
-        return errors;
-    }
-
-    private void respondToOffsetCommitRequest(final Map<TopicPartition, Long> expectedOffsets, Errors error) {
-        Map<TopicPartition, Errors> errors = partitionErrors(expectedOffsets.keySet(), error);
-        client.respond(offsetCommitRequestMatcher(expectedOffsets), offsetCommitResponse(errors));
-    }
-
-    private MockClient.RequestMatcher offsetCommitRequestMatcher(final Map<TopicPartition, Long> expectedOffsets) {
+    private MockClient.RequestMatcher offsetCommitRequestMatcher(
+        Map<TopicPartition, Long> expectedOffsets,
+        Consumer<OffsetCommitRequest> requestConsumer

Review Comment:
   @dajac Sure, that makes sense to get rid of the consumer, since it is mixing test design pattern and overlap the responsibilities of the matcher as you pointed out.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1203965540


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -129,16 +150,19 @@
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
 
 public abstract class ConsumerCoordinatorTest {
-    private final String topic1 = "test1";
-    private final String topic2 = "test2";
-    private final TopicPartition t1p = new TopicPartition(topic1, 0);
-    private final TopicPartition t2p = new TopicPartition(topic2, 0);
-    private final String groupId = "test-group";
+    private static String topic1 = "test1";
+    private static String topic2 = "test2";
+    private static TopicPartition t1p = new TopicPartition(topic1, 0);
+    private static TopicIdPartition ti1p = new TopicIdPartition(Uuid.randomUuid(), t1p);

Review Comment:
   I think David probably hints at consolidating both in one defining object to ease future updates of topic-partition to topic ids. I updated the test class as per David's comment, I am happy to revert if this brings too many LoC changes.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1450362232

   Please note that as mentioned by David above, in the current state, topic ids are not provided by the group coordinator when constructing the response, hence are not returned with the `OffsetCommitResponse`. A change to the PR will be made to address this shortly.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1104760648


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java:
##########
@@ -78,11 +89,18 @@ public OffsetCommitRequestData data() {
         return data;
     }
 
-    public Map<TopicPartition, Long> offsets() {
+    public Map<TopicPartition, Long> offsets(TopicResolver topicResolver) {
         Map<TopicPartition, Long> offsets = new HashMap<>();
         for (OffsetCommitRequestTopic topic : data.topics()) {
+            String topicName = topic.name();
+
+            if (version() >= 9 && topicName == null) {

Review Comment:
   Note 2: We could fail partially - just for the given topic entry - rather than the entire response.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1455667030

   Took a look at the authorizer tests (`AuthorizerIntegrationTests`). It seems that authorization with topic and group `READ` permissions and unknown topic name is not currently being tested. We could add this use case and extend it for topic ids in a separate PR?


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1122265994


##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.ConsumerConfig
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.message.OffsetCommitRequestData
+import org.apache.kafka.common.message.OffsetCommitRequestData.{OffsetCommitRequestPartition, OffsetCommitRequestTopic}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.junit.jupiter.api.Assertions.assertEquals
+import org.junit.jupiter.api.Test
+
+import java.util.Collections.singletonList
+import java.util.Properties
+
+class OffsetCommitRequestTest extends BaseRequestTest {

Review Comment:
   Adding more tests to this class.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1121488495


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -425,35 +425,63 @@ class KafkaApis(val requestChannel: RequestChannel,
       requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
       CompletableFuture.completedFuture[Unit](())
     } else {
+      val topicNames =
+        if (offsetCommitRequest.version() >= 9)
+          metadataCache.topicIdsToNames()
+        else
+          Collections.emptyMap[Uuid, String]()
+
+      val resolvedTopics = new ArrayBuffer[OffsetCommitRequestData.OffsetCommitRequestTopic]()
+      offsetCommitRequest.data.topics.forEach { topic =>
+        var topicName = topic.name()
+        if (Utils.isBlank(topicName)) {
+          // Expected for requests version >= 9 which rely on topic IDs exclusively.
+          topicName = topicNames.get(topic.topicId())
+        }
+        if (topicName != null) {
+          topic.setName(topicName)
+          resolvedTopics += topic
+        }
+      }
+
       val authorizedTopics = authHelper.filterByAuthorized(
         request.context,
         READ,
         TOPIC,
-        offsetCommitRequest.data.topics.asScala
+        resolvedTopics
       )(_.name)
 
-      val responseBuilder = new OffsetCommitResponse.Builder()
+      val responseBuilder = OffsetCommitResponse.newBuilder(offsetCommitRequest.version())
       val authorizedTopicsRequest = new mutable.ArrayBuffer[OffsetCommitRequestData.OffsetCommitRequestTopic]()
       offsetCommitRequest.data.topics.forEach { topic =>
-        if (!authorizedTopics.contains(topic.name)) {
+        if (Utils.isBlank(topic.name)) {

Review Comment:
   I would move this up and do it in the first iteration.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1450155072

   Thanks David, included all your comments in the PR. I now work on:
   - How to limit the highest version of the OffsetCommit API to 8 in the admin client;
   - Integration test mentioned in a prior review.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1130671265


##########
clients/src/main/java/org/apache/kafka/common/TopicIdAndNameBiMapping.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common;
+
+import org.apache.kafka.common.errors.InvalidTopicException;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Encapsulates the mapping between topic names and ids assuming a 1:1 relationship between
+ * a name and an id.
+ * <p></p>
+ * Note that this class intends to be used for the (reverse) lookup of topic IDs/names, but
+ * not to characterize the set of topics which are known by a client. Use the
+ * {@link org.apache.kafka.clients.MetadataCache} for that purpose.
+ */
+public class TopicIdAndNameBiMapping {
+    private final Map<String, Uuid> topicIds;
+    private final Map<Uuid, String> topicNames;
+
+    /**
+     * A mapping which universe of topic ids and names is captured from the input map. The reverse association
+     * between a topic ID and a topic name is computed by this method. If there are more than one topic name
+     * resolving to the same topic ID, an {@link InvalidTopicException} is thrown.
+     */
+    public static TopicIdAndNameBiMapping fromTopicIds(Map<String, Uuid> topicIds) {

Review Comment:
   Both names are fine for me. I leave it up to you.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1130670740


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -119,28 +131,52 @@ public boolean shouldClientThrottle(short version) {
         return version >= 4;
     }
 
+    public short version() {
+        return version;
+    }
+
     public static class Builder {
         OffsetCommitResponseData data = new OffsetCommitResponseData();
         HashMap<String, OffsetCommitResponseTopic> byTopicName = new HashMap<>();
+        private final TopicResolver topicResolver;
+        private final short version;
+
+        public Builder(TopicResolver topicResolver, short version) {

Review Comment:
   I actually wonder if we should do it the other way around. We could do KAFKA-14793 first, merge it, and update this one accordingly. Without KAFKA-14793, the contract of the not really respected and it feels a bit weird to work around it here instead of fixing the real issue. Is KAFKA-14793 complicated? What do you think?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1168675481


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1283,10 +1289,16 @@ RequestFuture<Void> sendOffsetCommitRequest(final Map<TopicPartition, OffsetAndM
                 return RequestFuture.failure(new IllegalArgumentException("Invalid offset: " + offsetAndMetadata.offset()));
             }
 
+            Uuid topicId = topicIdAndNames.getTopicIdOrZero(topicPartition.topic());

Review Comment:
   nit: `topicIdOrZero`?



##########
clients/src/main/java/org/apache/kafka/common/TopicIdAndNameBiMap.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common;
+
+import org.apache.kafka.common.errors.InvalidTopicException;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Encapsulates the mapping between topic names and ids assuming a 1:1 relationship between
+ * a name and an id.
+ * <p></p>
+ * Note that this class intends to be used for the (reverse) lookup of topic IDs/names, but
+ * not to characterize the set of topics which are known by a client. Use the
+ * {@link org.apache.kafka.clients.MetadataCache} for that purpose.
+ */
+public class TopicIdAndNameBiMap {
+    private final Map<String, Uuid> topicIds;
+    private final Map<Uuid, String> topicNames;
+
+    /**
+     * A mapping which universe of topic ids and names is captured from the input map. The reverse association
+     * between a topic ID and a topic name is computed by this method. If there are more than one topic name
+     * resolving to the same topic ID, an {@link InvalidTopicException} is thrown.
+     */
+    public static TopicIdAndNameBiMap fromTopicIds(Map<String, Uuid> topicIds) {
+        Map<Uuid, String> topicNames = new HashMap<>(topicIds.size());
+
+        for (Map.Entry<String, Uuid> e: topicIds.entrySet()) {
+            String conflicting = topicNames.putIfAbsent(e.getValue(), e.getKey());
+            if (conflicting != null) {
+                throw new IllegalStateException(
+                        "Topic " + e.getKey() + " shares the same ID " + e.getValue() + " as topic " + conflicting);
+            }
+        }
+
+        return new TopicIdAndNameBiMap(topicIds, topicNames);
+    }
+
+    /**
+     * A mapping which acts as a wrapper around the input mapping of topic ids from/to topic names.
+     * No validation is performed about the consistency of the mapping. This method is to be preferred
+     * when the copy of the input maps needs to be avoided.
+     */
+    public static TopicIdAndNameBiMap wrap(Map<String, Uuid> topicIds, Map<Uuid, String> topicNames) {
+        return new TopicIdAndNameBiMap(topicIds, topicNames);
+    }
+
+    /**
+     * Used when no mapping between topic name and id exists.
+     */
+    public static TopicIdAndNameBiMap emptyMapping() {
+        return fromTopicIds(Collections.emptyMap());
+    }
+
+    private TopicIdAndNameBiMap(Map<String, Uuid> topicIds, Map<Uuid, String> topicNames) {
+        this.topicIds = Collections.unmodifiableMap(topicIds);
+        this.topicNames = Collections.unmodifiableMap(topicNames);
+    }
+
+    /**
+     * Returns the ID of the topic with the given name, if that association exists.
+     */
+    public Uuid getTopicIdOrZero(String name) {
+        return Optional.ofNullable(topicIds.get(name)).orElse(Uuid.ZERO_UUID);

Review Comment:
   nit: Should we replace `ofNullable` by a simple `if/else` statement? Allocating an optional does not seem necessary here.



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2707,11 +2742,271 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_MEMBER_ID);
         assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testCommitOffsetUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+            prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_TOPIC_ID));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertFalse(coordinator.commitOffsetsSync(offsets, time.timer(timeoutMs)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testRetryCommitUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.UNKNOWN_TOPIC_ID)));
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.NONE)));
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertTrue(coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));
+

Review Comment:
   nit: This empty line could be removed.



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -3981,23 +4348,13 @@ private void prepareJoinAndSyncResponse(String consumerId, int generation, List<
         }, syncGroupResponse(assignment, Errors.NONE));
     }
 
-    private Map<TopicPartition, Errors> partitionErrors(Collection<TopicPartition> partitions, Errors error) {
-        final Map<TopicPartition, Errors> errors = new HashMap<>();
-        for (TopicPartition partition : partitions) {
-            errors.put(partition, error);
-        }
-        return errors;
-    }
-
-    private void respondToOffsetCommitRequest(final Map<TopicPartition, Long> expectedOffsets, Errors error) {
-        Map<TopicPartition, Errors> errors = partitionErrors(expectedOffsets.keySet(), error);
-        client.respond(offsetCommitRequestMatcher(expectedOffsets), offsetCommitResponse(errors));
-    }
-
-    private MockClient.RequestMatcher offsetCommitRequestMatcher(final Map<TopicPartition, Long> expectedOffsets) {
+    private MockClient.RequestMatcher offsetCommitRequestMatcher(
+        Map<TopicPartition, Long> expectedOffsets,
+        Consumer<OffsetCommitRequest> requestConsumer

Review Comment:
   I am not sure to follow why we need this `Consumer` here. Couldn't we just have a matcher which verifies what we want/need?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1331,21 +1343,28 @@ RequestFuture<Void> sendOffsetCommitRequest(final Map<TopicPartition, OffsetAndM
                         .setGenerationId(generation.generationId)
                         .setMemberId(generation.memberId)
                         .setGroupInstanceId(groupInstanceId)
-                        .setTopics(new ArrayList<>(requestTopicDataMap.values()))
+                        .setTopics(new ArrayList<>(requestTopicDataMap.values())),
+                canUseTopicIds
         );
 
         log.trace("Sending OffsetCommit request with {} to coordinator {}", offsets, coordinator);
 
         return client.send(coordinator, builder)
-                .compose(new OffsetCommitResponseHandler(offsets, generation));
+                .compose(new OffsetCommitResponseHandler(offsets, generation, topicIdAndNames));
     }
 
     private class OffsetCommitResponseHandler extends CoordinatorResponseHandler<OffsetCommitResponse, Void> {
         private final Map<TopicPartition, OffsetAndMetadata> offsets;
+        private final TopicIdAndNameBiMap topicIdAndNames;
 
-        private OffsetCommitResponseHandler(Map<TopicPartition, OffsetAndMetadata> offsets, Generation generation) {
+        private OffsetCommitResponseHandler(
+                Map<TopicPartition, OffsetAndMetadata> offsets,
+                Generation generation,
+                TopicIdAndNameBiMap topicIdAndNames

Review Comment:
   nit: Indentation seems to be off here.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1354,8 +1373,21 @@ public void handle(OffsetCommitResponse commitResponse, RequestFuture<Void> futu
             Set<String> unauthorizedTopics = new HashSet<>();
 
             for (OffsetCommitResponseData.OffsetCommitResponseTopic topic : commitResponse.data().topics()) {
+                String topicName = topic.name();
+
+                if (this.response.requestHeader().apiVersion() >= 9) {
+                    topicName = topicIdAndNames.getTopicName(topic.topicId()).orElse(null);

Review Comment:
   nit: `topicNameOrNull` and get rid of the `Optional`?



##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -46,6 +49,8 @@
  *   - {@link Errors#INVALID_COMMIT_OFFSET_SIZE}
  *   - {@link Errors#TOPIC_AUTHORIZATION_FAILED}
  *   - {@link Errors#GROUP_AUTHORIZATION_FAILED}
+ *   - {@link Errors#UNKNOWN_MEMBER_ID}

Review Comment:
   nit: This one is already in the list (L47).



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -129,16 +150,19 @@
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
 
 public abstract class ConsumerCoordinatorTest {
-    private final String topic1 = "test1";
-    private final String topic2 = "test2";
-    private final TopicPartition t1p = new TopicPartition(topic1, 0);
-    private final TopicPartition t2p = new TopicPartition(topic2, 0);
-    private final String groupId = "test-group";
+    private static String topic1 = "test1";
+    private static String topic2 = "test2";
+    private static TopicPartition t1p = new TopicPartition(topic1, 0);
+    private static TopicIdPartition ti1p = new TopicIdPartition(Uuid.randomUuid(), t1p);

Review Comment:
   nit: Could we try to combine those? `private static TopicIdPartition t1p = new new TopicIdPartition(Uuid.randomUuid(), 0, topic1)`?



##########
clients/src/main/java/org/apache/kafka/common/TopicIdAndNameBiMap.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common;
+
+import org.apache.kafka.common.errors.InvalidTopicException;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Encapsulates the mapping between topic names and ids assuming a 1:1 relationship between
+ * a name and an id.
+ * <p></p>

Review Comment:
   nit: Should we remove `<p></p>`?



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -159,13 +183,22 @@ public abstract class ConsumerCoordinatorTest {
     private final String consumerId = "consumer";
     private final String consumerId2 = "consumer2";
 
+    private final Map<String, Uuid> topicIds = new HashMap<>();
+    {
+        topicIds.put(topic1, ti1p.topicId());
+        topicIds.put(topic2, ti2p.topicId());
+    }
+
+    private final Map<String, Integer> partitionCounts = new HashMap<>();
+    {
+        partitionCounts.put(topic1, 1);
+        partitionCounts.put(topic2, 1);
+    }
+
+    private final TopicIdAndNameBiMap topicIdAndNames = TopicIdAndNameBiMap.fromTopicIds(topicIds);

Review Comment:
   nit: `topicIdAndNameBiMapping`?



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2707,11 +2742,271 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_MEMBER_ID);
         assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testCommitOffsetUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+            prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_TOPIC_ID));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertFalse(coordinator.commitOffsetsSync(offsets, time.timer(timeoutMs)));
+

Review Comment:
   nit: This empty line could be removed.



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2707,11 +2742,271 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_MEMBER_ID);
         assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testCommitOffsetUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+            prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_TOPIC_ID));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertFalse(coordinator.commitOffsetsSync(offsets, time.timer(timeoutMs)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testRetryCommitUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.UNKNOWN_TOPIC_ID)));
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.NONE)));
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertTrue(coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                // Unlike the commit offset sync API, the async API does not retry.

Review Comment:
   If the outcome of the test is different in this case, isn't it a bit weird to combine them in the same unit test?



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2707,11 +2742,271 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_MEMBER_ID);
         assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testCommitOffsetUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+            prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_TOPIC_ID));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertFalse(coordinator.commitOffsetsSync(offsets, time.timer(timeoutMs)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testRetryCommitUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.UNKNOWN_TOPIC_ID)));
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.NONE)));
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertTrue(coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                // Unlike the commit offset sync API, the async API does not retry.
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    static Stream<Arguments> commitOffsetTestArgs() {

Review Comment:
   This goes a bit too far in my opinion. We usually prefer to have simpler parameterized tests. Could we simplify this somehow and bring stuck back in the main unit test?



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2707,11 +2742,271 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_MEMBER_ID);
         assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testCommitOffsetUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+            prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_TOPIC_ID));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertFalse(coordinator.commitOffsetsSync(offsets, time.timer(timeoutMs)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testRetryCommitUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.UNKNOWN_TOPIC_ID)));
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.NONE)));
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertTrue(coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                // Unlike the commit offset sync API, the async API does not retry.
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    static Stream<Arguments> commitOffsetTestArgs() {
+        Map<TopicIdPartition, Long> byTopicIdOffsets = new HashMap<>();
+        byTopicIdOffsets.put(ti1p, 100L);
+        byTopicIdOffsets.put(ti2p, 200L);
+
+        TopicIdPartition unknownTopicIdPartition =
+            new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("topic3", 5));
+
+        Map<TopicIdPartition, Long> byTopicNameOffsets = new HashMap<>();
+        byTopicNameOffsets.put(ti1p, 100L);
+        byTopicNameOffsets.put(ti2p, 200L);
+        byTopicNameOffsets.put(unknownTopicIdPartition, 300L);
+
+        OffsetCommitRequestData byTopicIdData = new OffsetCommitRequestData()
+            .setGroupId(groupId)
+            .setGenerationId(OffsetCommitRequest.DEFAULT_GENERATION_ID)
+            .setTopics(Arrays.asList(
+                new OffsetCommitRequestTopic()
+                    .setTopicId(ti1p.topicId())
+                    .setName(topic1)
+                    .setPartitions(singletonList(new OffsetCommitRequestPartition()
+                        .setPartitionIndex(t1p.partition())
+                        .setCommittedOffset(100L)
+                        .setCommittedMetadata("metadata"))),
+                new OffsetCommitRequestTopic()
+                    .setTopicId(ti2p.topicId())
+                    .setName(topic2)
+                    .setPartitions(singletonList(new OffsetCommitRequestPartition()
+                        .setPartitionIndex(t2p.partition())
+                        .setCommittedOffset(200L)
+                        .setCommittedMetadata("metadata")))
+            ));
+
+        OffsetCommitRequestData byTopicNameData = byTopicIdData.duplicate();
+        byTopicNameData.topics().add(new OffsetCommitRequestTopic()
+            .setName(unknownTopicIdPartition.topic())
+            .setPartitions(singletonList(new OffsetCommitRequestPartition()
+                .setPartitionIndex(5)
+                .setCommittedOffset(300L)
+                .setCommittedMetadata("metadata")))
+        );
+
+        return Stream.of(
+            Arguments.of(true, byTopicIdOffsets, byTopicIdData, (short) 9),
+            Arguments.of(false, byTopicIdOffsets, byTopicIdData, (short) 9),
+            Arguments.of(true, byTopicNameOffsets, byTopicNameData, (short) 8),
+            Arguments.of(false, byTopicNameOffsets, byTopicNameData, (short) 8)
+        );
+    }
+
+    private static Map<TopicPartition, OffsetAndMetadata> offsetAndMetadata(Map<TopicIdPartition, Long> offsets) {
+        return offsets.entrySet().stream()
+            .map(e -> new SimpleEntry<>(e.getKey().topicPartition(), new OffsetAndMetadata(e.getValue(), "metadata")))
+            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+    }
+
+    @ParameterizedTest
+    @MethodSource("commitOffsetTestArgs")
+    public void testTopicIdsArePopulatedByTheConsumerCoordinator(
+            boolean commitSync,
+            Map<TopicIdPartition, Long> offsets,
+            OffsetCommitRequestData expectedRequestData,
+            short expectedRequestVersion) {
+
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        OffsetCommitRequestCaptor captor = new OffsetCommitRequestCaptor();
+        prepareOffsetCommitRequest(offsets, Errors.NONE, false, captor);
+
+        Map<TopicPartition, OffsetAndMetadata> input = offsetAndMetadata(offsets);
+
+        if (commitSync) {
+            assertTrue(coordinator.commitOffsetsSync(input, time.timer(Long.MAX_VALUE)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(input, (inputOffsets, exception) -> {
+                // Notes:
+                // 1) The offsets passed to the callback are the same object provided to the offset commit method.
+                //    The validation on the offsets is not required but defensive.
+                // 2) We validate that the commit was successful, which is the case if the exception is null.
+                // 3) We validate this callback was invoked, which is not necessary but defensive.
+                assertSame(inputOffsets, input);
+                assertNull(exception);
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+
+        // The consumer does not provide a guarantee on the order of occurrence of topics and partitions in the
+        // OffsetCommit request, since a map of offsets is provided to the consumer API. Here, both requests
+        // are asserted to be identical irrespective of the order in which topic and partitions appear in the requests.
+        assertRequestEquals(
+            new OffsetCommitRequest(expectedRequestData, expectedRequestVersion),
+            captor.request
+        );
+    }
+
+    @ParameterizedTest
+    @NullSource
+    @ValueSource(strings = { "", "test1" })
+    public void testInvalidTopicIdReturnedByBrokerWhenCommittingOffsetSync(String topicName) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(t1p, new OffsetAndMetadata(100L, "m"));
+
+        // The following offset commit response is valid and the authorization failure results in failing the
+        // offset commit invocation.
+        client.prepareResponse(offsetCommitResponse(topicName, ti1p.topicId(), Errors.GROUP_AUTHORIZATION_FAILED));
+        assertThrows(GroupAuthorizationException.class,
+            () -> coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));
+
+        // The following offset commit response defines a topic incorrectly. The coordinator ignores the topic,
+        // and the group authorization failure is therefore not propagated.
+        client.prepareResponse(offsetCommitResponse(topicName, Uuid.ZERO_UUID, Errors.GROUP_AUTHORIZATION_FAILED));
+        assertTrue(coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));

Review Comment:
   This one made me think that we are probably not doing the right thing in the implementation. In this particular case, if we have only one committed offset and we don't have a response for it because the topic id is wrong, I think that `commitOffsetsSync` should not succeed because we actually don't know if the offset was committed or not. What do you think?
   
   One way around this would be to verify that we have received a response for each topic-partitions.



##########
clients/src/test/java/org/apache/kafka/common/requests/OffsetCommitRequestTest.java:
##########
@@ -54,89 +68,204 @@ public class OffsetCommitRequestTest {
     protected static int throttleTimeMs = 10;
 
     private static OffsetCommitRequestData data;
-    private static List<OffsetCommitRequestTopic> topics;
+    private static OffsetCommitRequestPartition requestPartitionOne;
+    private static OffsetCommitRequestPartition requestPartitionTwo;
 
     @BeforeEach
     public void setUp() {
-        topics = Arrays.asList(
-            new OffsetCommitRequestTopic()
-                .setName(topicOne)
-                .setPartitions(Collections.singletonList(
-                    new OffsetCommitRequestPartition()
-                        .setPartitionIndex(partitionOne)
-                        .setCommittedOffset(offset)
-                        .setCommittedLeaderEpoch(leaderEpoch)
-                        .setCommittedMetadata(metadata)
-                )),
-            new OffsetCommitRequestTopic()
-                .setName(topicTwo)
-                .setPartitions(Collections.singletonList(
-                    new OffsetCommitRequestPartition()
-                        .setPartitionIndex(partitionTwo)
-                        .setCommittedOffset(offset)
-                        .setCommittedLeaderEpoch(leaderEpoch)
-                        .setCommittedMetadata(metadata)
-                ))
-        );
+        requestPartitionOne = new OffsetCommitRequestPartition()
+            .setPartitionIndex(partitionOne)
+            .setCommittedOffset(offset)
+            .setCommittedLeaderEpoch(leaderEpoch)
+            .setCommittedMetadata(metadata);
+
+        requestPartitionTwo = new OffsetCommitRequestPartition()
+            .setPartitionIndex(partitionTwo)
+            .setCommittedOffset(offset)
+            .setCommittedLeaderEpoch(leaderEpoch)
+            .setCommittedMetadata(metadata);
+
         data = new OffsetCommitRequestData()
-                   .setGroupId(groupId)
-                   .setTopics(topics);
+           .setGroupId(groupId)
+           .setTopics(Arrays.asList(
+               new OffsetCommitRequestTopic()
+                   .setTopicId(topicOneId)
+                   .setName(topicOne)
+                   .setPartitions(Collections.singletonList(requestPartitionOne)),
+               new OffsetCommitRequestTopic()
+                   .setTopicId(topicTwoId)
+                   .setName(topicTwo)
+                   .setPartitions(Collections.singletonList(requestPartitionTwo))
+           ));
     }
 
-    @Test
-    public void testConstructor() {
-        Map<TopicPartition, Long> expectedOffsets = new HashMap<>();
-        expectedOffsets.put(new TopicPartition(topicOne, partitionOne), offset);
-        expectedOffsets.put(new TopicPartition(topicTwo, partitionTwo), offset);
+    public static Map<TopicPartition, Long> offsets(
+        OffsetCommitRequest request,
+        TopicIdAndNameBiMap topicIdAndNames
+    ) {
+        Map<TopicPartition, Long> offsets = new HashMap<>();
+        for (OffsetCommitRequestTopic topic : request.data().topics()) {
+            String topicName = topic.name();
 
-        OffsetCommitRequest.Builder builder = new OffsetCommitRequest.Builder(data);
+            if (request.version() >= 9) {
+                topicName = topicIdAndNames.getTopicName(topic.topicId()).orElseThrow(
+                    () -> new UnknownTopicIdException("Topic with ID " + topic.topicId() + " not found."));
+            }
 
-        for (short version : ApiKeys.TXN_OFFSET_COMMIT.allVersions()) {
-            OffsetCommitRequest request = builder.build(version);
-            assertEquals(expectedOffsets, request.offsets());
+            for (OffsetCommitRequestData.OffsetCommitRequestPartition partition : topic.partitions()) {
+                offsets.put(new TopicPartition(topicName, partition.partitionIndex()), partition.committedOffset());
+            }
+        }
+        return offsets;
+    }
 
-            OffsetCommitResponse response = request.getErrorResponse(throttleTimeMs, Errors.NOT_COORDINATOR.exception());
+    @ParameterizedTest
+    @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_COMMIT)
+    public void testConstructor(short version) {
+        OffsetCommitRequest request = new OffsetCommitRequest.Builder(data, true).build(version);
+        OffsetCommitResponse response = request.getErrorResponse(throttleTimeMs, Errors.NOT_COORDINATOR.exception());
 
-            assertEquals(Collections.singletonMap(Errors.NOT_COORDINATOR, 2), response.errorCounts());
-            assertEquals(throttleTimeMs, response.throttleTimeMs());
-        }
+        assertEquals(data, request.data());
+        assertEquals(Collections.singletonMap(Errors.NOT_COORDINATOR, 2), response.errorCounts());
+        assertEquals(throttleTimeMs, response.throttleTimeMs());
     }
 
     @Test
     public void testGetErrorResponseTopics() {
         List<OffsetCommitResponseTopic> expectedTopics = Arrays.asList(
             new OffsetCommitResponseTopic()
                 .setName(topicOne)
+                .setTopicId(topicOneId)
                 .setPartitions(Collections.singletonList(
                     new OffsetCommitResponsePartition()
                         .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
                         .setPartitionIndex(partitionOne))),
             new OffsetCommitResponseTopic()
                 .setName(topicTwo)
+                .setTopicId(topicTwoId)
                 .setPartitions(Collections.singletonList(
                     new OffsetCommitResponsePartition()
                         .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
                         .setPartitionIndex(partitionTwo)))
         );
-        assertEquals(expectedTopics, getErrorResponseTopics(topics, Errors.UNKNOWN_MEMBER_ID));
+        assertEquals(expectedTopics, getErrorResponseTopics(data.topics(), Errors.UNKNOWN_MEMBER_ID));
     }
 
-    @Test
-    public void testVersionSupportForGroupInstanceId() {
+    @ParameterizedTest
+    @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_COMMIT)
+    public void testVersionSupportForGroupInstanceId(short version) {
         OffsetCommitRequest.Builder builder = new OffsetCommitRequest.Builder(
             new OffsetCommitRequestData()
                 .setGroupId(groupId)
                 .setMemberId(memberId)
-                .setGroupInstanceId(groupInstanceId)
+                .setGroupInstanceId(groupInstanceId),
+            true
         );
 
-        for (short version : ApiKeys.OFFSET_COMMIT.allVersions()) {
-            if (version >= 7) {
-                builder.build(version);
-            } else {
-                final short finalVersion = version;
-                assertThrows(UnsupportedVersionException.class, () -> builder.build(finalVersion));
-            }
+        if (version >= 7) {
+            builder.build(version);
+        } else {
+            final short finalVersion = version;
+            assertThrows(UnsupportedVersionException.class, () -> builder.build(finalVersion));
+        }
+    }
+
+    @ParameterizedTest
+    @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_COMMIT)
+    public void testResolvesTopicNameIfRequiredWhenListingOffsets(short version) {
+        OffsetCommitRequest request = new OffsetCommitRequest.Builder(data, true).build(version);
+        List<OffsetCommitRequestTopic> topics = request.data().topics();
+
+        assertEquals(2, topics.stream().flatMap(t -> t.partitions().stream()).count());
+        assertEquals(requestPartitionOne, topics.get(0).partitions().get(0));
+        assertEquals(requestPartitionTwo, topics.get(1).partitions().get(0));
+    }
+
+    @Test
+    public void testUnresolvableTopicIdWhenListingOffset() {
+        OffsetCommitRequest request = new OffsetCommitRequest.Builder(data.duplicate(), true).build((short) 9);
+        assertThrows(UnknownTopicIdException.class,
+            () -> OffsetCommitRequestTest.offsets(request, TopicIdAndNameBiMap.emptyMapping()));
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void maxAllowedVersionIsEightIfRequestCannotUseTopicIds(boolean canUseTopicIds) {
+        OffsetCommitRequest.Builder builder = new OffsetCommitRequest.Builder(data.duplicate(), canUseTopicIds);
+        assertEquals(canUseTopicIds ? 9 : 8, builder.build(builder.latestAllowedVersion()).version());
+    }
+
+    /**
+     * Compares the two {@link OffsetCommitRequest} independently of the order in which the
+     * {@link OffsetCommitRequestTopic} and {@link OffsetCommitRequestPartition} are defined in the response.
+     */
+    public static void assertRequestEquals(OffsetCommitRequest expectedRequest, OffsetCommitRequest actualRequest) {
+        if (expectedRequest.version() > 9 || actualRequest.version() > 9) {
+            throw new AssertionError("A new version of OffsetCommitRequest has been detected. Please " +
+                    "review the equality contract enforced here and add/remove fields accordingly.");
         }
+
+        OffsetCommitRequestData expected = expectedRequest.data();
+        OffsetCommitRequestData actual = actualRequest.data();
+
+        assertEquals(expectedRequest.version(), actualRequest.version());
+        assertEquals(expected.groupId(), actual.groupId(), "Group id mismatch");
+        assertEquals(expected.groupInstanceId(), actual.groupInstanceId(), "Group instance id mismatch");
+        assertEquals(expected.generationId(), actual.generationId(), "Generation id mismatch");
+        assertEquals(expected.memberId(), actual.memberId(), "Member id mismatch");
+        assertEquals(expected.retentionTimeMs(), actual.retentionTimeMs(), "Retention time mismatch");
+
+        Function<OffsetCommitRequestTopic, List<OffsetCommitRequestPartition>> partitionSelector =
+            OffsetCommitRequestTopic::partitions;
+
+        Function<OffsetCommitRequestTopic, NameAndId> topicClassifier =
+            topic -> new NameAndId(topic.name(), topic.topicId());
+
+        BiFunction<NameAndId, OffsetCommitRequestPartition, TopicIdPartition> partitionClassifier =
+            (nameAndId, p) -> new TopicIdPartition(nameAndId.id(), p.partitionIndex(), nameAndId.name());
+
+        Function<OffsetCommitRequestData, Map<TopicIdPartition, OffsetCommitRequestPartition>> partitioner =
+            request -> partition(request.topics(), partitionSelector, topicClassifier, partitionClassifier);
+
+        assertEquals(partitioner.apply(expected), partitioner.apply(actual));
+    }
+
+    /**
+     * Compares the two {@link OffsetCommitResponse} independently of the order in which the
+     * {@link OffsetCommitResponseTopic} and {@link OffsetCommitResponsePartition} are defined in the response.
+     */
+    public static void assertResponseEquals(OffsetCommitResponse expected, OffsetCommitResponse actual) {
+        assertEquals(expected.throttleTimeMs(), actual.throttleTimeMs());
+        assertEquals(expected.errorCounts(), actual.errorCounts());
+
+        Function<OffsetCommitResponseTopic, List<OffsetCommitResponsePartition>> partitionSelector =
+            OffsetCommitResponseTopic::partitions;
+
+        Function<OffsetCommitResponseTopic, NameAndId> topicClassifier =
+            topic -> new NameAndId(topic.name(), topic.topicId());
+
+        BiFunction<NameAndId, OffsetCommitResponsePartition, TopicIdPartition> partitionClassifier =
+            (nameAndId, p) -> new TopicIdPartition(nameAndId.id(), p.partitionIndex(), nameAndId.name());
+
+        Function<OffsetCommitResponse, Map<TopicIdPartition, OffsetCommitResponsePartition>> partitioner =
+            response -> partition(response.data().topics(), partitionSelector, topicClassifier, partitionClassifier);
+
+        assertEquals(partitioner.apply(expected), partitioner.apply(actual));
+    }
+
+    private static <T, P> Map<TopicIdPartition, P> partition(

Review Comment:
   This seems to be a quite complicated way to group `OffsetCommitRequestPartition` or `OffsetCommitResponsePartition` by `TopicIdPartition`, no? I would just write two methods to do just this.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1116198012


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -151,11 +154,12 @@ public Builder addPartition(
 
         public <P> Builder addPartitions(
             String topicName,
+            Uuid topicId,
             List<P> partitions,
             Function<P, Integer> partitionIndex,
             Errors error
         ) {
-            final OffsetCommitResponseTopic topicResponse = getOrCreateTopic(topicName);
+            final OffsetCommitResponseTopic topicResponse = getOrCreateTopic(topicName, topicId);

Review Comment:
   You are right, thanks for finding this bug (again!). I followed the approach you suggest here in the builder of the `OffsetCommitResponse`, please let me know if the semantics make sense.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1204151150


##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.KafkaApisTest.{NameAndId, newOffsetCommitRequestData, newOffsetCommitResponseData}
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetCommitRequestTest.assertResponseEquals
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
+
+import java.util.Optional.empty
+import java.util.Properties
+import scala.collection.{Map, Seq}
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava}
+
+class OffsetCommitRequestTest extends BaseRequestTest {
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val groupId = "groupId"
+
+  var consumer: KafkaConsumer[_, _] = _
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    consumer = createConsumer()
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    if (consumer != null)
+      Utils.closeQuietly(consumer, "KafkaConsumer")
+    super.tearDown()
+  }
+
+  def createTopics(topicNames: String*): Seq[NameAndId] = {
+    topicNames.map(topic => {
+      createTopic(topic)
+      val topicId: Uuid = getTopicIds().get(topic) match {
+        case Some(x) => x
+        case _ => throw new AssertionError("Topic ID not found for " + topic)
+      }
+      NameAndId(topic, topicId)
+    })
+  }
+
+
+  @Test
+  def testTopicIdsArePopulatedInOffsetCommitResponses(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")

Review Comment:
   @dajac Sure, I removed the method `createTopics` and use `getTopicIds` instead.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1741511578

   Closing this PR for now as the topic id work will be done later. We can re-open it when we resume the work.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] clolov commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "clolov (via GitHub)" <gi...@apache.org>.
clolov commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1197988726


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -3981,23 +4348,13 @@ private void prepareJoinAndSyncResponse(String consumerId, int generation, List<
         }, syncGroupResponse(assignment, Errors.NONE));
     }
 
-    private Map<TopicPartition, Errors> partitionErrors(Collection<TopicPartition> partitions, Errors error) {
-        final Map<TopicPartition, Errors> errors = new HashMap<>();
-        for (TopicPartition partition : partitions) {
-            errors.put(partition, error);
-        }
-        return errors;
-    }
-
-    private void respondToOffsetCommitRequest(final Map<TopicPartition, Long> expectedOffsets, Errors error) {
-        Map<TopicPartition, Errors> errors = partitionErrors(expectedOffsets.keySet(), error);
-        client.respond(offsetCommitRequestMatcher(expectedOffsets), offsetCommitResponse(errors));
-    }
-
-    private MockClient.RequestMatcher offsetCommitRequestMatcher(final Map<TopicPartition, Long> expectedOffsets) {
+    private MockClient.RequestMatcher offsetCommitRequestMatcher(
+        Map<TopicPartition, Long> expectedOffsets,
+        Consumer<OffsetCommitRequest> requestConsumer

Review Comment:
   From my reading of the code this consumer is a captor. We validate some of the things in this method and we validate the overall captured value elsewhere in individual tests. I am not too certain how this can be simplified to just a matcher to be honest.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1204079800


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2707,11 +2742,271 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_MEMBER_ID);
         assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testCommitOffsetUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+            prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_TOPIC_ID));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertFalse(coordinator.commitOffsetsSync(offsets, time.timer(timeoutMs)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testRetryCommitUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.UNKNOWN_TOPIC_ID)));
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.NONE)));
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertTrue(coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                // Unlike the commit offset sync API, the async API does not retry.
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    static Stream<Arguments> commitOffsetTestArgs() {

Review Comment:
   Did the change to provide higher cohesion.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1124559198


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -167,8 +213,24 @@ public <P> Builder addPartitions(
         }
 
         public Builder merge(
-            OffsetCommitResponseData newData
+            OffsetCommitResponseData newData,
+            Logger logger
         ) {
+            if (version >= 9) {

Review Comment:
   Not strictly needed. We can remove the condition and the logger as well.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1454676488

   Hello David, found a case where the use of OffsetCommit requests version 9 from the consumer resulted in an error in the ACL authorization tests. The reason is that when the commit offsets are performed by the tests to exercise ACLs, the topic IDs are not yet present in the consumer metadata cache, in this case because the consumer hasn't subscribed to the topics or be assigned any of their partitions. As a result, the OffsetCommitRequest version 9 was sent with zero topic ids.
   
   In order to avoid this, we could:
   
   1. Enforce a metadata update for any topic in the offsets to commit, using the transient topics list exposed by the `ConsumerMetadata` and used for instance by the `OffsetFetcher` to fetch offsets for out-of-band topic-partitions which aren't part of any subscription. This approach however adds complexity to the offset commit implementation in the consumer and only address this specific use case. There could be other cases where metadata hasn't converged yet and the topic id would not be available to the consumer.
   
   2. Adds a condition on the support of topic ids when constructing the `OffsetCommitRequest`. This is the approach used when constructing the `Fetch` request. The advantage of the approach is that it provides the invariant that any OffsetCommitRequest version >= 9 have valid (non-zero) topic ids in it. One downside is that if a bug in the consumer makes a topic id unavailable, the consumer will keep using version <= 8 permanently and silently, while we would want to know about it as implementors to address any potential gap.
   
   The second approach is currently implemented in the PR. Happy to discuss more about it.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1125501983


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1328,13 +1335,16 @@ RequestFuture<Void> sendOffsetCommitRequest(final Map<TopicPartition, OffsetAndM
             groupInstanceId = null;
         }
 
+        boolean canUseTopicIds = topicPartitionsWithoutTopicId == 0;

Review Comment:
   Should we add tests to cover this new logic?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1121499577


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -188,12 +188,73 @@ public Builder merge(
                     }
                 });
             }
-
             return this;
         }
 
-        public OffsetCommitResponse build() {
+        public final OffsetCommitResponse build() {
             return new OffsetCommitResponse(data);
         }
+
+        protected abstract void validate(String topicName, Uuid topicId);
+
+        protected abstract T classifer(String topicName, Uuid topicId);
+
+        protected abstract OffsetCommitResponseTopic newTopic(String topicName, Uuid topicId);
+        private OffsetCommitResponseTopic getOrCreateTopic(String topicName, Uuid topicId) {
+            T topicKey = classifer(topicName, topicId);
+            OffsetCommitResponseTopic topic = topics.get(topicKey);
+            if (topic == null) {
+                topic = newTopic(topicName, topicId);
+                data.topics().add(topic);
+                topics.put(topicKey, topic);
+            }
+            return topic;
+        }
+    }
+
+    public static final class BuilderByTopicId extends Builder<Uuid> {
+        protected BuilderByTopicId(short version) {
+            super(version);
+        }
+
+        @Override
+        protected void validate(String topicName, Uuid topicId) {
+            if (topicId == null || Uuid.ZERO_UUID.equals(topicId))
+                throw new UnsupportedVersionException("OffsetCommitResponse version " + version +
+                        " does not support zero topic IDs.");
+        }
+
+        @Override
+        protected Uuid classifer(String topicName, Uuid topicId) {
+            return topicId;
+        }
+
+        @Override
+        protected OffsetCommitResponseTopic newTopic(String topicName, Uuid topicId) {
+            return new OffsetCommitResponseTopic().setName(null).setTopicId(topicId);

Review Comment:
   We could get this in the base class and always set both of them. The serialization framework knows what to do.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1129482786


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -167,8 +213,24 @@ public <P> Builder addPartitions(
         }
 
         public Builder merge(
-            OffsetCommitResponseData newData
+            OffsetCommitResponseData newData,
+            Logger logger
         ) {
+            if (version >= 9) {
+                // This method is called after the group coordinator committed the offsets. The group coordinator
+                // provides the OffsetCommitResponseData it built in the process. As of now, this data does
+                // not contain topic ids, so we resolve them here.
+                newData.topics().forEach(topic -> {
+                    Uuid topicId = topicResolver.getTopicId(topic.name()).orElse(Uuid.ZERO_UUID);
+                    if (Uuid.ZERO_UUID.equals(topicId)) {

Review Comment:
   Modified the PR so that the server now sends an `UNKNOWN_SERVER_ERROR` when this happens, in the code moved to the future handler in `KafkaApis`. Would this behaviour be acceptable?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1460209520

   Hi David, thanks for the review. I addressed all your comments and updated the PR. I am just adding a test to add coverage on addition of topic ids in the response generated by the coordinator.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1105985661


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -425,35 +425,72 @@ class KafkaApis(val requestChannel: RequestChannel,
       requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
       CompletableFuture.completedFuture[Unit](())
     } else {
+      val topicNames =
+        if (offsetCommitRequest.version() >= 9)
+          metadataCache.topicIdsToNames()
+        else
+          Collections.emptyMap[Uuid, String]()
+
+      // For version < 9, lookup from topicNames fails and the topic name (which cannot be null) is returned.
+      // For version >= 9, if lookup from topicNames fails, there are two possibilities:
+      //
+      // a) The topic ID was left to default and the topic name should have been populated as a fallback instead.
+      //    If none was provided, null is returned.
+      //
+      // b) The topic ID was not default but is not present in the local topic IDs cache. In this case, because
+      //    clients should make exclusive use of topic name or topic ID, the topic name should be null. If however
+      //    the client provided a topic name, we do not want to use it, because any topic with the same name
+      //    present locally would then have a topic ID which does not match the topic ID in the request.
+      def resolveTopicName(topic: OffsetCommitRequestData.OffsetCommitRequestTopic): String = {
+          val resolvedFromId = topicNames.get(topic.topicId())
+          if (resolvedFromId != null)
+            resolvedFromId
+          else if (offsetCommitRequest.version() < 9 || Uuid.ZERO_UUID.equals(topic.topicId)) {
+            topic.name()
+          } else {
+            null
+          }
+      }
+
       val authorizedTopics = authHelper.filterByAuthorized(
         request.context,
         READ,
         TOPIC,
-        offsetCommitRequest.data.topics.asScala
-      )(_.name)
+        offsetCommitRequest.data.topics.asScala.filter(topic => resolveTopicName(topic) != null)
+      )(resolveTopicName)
 
       val responseBuilder = new OffsetCommitResponse.Builder()
       val authorizedTopicsRequest = new mutable.ArrayBuffer[OffsetCommitRequestData.OffsetCommitRequestTopic]()
       offsetCommitRequest.data.topics.forEach { topic =>
-        if (!authorizedTopics.contains(topic.name)) {
+        val topicName = resolveTopicName(topic)
+        if (topicName == null) {
+          // Topic name cannot be null for version < 9. From version >= 9, topicName is null iff it cannot
+          // be resolved from the local topic IDs cache or topic ID was left to default but no fallback topic
+          // name was provided.
+          responseBuilder.addPartitions[OffsetCommitRequestData.OffsetCommitRequestPartition](

Review Comment:
   Apologies, you are right. This hints that perhaps we should reconstruct the list of `OffsetCommitRequestTopic` and use it internally to avoid any such mistake?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1428396257

   @Hangleton Thanks. I will take a look later this week.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1185779424


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2707,11 +2742,271 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_MEMBER_ID);
         assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testCommitOffsetUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+            prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_TOPIC_ID));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertFalse(coordinator.commitOffsetsSync(offsets, time.timer(timeoutMs)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testRetryCommitUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.UNKNOWN_TOPIC_ID)));
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.NONE)));
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertTrue(coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                // Unlike the commit offset sync API, the async API does not retry.

Review Comment:
   That is right. Christo, maybe you can create two separate tests for these cases and factor in common code in a method?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1131047710


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -119,28 +131,52 @@ public boolean shouldClientThrottle(short version) {
         return version >= 4;
     }
 
+    public short version() {
+        return version;
+    }
+
     public static class Builder {
         OffsetCommitResponseData data = new OffsetCommitResponseData();
         HashMap<String, OffsetCommitResponseTopic> byTopicName = new HashMap<>();
+        private final TopicResolver topicResolver;
+        private final short version;
+
+        public Builder(TopicResolver topicResolver, short version) {

Review Comment:
   Thanks for the answer. If I understand correctly, we would then have a resolution of topic ids from topic-name-based data, so this may not prevent offsets from a topic to be provided as those of another topic with the same name (defined at different point in time in the server)?
   
   The resolution can be done in the group coordinator layer, assuming it has access to the topic id resolved upstream by the request handler. Because we want to preserve the same mapping used when request processing started, we need to ensure the right ids are used within the adaptor's `GroupCoordinator#commitOffsets` method(). Since the mapping returned from the metadata cache depends on the snapshot used at the time the mapping is requested, if the adaptor retrieves it from the metadata cache internally, there is no guarantee the metadata is the same hence that the topic IDs registered with the broker are the same.
   
   This means that the topic ids need to be propagated from the request handler (`KafkaApis`) to the coordinator adaptor somehow. Without a change in the method and contract implemented by the coordinator, these ids could be transferred via the `OffsetCommitRequestData` DTO directly, which means a change in the API schema would be required prior to the change. Alternatively, we may want to change the interface of the coordinator and change the signature of the offset commit method to allow for the propagation of topic ids.
   
   I may be missing the entire thing though?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1449936916

   Sorry, used the wrong button...


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1116196039


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java:
##########
@@ -78,11 +111,18 @@ public OffsetCommitRequestData data() {
         return data;
     }
 
-    public Map<TopicPartition, Long> offsets() {
+    public Map<TopicPartition, Long> offsets(TopicResolver topicResolver) {

Review Comment:
   Sure, I used the auto-generated class in the unit test for the `OffsetCommitRequest`. I moved this method to the unit test class as it is used from other unit tests in for the consumer coordinator, from where using the full-fledged request object would be less convenient.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1442380719

   Many thanks David for the review. I updated the PR to cover all the points you raised. I still have more unit tests to add for the broker-side code paths, need to check if any further change/test is required on the admin client side. Will update the PR with those. Thanks!


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1571989946

   Hello David (@dajac), I was discussing this with Christo today as part of his work on the OffsetFetch API. Would you like this PR on OffsetCommit to be split to make the review easier and reduce risks?


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1116202452


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -425,35 +425,73 @@ class KafkaApis(val requestChannel: RequestChannel,
       requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
       CompletableFuture.completedFuture[Unit](())
     } else {
+      val topicNames =
+        if (offsetCommitRequest.version() >= 9)
+          metadataCache.topicIdsToNames()
+        else
+          Collections.emptyMap[Uuid, String]()
+
+      // For version < 9, lookup from topicNames fails and the topic name (which cannot be null) is returned.
+      // For version >= 9, if lookup from topicNames fails, there are two possibilities:
+      //
+      // a) The topic ID was left to default and the topic name should have been populated as a fallback instead.
+      //    If none was provided, null is returned.
+      //
+      // b) The topic ID was not default but is not present in the local topic IDs cache. In this case, because
+      //    clients should make exclusive use of topic name or topic ID, the topic name should be null. If however
+      //    the client provided a topic name, we do not want to use it, because any topic with the same name
+      //    present locally would then have a topic ID which does not match the topic ID in the request.
+      def resolveTopicName(topic: OffsetCommitRequestData.OffsetCommitRequestTopic): Option[String] = {
+          val resolvedFromId = topicNames.get(topic.topicId())
+          if (resolvedFromId != null)
+            Some(resolvedFromId)
+          else if (offsetCommitRequest.version() < 9 || Uuid.ZERO_UUID.equals(topic.topicId)) {
+            Option(topic.name())
+          } else {
+            None
+          }
+      }
+
+      offsetCommitRequest.data.topics.forEach { topic => resolveTopicName(topic).foreach(topic.setName _) }

Review Comment:
   Thanks, I built the list of resolved topics and pass it to the authorizer, inlining name resolution. If a topic has both a name and id defined, the broker fails fast the request and returns an `INVALID_REQUEST`. Is this what you had in mind? Should we send more information to the client in that case?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1116204122


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -1369,7 +1387,7 @@ public void testJoinPrepareWithDisableAutoCommit() {
         try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.of("group-id"), true)) {
             coordinator.ensureActiveGroup();
 
-            prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.NONE);
+            prepareOffsetCommitRequest(new OffsetCommitResponseSpec().expectedOffsets(singletonMap(t1p, 100L)));

Review Comment:
   Sure, I reverted this refactoring and use the response class instead.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1437297790

   Hello David, I updated the PR to take into account your comments and have been adding tests. Almost ready for review.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1122175488


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -119,43 +130,62 @@ public boolean shouldClientThrottle(short version) {
         return version >= 4;
     }
 
-    public static class Builder {
-        OffsetCommitResponseData data = new OffsetCommitResponseData();
-        HashMap<String, OffsetCommitResponseTopic> byTopicName = new HashMap<>();
+    public short version() {
+        return version;
+    }
 
-        private OffsetCommitResponseTopic getOrCreateTopic(
-            String topicName
-        ) {
-            OffsetCommitResponseTopic topic = byTopicName.get(topicName);
-            if (topic == null) {
-                topic = new OffsetCommitResponseTopic().setName(topicName);
-                data.topics().add(topic);
-                byTopicName.put(topicName, topic);
-            }
-            return topic;
+    public static Builder<?> newBuilder(TopicResolver topicResolver, short version) {
+        if (version >= 9) {
+            return new Builder<>(topicResolver, new ByTopicId(), version);
+        } else {
+            return new Builder<>(topicResolver, new ByTopicName(), version);
         }
+    }
 
-        public Builder addPartition(
-            String topicName,
-            int partitionIndex,
-            Errors error
-        ) {
-            final OffsetCommitResponseTopic topicResponse = getOrCreateTopic(topicName);
+    public static final class Builder<T> {
+        private final TopicResolver topicResolver;
+        private final TopicClassifier<T> topicClassifier;
+        private final short version;
 
-            topicResponse.partitions().add(new OffsetCommitResponsePartition()
-                .setPartitionIndex(partitionIndex)
-                .setErrorCode(error.code()));
+        private OffsetCommitResponseData data = new OffsetCommitResponseData();
+        private final Map<T, OffsetCommitResponseTopic> topics = new HashMap<>();
+
+        protected Builder(TopicResolver topicResolver, TopicClassifier<T> topicClassifier, short version) {
+            this.topicResolver = topicResolver;
+            this.topicClassifier = topicClassifier;
+            this.version = version;
+        }
+
+        public Builder<T> addPartition(String topicName, Uuid topicId, int partitionIndex, Errors error) {
+            Uuid resolvedTopicId = maybeResolveTopicId(topicName, topicId);
+
+            if (version >= 9 && Uuid.ZERO_UUID.equals(resolvedTopicId)) {
+                Errors reported = error != Errors.NONE ? error : Errors.UNKNOWN_TOPIC_ID;

Review Comment:
   This case shouldn't be reachable because once we have proceeded with constructing the response via `addPartition` all topic ids are supposed to have been resolved successfully. Here, we choose to add the topic to the response with the error code `UNKNOWN_TOPIC_ID` if no error is already set. Any existing error is not overwritten.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1443590758

   - Added a test for `INVALID_REQUEST` in case of definition of topic id and name for a topic in the `OffsetCommit` request.
   - Added a test exercising a valid `OffsetCommit` request across all schema version. We can see that the topic id and name is propagated to the group coordinator in all cases.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1449920535

   Hello David, thanks for the fast review. Apologies for being slow, I hadn't finished the previous revision. Will include your comments. Working on it right now. Thanks!


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1469825143

   @Hangleton I just merge https://github.com/apache/kafka/pull/13378. We can update this PR now.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1129283573


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java:
##########
@@ -49,7 +46,14 @@ public static class Builder extends AbstractRequest.Builder<OffsetCommitRequest>
         private final OffsetCommitRequestData data;
 
         public Builder(OffsetCommitRequestData data) {

Review Comment:
   It was only used in tests, so best to have it removed.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1129407685


##########
clients/src/main/java/org/apache/kafka/common/TopicResolver.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common;
+
+import org.apache.kafka.common.errors.InvalidTopicException;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Encapsulates the resolution of a topic name from its ID, or its ID from its name, using a local
+ * bidirectional mapping. This resolver assumes there is a bijection between topic IDs and topic names.
+ * <p></p>
+ * Note that this class intends to be used for the (reverse) lookup of topic IDs/names, but
+ * not to characterize the set of topics which are known by a client. Use the
+ * {@link org.apache.kafka.clients.MetadataCache} for that purpose.
+ */
+//@Immutable, @ThreadSafe
+public class TopicResolver {
+    private final Map<String, Uuid> topicIds;
+    private final Map<Uuid, String> topicNames;
+
+    /**
+     * A resolver which universe of topic ids and names is captured from the input map. The reverse association
+     * between a topic ID and a topic name is computed by this method. If there are more than one topic name
+     * resolving to the same topic ID, an {@link InvalidTopicException} is thrown.
+     */
+    public static TopicResolver fromTopicIds(Map<String, Uuid> topicIds) {
+        Map<Uuid, String> topicNames = new HashMap<>(topicIds.size());
+
+        for (Map.Entry<String, Uuid> e: topicIds.entrySet()) {
+            String conflicting = topicNames.putIfAbsent(e.getValue(), e.getKey());
+            if (conflicting != null) {
+                throw new InvalidTopicException(
+                        "Topic " + e.getKey() + " shares the same ID " + e.getValue() + " as topic " + conflicting);
+            }
+        }
+
+        return new TopicResolver(topicIds, topicNames);
+    }
+
+    /**
+     * A resolver which acts as a wrapper around the input mapping of topic ids from/to topic names.
+     * No validation is performed about the consistency of the mapping. This method is to be preferred
+     * when the copy of the input maps needs to be avoided.
+     */
+    public static TopicResolver wrap(Map<String, Uuid> topicIds, Map<Uuid, String> topicNames) {
+        return new TopicResolver(topicIds, topicNames);
+    }
+
+    /**
+     * A resolver with no existing mapping between any topic name and id.
+     */
+    public static TopicResolver emptyResolver() {
+        return fromTopicIds(Collections.emptyMap());
+    }
+
+    private TopicResolver(Map<String, Uuid> topicIds, Map<Uuid, String> topicNames) {
+        this.topicIds = Collections.unmodifiableMap(topicIds);
+        this.topicNames = Collections.unmodifiableMap(topicNames);
+    }
+
+    /**
+     * Returns the ID of the topic with the given name, if that association exists.
+     */
+    public Optional<Uuid> getTopicId(String name) {

Review Comment:
   Yes, there is a code smell here.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1129167871


##########
clients/src/main/java/org/apache/kafka/common/TopicResolver.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common;
+
+import org.apache.kafka.common.errors.InvalidTopicException;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Encapsulates the resolution of a topic name from its ID, or its ID from its name, using a local
+ * bidirectional mapping. This resolver assumes there is a bijection between topic IDs and topic names.
+ * <p></p>
+ * Note that this class intends to be used for the (reverse) lookup of topic IDs/names, but
+ * not to characterize the set of topics which are known by a client. Use the
+ * {@link org.apache.kafka.clients.MetadataCache} for that purpose.
+ */
+//@Immutable, @ThreadSafe
+public class TopicResolver {

Review Comment:
   I agree with you and am not satisfied either with `TopicResolver` but could not find a better name. `TopicIdResolver` would be misleading because this class treats topic ids and names symmetrically. One of the closest entity with similar purposes as this is in [`MetadataCache#L93`](https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/MetadataCache.scala#L93) where `topicIdInfo` is used to refer to the bidirectional mapping. The suffix `Info` could be used here as well although it is not strictly aligned with other uses of that suffix such as in [`TopicPartitionInfo`](https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/TopicPartitionInfo.java).  Interestingly another entity for which may have had to be assigned a generic name is [`TopicCollection`](https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/TopicCollection.java).
   
   Using another name to refer to the dual name/id reference such as `TopicRefResolver` introduces yet another noun (_reference_) not used elsewhere in the codebase and which can be confusing.
   
   So, I am not sure about what could be a better name but maybe `TopicInfoResolver` or `TopicIdInfoResolver` or `TopicIdInfo` or `TopicIdResolver` may sound better albeit still ambiguous and partially incorrect?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1129452978


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -119,28 +131,52 @@ public boolean shouldClientThrottle(short version) {
         return version >= 4;
     }
 
+    public short version() {
+        return version;
+    }
+
     public static class Builder {
         OffsetCommitResponseData data = new OffsetCommitResponseData();
         HashMap<String, OffsetCommitResponseTopic> byTopicName = new HashMap<>();
+        private final TopicResolver topicResolver;
+        private final short version;
+
+        public Builder(TopicResolver topicResolver, short version) {

Review Comment:
   I see what you mean. I moved this logic in the callback of the future which merges the results from the coordinator with those created by the request handling method. I thought about extending the support id in internal layers (group coordinator) in a PR of its own. So, eventually, the coordinator will return results populated with topic ids when applicable.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] clolov commented on pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "clolov (via GitHub)" <gi...@apache.org>.
clolov commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1553270438

   Heya @dajac! I hope I have addressed your comments on all files except `ConsumerCoordinatorTest` and the `OffsetCommitRequestTest`, could you review everything except those two and confirm whether this is the case? As far as I understand your general concerns with `ConsumerCoordinatorTest` and `OffsetCommitRequestTest` is that those tests are either not parameterised in a simple way or they are not parameterised at all - am I correct? For the ones which are not parameterised simply I cannot think of an easier approach - the setup is just long-winded, but the main idea behind the arguments is https://github.com/apache/kafka/pull/13240/files#diff-964e16515361b7d22acbad4795f13abe6af513be9588952bc6427aa7ce00938dR2875-R2880. I guess we can split them into separate tests were we vary just one of the arguments rather than all of them if that's what you mean? The ones which are not parameterised at all I believe are not parameterised because the same functions are used for tests which onl
 y test behaviour in versions >= 9. I am happy to implement any suggestions you might have to improve on what's already there.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1203992806


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2707,11 +2742,271 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_MEMBER_ID);
         assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testCommitOffsetUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+            prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_TOPIC_ID));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertFalse(coordinator.commitOffsetsSync(offsets, time.timer(timeoutMs)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testRetryCommitUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.UNKNOWN_TOPIC_ID)));
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.NONE)));
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertTrue(coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                // Unlike the commit offset sync API, the async API does not retry.
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    static Stream<Arguments> commitOffsetTestArgs() {

Review Comment:
   I think I see what you mean, it is rather heavy-weight and lacks single scope which is preferable for unit tests. Updating accordingly.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1204087015


##########
core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala:
##########
@@ -489,24 +489,23 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
   }
 
   private def createOffsetCommitRequest = {
-    new requests.OffsetCommitRequest.Builder(
-        new OffsetCommitRequestData()
-          .setGroupId(group)
-          .setMemberId(JoinGroupRequest.UNKNOWN_MEMBER_ID)
-          .setGenerationId(1)
-          .setTopics(Collections.singletonList(
-            new OffsetCommitRequestData.OffsetCommitRequestTopic()
-              .setName(topic)
-              .setPartitions(Collections.singletonList(
-                new OffsetCommitRequestData.OffsetCommitRequestPartition()
-                  .setPartitionIndex(part)
-                  .setCommittedOffset(0)
-                  .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
-                  .setCommitTimestamp(OffsetCommitRequest.DEFAULT_TIMESTAMP)
-                  .setCommittedMetadata("metadata")
-              )))
-          )
-    ).build()
+    val data = new OffsetCommitRequestData()
+      .setGroupId(group)
+      .setMemberId(JoinGroupRequest.UNKNOWN_MEMBER_ID)
+      .setGenerationId(1)
+      .setTopics(Collections.singletonList(
+        new OffsetCommitRequestData.OffsetCommitRequestTopic()
+          .setName(topic)
+          .setPartitions(Collections.singletonList(
+            new OffsetCommitRequestData.OffsetCommitRequestPartition()
+              .setPartitionIndex(part)
+              .setCommittedOffset(0)
+              .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
+              .setCommitTimestamp(OffsetCommitRequest.DEFAULT_TIMESTAMP)
+              .setCommittedMetadata("metadata")
+          )))
+      )
+    new requests.OffsetCommitRequest.Builder(data, true).build()

Review Comment:
   That is true.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1204356253


##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.KafkaApisTest.{NameAndId, newOffsetCommitRequestData, newOffsetCommitResponseData}
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetCommitRequestTest.assertResponseEquals
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
+
+import java.util.Optional.empty
+import java.util.Properties
+import scala.collection.{Map, Seq}
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava}
+
+class OffsetCommitRequestTest extends BaseRequestTest {
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val groupId = "groupId"
+
+  var consumer: KafkaConsumer[_, _] = _
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    consumer = createConsumer()
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    if (consumer != null)
+      Utils.closeQuietly(consumer, "KafkaConsumer")
+    super.tearDown()
+  }
+
+  def createTopics(topicNames: String*): Seq[NameAndId] = {
+    topicNames.map(topic => {
+      createTopic(topic)
+      val topicId: Uuid = getTopicIds().get(topic) match {
+        case Some(x) => x
+        case _ => throw new AssertionError("Topic ID not found for " + topic)
+      }
+      NameAndId(topic, topicId)
+    })
+  }
+
+
+  @Test
+  def testTopicIdsArePopulatedInOffsetCommitResponses(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+    consumer.subscribe(topics.map(_.name).asJava)
+
+    sendOffsetCommitRequest(
+      topics.map((_, ListMap(0 -> offset))),
+      topics.map((_, Map(0 -> Errors.NONE))),
+      ApiKeys.OFFSET_COMMIT.allVersions().asScala
+    )
+  }
+
+  @Test
+  def testCommitOffsetFromConsumer(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+    consumer.commitSync(offsetsToCommit(topics, offset))
+  }

Review Comment:
   You are right, this is exercised in `OffsetFetchRequestTest`, `PlaintextConsumerTest`, `GroupCoordinatorIntegrationTest` and `AuthorizerIntegrationTest`. So, I removed this test to avoid duplication.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1204079800


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java:
##########
@@ -2707,11 +2742,271 @@ public void testCommitOffsetUnknownMemberId() {
         client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
         coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
 
-        prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID);
+        prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_MEMBER_ID);
         assertThrows(CommitFailedException.class, () -> coordinator.commitOffsetsSync(singletonMap(t1p,
                 new OffsetAndMetadata(100L, "metadata")), time.timer(Long.MAX_VALUE)));
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testCommitOffsetUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        // Prepare five OffsetCommit requests which return a retriable UNKNOWN_TOPIC_ID error.
+        // Set the timeout accordingly so that commitOffsetsSync completes on the fifth attempt.
+        // Note that the timer (MockTime) only ticks when its sleep(long) method is invoked.
+        // Because the ConsumerNetworkClient does not call sleep() in its network-level poll(.), this
+        // timer never moves forward once the network client is invoked. If there is no available
+        // response to consume, its internal poll loop never completes. Hence, the timeout needs to be
+        // enforced in the ConsumerCoordinator, and we need to make sure there are enough responses
+        // queued in the MockClient to satisfy all invocations of the ConsumerNetworkClient#poll(.).
+        int offsetCommitCalls = 5;
+        long timeoutMs = rebalanceConfig.retryBackoffMs * offsetCommitCalls;
+
+        IntStream.range(0, offsetCommitCalls).forEach(__ ->
+            prepareOffsetCommitRequest(singletonMap(ti1p, 100L), Errors.UNKNOWN_TOPIC_ID));
+
+        // UnknownTopicIdException is retriable, hence will be retried by the coordinator as long as
+        // the timeout allows. Note that since topic ids are not part of the public API of the consumer,
+        // we cannot throw an UnknownTopicId to the user. By design, a false boolean indicating the
+        // offset commit failed is returned.
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertFalse(coordinator.commitOffsetsSync(offsets, time.timer(timeoutMs)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = { true, false })
+    public void testRetryCommitUnknownTopicId(boolean commitSync) {
+        client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+        coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
+
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.UNKNOWN_TOPIC_ID)));
+        client.prepareResponse(offsetCommitResponse(singletonMap(ti1p, Errors.NONE)));
+
+        Map<TopicPartition, OffsetAndMetadata> offsets = singletonMap(
+            t1p,
+            new OffsetAndMetadata(100L, "metadata")
+        );
+
+        if (commitSync) {
+            assertTrue(coordinator.commitOffsetsSync(offsets, time.timer(Long.MAX_VALUE)));
+
+        } else {
+            AtomicBoolean callbackInvoked = new AtomicBoolean();
+            coordinator.commitOffsetsAsync(offsets, (inputOffsets, exception) -> {
+                // Unlike the commit offset sync API, the async API does not retry.
+                assertSame(inputOffsets, offsets);
+                assertEquals(RetriableCommitFailedException.class, exception.getClass());
+                assertEquals(UnknownTopicOrPartitionException.class, exception.getCause().getClass());
+                callbackInvoked.set(true);
+            });
+
+            coordinator.invokeCompletedOffsetCommitCallbacks();
+            assertTrue(callbackInvoked.get());
+        }
+    }
+
+    static Stream<Arguments> commitOffsetTestArgs() {

Review Comment:
   Did the change to provide higher cohesion to the tests. I split the initial test method in two separated test methods.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1129452978


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -119,28 +131,52 @@ public boolean shouldClientThrottle(short version) {
         return version >= 4;
     }
 
+    public short version() {
+        return version;
+    }
+
     public static class Builder {
         OffsetCommitResponseData data = new OffsetCommitResponseData();
         HashMap<String, OffsetCommitResponseTopic> byTopicName = new HashMap<>();
+        private final TopicResolver topicResolver;
+        private final short version;
+
+        public Builder(TopicResolver topicResolver, short version) {

Review Comment:
   I see what you mean. I moved this logic in the callback of the future which merges the results from the coordinator with those created by the request handling method. I thought about extending the support id in internal layers (group coordinator) in a PR of its own. So, eventually, the coordinator will return results populated with topic ids when applicable.
   
   Added [this JIRA](https://issues.apache.org/jira/browse/KAFKA-14793) to track this work, if that is ok?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1131139080


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -119,28 +131,52 @@ public boolean shouldClientThrottle(short version) {
         return version >= 4;
     }
 
+    public short version() {
+        return version;
+    }
+
     public static class Builder {
         OffsetCommitResponseData data = new OffsetCommitResponseData();
         HashMap<String, OffsetCommitResponseTopic> byTopicName = new HashMap<>();
+        private final TopicResolver topicResolver;
+        private final short version;
+
+        public Builder(TopicResolver topicResolver, short version) {

Review Comment:
   Sounds good. Thanks for your guidance. As you mentioned, this PR is already quite large, so if you agree, I will go ahead and implement this change first, in a PR of its own. Thanks!



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1129272424


##########
clients/src/main/java/org/apache/kafka/common/TopicResolver.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common;
+
+import org.apache.kafka.common.errors.InvalidTopicException;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Encapsulates the resolution of a topic name from its ID, or its ID from its name, using a local
+ * bidirectional mapping. This resolver assumes there is a bijection between topic IDs and topic names.
+ * <p></p>
+ * Note that this class intends to be used for the (reverse) lookup of topic IDs/names, but
+ * not to characterize the set of topics which are known by a client. Use the
+ * {@link org.apache.kafka.clients.MetadataCache} for that purpose.
+ */
+//@Immutable, @ThreadSafe
+public class TopicResolver {

Review Comment:
   I would tend to have a preference for a business type which conveys semantics versus a generic data structure, but that is not very important here especially since the entity exposing the bidirectional mapping is relatively short-lived when used in the code. One advantage of a generic DS is that it can be reused for other purposes. Another thing is that there is no functionality provided outside that of a bimap and since no extension is foreseen, there is no need to expose a specialized type. Very happy to expose it as a bimap. I could not find an existing implementation in the codebase or its dependencies, although there is a bidirectional multimap defined within restricted scope [here](https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java#L459-L489).



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1129482786


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -167,8 +213,24 @@ public <P> Builder addPartitions(
         }
 
         public Builder merge(
-            OffsetCommitResponseData newData
+            OffsetCommitResponseData newData,
+            Logger logger
         ) {
+            if (version >= 9) {
+                // This method is called after the group coordinator committed the offsets. The group coordinator
+                // provides the OffsetCommitResponseData it built in the process. As of now, this data does
+                // not contain topic ids, so we resolve them here.
+                newData.topics().forEach(topic -> {
+                    Uuid topicId = topicResolver.getTopicId(topic.name()).orElse(Uuid.ZERO_UUID);
+                    if (Uuid.ZERO_UUID.equals(topicId)) {

Review Comment:
   The server now sends an `UNKNOWN_SERVER_ERROR` when this happens, in the code moved to the future handler in `KafkaApis`. Would this behaviour be acceptable?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1131241761


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -119,28 +131,52 @@ public boolean shouldClientThrottle(short version) {
         return version >= 4;
     }
 
+    public short version() {
+        return version;
+    }
+
     public static class Builder {
         OffsetCommitResponseData data = new OffsetCommitResponseData();
         HashMap<String, OffsetCommitResponseTopic> byTopicName = new HashMap<>();
+        private final TopicResolver topicResolver;
+        private final short version;
+
+        public Builder(TopicResolver topicResolver, short version) {

Review Comment:
   Sounds good to me. Thanks!



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1129266598


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1379,7 +1411,8 @@ public void handle(OffsetCommitResponse commitResponse, RequestFuture<Void> futu
                             future.raise(error);
                             return;
                         } else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS
-                                || error == Errors.UNKNOWN_TOPIC_OR_PARTITION) {
+                                || error == Errors.UNKNOWN_TOPIC_OR_PARTITION
+                                || error == Errors.UNKNOWN_TOPIC_ID) {

Review Comment:
   With the synchronous API in the consumer, the error is not surfaced (only `true`/`false`). However, I added the missing tests to exercise the asynchronous API for this use case, and it did expose the `UnknownTopicIdException` to the user. Since it violates the API contract which exclusively relies on topic names, I raised the error `UNKNOWN_TOPIC_OR_PARTITION` when an `UNKNOWN_TOPIC_ID` is returned in the offset commit response. Do you think this is sensible?
   
   I added the corresponding unit tests for the consumer coordinator.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1129194029


##########
clients/src/main/java/org/apache/kafka/common/TopicResolver.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common;
+
+import org.apache.kafka.common.errors.InvalidTopicException;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Encapsulates the resolution of a topic name from its ID, or its ID from its name, using a local
+ * bidirectional mapping. This resolver assumes there is a bijection between topic IDs and topic names.
+ * <p></p>
+ * Note that this class intends to be used for the (reverse) lookup of topic IDs/names, but
+ * not to characterize the set of topics which are known by a client. Use the
+ * {@link org.apache.kafka.clients.MetadataCache} for that purpose.
+ */
+//@Immutable, @ThreadSafe
+public class TopicResolver {

Review Comment:
   Another way would be to implement a minimal and generic BiMap that we could use here. Would it be an option?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1129371713


##########
core/src/main/scala/kafka/server/MetadataCache.scala:
##########
@@ -90,6 +90,10 @@ trait MetadataCache {
 
   def topicIdsToNames(): util.Map[Uuid, String]
 
+  def topicResolver(): TopicResolver = {
+    TopicResolver.wrap(topicNamesToIds(), topicIdsToNames())

Review Comment:
   Oops...



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1116203414


##########
core/src/test/scala/unit/kafka/server/KafkaApisTest.scala:
##########
@@ -1351,8 +1351,11 @@ class KafkaApisTest {
 
   @Test
   def testHandleOffsetCommitRequestTopicsAndPartitionsValidation(): Unit = {

Review Comment:
   Sure, I added another unresolvable topic to the request/response. I will add more cases covering more of the possible code paths.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1562458362

   Hi David (@dajac), thanks for the review and apologies for the delayed reply. Thanks to Christo's help, I believe most of your comments have been addressed. I have one question regarding the behaviour of the offset commit consumer API that you identified [here](https://github.com/apache/kafka/pull/13240#discussion_r1168735466). Thanks!


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1204361177


##########
core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala:
##########
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package unit.kafka.server
+
+import kafka.server.KafkaApisTest.{NameAndId, newOffsetCommitRequestData, newOffsetCommitResponseData}
+import kafka.server.{BaseRequestTest, KafkaConfig}
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetCommitRequestTest.assertResponseEquals
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetCommitResponse}
+import org.apache.kafka.common.utils.Utils
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
+
+import java.util.Optional.empty
+import java.util.Properties
+import scala.collection.{Map, Seq}
+import scala.collection.immutable.ListMap
+import scala.jdk.CollectionConverters.{ListHasAsScala, MapHasAsJava, SeqHasAsJava}
+
+class OffsetCommitRequestTest extends BaseRequestTest {
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val groupId = "groupId"
+
+  var consumer: KafkaConsumer[_, _] = _
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    consumer = createConsumer()
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    if (consumer != null)
+      Utils.closeQuietly(consumer, "KafkaConsumer")
+    super.tearDown()
+  }
+
+  def createTopics(topicNames: String*): Seq[NameAndId] = {
+    topicNames.map(topic => {
+      createTopic(topic)
+      val topicId: Uuid = getTopicIds().get(topic) match {
+        case Some(x) => x
+        case _ => throw new AssertionError("Topic ID not found for " + topic)
+      }
+      NameAndId(topic, topicId)
+    })
+  }
+
+
+  @Test
+  def testTopicIdsArePopulatedInOffsetCommitResponses(): Unit = {
+    val topics = createTopics("topic1", "topic2", "topic3")
+    consumer.subscribe(topics.map(_.name).asJava)
+
+    sendOffsetCommitRequest(
+      topics.map((_, ListMap(0 -> offset))),
+      topics.map((_, Map(0 -> Errors.NONE))),
+      ApiKeys.OFFSET_COMMIT.allVersions().asScala

Review Comment:
   Added parameterization as David suggested.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1121506569


##########
clients/src/test/java/org/apache/kafka/common/requests/OffsetCommitResponseTest.java:
##########
@@ -17,37 +17,64 @@
 package org.apache.kafka.common.requests;
 
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
 import org.apache.kafka.common.message.OffsetCommitResponseData;
 import org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponsePartition;
 import org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponseTopic;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
 import org.apache.kafka.common.protocol.MessageUtil;
+import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
 
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
+import static java.util.Arrays.asList;
+import static java.util.Collections.singletonList;
+import static java.util.function.Function.identity;
 import static org.apache.kafka.common.requests.AbstractResponse.DEFAULT_THROTTLE_TIME;
 import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
 
 public class OffsetCommitResponseTest {
 
     protected final int throttleTimeMs = 10;
 
-    protected final String topicOne = "topic1";
+    protected final String topic1 = "topic1";
+    protected final Uuid topic1Id = Uuid.randomUuid();
     protected final int partitionOne = 1;
+    protected final int partitionTwo = 2;
     protected final Errors errorOne = Errors.COORDINATOR_NOT_AVAILABLE;
     protected final Errors errorTwo = Errors.NOT_COORDINATOR;
-    protected final String topicTwo = "topic2";
-    protected final int partitionTwo = 2;
-
-    protected TopicPartition tp1 = new TopicPartition(topicOne, partitionOne);
-    protected TopicPartition tp2 = new TopicPartition(topicTwo, partitionTwo);
+    protected final String topic2 = "topic2";
+    protected final int p3 = 3;
+    protected final int p4 = 4;
+    protected final String topic3 = "topic3";
+    protected final int p5 = 5;
+    protected final int p6 = 6;
+    protected final String topic4 = "topic4";
+    protected final Uuid topic4Id = Uuid.randomUuid();
+    protected final int p7 = 7;
+    protected final int p8 = 8;
+    protected final Uuid topic5Id = Uuid.randomUuid();
+    protected final int p9 = 9;
+    protected final int p10 = 10;
+    protected final String topic6 = "topic6";
+    protected final Uuid topic6Id = Uuid.randomUuid();
+    protected final int p11 = 11;
+    protected final int p12 = 12;

Review Comment:
   I am not a fan of all those attributes in test. One or two are fine if they are really re-used on all the tests. Otherwise, it may be better to check define what you need in tests. I would also use `TopicIdPartition` when relevant so you can basically group the name, id, and partition together.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1121501980


##########
clients/src/test/java/org/apache/kafka/common/requests/TxnOffsetCommitRequestTest.java:
##########
@@ -80,10 +81,10 @@ public void setUp() {
         );
     }
 
-    @Test
     @Override
-    public void testConstructor() {
-
+    @ParameterizedTest
+    @ApiKeyVersionsSource(apiKey = ApiKeys.TXN_OFFSET_COMMIT)
+    public void testConstructor(short version) {

Review Comment:
   Is this change related to the PR? If not, I would rather do it in a separate PR.



##########
clients/src/test/java/org/apache/kafka/common/requests/TxnOffsetCommitResponseTest.java:
##########
@@ -38,9 +40,9 @@ public void testConstructorWithErrorResponse() {
         assertEquals(throttleTimeMs, response.throttleTimeMs());
     }
 
-    @Test
-    @Override
-    public void testParse() {
+    @ParameterizedTest
+    @ApiKeyVersionsSource(apiKey = ApiKeys.TXN_OFFSET_COMMIT)

Review Comment:
   Same question here.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1119798298


##########
clients/src/main/resources/common/message/OffsetCommitRequest.json:
##########
@@ -47,8 +49,10 @@
       "about": "The time period in ms to retain the offset." },
     { "name": "Topics", "type": "[]OffsetCommitRequestTopic", "versions": "0+",
       "about": "The topics to commit offsets for.",  "fields": [
-      { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName",
+      { "name": "Name", "type": "string", "versions": "0+", "nullableVersions": "9+", "entityType": "topicName",
         "about": "The topic name." },
+      { "name": "TopicId", "type": "uuid", "versions": "9+",
+        "about": "The unique topic ID." },

Review Comment:
   Hi David, thanks for the follow-up and clarifying. This is all good, I am working on adapting the PR. Thanks!



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1121489116


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -425,35 +425,63 @@ class KafkaApis(val requestChannel: RequestChannel,
       requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
       CompletableFuture.completedFuture[Unit](())
     } else {
+      val topicNames =
+        if (offsetCommitRequest.version() >= 9)
+          metadataCache.topicIdsToNames()
+        else
+          Collections.emptyMap[Uuid, String]()
+
+      val resolvedTopics = new ArrayBuffer[OffsetCommitRequestData.OffsetCommitRequestTopic]()
+      offsetCommitRequest.data.topics.forEach { topic =>
+        var topicName = topic.name()
+        if (Utils.isBlank(topicName)) {
+          // Expected for requests version >= 9 which rely on topic IDs exclusively.
+          topicName = topicNames.get(topic.topicId())
+        }
+        if (topicName != null) {
+          topic.setName(topicName)
+          resolvedTopics += topic
+        }
+      }
+
       val authorizedTopics = authHelper.filterByAuthorized(
         request.context,
         READ,
         TOPIC,
-        offsetCommitRequest.data.topics.asScala
+        resolvedTopics
       )(_.name)
 
-      val responseBuilder = new OffsetCommitResponse.Builder()
+      val responseBuilder = OffsetCommitResponse.newBuilder(offsetCommitRequest.version())
       val authorizedTopicsRequest = new mutable.ArrayBuffer[OffsetCommitRequestData.OffsetCommitRequestTopic]()
       offsetCommitRequest.data.topics.forEach { topic =>

Review Comment:
   You could use `resolvedTopics` instead of `offsetCommitRequest.data.topics` here.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1121494379


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java:
##########
@@ -57,8 +55,22 @@ public Builder(OffsetCommitRequestData data) {
         public OffsetCommitRequest build(short version) {
             if (data.groupInstanceId() != null && version < 7) {
                 throw new UnsupportedVersionException("The broker offset commit protocol version " +
-                        version + " does not support usage of config group.instance.id.");
+                    version + " does not support usage of config group.instance.id.");
             }
+
+            data.topics().forEach(topic -> {

Review Comment:
   I think that we could remove those checks now.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1122096890


##########
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java:
##########
@@ -149,13 +161,15 @@ public Builder addPartition(
             return this;
         }
 
-        public <P> Builder addPartitions(
-            String topicName,
-            List<P> partitions,
-            Function<P, Integer> partitionIndex,
-            Errors error
+        public final <P> Builder<T> addPartitions(

Review Comment:
   There is still a problem here if `topicName` and `topicId` are both undefined in which case we should do what was done before and add to the response without caching.



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -425,35 +426,59 @@ class KafkaApis(val requestChannel: RequestChannel,
       requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
       CompletableFuture.completedFuture[Unit](())
     } else {
+      val topicResolver = TopicResolver.fromTopicIds(metadataCache.topicNamesToIds())

Review Comment:
   Move the `TopicResolver` in the `MetadataCache` or create it without copying the map of topic ids as this is costly.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1452495333

   Thanks David for the review, have a few more tests to add but this should be eligible to another pass.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac closed pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac closed pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9
URL: https://github.com/apache/kafka/pull/13240


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13240: KAFKA-14690: Add topic IDs to OffsetCommit API and propagate for request version >= 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13240:
URL: https://github.com/apache/kafka/pull/13240#discussion_r1105854122


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -425,35 +425,72 @@ class KafkaApis(val requestChannel: RequestChannel,
       requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
       CompletableFuture.completedFuture[Unit](())
     } else {
+      val topicNames =
+        if (offsetCommitRequest.version() >= 9)
+          metadataCache.topicIdsToNames()
+        else
+          Collections.emptyMap[Uuid, String]()
+
+      // For version < 9, lookup from topicNames fails and the topic name (which cannot be null) is returned.
+      // For version >= 9, if lookup from topicNames fails, there are two possibilities:
+      //
+      // a) The topic ID was left to default and the topic name should have been populated as a fallback instead.
+      //    If none was provided, null is returned.
+      //
+      // b) The topic ID was not default but is not present in the local topic IDs cache. In this case, because
+      //    clients should make exclusive use of topic name or topic ID, the topic name should be null. If however
+      //    the client provided a topic name, we do not want to use it, because any topic with the same name
+      //    present locally would then have a topic ID which does not match the topic ID in the request.
+      def resolveTopicName(topic: OffsetCommitRequestData.OffsetCommitRequestTopic): String = {
+          val resolvedFromId = topicNames.get(topic.topicId())
+          if (resolvedFromId != null)
+            resolvedFromId
+          else if (offsetCommitRequest.version() < 9 || Uuid.ZERO_UUID.equals(topic.topicId)) {
+            topic.name()
+          } else {
+            null
+          }
+      }
+
       val authorizedTopics = authHelper.filterByAuthorized(
         request.context,
         READ,
         TOPIC,
-        offsetCommitRequest.data.topics.asScala
-      )(_.name)
+        offsetCommitRequest.data.topics.asScala.filter(topic => resolveTopicName(topic) != null)

Review Comment:
   We are calling `resolveTopicName` three times. I think that it would be better to iterate once over the topics to resolve the topic ids and build the list of topic names (if one was found) while doing this. Then, we can check the authorization and do the rest.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on pull request #13240: KAFKA-14690: Add topic IDs to the OffsetCommit API version 9

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on PR #13240:
URL: https://github.com/apache/kafka/pull/13240#issuecomment-1505120343

   @Hangleton There are a few conflicts. Could you please rebase the PR? I plan to make another pass on it afterwards.


-- 
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-unsubscribe@kafka.apache.org

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