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

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

Reply via email to