rajinisivaram commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664880346



##########
File path: clients/src/main/resources/common/message/OffsetFetchResponse.json
##########
@@ -30,30 +30,57 @@
   // Version 6 is the first flexible version.
   //
   // Version 7 adds pending offset commit as new error response on partition 
level.
-  "validVersions": "0-7",
+  //
+  // Version 8 is adding support for fetching offsets for multiple groups
+  "validVersions": "0-8",
   "flexibleVersions": "6+",
   "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": "[]OffsetFetchResponseTopic", "versions": 
"0+", 
+    { "name": "Topics", "type": "[]OffsetFetchResponseTopic", "versions": 
"0-7",
       "about": "The responses per topic.", "fields": [
-      { "name": "Name", "type": "string", "versions": "0+", "entityType": 
"topicName",
+      { "name": "Name", "type": "string", "versions": "0-7", "entityType": 
"topicName",
         "about": "The topic name." },
-      { "name": "Partitions", "type": "[]OffsetFetchResponsePartition", 
"versions": "0+",
+      { "name": "Partitions", "type": "[]OffsetFetchResponsePartition", 
"versions": "0-7",
         "about": "The responses per partition", "fields": [
-        { "name": "PartitionIndex", "type": "int32", "versions": "0+",
+        { "name": "PartitionIndex", "type": "int32", "versions": "0-7",
           "about": "The partition index." },
-        { "name": "CommittedOffset", "type": "int64", "versions": "0+",
+        { "name": "CommittedOffset", "type": "int64", "versions": "0-7",
           "about": "The committed message offset." },
-        { "name": "CommittedLeaderEpoch", "type": "int32", "versions": "5+", 
"default": "-1",
+        { "name": "CommittedLeaderEpoch", "type": "int32", "versions": "5-7", 
"default": "-1",
           "ignorable": true, "about": "The leader epoch." },
-        { "name": "Metadata", "type": "string", "versions": "0+", 
"nullableVersions": "0+",
+        { "name": "Metadata", "type": "string", "versions": "0-7", 
"nullableVersions": "0-7",
           "about": "The partition metadata." },
-        { "name": "ErrorCode", "type": "int16", "versions": "0+",
+        { "name": "ErrorCode", "type": "int16", "versions": "0-7",
           "about": "The error code, or 0 if there was no error." }
       ]}
     ]},
-    { "name": "ErrorCode", "type": "int16", "versions": "2+", "default": "0", 
"ignorable": true,
-      "about": "The top-level error code, or 0 if there was no error." }
+    { "name": "ErrorCode", "type": "int16", "versions": "2-7", "default": "0", 
"ignorable": true,
+      "about": "The top-level error code, or 0 if there was no error." },
+    {"name": "GroupIds", "type": "[]OffsetFetchResponseGroup", "versions": 
"8+",

Review comment:
       As with the response, should we call this `Groups` rather than 
`GroupIds`?

##########
File path: clients/src/main/resources/common/message/OffsetFetchRequest.json
##########
@@ -31,19 +31,33 @@
   // Version 6 is the first flexible version.
   //
   // Version 7 is adding the require stable flag.
-  "validVersions": "0-7",
+  //
+  // Version 8 is adding support for fetching offsets for multiple groups at a 
time
+  "validVersions": "0-8",
   "flexibleVersions": "6+",
   "fields": [
-    { "name": "GroupId", "type": "string", "versions": "0+", "entityType": 
"groupId",
+    { "name": "GroupId", "type": "string", "versions": "0-7", "entityType": 
"groupId",
       "about": "The group to fetch offsets for." },
-    { "name": "Topics", "type": "[]OffsetFetchRequestTopic", "versions": "0+", 
"nullableVersions": "2+",
+    { "name": "Topics", "type": "[]OffsetFetchRequestTopic", "versions": 
"0-7", "nullableVersions": "2-7",
       "about": "Each topic we would like to fetch offsets for, or null to 
fetch offsets for all topics.", "fields": [
-      { "name": "Name", "type": "string", "versions": "0+", "entityType": 
"topicName",
+      { "name": "Name", "type": "string", "versions": "0-7", "entityType": 
"topicName",
         "about": "The topic name."},
-      { "name": "PartitionIndexes", "type": "[]int32", "versions": "0+",
+      { "name": "PartitionIndexes", "type": "[]int32", "versions": "0-7",
         "about": "The partition indexes we would like to fetch offsets for." }
     ]},
+    { "name": "GroupIds", "type": "[]OffsetFetchRequestGroup", "versions": 
"8+",

Review comment:
       Should we call this `Groups` rather than `GroupIds` since it is not just 
the group id?

##########
File path: 
core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
##########
@@ -1358,17 +1367,233 @@ class AuthorizerIntegrationTest extends 
BaseRequestTest {
     // note there's only one broker, so no need to lookup the group coordinator
 
     // without describe permission on the topic, we shouldn't be able to fetch 
offsets
-    val offsetFetchRequest = new requests.OffsetFetchRequest.Builder(group, 
false, null, false).build()
+    val offsetFetchRequest = createOffsetFetchRequestAllPartitions
     var offsetFetchResponse = 
connectAndReceive[OffsetFetchResponse](offsetFetchRequest)
-    assertEquals(Errors.NONE, offsetFetchResponse.error)
-    assertTrue(offsetFetchResponse.responseData.isEmpty)
+    assertEquals(Errors.NONE, offsetFetchResponse.groupLevelError(group))
+    assertTrue(offsetFetchResponse.partitionDataMap(group).isEmpty)
 
     // now add describe permission on the topic and verify that the offset can 
be fetched
     addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, 
WildcardHost, DESCRIBE, ALLOW)), topicResource)
     offsetFetchResponse = 
connectAndReceive[OffsetFetchResponse](offsetFetchRequest)
-    assertEquals(Errors.NONE, offsetFetchResponse.error)
-    assertTrue(offsetFetchResponse.responseData.containsKey(tp))
-    assertEquals(offset, offsetFetchResponse.responseData.get(tp).offset)
+    assertEquals(Errors.NONE, offsetFetchResponse.groupLevelError(group))
+    assertTrue(offsetFetchResponse.partitionDataMap(group).containsKey(tp))
+    assertEquals(offset, 
offsetFetchResponse.partitionDataMap(group).get(tp).offset)
+  }
+
+  @Test
+  def testOffsetFetchMultipleGroupsAuthorization(): Unit = {
+    val groupOne = "group1"
+    val groupOneResource = new ResourcePattern(GROUP, groupOne, LITERAL)
+    val groupTwo = "group2"
+    val groupTwoResource = new ResourcePattern(GROUP, groupTwo, LITERAL)
+    val groupThree = "group3"
+    val groupThreeResource = new ResourcePattern(GROUP, groupThree, LITERAL)
+    val groupFour = "group4"
+    val groupFourResource = new ResourcePattern(GROUP, groupFour, LITERAL)
+    val groupFive = "group5"
+    val groupFiveResource = new ResourcePattern(GROUP, groupFive, LITERAL)

Review comment:
       We could do something like:
   ```
   val groups = (0 until 5).map(i => s"group$i")
   val groupResources = groups.map(group => new ResourcePattern(GROUP, group, 
LITERAL))
   ```

##########
File path: core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala
##########
@@ -0,0 +1,241 @@
+/**
+ * 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 kafka.server
+
+import kafka.utils.TestUtils
+import org.apache.kafka.clients.consumer.{ConsumerConfig, OffsetAndMetadata}
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData
+import org.apache.kafka.common.requests.{AbstractResponse, OffsetFetchRequest, 
OffsetFetchResponse}
+import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
+import org.junit.jupiter.api.{BeforeEach, Test}
+
+import java.util
+import java.util.Collections.singletonList
+import scala.jdk.CollectionConverters._
+import java.util.{Optional, Properties}
+
+class OffsetFetchRequestTest extends BaseRequestTest{

Review comment:
       nit: space before {

##########
File path: core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala
##########
@@ -0,0 +1,227 @@
+/**
+ * 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 kafka.server
+
+import kafka.utils.TestUtils
+import org.apache.kafka.clients.consumer.{ConsumerConfig, OffsetAndMetadata}
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.{AbstractResponse, OffsetFetchRequest, 
OffsetFetchResponse}
+import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
+import org.junit.jupiter.api.{BeforeEach, Test}
+
+import java.util
+import java.util.Collections.singletonList
+import scala.jdk.CollectionConverters._
+import java.util.{Optional, Properties}
+
+class OffsetFetchRequestTest extends BaseRequestTest{
+
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val leaderEpoch: Optional[Integer] = Optional.of(3)
+  val metadata = "metadata"
+
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.BrokerIdProp, brokerId.toString)
+    properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
+    properties.put(KafkaConfig.TransactionsTopicPartitionsProp, "1")
+    properties.put(KafkaConfig.TransactionsTopicReplicationFactorProp, "1")
+    properties.put(KafkaConfig.TransactionsTopicMinISRProp, "1")
+  }
+
+  @BeforeEach
+  override def setUp(): Unit = {
+    doSetup(createOffsetsTopic = false)
+
+    TestUtils.createOffsetsTopic(zkClient, servers)
+  }
+
+  @Test
+  def testOffsetFetchRequestLessThanV8(): Unit = {
+    val topic = "topic"
+    createTopic(topic)
+
+    val groupId = "groupId"
+    val tpList = singletonList(new TopicPartition(topic, 0))
+    val topicOffsets = tpList.asScala.map{
+      tp => (tp, new OffsetAndMetadata(offset, leaderEpoch, metadata))
+    }.toMap.asJava
+
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+    val consumer = createConsumer()
+    consumer.assign(tpList)
+    consumer.commitSync(topicOffsets)
+    consumer.close()
+    // testing from version 1 onward since version 0 read offsets from ZK
+    for (version <- 1 to ApiKeys.OFFSET_FETCH.latestVersion()) {
+      if (version < 8) {
+        val request =
+          if (version < 7) {
+            new OffsetFetchRequest.Builder(
+              groupId, false, tpList, false)
+              .build(version.asInstanceOf[Short])
+          } else {
+            new OffsetFetchRequest.Builder(
+              groupId, false, tpList, true)
+              .build(version.asInstanceOf[Short])
+          }
+        val response = connectAndReceive[OffsetFetchResponse](request)
+        val topicData = response.data().topics().get(0)
+        val partitionData = topicData.partitions().get(0)
+        if (version < 3) {
+          assertEquals(AbstractResponse.DEFAULT_THROTTLE_TIME, 
response.throttleTimeMs())
+        }
+        assertEquals(Errors.NONE, response.error())
+        assertEquals(topic, topicData.name())
+        assertEquals(0, partitionData.partitionIndex())
+        assertEquals(offset, partitionData.committedOffset())
+        if (version >= 5) {
+          // committed leader epoch introduced with V5
+          assertEquals(leaderEpoch.get(), partitionData.committedLeaderEpoch())
+        }
+        assertEquals(metadata, partitionData.metadata())
+        assertEquals(Errors.NONE.code(), partitionData.errorCode())
+      }
+    }
+  }
+
+  @Test
+  def testOffsetFetchRequestV8AndAbove(): Unit = {
+    val groupOne = "group1"
+    val groupTwo = "group2"
+    val groupThree = "group3"
+    val groupFour = "group4"
+    val groupFive = "group5"

Review comment:
       We can do:
   ```
   val groups = (0 until 5).map(i => s"group$i")
   ```
   And use groups(0), groups(1) etc. instead of groupOne, groupTwo.




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