You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/07/02 19:28:11 UTC

[GitHub] [kafka] skaundinya15 opened a new pull request #10962: KIP-709: Implement request/response for offsetFetch batching

skaundinya15 opened a new pull request #10962:
URL: https://github.com/apache/kafka/pull/10962


   This implements the request and response portion of KIP-709: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=173084258. The Admin APIs will be implemented in a follow up PR.
   
   It updates the OffsetFetch request and response to support fetching offsets for multiple consumer groups at a time. If the broker does not support the new OffsetFetch version, clients can revert to the previous behaviour and use a request for each 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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663419332



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -154,14 +166,88 @@ public OffsetFetchResponse(int throttleTimeMs, Errors error, Map<TopicPartition,
         this.error = error;
     }
 
+    /**
+     * Constructor without throttle time for version 8 and above.
+     * @param errors Error code on a per group level basis
+     * @param responseData Fetched offset information grouped group id
+     */
+    public OffsetFetchResponse(Map<String, Errors> errors, Map<String, Map<TopicPartition, PartitionData>> responseData) {

Review comment:
       I added this constructor to keep the precedent of having a constructor with no throttle time and just passing in the `DEFAULT_THROTTLE_TIME` - if it's not needed it can definitely be 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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664702184



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +315,28 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public boolean isAllPartitionsForGroup(String groupId) {
+        OffsetFetchRequestGroup group = data
+            .groupIds()
+            .stream()
+            .filter(g -> g.groupId().equals(groupId))
+            .collect(toSingleton());
+        return group.topics() == ALL_TOPIC_PARTITIONS_BATCH;
+    }
+
+    // Custom collector to filter a single element
+    private <T> Collector<T, ?, T> toSingleton() {

Review comment:
       Makes sense, for now I will remove it and convert the methods that used this to collect a `List` and get the first element in the list.




-- 
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] rajinisivaram commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664918688



##########
File path: core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
##########
@@ -1358,17 +1367,222 @@ 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 groups = (0 until 5).map(i => s"group$i")
+    val groupResources = groups.map(group => new ResourcePattern(GROUP, group, LITERAL))
+
+    val topic1 = "topic1"
+    val topic1List = singletonList(new TopicPartition(topic1, 0))
+    val topicOneResource = new ResourcePattern(TOPIC, topic1, LITERAL)
+    val topic2 = "topic2"
+    val topic1And2List = util.Arrays.asList(
+      new TopicPartition(topic1, 0),
+      new TopicPartition(topic2, 0),
+      new TopicPartition(topic2, 1))
+    val topicTwoResource = new ResourcePattern(TOPIC, topic2, LITERAL)
+    val topic3 = "topic3"
+    val allTopicsList = util.Arrays.asList(
+      new TopicPartition(topic1, 0),
+      new TopicPartition(topic2, 0),
+      new TopicPartition(topic2, 1),
+      new TopicPartition(topic3, 0),
+      new TopicPartition(topic3, 1),
+      new TopicPartition(topic3, 2))
+    val topicThreeResource = new ResourcePattern(TOPIC, topic3, LITERAL)
+
+    // create group to partition map to build batched offsetFetch request
+    val groupToPartitionMap = new util.HashMap[String, util.List[TopicPartition]]()
+    groupToPartitionMap.put(groups(1), topic1List)

Review comment:
       groups(0) to group(4) 

##########
File path: core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala
##########
@@ -0,0 +1,237 @@
+/**
+ * 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 {
+
+  override def brokerCount: Int = 1
+
+  val brokerId: Integer = 0
+  val offset = 15L
+  val leaderEpoch: Optional[Integer] = Optional.of(3)
+  val metadata = "metadata"
+  val topic = "topic"
+  val groupId = "groupId"
+  val groups: Seq[String] = (0 until 5).map(i => s"group$i")
+
+  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 testOffsetFetchRequestSingleGroup(): Unit = {
+    createTopic(topic)
+
+    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)
+    commitOffsets(tpList, topicOffsets)
+
+    // 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())
+        }
+        verifySingleGroupResponse(version.asInstanceOf[Short],
+          response.error().code(), partitionData.errorCode(), topicData.name(),
+          partitionData.partitionIndex(), partitionData.committedOffset(),
+          partitionData.committedLeaderEpoch(), partitionData.metadata())
+      } else {
+        val request = new OffsetFetchRequest.Builder(
+          Map(groupId -> tpList).asJava, false, false)
+          .build(version.asInstanceOf[Short])
+        val response = connectAndReceive[OffsetFetchResponse](request)
+        val groupData = response.data().groups().get(0)
+        val topicData = groupData.topics().get(0)
+        val partitionData = topicData.partitions().get(0)
+        verifySingleGroupResponse(version.asInstanceOf[Short],
+          groupData.errorCode(), partitionData.errorCode(), topicData.name(),
+          partitionData.partitionIndex(), partitionData.committedOffset(),
+          partitionData.committedLeaderEpoch(), partitionData.metadata())
+      }
+    }
+  }
+
+  @Test
+  def testOffsetFetchRequestV8AndAbove(): Unit = {
+
+    val topic1 = "topic1"
+    val topic1List = singletonList(new TopicPartition(topic1, 0))
+    val topic2 = "topic2"
+    val topic1And2List = util.Arrays.asList(
+      new TopicPartition(topic1, 0),
+      new TopicPartition(topic2, 0),
+      new TopicPartition(topic2, 1))
+    val topic3 = "topic3"
+    val allTopicsList = util.Arrays.asList(
+      new TopicPartition(topic1, 0),
+      new TopicPartition(topic2, 0),
+      new TopicPartition(topic2, 1),
+      new TopicPartition(topic3, 0),
+      new TopicPartition(topic3, 1),
+      new TopicPartition(topic3, 2))
+
+    // create group to partition map to build batched offsetFetch request
+    val groupToPartitionMap: util.Map[String, util.List[TopicPartition]] =
+      new util.HashMap[String, util.List[TopicPartition]]()
+    groupToPartitionMap.put(groups(1), topic1List)

Review comment:
       groups(0) to groups(4)




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664803616



##########
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:
       I refactored this test class a bit to make use of more helper methods - is there something additional we can do here to make it work on a collection of groups? Are you suggesting adding an additional method that can create some number of groups that we can use the test with?




-- 
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] skaundinya15 commented on a change in pull request #10962: KAFKA-12234: Implement request/response for offsetFetch batching (KIP-709)

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r666432011



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +315,28 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public boolean isAllPartitionsForGroup(String groupId) {
+        OffsetFetchRequestGroup group = data
+            .groupIds()
+            .stream()
+            .filter(g -> g.groupId().equals(groupId))
+            .collect(toSingleton());
+        return group.topics() == ALL_TOPIC_PARTITIONS_BATCH;
+    }
+
+    // Custom collector to filter a single element
+    private <T> Collector<T, ?, T> toSingleton() {

Review comment:
       Opened https://github.com/apache/kafka/pull/11000 for this. We can continue the discussion 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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663419332



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -154,14 +166,88 @@ public OffsetFetchResponse(int throttleTimeMs, Errors error, Map<TopicPartition,
         this.error = error;
     }
 
+    /**
+     * Constructor without throttle time for version 8 and above.
+     * @param errors Error code on a per group level basis
+     * @param responseData Fetched offset information grouped group id
+     */
+    public OffsetFetchResponse(Map<String, Errors> errors, Map<String, Map<TopicPartition, PartitionData>> responseData) {

Review comment:
       This constructor allows us to initialize errors and the topic partitions for multiple groups. Since the KIP is for implementing batching of the `offsetFetch` API, this constructor allows for exactly that. The request allows for multiple groups to get offsets for, and the response is so we can return all of those offsets for each group requested.




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663419209



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -65,6 +69,8 @@
 
     private final OffsetFetchResponseData data;
     private final Errors error;
+    private final Map<String, Errors> groupLevelErrors = new HashMap<>();
+    private final Map<String, Map<TopicPartition, PartitionData>> groupToPartitionData = new HashMap<>();

Review comment:
       We do have it in `data` but it is not easy to query them quickly. Putting them in a map makes it faster to query them. Otherwise we would have to always parse through all the list of group level errors and partition level data in order to get something that is specific to a particular group id. I figured it would be more worth it to cache these upon initialization of the request in a map so that whenever we want to figure out if a group has a specific error or what topic partitions it is trying to fetch offsets for, it would be easily returnable. 




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663419495



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -154,14 +166,88 @@ public OffsetFetchResponse(int throttleTimeMs, Errors error, Map<TopicPartition,
         this.error = error;
     }
 
+    /**
+     * Constructor without throttle time for version 8 and above.
+     * @param errors Error code on a per group level basis
+     * @param responseData Fetched offset information grouped group id
+     */
+    public OffsetFetchResponse(Map<String, Errors> errors, Map<String, Map<TopicPartition, PartitionData>> responseData) {
+        this(DEFAULT_THROTTLE_TIME, errors, responseData);
+    }
+
+    /**
+     * Constructor with throttle time for version 8 and above.
+     * @param throttleTimeMs The time in milliseconds that this response was throttled
+     * @param errors Potential coordinator or group level error code (for api version 2 and later)
+     * @param responseData Fetched offset information grouped by topic-partition and by group
+     */
+    public OffsetFetchResponse(int throttleTimeMs, Map<String, Errors> errors, Map<String,
+        Map<TopicPartition, PartitionData>> responseData) {
+        super(ApiKeys.OFFSET_FETCH);
+        List<OffsetFetchResponseGroup> groupList = new ArrayList<>();
+        for (Entry<String, Map<TopicPartition, PartitionData>> entry : responseData.entrySet()) {
+            Map<String, OffsetFetchResponseTopics> offsetFetchResponseTopicsMap = new HashMap<>();
+            for (Entry<TopicPartition, PartitionData> partitionEntry :
+                responseData.get(entry.getKey()).entrySet()) {

Review comment:
       I think we still want `entry.getKey()` as that returns the `String` group Id. `entry.getValue()` returns `Map<TopicPartition, PartitionData>`. I will change it so that we save `entry.getKey()` in a local variable so we don't have to keep calling 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] rajinisivaram commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663488495



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##########
@@ -1308,29 +1308,31 @@ private OffsetFetchResponseHandler() {
 
         @Override
         public void handle(OffsetFetchResponse response, RequestFuture<Map<TopicPartition, OffsetAndMetadata>> future) {
-            if (response.hasError()) {
-                Errors error = response.error();
-                log.debug("Offset fetch failed: {}", error.message());
+            Errors responseError = response.groupLevelError(rebalanceConfig.groupId);

Review comment:
       We could just call this `error` and then won't require the remaining changes below.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -78,26 +85,107 @@ boolean isAllTopicPartitions() {
             return this.data.topics() == ALL_TOPIC_PARTITIONS;
         }
 
+        public Builder(Map<String, List<TopicPartition>> groupIdToTopicPartitionMap,
+            boolean requireStable,

Review comment:
       nit: indentation

##########
File path: clients/src/test/java/org/apache/kafka/common/message/MessageTest.java
##########
@@ -662,20 +669,179 @@ public void testOffsetFetchVersions() throws Exception {
                       .setErrorCode(Errors.NOT_COORDINATOR.code())
                       .setThrottleTimeMs(10);
         for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
-            OffsetFetchResponseData responseData = response.get();
-            if (version <= 1) {
-                responseData.setErrorCode(Errors.NONE.code());
+            if (version < 8) {

Review comment:
       As before, we should have `if version >= 8` as well.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java
##########
@@ -87,12 +87,16 @@ public String apiName() {
         Map<CoordinatorKey, Throwable> failed = new HashMap<>();
         List<CoordinatorKey> unmapped = new ArrayList<>();
 
-        if (response.error() != Errors.NONE) {
-            handleError(groupId, response.error(), failed, unmapped);
+        Errors responseError = response.groupLevelError(groupId.idValue);
+        if (responseError != Errors.NONE) {
+            handleError(groupId, responseError, failed, unmapped);
         } else {
             final Map<TopicPartition, OffsetAndMetadata> groupOffsetsListing = new HashMap<>();
-            for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry :
-                response.responseData().entrySet()) {
+            // if entry for group level response data is null, we are getting back an older version
+            // of the response

Review comment:
       Is this comment required?

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -1255,81 +1256,143 @@ class KafkaApis(val requestChannel: RequestChannel,
    * Handle an offset fetch request
    */
   def handleOffsetFetchRequest(request: RequestChannel.Request): Unit = {
+    val version = request.header.apiVersion
+    if (version == 0) {
+      // reading offsets from ZK
+      handleOffsetFetchRequestV0(request)
+    } else if (version >= 1 && version <= 7) {
+      // reading offsets from Kafka
+      handleOffsetFetchRequestBetweenV1AndV7(request)
+    } else {
+      // batching offset reads for multiple groups starts with version 8 and greater
+      handleOffsetFetchRequestV8AndAbove(request)
+    }
+  }
+
+  private def handleOffsetFetchRequestV0(request: RequestChannel.Request): Unit = {
     val header = request.header
     val offsetFetchRequest = request.body[OffsetFetchRequest]
 
-    def partitionByAuthorized(seq: Seq[TopicPartition]): (Seq[TopicPartition], Seq[TopicPartition]) =
-      authHelper.partitionSeqByAuthorized(request.context, DESCRIBE, TOPIC, seq)(_.topic)
-
     def createResponse(requestThrottleMs: Int): AbstractResponse = {
       val offsetFetchResponse =
-        // reject the request if not authorized to the group
+      // reject the request if not authorized to the group

Review comment:
       nit: revert indentation change

##########
File path: core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
##########
@@ -1358,17 +1367,241 @@ 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.responseData(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.responseData(group).containsKey(tp))
+    assertEquals(offset, offsetFetchResponse.responseData(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)
+
+    val topic1 = "topic1"
+    val topic1List = singletonList(new TopicPartition(topic1, 0))
+    val topicOneResource = new ResourcePattern(TOPIC, topic1, LITERAL)
+    val topic2 = "topic2"
+    val topic1And2List = util.Arrays.asList(
+      new TopicPartition(topic1, 0),
+      new TopicPartition(topic2, 0),
+      new TopicPartition(topic2, 1))
+    val topicTwoResource = new ResourcePattern(TOPIC, topic2, LITERAL)
+    val topic3 = "topic3"
+    val allTopicsList = util.Arrays.asList(
+      new TopicPartition(topic1, 0),
+      new TopicPartition(topic2, 0),
+      new TopicPartition(topic2, 1),
+      new TopicPartition(topic3, 0),
+      new TopicPartition(topic3, 1),
+      new TopicPartition(topic3, 2))
+    val topicThreeResource = new ResourcePattern(TOPIC, topic3, LITERAL)
+
+    // create group to partition map to build batched offsetFetch request
+    val groupToPartitionMap = new util.HashMap[String, util.List[TopicPartition]]()
+    groupToPartitionMap.put(groupOne, topic1List)
+    groupToPartitionMap.put(groupTwo, topic1And2List)
+    groupToPartitionMap.put(groupThree, allTopicsList)
+    groupToPartitionMap.put(groupFour, null)
+    groupToPartitionMap.put(groupFive, null)
+
+    createTopic(topic1)
+    createTopic(topic2, numPartitions = 2)
+    createTopic(topic3, numPartitions = 3)
+
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupOneResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupTwoResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupThreeResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupFourResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupFiveResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicOneResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicTwoResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicThreeResource)
+
+    val offset = 15L
+    val leaderEpoch: Optional[Integer] = Optional.of(1)
+    val metadata = "metadata"
+    val topicOneOffsets = topic1List.asScala.map{

Review comment:
       nit: space before braces (multiple places)

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -78,26 +85,107 @@ boolean isAllTopicPartitions() {
             return this.data.topics() == ALL_TOPIC_PARTITIONS;
         }
 
+        public Builder(Map<String, List<TopicPartition>> groupIdToTopicPartitionMap,
+            boolean requireStable,
+            boolean throwOnFetchStableOffsetsUnsupported) {
+            super(ApiKeys.OFFSET_FETCH);
+
+            List<OffsetFetchRequestGroup> groups = new ArrayList<>();
+            for (Entry<String, List<TopicPartition>> entry : groupIdToTopicPartitionMap.entrySet()) {
+                String groupName = entry.getKey();
+                List<TopicPartition> tpList = entry.getValue();
+                final List<OffsetFetchRequestTopics> topics;
+                if (tpList != null) {
+                    Map<String, OffsetFetchRequestTopics> offsetFetchRequestTopicMap =
+                        new HashMap<>();
+                    for (TopicPartition topicPartition : tpList) {
+                        String topicName = topicPartition.topic();
+                        OffsetFetchRequestTopics topic = offsetFetchRequestTopicMap.getOrDefault(
+                            topicName, new OffsetFetchRequestTopics().setName(topicName));
+                        topic.partitionIndexes().add(topicPartition.partition());
+                        offsetFetchRequestTopicMap.put(topicName, topic);
+                    }
+                    topics = new ArrayList<>(offsetFetchRequestTopicMap.values());
+                } else {
+                    topics = ALL_TOPIC_PARTITIONS_BATCH;
+                }
+                groups.add(new OffsetFetchRequestGroup()
+                    .setGroupId(groupName)
+                    .setTopics(topics));
+            }
+            this.data = new OffsetFetchRequestData()
+                .setGroupIds(groups)
+                .setRequireStable(requireStable);
+            this.throwOnFetchStableOffsetsUnsupported = throwOnFetchStableOffsetsUnsupported;
+        }
+
         @Override
         public OffsetFetchRequest build(short version) {
             if (isAllTopicPartitions() && version < 2) {
                 throw new UnsupportedVersionException("The broker only supports OffsetFetchRequest " +
                     "v" + version + ", but we need v2 or newer to request all topic partitions.");
             }
-
+            if (data.groupIds().size() > 1 && version < 8) {
+                throw new NoBatchedOffsetFetchRequestException("Broker does not support"
+                    + " batching groups for fetch offset request on version " + version);
+            }
             if (data.requireStable() && version < 7) {
                 if (throwOnFetchStableOffsetsUnsupported) {
                     throw new UnsupportedVersionException("Broker unexpectedly " +
                         "doesn't support requireStable flag on version " + version);
                 } else {
                     log.trace("Fallback the requireStable flag to false as broker " +
-                                  "only supports OffsetFetchRequest version {}. Need " +
-                                  "v7 or newer to enable this feature", version);
+                        "only supports OffsetFetchRequest version {}. Need " +
+                        "v7 or newer to enable this feature", version);
 
                     return new OffsetFetchRequest(data.setRequireStable(false), version);
                 }
             }
-
+            if (version < 8) {
+                OffsetFetchRequestData oldDataFormat = null;
+                if (!data.groupIds().isEmpty()) {
+                    OffsetFetchRequestGroup group = data.groupIds().get(0);
+                    String groupName = group.groupId();
+                    List<OffsetFetchRequestTopics> topics = group.topics();
+                    List<OffsetFetchRequestTopic> oldFormatTopics = null;
+                    if (topics != null) {
+                        oldFormatTopics = topics
+                            .stream()
+                            .map(t ->
+                                new OffsetFetchRequestTopic()
+                                    .setName(t.name())
+                                    .setPartitionIndexes(t.partitionIndexes()))
+                            .collect(Collectors.toList());
+                    }
+                    oldDataFormat = new OffsetFetchRequestData()
+                        .setGroupId(groupName)
+                        .setTopics(oldFormatTopics)
+                        .setRequireStable(data.requireStable());
+                }
+                return new OffsetFetchRequest(oldDataFormat == null ? data : oldDataFormat, version);
+            }
+            // version 8 but have used old format of request, convert to version 8 of request

Review comment:
       May be better to put the following code in an else statement and put the comment at the start of the `if` block since we may be converting for version < 8 as well in the block above.

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchRequestTest.java
##########
@@ -76,62 +73,169 @@ public void testConstructor() {
         }
 
         for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
-            OffsetFetchRequest request = builder.build(version);
-            assertFalse(request.isAllPartitions());
-            assertEquals(groupId, request.groupId());
-            assertEquals(partitions, request.partitions());
-
-            OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
-            assertEquals(Errors.NONE, response.error());
-            assertFalse(response.hasError());
-            assertEquals(Collections.singletonMap(Errors.NONE, version <= (short) 1 ? 3 : 1), response.errorCounts(),
-                "Incorrect error count for version " + version);
-
-            if (version <= 1) {
-                assertEquals(expectedData, response.responseData());
+            if (version < 8) {
+                builder = new OffsetFetchRequest.Builder(
+                    group1,
+                    false,
+                    partitions,
+                    false);
+                assertFalse(builder.isAllTopicPartitions());
+                OffsetFetchRequest request = builder.build(version);
+                assertFalse(request.isAllPartitions());
+                assertEquals(group1, request.groupId());
+                assertEquals(partitions, request.partitions());
+
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.error());
+                assertFalse(response.hasError());
+                assertEquals(Collections.singletonMap(Errors.NONE, version <= (short) 1 ? 3 : 1), response.errorCounts(),
+                    "Incorrect error count for version " + version);
+
+                if (version <= 1) {
+                    assertEquals(expectedData, response.responseDataV0ToV7());
+                }
+
+                if (version >= 3) {
+                    assertEquals(throttleTimeMs, response.throttleTimeMs());
+                } else {
+                    assertEquals(DEFAULT_THROTTLE_TIME, response.throttleTimeMs());
+                }
+            } else {
+                builder = new Builder(Collections.singletonMap(group1, partitions), false, false);
+                OffsetFetchRequest request = builder.build(version);
+                Map<String, List<TopicPartition>> groupToPartitionMap =
+                    request.groupIdsToPartitions();
+                Map<String, List<OffsetFetchRequestTopics>> groupToTopicMap =
+                    request.groupIdsToTopics();
+                assertFalse(request.isAllPartitionsForGroup(group1));
+                assertTrue(groupToPartitionMap.containsKey(group1) && groupToTopicMap.containsKey(
+                    group1));
+                assertEquals(partitions, groupToPartitionMap.get(group1));
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.groupLevelError(group1));
+                assertFalse(response.groupHasError(group1));
+                assertEquals(Collections.singletonMap(Errors.NONE, 1), response.errorCounts(),
+                    "Incorrect error count for version " + version);
+                assertEquals(throttleTimeMs, response.throttleTimeMs());
             }
+        }
+    }
+
+    @Test
+    public void testConstructorWithMultipleGroups() {
+        List<TopicPartition> topic1Partitions = Arrays.asList(
+            new TopicPartition(topicOne, partitionOne),
+            new TopicPartition(topicOne, partitionTwo));
+        List<TopicPartition> topic2Partitions = Arrays.asList(
+            new TopicPartition(topicTwo, partitionOne),
+            new TopicPartition(topicTwo, partitionTwo));
+        List<TopicPartition> topic3Partitions = Arrays.asList(
+            new TopicPartition(topicThree, partitionOne),
+            new TopicPartition(topicThree, partitionTwo));
+        Map<String, List<TopicPartition>> groupToTp = new HashMap<>();
+        groupToTp.put(group1, topic1Partitions);
+        groupToTp.put(group2, topic2Partitions);
+        groupToTp.put(group3, topic3Partitions);
+        groupToTp.put(group4, null);
+        groupToTp.put(group5, null);
+        int throttleTimeMs = 10;
 
-            if (version >= 3) {
+        for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
+            if (version >= 8) {
+                builder = new Builder(groupToTp, false, false);
+                OffsetFetchRequest request = builder.build(version);
+                Map<String, List<TopicPartition>> groupToPartitionMap =
+                    request.groupIdsToPartitions();
+                Map<String, List<OffsetFetchRequestTopics>> groupToTopicMap =
+                    request.groupIdsToTopics();
+                assertEquals(groupToTp.keySet(), groupToTopicMap.keySet());
+                assertEquals(groupToTp.keySet(), groupToPartitionMap.keySet());
+                assertFalse(request.isAllPartitionsForGroup(group1));
+                assertFalse(request.isAllPartitionsForGroup(group2));
+                assertFalse(request.isAllPartitionsForGroup(group3));
+                assertTrue(request.isAllPartitionsForGroup(group4));
+                assertTrue(request.isAllPartitionsForGroup(group5));
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.groupLevelError(group1));
+                assertEquals(Errors.NONE, response.groupLevelError(group2));
+                assertEquals(Errors.NONE, response.groupLevelError(group3));
+                assertEquals(Errors.NONE, response.groupLevelError(group4));
+                assertEquals(Errors.NONE, response.groupLevelError(group5));

Review comment:
       List of groups will enable us to use loop here.

##########
File path: clients/src/test/java/org/apache/kafka/common/message/MessageTest.java
##########
@@ -633,17 +638,19 @@ public void testOffsetFetchVersions() throws Exception {
                                                        .setRequireStable(true);
 
         for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
-            final short finalVersion = version;
-            if (version < 2) {
-                assertThrows(NullPointerException.class, () -> testAllMessageRoundTripsFromVersion(finalVersion, allPartitionData));
-            } else {
-                testAllMessageRoundTripsFromVersion(version, allPartitionData);
-            }
-
-            if (version < 7) {
-                assertThrows(UnsupportedVersionException.class, () -> testAllMessageRoundTripsFromVersion(finalVersion, requireStableData));
-            } else {
-                testAllMessageRoundTripsFromVersion(finalVersion, requireStableData);
+            if (version < 8) {

Review comment:
       We should probably just add this test for v8 and above as well. The other v8 test can just focus on multiple groups, this one can test the common case of single group that is used by consumers. 

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchRequestTest.java
##########
@@ -76,62 +73,169 @@ public void testConstructor() {
         }
 
         for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
-            OffsetFetchRequest request = builder.build(version);
-            assertFalse(request.isAllPartitions());
-            assertEquals(groupId, request.groupId());
-            assertEquals(partitions, request.partitions());
-
-            OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
-            assertEquals(Errors.NONE, response.error());
-            assertFalse(response.hasError());
-            assertEquals(Collections.singletonMap(Errors.NONE, version <= (short) 1 ? 3 : 1), response.errorCounts(),
-                "Incorrect error count for version " + version);
-
-            if (version <= 1) {
-                assertEquals(expectedData, response.responseData());
+            if (version < 8) {
+                builder = new OffsetFetchRequest.Builder(
+                    group1,
+                    false,
+                    partitions,
+                    false);
+                assertFalse(builder.isAllTopicPartitions());
+                OffsetFetchRequest request = builder.build(version);
+                assertFalse(request.isAllPartitions());
+                assertEquals(group1, request.groupId());
+                assertEquals(partitions, request.partitions());
+
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.error());
+                assertFalse(response.hasError());
+                assertEquals(Collections.singletonMap(Errors.NONE, version <= (short) 1 ? 3 : 1), response.errorCounts(),
+                    "Incorrect error count for version " + version);
+
+                if (version <= 1) {
+                    assertEquals(expectedData, response.responseDataV0ToV7());
+                }
+
+                if (version >= 3) {
+                    assertEquals(throttleTimeMs, response.throttleTimeMs());
+                } else {
+                    assertEquals(DEFAULT_THROTTLE_TIME, response.throttleTimeMs());
+                }
+            } else {
+                builder = new Builder(Collections.singletonMap(group1, partitions), false, false);
+                OffsetFetchRequest request = builder.build(version);
+                Map<String, List<TopicPartition>> groupToPartitionMap =
+                    request.groupIdsToPartitions();
+                Map<String, List<OffsetFetchRequestTopics>> groupToTopicMap =
+                    request.groupIdsToTopics();
+                assertFalse(request.isAllPartitionsForGroup(group1));
+                assertTrue(groupToPartitionMap.containsKey(group1) && groupToTopicMap.containsKey(
+                    group1));
+                assertEquals(partitions, groupToPartitionMap.get(group1));
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.groupLevelError(group1));
+                assertFalse(response.groupHasError(group1));
+                assertEquals(Collections.singletonMap(Errors.NONE, 1), response.errorCounts(),
+                    "Incorrect error count for version " + version);
+                assertEquals(throttleTimeMs, response.throttleTimeMs());
             }
+        }
+    }
+
+    @Test
+    public void testConstructorWithMultipleGroups() {
+        List<TopicPartition> topic1Partitions = Arrays.asList(
+            new TopicPartition(topicOne, partitionOne),
+            new TopicPartition(topicOne, partitionTwo));
+        List<TopicPartition> topic2Partitions = Arrays.asList(
+            new TopicPartition(topicTwo, partitionOne),
+            new TopicPartition(topicTwo, partitionTwo));
+        List<TopicPartition> topic3Partitions = Arrays.asList(
+            new TopicPartition(topicThree, partitionOne),
+            new TopicPartition(topicThree, partitionTwo));
+        Map<String, List<TopicPartition>> groupToTp = new HashMap<>();
+        groupToTp.put(group1, topic1Partitions);
+        groupToTp.put(group2, topic2Partitions);
+        groupToTp.put(group3, topic3Partitions);
+        groupToTp.put(group4, null);
+        groupToTp.put(group5, null);
+        int throttleTimeMs = 10;
 
-            if (version >= 3) {
+        for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
+            if (version >= 8) {
+                builder = new Builder(groupToTp, false, false);
+                OffsetFetchRequest request = builder.build(version);
+                Map<String, List<TopicPartition>> groupToPartitionMap =
+                    request.groupIdsToPartitions();
+                Map<String, List<OffsetFetchRequestTopics>> groupToTopicMap =
+                    request.groupIdsToTopics();
+                assertEquals(groupToTp.keySet(), groupToTopicMap.keySet());
+                assertEquals(groupToTp.keySet(), groupToPartitionMap.keySet());
+                assertFalse(request.isAllPartitionsForGroup(group1));
+                assertFalse(request.isAllPartitionsForGroup(group2));
+                assertFalse(request.isAllPartitionsForGroup(group3));
+                assertTrue(request.isAllPartitionsForGroup(group4));
+                assertTrue(request.isAllPartitionsForGroup(group5));
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.groupLevelError(group1));
+                assertEquals(Errors.NONE, response.groupLevelError(group2));
+                assertEquals(Errors.NONE, response.groupLevelError(group3));
+                assertEquals(Errors.NONE, response.groupLevelError(group4));
+                assertEquals(Errors.NONE, response.groupLevelError(group5));
+                assertFalse(response.groupHasError(group1));
+                assertFalse(response.groupHasError(group2));
+                assertFalse(response.groupHasError(group3));
+                assertFalse(response.groupHasError(group4));
+                assertFalse(response.groupHasError(group5));
+                assertEquals(Collections.singletonMap(Errors.NONE, 5), response.errorCounts(),
+                    "Incorrect error count for version " + version);
                 assertEquals(throttleTimeMs, response.throttleTimeMs());
-            } else {
-                assertEquals(DEFAULT_THROTTLE_TIME, response.throttleTimeMs());
             }
         }
     }
 
     @Test
-    public void testConstructorFailForUnsupportedRequireStable() {
+    public void testBuildThrowForUnsupportedBatchRequest() {
         for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
-            // The builder needs to be initialized every cycle as the internal data `requireStable` flag is flipped.
-            builder = new OffsetFetchRequest.Builder(groupId, true, null, false);
-            final short finalVersion = version;
-            if (version < 2) {
-                assertThrows(UnsupportedVersionException.class, () -> builder.build(finalVersion));
-            } else {
-                OffsetFetchRequest request = builder.build(finalVersion);
-                assertEquals(groupId, request.groupId());
-                assertNull(request.partitions());
-                assertTrue(request.isAllPartitions());
-                if (version < 7) {
-                    assertFalse(request.requireStable());
-                } else {
-                    assertTrue(request.requireStable());
-                }
+            if (version < 8) {
+                Map<String, List<TopicPartition>> groupPartitionMap = new HashMap<>();
+                groupPartitionMap.put(group1, null);
+                groupPartitionMap.put(group2, null);
+                builder = new Builder(groupPartitionMap, true, false);
+                final short finalVersion = version;
+                assertThrows(NoBatchedOffsetFetchRequestException.class, () -> builder.build(finalVersion));
             }
         }
     }
 
     @Test
-    public void testBuildThrowForUnsupportedRequireStable() {
+    public void testConstructorFailForUnsupportedRequireStable() {
         for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
-            builder = new OffsetFetchRequest.Builder(groupId, true, null, true);
-            if (version < 7) {
+            if (version < 8) {
+                // The builder needs to be initialized every cycle as the internal data `requireStable` flag is flipped.
+                builder = new OffsetFetchRequest.Builder(group1, true, null, false);
                 final short finalVersion = version;
-                assertThrows(UnsupportedVersionException.class, () -> builder.build(finalVersion));
+                if (version < 2) {
+                    assertThrows(UnsupportedVersionException.class, () -> builder.build(finalVersion));
+                } else {
+                    OffsetFetchRequest request = builder.build(finalVersion);
+                    assertEquals(group1, request.groupId());
+                    assertNull(request.partitions());
+                    assertTrue(request.isAllPartitions());
+                    if (version < 7) {
+                        assertFalse(request.requireStable());
+                    } else {
+                        assertTrue(request.requireStable());
+                    }
+                }
             } else {
+                builder = new Builder(Collections.singletonMap(group1, null), true, false);
                 OffsetFetchRequest request = builder.build(version);
+                Map<String, List<TopicPartition>> groupToPartitionMap =
+                    request.groupIdsToPartitions();
+                Map<String, List<OffsetFetchRequestTopics>> groupToTopicMap =
+                    request.groupIdsToTopics();
+                assertTrue(groupToPartitionMap.containsKey(group1) && groupToTopicMap.containsKey(
+                    group1));
+                assertNull(groupToPartitionMap.get(group1));
+                assertTrue(request.isAllPartitionsForGroup(group1));
                 assertTrue(request.requireStable());
             }
         }
     }
+
+    @Test
+    public void testBuildThrowForUnsupportedRequireStable() {
+        for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
+            if (version < 8) {

Review comment:
       As before, it will be good to include the same logic for `version >=8` as well. Based on the number of tests that use the pattern of `ApiKeys.OFFSET_FETCH.allVersions()` followed by `if (version < 8)`, it may be good to add the list of versions < 8 and use that in these cases if we require this pattern.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -154,14 +167,74 @@ public OffsetFetchResponse(int throttleTimeMs, Errors error, Map<TopicPartition,
         this.error = error;
     }
 
+    /**
+     * Constructor without throttle time for version 8 and above.
+     * @param errors Error code on a per group level basis
+     * @param responseData Fetched offset information grouped group id
+     */
+    public OffsetFetchResponse(Map<String, Errors> errors, Map<String, Map<TopicPartition, PartitionData>> responseData) {
+        this(DEFAULT_THROTTLE_TIME, errors, responseData);
+    }
+
+    /**
+     * Constructor with throttle time for version 8 and above.
+     * @param throttleTimeMs The time in milliseconds that this response was throttled
+     * @param errors Potential coordinator or group level error code
+     * @param responseData Fetched offset information grouped by topic-partition and by group
+     */
+    public OffsetFetchResponse(int throttleTimeMs, Map<String, Errors> errors, Map<String,
+        Map<TopicPartition, PartitionData>> responseData) {

Review comment:
       nit: indentation

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +315,28 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public boolean isAllPartitionsForGroup(String groupId) {
+        OffsetFetchRequestGroup group = data
+            .groupIds()
+            .stream()
+            .filter(g -> g.groupId().equals(groupId))
+            .collect(toSingleton());

Review comment:
       This is going to throw IllegalStateException if a group is included multiple times in a request. If that is an invalid request, we should throw InvalidRequestException before it gets here to ensure we don't return UNKNOWN_SERVER_ERROR. More likely, it is valid and we should not throw an exception.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -113,8 +120,14 @@ public int hashCode() {
         }
     }
 
+    public OffsetFetchResponse(OffsetFetchResponseData data) {
+        super(ApiKeys.OFFSET_FETCH);
+        this.data = data;
+        this.error = null;
+    }
+
     /**
-     * Constructor for all versions without throttle time.
+     * Constructor without throttle time for version 0 to version 7.

Review comment:
       It is not actually for version 0 to 7, it is for the versions without throttle time.

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchRequestTest.java
##########
@@ -44,25 +46,20 @@
     private final int partitionOne = 1;
     private final String topicTwo = "topic2";
     private final int partitionTwo = 2;
-    private final String groupId = "groupId";
+    private final String topicThree = "topic3";
+    private final String group1 = "group1";
+    private final String group2 = "group2";
+    private final String group3 = "group3";
+    private final String group4 = "group4";
+    private final String group5 = "group5";

Review comment:
       Maybe better to define `List<String> groups` to avoid repetition here and in the test logic

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -185,21 +249,46 @@ public boolean hasError() {
         return error != Errors.NONE;
     }
 
+    public boolean groupHasError(String groupId) {
+        return groupLevelErrors.get(groupId) != Errors.NONE;
+    }
+
     public Errors error() {
         return error;
     }
 
+    public Errors groupLevelError(String groupId) {
+        if (error != null) {
+            return error;
+        }
+        return groupLevelErrors.get(groupId);
+    }
+
     @Override
     public Map<Errors, Integer> errorCounts() {
         Map<Errors, Integer> counts = new HashMap<>();
-        updateErrorCounts(counts, error);
-        data.topics().forEach(topic ->
-                topic.partitions().forEach(partition ->
+        if (!groupLevelErrors.isEmpty()) {
+            // built response with v8 or above
+            for (Map.Entry<String, Errors> entry : groupLevelErrors.entrySet()) {
+                updateErrorCounts(counts, entry.getValue());
+            }
+            for (OffsetFetchResponseGroup group : data.groupIds()) {
+                group.topics().forEach(topic ->
+                    topic.partitions().forEach(partition ->
                         updateErrorCounts(counts, Errors.forCode(partition.errorCode()))));
+            }
+        } else {
+            // built response with v0-v7
+            updateErrorCounts(counts, error);
+            data.topics().forEach(topic ->
+                topic.partitions().forEach(partition ->
+                    updateErrorCounts(counts, Errors.forCode(partition.errorCode()))));
+        }
         return counts;
     }
 
-    public Map<TopicPartition, PartitionData> responseData() {
+    //public for testing purposes
+    public Map<TopicPartition, PartitionData> responseDataV0ToV7() {

Review comment:
       package-private should be sufficient for unit tests? May also be possible keep this private and call `responseData()` directly to ensure the right data is returned based on version.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -214,6 +303,46 @@ public Errors error() {
         return responseData;
     }
 
+    private Map<TopicPartition, PartitionData> buildResponseData(String groupId) {
+        Map<TopicPartition, PartitionData> responseData = new HashMap<>();
+        OffsetFetchResponseGroup group = data
+            .groupIds()
+            .stream()
+            .filter(g -> g.groupId().equals(groupId))
+            .collect(toSingleton());
+        for (OffsetFetchResponseTopics topic : group.topics()) {
+            for (OffsetFetchResponsePartitions partition : topic.partitions()) {
+                responseData.put(new TopicPartition(topic.name(), partition.partitionIndex()),
+                    new PartitionData(partition.committedOffset(),
+                        RequestUtils.getLeaderEpoch(partition.committedLeaderEpoch()),
+                        partition.metadata(),
+                        Errors.forCode(partition.errorCode()))
+                );
+            }
+        }
+        return responseData;
+    }
+
+    // Custom collector to filter a single element
+    private <T> Collector<T, ?, T> toSingleton() {

Review comment:
       See earlier note about this method.

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##########
@@ -1308,29 +1308,31 @@ private OffsetFetchResponseHandler() {
 
         @Override
         public void handle(OffsetFetchResponse response, RequestFuture<Map<TopicPartition, OffsetAndMetadata>> future) {
-            if (response.hasError()) {
-                Errors error = response.error();
-                log.debug("Offset fetch failed: {}", error.message());
+            Errors responseError = response.groupLevelError(rebalanceConfig.groupId);
+            if (responseError != Errors.NONE) {
+                log.debug("Offset fetch failed: {}", responseError.message());
 
-                if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS) {
+                if (responseError == Errors.COORDINATOR_LOAD_IN_PROGRESS) {
                     // just retry
-                    future.raise(error);
-                } else if (error == Errors.NOT_COORDINATOR) {
+                    future.raise(responseError);
+                } else if (responseError == Errors.NOT_COORDINATOR) {
                     // re-discover the coordinator and retry
-                    markCoordinatorUnknown(error);
-                    future.raise(error);
-                } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) {
+                    markCoordinatorUnknown(responseError);
+                    future.raise(responseError);
+                } else if (responseError == Errors.GROUP_AUTHORIZATION_FAILED) {
                     future.raise(GroupAuthorizationException.forGroupId(rebalanceConfig.groupId));
                 } else {
-                    future.raise(new KafkaException("Unexpected error in fetch offset response: " + error.message()));
+                    future.raise(new KafkaException("Unexpected error in fetch offset response: " + responseError.message()));
                 }
                 return;
             }
 
             Set<String> unauthorizedTopics = null;
-            Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>(response.responseData().size());
+            Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData =

Review comment:
       as before, rename responseData?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java
##########
@@ -87,12 +87,16 @@ public String apiName() {
         Map<CoordinatorKey, Throwable> failed = new HashMap<>();
         List<CoordinatorKey> unmapped = new ArrayList<>();
 
-        if (response.error() != Errors.NONE) {
-            handleError(groupId, response.error(), failed, unmapped);
+        Errors responseError = response.groupLevelError(groupId.idValue);
+        if (responseError != Errors.NONE) {
+            handleError(groupId, responseError, failed, unmapped);
         } else {
             final Map<TopicPartition, OffsetAndMetadata> groupOffsetsListing = new HashMap<>();
-            for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry :
-                response.responseData().entrySet()) {
+            // if entry for group level response data is null, we are getting back an older version
+            // of the response
+            Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData =
+                response.responseData(groupId.idValue);

Review comment:
       Perhaps call the method and variable partitionData or offsetData rather than responseData since responseData is too close to response.data?

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +315,28 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public boolean isAllPartitionsForGroup(String groupId) {
+        OffsetFetchRequestGroup group = data
+            .groupIds()
+            .stream()
+            .filter(g -> g.groupId().equals(groupId))
+            .collect(toSingleton());
+        return group.topics() == ALL_TOPIC_PARTITIONS_BATCH;
+    }
+
+    // Custom collector to filter a single element
+    private <T> Collector<T, ?, T> toSingleton() {

Review comment:
       We seem to have two copies of this method in the PR. We need to decide first whether this logic is correct first. If we really do want the method, we need to make it shared.

##########
File path: core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
##########
@@ -1358,17 +1367,241 @@ 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.responseData(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.responseData(group).containsKey(tp))
+    assertEquals(offset, offsetFetchResponse.responseData(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)
+
+    val topic1 = "topic1"
+    val topic1List = singletonList(new TopicPartition(topic1, 0))
+    val topicOneResource = new ResourcePattern(TOPIC, topic1, LITERAL)
+    val topic2 = "topic2"
+    val topic1And2List = util.Arrays.asList(
+      new TopicPartition(topic1, 0),
+      new TopicPartition(topic2, 0),
+      new TopicPartition(topic2, 1))
+    val topicTwoResource = new ResourcePattern(TOPIC, topic2, LITERAL)
+    val topic3 = "topic3"
+    val allTopicsList = util.Arrays.asList(
+      new TopicPartition(topic1, 0),
+      new TopicPartition(topic2, 0),
+      new TopicPartition(topic2, 1),
+      new TopicPartition(topic3, 0),
+      new TopicPartition(topic3, 1),
+      new TopicPartition(topic3, 2))
+    val topicThreeResource = new ResourcePattern(TOPIC, topic3, LITERAL)
+
+    // create group to partition map to build batched offsetFetch request
+    val groupToPartitionMap = new util.HashMap[String, util.List[TopicPartition]]()
+    groupToPartitionMap.put(groupOne, topic1List)
+    groupToPartitionMap.put(groupTwo, topic1And2List)
+    groupToPartitionMap.put(groupThree, allTopicsList)
+    groupToPartitionMap.put(groupFour, null)
+    groupToPartitionMap.put(groupFive, null)
+
+    createTopic(topic1)
+    createTopic(topic2, numPartitions = 2)
+    createTopic(topic3, numPartitions = 3)
+
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupOneResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupTwoResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupThreeResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupFourResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupFiveResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicOneResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicTwoResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicThreeResource)

Review comment:
       Looks like they are are all the same with different resources, better to put them in a collection and call `addAndVerifyAcls` over the collection

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
##########
@@ -1580,21 +1623,83 @@ private OffsetCommitResponse createOffsetCommitResponse() {
     }
 
     private OffsetFetchRequest createOffsetFetchRequest(int version, boolean requireStable) {
-        return new OffsetFetchRequest.Builder("group1", requireStable, Collections.singletonList(new TopicPartition("test11", 1)), false)
+        if (version < 8) {
+            return new OffsetFetchRequest.Builder(
+                "group1",
+                requireStable,
+                Collections.singletonList(
+                    new TopicPartition("test11", 1)),
+                false)
                 .build((short) version);
+        }
+        return new OffsetFetchRequest.Builder(
+            Collections.singletonMap(
+                "group1",
+                Collections.singletonList(
+                    new TopicPartition("test11", 1))),

Review comment:
       nit: unnecessary newlines

##########
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+",
+      "about": "The responses per group id.", "fields": [
+      { "name": "groupId", "type": "string", "versions": "8+", "entityType": "groupId",
+        "about": "The group ID." },
+      { "name": "Topics", "type": "[]OffsetFetchResponseTopics", "versions": "8+",
+        "about": "The responses per topic.", "fields": [
+        { "name": "Name", "type": "string", "versions": "8+", "entityType": "topicName",
+          "about": "The topic name." },
+        { "name": "Partitions", "type": "[]OffsetFetchResponsePartitions", "versions": "8+",
+          "about": "The responses per partition", "fields": [
+          { "name": "PartitionIndex", "type": "int32", "versions": "8+",
+            "about": "The partition index." },
+          { "name": "CommittedOffset", "type": "int64", "versions": "8+",
+            "about": "The committed message offset." },
+          { "name": "CommittedLeaderEpoch", "type": "int32", "versions": "8+", "default": "-1",
+            "ignorable": true, "about": "The leader epoch." },
+          { "name": "Metadata", "type": "string", "versions": "8+", "nullableVersions": "8+",
+            "about": "The partition metadata." },
+          { "name": "ErrorCode", "type": "int16", "versions": "8+",
+            "about": "The partition-level error code, or 0 if there was no error." }
+        ]}
+      ]},
+      { "name": "ErrorCode", "type": "int16", "versions": "8+", "default": "0", "ignorable": true,

Review comment:
       Don't think this should be ignorable

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchRequestTest.java
##########
@@ -76,62 +73,169 @@ public void testConstructor() {
         }
 
         for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
-            OffsetFetchRequest request = builder.build(version);
-            assertFalse(request.isAllPartitions());
-            assertEquals(groupId, request.groupId());
-            assertEquals(partitions, request.partitions());
-
-            OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
-            assertEquals(Errors.NONE, response.error());
-            assertFalse(response.hasError());
-            assertEquals(Collections.singletonMap(Errors.NONE, version <= (short) 1 ? 3 : 1), response.errorCounts(),
-                "Incorrect error count for version " + version);
-
-            if (version <= 1) {
-                assertEquals(expectedData, response.responseData());
+            if (version < 8) {
+                builder = new OffsetFetchRequest.Builder(
+                    group1,
+                    false,
+                    partitions,
+                    false);
+                assertFalse(builder.isAllTopicPartitions());
+                OffsetFetchRequest request = builder.build(version);
+                assertFalse(request.isAllPartitions());
+                assertEquals(group1, request.groupId());
+                assertEquals(partitions, request.partitions());
+
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.error());
+                assertFalse(response.hasError());
+                assertEquals(Collections.singletonMap(Errors.NONE, version <= (short) 1 ? 3 : 1), response.errorCounts(),
+                    "Incorrect error count for version " + version);
+
+                if (version <= 1) {
+                    assertEquals(expectedData, response.responseDataV0ToV7());
+                }
+
+                if (version >= 3) {
+                    assertEquals(throttleTimeMs, response.throttleTimeMs());
+                } else {
+                    assertEquals(DEFAULT_THROTTLE_TIME, response.throttleTimeMs());
+                }
+            } else {
+                builder = new Builder(Collections.singletonMap(group1, partitions), false, false);
+                OffsetFetchRequest request = builder.build(version);
+                Map<String, List<TopicPartition>> groupToPartitionMap =
+                    request.groupIdsToPartitions();
+                Map<String, List<OffsetFetchRequestTopics>> groupToTopicMap =
+                    request.groupIdsToTopics();
+                assertFalse(request.isAllPartitionsForGroup(group1));
+                assertTrue(groupToPartitionMap.containsKey(group1) && groupToTopicMap.containsKey(
+                    group1));
+                assertEquals(partitions, groupToPartitionMap.get(group1));
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.groupLevelError(group1));
+                assertFalse(response.groupHasError(group1));
+                assertEquals(Collections.singletonMap(Errors.NONE, 1), response.errorCounts(),
+                    "Incorrect error count for version " + version);
+                assertEquals(throttleTimeMs, response.throttleTimeMs());
             }
+        }
+    }
+
+    @Test
+    public void testConstructorWithMultipleGroups() {
+        List<TopicPartition> topic1Partitions = Arrays.asList(
+            new TopicPartition(topicOne, partitionOne),
+            new TopicPartition(topicOne, partitionTwo));
+        List<TopicPartition> topic2Partitions = Arrays.asList(
+            new TopicPartition(topicTwo, partitionOne),
+            new TopicPartition(topicTwo, partitionTwo));
+        List<TopicPartition> topic3Partitions = Arrays.asList(
+            new TopicPartition(topicThree, partitionOne),
+            new TopicPartition(topicThree, partitionTwo));
+        Map<String, List<TopicPartition>> groupToTp = new HashMap<>();
+        groupToTp.put(group1, topic1Partitions);
+        groupToTp.put(group2, topic2Partitions);
+        groupToTp.put(group3, topic3Partitions);
+        groupToTp.put(group4, null);
+        groupToTp.put(group5, null);
+        int throttleTimeMs = 10;
 
-            if (version >= 3) {
+        for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
+            if (version >= 8) {
+                builder = new Builder(groupToTp, false, false);
+                OffsetFetchRequest request = builder.build(version);
+                Map<String, List<TopicPartition>> groupToPartitionMap =
+                    request.groupIdsToPartitions();
+                Map<String, List<OffsetFetchRequestTopics>> groupToTopicMap =
+                    request.groupIdsToTopics();
+                assertEquals(groupToTp.keySet(), groupToTopicMap.keySet());
+                assertEquals(groupToTp.keySet(), groupToPartitionMap.keySet());
+                assertFalse(request.isAllPartitionsForGroup(group1));
+                assertFalse(request.isAllPartitionsForGroup(group2));
+                assertFalse(request.isAllPartitionsForGroup(group3));
+                assertTrue(request.isAllPartitionsForGroup(group4));
+                assertTrue(request.isAllPartitionsForGroup(group5));
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.groupLevelError(group1));
+                assertEquals(Errors.NONE, response.groupLevelError(group2));
+                assertEquals(Errors.NONE, response.groupLevelError(group3));
+                assertEquals(Errors.NONE, response.groupLevelError(group4));
+                assertEquals(Errors.NONE, response.groupLevelError(group5));
+                assertFalse(response.groupHasError(group1));
+                assertFalse(response.groupHasError(group2));
+                assertFalse(response.groupHasError(group3));
+                assertFalse(response.groupHasError(group4));
+                assertFalse(response.groupHasError(group5));
+                assertEquals(Collections.singletonMap(Errors.NONE, 5), response.errorCounts(),
+                    "Incorrect error count for version " + version);
                 assertEquals(throttleTimeMs, response.throttleTimeMs());
-            } else {
-                assertEquals(DEFAULT_THROTTLE_TIME, response.throttleTimeMs());
             }
         }
     }
 
     @Test
-    public void testConstructorFailForUnsupportedRequireStable() {
+    public void testBuildThrowForUnsupportedBatchRequest() {
         for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
-            // The builder needs to be initialized every cycle as the internal data `requireStable` flag is flipped.
-            builder = new OffsetFetchRequest.Builder(groupId, true, null, false);
-            final short finalVersion = version;
-            if (version < 2) {
-                assertThrows(UnsupportedVersionException.class, () -> builder.build(finalVersion));
-            } else {
-                OffsetFetchRequest request = builder.build(finalVersion);
-                assertEquals(groupId, request.groupId());
-                assertNull(request.partitions());
-                assertTrue(request.isAllPartitions());
-                if (version < 7) {
-                    assertFalse(request.requireStable());
-                } else {
-                    assertTrue(request.requireStable());
-                }
+            if (version < 8) {
+                Map<String, List<TopicPartition>> groupPartitionMap = new HashMap<>();
+                groupPartitionMap.put(group1, null);
+                groupPartitionMap.put(group2, null);
+                builder = new Builder(groupPartitionMap, true, false);
+                final short finalVersion = version;
+                assertThrows(NoBatchedOffsetFetchRequestException.class, () -> builder.build(finalVersion));

Review comment:
       As before, it will be good to include test for `version >=8` as well

##########
File path: core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
##########
@@ -1358,17 +1367,241 @@ 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.responseData(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.responseData(group).containsKey(tp))
+    assertEquals(offset, offsetFetchResponse.responseData(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)
+
+    val topic1 = "topic1"
+    val topic1List = singletonList(new TopicPartition(topic1, 0))
+    val topicOneResource = new ResourcePattern(TOPIC, topic1, LITERAL)
+    val topic2 = "topic2"
+    val topic1And2List = util.Arrays.asList(
+      new TopicPartition(topic1, 0),
+      new TopicPartition(topic2, 0),
+      new TopicPartition(topic2, 1))
+    val topicTwoResource = new ResourcePattern(TOPIC, topic2, LITERAL)
+    val topic3 = "topic3"
+    val allTopicsList = util.Arrays.asList(
+      new TopicPartition(topic1, 0),
+      new TopicPartition(topic2, 0),
+      new TopicPartition(topic2, 1),
+      new TopicPartition(topic3, 0),
+      new TopicPartition(topic3, 1),
+      new TopicPartition(topic3, 2))
+    val topicThreeResource = new ResourcePattern(TOPIC, topic3, LITERAL)
+
+    // create group to partition map to build batched offsetFetch request
+    val groupToPartitionMap = new util.HashMap[String, util.List[TopicPartition]]()
+    groupToPartitionMap.put(groupOne, topic1List)
+    groupToPartitionMap.put(groupTwo, topic1And2List)
+    groupToPartitionMap.put(groupThree, allTopicsList)
+    groupToPartitionMap.put(groupFour, null)
+    groupToPartitionMap.put(groupFive, null)
+
+    createTopic(topic1)
+    createTopic(topic2, numPartitions = 2)
+    createTopic(topic3, numPartitions = 3)
+
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupOneResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupTwoResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupThreeResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupFourResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupFiveResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicOneResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicTwoResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicThreeResource)
+
+    val offset = 15L
+    val leaderEpoch: Optional[Integer] = Optional.of(1)
+    val metadata = "metadata"
+    val topicOneOffsets = topic1List.asScala.map{
+      tp => (tp, new OffsetAndMetadata(offset, leaderEpoch, metadata))
+    }.toMap.asJava
+    val topicOneAndTwoOffsets = topic1And2List.asScala.map{
+      tp => (tp, new OffsetAndMetadata(offset, leaderEpoch, metadata))
+    }.toMap.asJava
+    val allTopicOffsets = allTopicsList.asScala.map{
+      tp => (tp, new OffsetAndMetadata(offset, leaderEpoch, metadata))
+    }.toMap.asJava
+
+    // create 5 consumers to commit offsets so we can fetch them later
+
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupOne)
+    var consumer = createConsumer()
+    consumer.assign(topic1List)
+    consumer.commitSync(topicOneOffsets)
+    consumer.close()

Review comment:
       Create a nested method to do this and call it for all cases.

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
##########
@@ -1580,21 +1623,83 @@ private OffsetCommitResponse createOffsetCommitResponse() {
     }
 
     private OffsetFetchRequest createOffsetFetchRequest(int version, boolean requireStable) {
-        return new OffsetFetchRequest.Builder("group1", requireStable, Collections.singletonList(new TopicPartition("test11", 1)), false)
+        if (version < 8) {
+            return new OffsetFetchRequest.Builder(
+                "group1",
+                requireStable,
+                Collections.singletonList(
+                    new TopicPartition("test11", 1)),

Review comment:
       nit: unnecessary newline

##########
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:
       If we move the single group case to the previous test, we can make this one work on a collection of groups. Instead of named groups like `groupOne`, we can just use group(1).

##########
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 = {

Review comment:
       Can we make this for all versions with a single group id? That is the scenario in the consumer and we want to make sure the methods in request/response classes handle any version for 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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664805037



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +315,28 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public boolean isAllPartitionsForGroup(String groupId) {
+        OffsetFetchRequestGroup group = data
+            .groupIds()
+            .stream()
+            .filter(g -> g.groupId().equals(groupId))
+            .collect(toSingleton());

Review comment:
       Ah gotcha, makes sense. For now I have removed the `toSingleton()` helper method and collected it into a list that gets the first element 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] rajinisivaram commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663488495



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##########
@@ -1308,29 +1308,31 @@ private OffsetFetchResponseHandler() {
 
         @Override
         public void handle(OffsetFetchResponse response, RequestFuture<Map<TopicPartition, OffsetAndMetadata>> future) {
-            if (response.hasError()) {
-                Errors error = response.error();
-                log.debug("Offset fetch failed: {}", error.message());
+            Errors responseError = response.groupLevelError(rebalanceConfig.groupId);

Review comment:
       We could just call this `error` and then won't require the remaining changes below.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -78,26 +85,107 @@ boolean isAllTopicPartitions() {
             return this.data.topics() == ALL_TOPIC_PARTITIONS;
         }
 
+        public Builder(Map<String, List<TopicPartition>> groupIdToTopicPartitionMap,
+            boolean requireStable,

Review comment:
       nit: indentation

##########
File path: clients/src/test/java/org/apache/kafka/common/message/MessageTest.java
##########
@@ -662,20 +669,179 @@ public void testOffsetFetchVersions() throws Exception {
                       .setErrorCode(Errors.NOT_COORDINATOR.code())
                       .setThrottleTimeMs(10);
         for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
-            OffsetFetchResponseData responseData = response.get();
-            if (version <= 1) {
-                responseData.setErrorCode(Errors.NONE.code());
+            if (version < 8) {

Review comment:
       As before, we should have `if version >= 8` as well.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java
##########
@@ -87,12 +87,16 @@ public String apiName() {
         Map<CoordinatorKey, Throwable> failed = new HashMap<>();
         List<CoordinatorKey> unmapped = new ArrayList<>();
 
-        if (response.error() != Errors.NONE) {
-            handleError(groupId, response.error(), failed, unmapped);
+        Errors responseError = response.groupLevelError(groupId.idValue);
+        if (responseError != Errors.NONE) {
+            handleError(groupId, responseError, failed, unmapped);
         } else {
             final Map<TopicPartition, OffsetAndMetadata> groupOffsetsListing = new HashMap<>();
-            for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry :
-                response.responseData().entrySet()) {
+            // if entry for group level response data is null, we are getting back an older version
+            // of the response

Review comment:
       Is this comment required?

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -1255,81 +1256,143 @@ class KafkaApis(val requestChannel: RequestChannel,
    * Handle an offset fetch request
    */
   def handleOffsetFetchRequest(request: RequestChannel.Request): Unit = {
+    val version = request.header.apiVersion
+    if (version == 0) {
+      // reading offsets from ZK
+      handleOffsetFetchRequestV0(request)
+    } else if (version >= 1 && version <= 7) {
+      // reading offsets from Kafka
+      handleOffsetFetchRequestBetweenV1AndV7(request)
+    } else {
+      // batching offset reads for multiple groups starts with version 8 and greater
+      handleOffsetFetchRequestV8AndAbove(request)
+    }
+  }
+
+  private def handleOffsetFetchRequestV0(request: RequestChannel.Request): Unit = {
     val header = request.header
     val offsetFetchRequest = request.body[OffsetFetchRequest]
 
-    def partitionByAuthorized(seq: Seq[TopicPartition]): (Seq[TopicPartition], Seq[TopicPartition]) =
-      authHelper.partitionSeqByAuthorized(request.context, DESCRIBE, TOPIC, seq)(_.topic)
-
     def createResponse(requestThrottleMs: Int): AbstractResponse = {
       val offsetFetchResponse =
-        // reject the request if not authorized to the group
+      // reject the request if not authorized to the group

Review comment:
       nit: revert indentation change

##########
File path: core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
##########
@@ -1358,17 +1367,241 @@ 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.responseData(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.responseData(group).containsKey(tp))
+    assertEquals(offset, offsetFetchResponse.responseData(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)
+
+    val topic1 = "topic1"
+    val topic1List = singletonList(new TopicPartition(topic1, 0))
+    val topicOneResource = new ResourcePattern(TOPIC, topic1, LITERAL)
+    val topic2 = "topic2"
+    val topic1And2List = util.Arrays.asList(
+      new TopicPartition(topic1, 0),
+      new TopicPartition(topic2, 0),
+      new TopicPartition(topic2, 1))
+    val topicTwoResource = new ResourcePattern(TOPIC, topic2, LITERAL)
+    val topic3 = "topic3"
+    val allTopicsList = util.Arrays.asList(
+      new TopicPartition(topic1, 0),
+      new TopicPartition(topic2, 0),
+      new TopicPartition(topic2, 1),
+      new TopicPartition(topic3, 0),
+      new TopicPartition(topic3, 1),
+      new TopicPartition(topic3, 2))
+    val topicThreeResource = new ResourcePattern(TOPIC, topic3, LITERAL)
+
+    // create group to partition map to build batched offsetFetch request
+    val groupToPartitionMap = new util.HashMap[String, util.List[TopicPartition]]()
+    groupToPartitionMap.put(groupOne, topic1List)
+    groupToPartitionMap.put(groupTwo, topic1And2List)
+    groupToPartitionMap.put(groupThree, allTopicsList)
+    groupToPartitionMap.put(groupFour, null)
+    groupToPartitionMap.put(groupFive, null)
+
+    createTopic(topic1)
+    createTopic(topic2, numPartitions = 2)
+    createTopic(topic3, numPartitions = 3)
+
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupOneResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupTwoResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupThreeResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupFourResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupFiveResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicOneResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicTwoResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicThreeResource)
+
+    val offset = 15L
+    val leaderEpoch: Optional[Integer] = Optional.of(1)
+    val metadata = "metadata"
+    val topicOneOffsets = topic1List.asScala.map{

Review comment:
       nit: space before braces (multiple places)

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -78,26 +85,107 @@ boolean isAllTopicPartitions() {
             return this.data.topics() == ALL_TOPIC_PARTITIONS;
         }
 
+        public Builder(Map<String, List<TopicPartition>> groupIdToTopicPartitionMap,
+            boolean requireStable,
+            boolean throwOnFetchStableOffsetsUnsupported) {
+            super(ApiKeys.OFFSET_FETCH);
+
+            List<OffsetFetchRequestGroup> groups = new ArrayList<>();
+            for (Entry<String, List<TopicPartition>> entry : groupIdToTopicPartitionMap.entrySet()) {
+                String groupName = entry.getKey();
+                List<TopicPartition> tpList = entry.getValue();
+                final List<OffsetFetchRequestTopics> topics;
+                if (tpList != null) {
+                    Map<String, OffsetFetchRequestTopics> offsetFetchRequestTopicMap =
+                        new HashMap<>();
+                    for (TopicPartition topicPartition : tpList) {
+                        String topicName = topicPartition.topic();
+                        OffsetFetchRequestTopics topic = offsetFetchRequestTopicMap.getOrDefault(
+                            topicName, new OffsetFetchRequestTopics().setName(topicName));
+                        topic.partitionIndexes().add(topicPartition.partition());
+                        offsetFetchRequestTopicMap.put(topicName, topic);
+                    }
+                    topics = new ArrayList<>(offsetFetchRequestTopicMap.values());
+                } else {
+                    topics = ALL_TOPIC_PARTITIONS_BATCH;
+                }
+                groups.add(new OffsetFetchRequestGroup()
+                    .setGroupId(groupName)
+                    .setTopics(topics));
+            }
+            this.data = new OffsetFetchRequestData()
+                .setGroupIds(groups)
+                .setRequireStable(requireStable);
+            this.throwOnFetchStableOffsetsUnsupported = throwOnFetchStableOffsetsUnsupported;
+        }
+
         @Override
         public OffsetFetchRequest build(short version) {
             if (isAllTopicPartitions() && version < 2) {
                 throw new UnsupportedVersionException("The broker only supports OffsetFetchRequest " +
                     "v" + version + ", but we need v2 or newer to request all topic partitions.");
             }
-
+            if (data.groupIds().size() > 1 && version < 8) {
+                throw new NoBatchedOffsetFetchRequestException("Broker does not support"
+                    + " batching groups for fetch offset request on version " + version);
+            }
             if (data.requireStable() && version < 7) {
                 if (throwOnFetchStableOffsetsUnsupported) {
                     throw new UnsupportedVersionException("Broker unexpectedly " +
                         "doesn't support requireStable flag on version " + version);
                 } else {
                     log.trace("Fallback the requireStable flag to false as broker " +
-                                  "only supports OffsetFetchRequest version {}. Need " +
-                                  "v7 or newer to enable this feature", version);
+                        "only supports OffsetFetchRequest version {}. Need " +
+                        "v7 or newer to enable this feature", version);
 
                     return new OffsetFetchRequest(data.setRequireStable(false), version);
                 }
             }
-
+            if (version < 8) {
+                OffsetFetchRequestData oldDataFormat = null;
+                if (!data.groupIds().isEmpty()) {
+                    OffsetFetchRequestGroup group = data.groupIds().get(0);
+                    String groupName = group.groupId();
+                    List<OffsetFetchRequestTopics> topics = group.topics();
+                    List<OffsetFetchRequestTopic> oldFormatTopics = null;
+                    if (topics != null) {
+                        oldFormatTopics = topics
+                            .stream()
+                            .map(t ->
+                                new OffsetFetchRequestTopic()
+                                    .setName(t.name())
+                                    .setPartitionIndexes(t.partitionIndexes()))
+                            .collect(Collectors.toList());
+                    }
+                    oldDataFormat = new OffsetFetchRequestData()
+                        .setGroupId(groupName)
+                        .setTopics(oldFormatTopics)
+                        .setRequireStable(data.requireStable());
+                }
+                return new OffsetFetchRequest(oldDataFormat == null ? data : oldDataFormat, version);
+            }
+            // version 8 but have used old format of request, convert to version 8 of request

Review comment:
       May be better to put the following code in an else statement and put the comment at the start of the `if` block since we may be converting for version < 8 as well in the block above.

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchRequestTest.java
##########
@@ -76,62 +73,169 @@ public void testConstructor() {
         }
 
         for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
-            OffsetFetchRequest request = builder.build(version);
-            assertFalse(request.isAllPartitions());
-            assertEquals(groupId, request.groupId());
-            assertEquals(partitions, request.partitions());
-
-            OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
-            assertEquals(Errors.NONE, response.error());
-            assertFalse(response.hasError());
-            assertEquals(Collections.singletonMap(Errors.NONE, version <= (short) 1 ? 3 : 1), response.errorCounts(),
-                "Incorrect error count for version " + version);
-
-            if (version <= 1) {
-                assertEquals(expectedData, response.responseData());
+            if (version < 8) {
+                builder = new OffsetFetchRequest.Builder(
+                    group1,
+                    false,
+                    partitions,
+                    false);
+                assertFalse(builder.isAllTopicPartitions());
+                OffsetFetchRequest request = builder.build(version);
+                assertFalse(request.isAllPartitions());
+                assertEquals(group1, request.groupId());
+                assertEquals(partitions, request.partitions());
+
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.error());
+                assertFalse(response.hasError());
+                assertEquals(Collections.singletonMap(Errors.NONE, version <= (short) 1 ? 3 : 1), response.errorCounts(),
+                    "Incorrect error count for version " + version);
+
+                if (version <= 1) {
+                    assertEquals(expectedData, response.responseDataV0ToV7());
+                }
+
+                if (version >= 3) {
+                    assertEquals(throttleTimeMs, response.throttleTimeMs());
+                } else {
+                    assertEquals(DEFAULT_THROTTLE_TIME, response.throttleTimeMs());
+                }
+            } else {
+                builder = new Builder(Collections.singletonMap(group1, partitions), false, false);
+                OffsetFetchRequest request = builder.build(version);
+                Map<String, List<TopicPartition>> groupToPartitionMap =
+                    request.groupIdsToPartitions();
+                Map<String, List<OffsetFetchRequestTopics>> groupToTopicMap =
+                    request.groupIdsToTopics();
+                assertFalse(request.isAllPartitionsForGroup(group1));
+                assertTrue(groupToPartitionMap.containsKey(group1) && groupToTopicMap.containsKey(
+                    group1));
+                assertEquals(partitions, groupToPartitionMap.get(group1));
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.groupLevelError(group1));
+                assertFalse(response.groupHasError(group1));
+                assertEquals(Collections.singletonMap(Errors.NONE, 1), response.errorCounts(),
+                    "Incorrect error count for version " + version);
+                assertEquals(throttleTimeMs, response.throttleTimeMs());
             }
+        }
+    }
+
+    @Test
+    public void testConstructorWithMultipleGroups() {
+        List<TopicPartition> topic1Partitions = Arrays.asList(
+            new TopicPartition(topicOne, partitionOne),
+            new TopicPartition(topicOne, partitionTwo));
+        List<TopicPartition> topic2Partitions = Arrays.asList(
+            new TopicPartition(topicTwo, partitionOne),
+            new TopicPartition(topicTwo, partitionTwo));
+        List<TopicPartition> topic3Partitions = Arrays.asList(
+            new TopicPartition(topicThree, partitionOne),
+            new TopicPartition(topicThree, partitionTwo));
+        Map<String, List<TopicPartition>> groupToTp = new HashMap<>();
+        groupToTp.put(group1, topic1Partitions);
+        groupToTp.put(group2, topic2Partitions);
+        groupToTp.put(group3, topic3Partitions);
+        groupToTp.put(group4, null);
+        groupToTp.put(group5, null);
+        int throttleTimeMs = 10;
 
-            if (version >= 3) {
+        for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
+            if (version >= 8) {
+                builder = new Builder(groupToTp, false, false);
+                OffsetFetchRequest request = builder.build(version);
+                Map<String, List<TopicPartition>> groupToPartitionMap =
+                    request.groupIdsToPartitions();
+                Map<String, List<OffsetFetchRequestTopics>> groupToTopicMap =
+                    request.groupIdsToTopics();
+                assertEquals(groupToTp.keySet(), groupToTopicMap.keySet());
+                assertEquals(groupToTp.keySet(), groupToPartitionMap.keySet());
+                assertFalse(request.isAllPartitionsForGroup(group1));
+                assertFalse(request.isAllPartitionsForGroup(group2));
+                assertFalse(request.isAllPartitionsForGroup(group3));
+                assertTrue(request.isAllPartitionsForGroup(group4));
+                assertTrue(request.isAllPartitionsForGroup(group5));
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.groupLevelError(group1));
+                assertEquals(Errors.NONE, response.groupLevelError(group2));
+                assertEquals(Errors.NONE, response.groupLevelError(group3));
+                assertEquals(Errors.NONE, response.groupLevelError(group4));
+                assertEquals(Errors.NONE, response.groupLevelError(group5));

Review comment:
       List of groups will enable us to use loop here.

##########
File path: clients/src/test/java/org/apache/kafka/common/message/MessageTest.java
##########
@@ -633,17 +638,19 @@ public void testOffsetFetchVersions() throws Exception {
                                                        .setRequireStable(true);
 
         for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
-            final short finalVersion = version;
-            if (version < 2) {
-                assertThrows(NullPointerException.class, () -> testAllMessageRoundTripsFromVersion(finalVersion, allPartitionData));
-            } else {
-                testAllMessageRoundTripsFromVersion(version, allPartitionData);
-            }
-
-            if (version < 7) {
-                assertThrows(UnsupportedVersionException.class, () -> testAllMessageRoundTripsFromVersion(finalVersion, requireStableData));
-            } else {
-                testAllMessageRoundTripsFromVersion(finalVersion, requireStableData);
+            if (version < 8) {

Review comment:
       We should probably just add this test for v8 and above as well. The other v8 test can just focus on multiple groups, this one can test the common case of single group that is used by consumers. 

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchRequestTest.java
##########
@@ -76,62 +73,169 @@ public void testConstructor() {
         }
 
         for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
-            OffsetFetchRequest request = builder.build(version);
-            assertFalse(request.isAllPartitions());
-            assertEquals(groupId, request.groupId());
-            assertEquals(partitions, request.partitions());
-
-            OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
-            assertEquals(Errors.NONE, response.error());
-            assertFalse(response.hasError());
-            assertEquals(Collections.singletonMap(Errors.NONE, version <= (short) 1 ? 3 : 1), response.errorCounts(),
-                "Incorrect error count for version " + version);
-
-            if (version <= 1) {
-                assertEquals(expectedData, response.responseData());
+            if (version < 8) {
+                builder = new OffsetFetchRequest.Builder(
+                    group1,
+                    false,
+                    partitions,
+                    false);
+                assertFalse(builder.isAllTopicPartitions());
+                OffsetFetchRequest request = builder.build(version);
+                assertFalse(request.isAllPartitions());
+                assertEquals(group1, request.groupId());
+                assertEquals(partitions, request.partitions());
+
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.error());
+                assertFalse(response.hasError());
+                assertEquals(Collections.singletonMap(Errors.NONE, version <= (short) 1 ? 3 : 1), response.errorCounts(),
+                    "Incorrect error count for version " + version);
+
+                if (version <= 1) {
+                    assertEquals(expectedData, response.responseDataV0ToV7());
+                }
+
+                if (version >= 3) {
+                    assertEquals(throttleTimeMs, response.throttleTimeMs());
+                } else {
+                    assertEquals(DEFAULT_THROTTLE_TIME, response.throttleTimeMs());
+                }
+            } else {
+                builder = new Builder(Collections.singletonMap(group1, partitions), false, false);
+                OffsetFetchRequest request = builder.build(version);
+                Map<String, List<TopicPartition>> groupToPartitionMap =
+                    request.groupIdsToPartitions();
+                Map<String, List<OffsetFetchRequestTopics>> groupToTopicMap =
+                    request.groupIdsToTopics();
+                assertFalse(request.isAllPartitionsForGroup(group1));
+                assertTrue(groupToPartitionMap.containsKey(group1) && groupToTopicMap.containsKey(
+                    group1));
+                assertEquals(partitions, groupToPartitionMap.get(group1));
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.groupLevelError(group1));
+                assertFalse(response.groupHasError(group1));
+                assertEquals(Collections.singletonMap(Errors.NONE, 1), response.errorCounts(),
+                    "Incorrect error count for version " + version);
+                assertEquals(throttleTimeMs, response.throttleTimeMs());
             }
+        }
+    }
+
+    @Test
+    public void testConstructorWithMultipleGroups() {
+        List<TopicPartition> topic1Partitions = Arrays.asList(
+            new TopicPartition(topicOne, partitionOne),
+            new TopicPartition(topicOne, partitionTwo));
+        List<TopicPartition> topic2Partitions = Arrays.asList(
+            new TopicPartition(topicTwo, partitionOne),
+            new TopicPartition(topicTwo, partitionTwo));
+        List<TopicPartition> topic3Partitions = Arrays.asList(
+            new TopicPartition(topicThree, partitionOne),
+            new TopicPartition(topicThree, partitionTwo));
+        Map<String, List<TopicPartition>> groupToTp = new HashMap<>();
+        groupToTp.put(group1, topic1Partitions);
+        groupToTp.put(group2, topic2Partitions);
+        groupToTp.put(group3, topic3Partitions);
+        groupToTp.put(group4, null);
+        groupToTp.put(group5, null);
+        int throttleTimeMs = 10;
 
-            if (version >= 3) {
+        for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
+            if (version >= 8) {
+                builder = new Builder(groupToTp, false, false);
+                OffsetFetchRequest request = builder.build(version);
+                Map<String, List<TopicPartition>> groupToPartitionMap =
+                    request.groupIdsToPartitions();
+                Map<String, List<OffsetFetchRequestTopics>> groupToTopicMap =
+                    request.groupIdsToTopics();
+                assertEquals(groupToTp.keySet(), groupToTopicMap.keySet());
+                assertEquals(groupToTp.keySet(), groupToPartitionMap.keySet());
+                assertFalse(request.isAllPartitionsForGroup(group1));
+                assertFalse(request.isAllPartitionsForGroup(group2));
+                assertFalse(request.isAllPartitionsForGroup(group3));
+                assertTrue(request.isAllPartitionsForGroup(group4));
+                assertTrue(request.isAllPartitionsForGroup(group5));
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.groupLevelError(group1));
+                assertEquals(Errors.NONE, response.groupLevelError(group2));
+                assertEquals(Errors.NONE, response.groupLevelError(group3));
+                assertEquals(Errors.NONE, response.groupLevelError(group4));
+                assertEquals(Errors.NONE, response.groupLevelError(group5));
+                assertFalse(response.groupHasError(group1));
+                assertFalse(response.groupHasError(group2));
+                assertFalse(response.groupHasError(group3));
+                assertFalse(response.groupHasError(group4));
+                assertFalse(response.groupHasError(group5));
+                assertEquals(Collections.singletonMap(Errors.NONE, 5), response.errorCounts(),
+                    "Incorrect error count for version " + version);
                 assertEquals(throttleTimeMs, response.throttleTimeMs());
-            } else {
-                assertEquals(DEFAULT_THROTTLE_TIME, response.throttleTimeMs());
             }
         }
     }
 
     @Test
-    public void testConstructorFailForUnsupportedRequireStable() {
+    public void testBuildThrowForUnsupportedBatchRequest() {
         for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
-            // The builder needs to be initialized every cycle as the internal data `requireStable` flag is flipped.
-            builder = new OffsetFetchRequest.Builder(groupId, true, null, false);
-            final short finalVersion = version;
-            if (version < 2) {
-                assertThrows(UnsupportedVersionException.class, () -> builder.build(finalVersion));
-            } else {
-                OffsetFetchRequest request = builder.build(finalVersion);
-                assertEquals(groupId, request.groupId());
-                assertNull(request.partitions());
-                assertTrue(request.isAllPartitions());
-                if (version < 7) {
-                    assertFalse(request.requireStable());
-                } else {
-                    assertTrue(request.requireStable());
-                }
+            if (version < 8) {
+                Map<String, List<TopicPartition>> groupPartitionMap = new HashMap<>();
+                groupPartitionMap.put(group1, null);
+                groupPartitionMap.put(group2, null);
+                builder = new Builder(groupPartitionMap, true, false);
+                final short finalVersion = version;
+                assertThrows(NoBatchedOffsetFetchRequestException.class, () -> builder.build(finalVersion));
             }
         }
     }
 
     @Test
-    public void testBuildThrowForUnsupportedRequireStable() {
+    public void testConstructorFailForUnsupportedRequireStable() {
         for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
-            builder = new OffsetFetchRequest.Builder(groupId, true, null, true);
-            if (version < 7) {
+            if (version < 8) {
+                // The builder needs to be initialized every cycle as the internal data `requireStable` flag is flipped.
+                builder = new OffsetFetchRequest.Builder(group1, true, null, false);
                 final short finalVersion = version;
-                assertThrows(UnsupportedVersionException.class, () -> builder.build(finalVersion));
+                if (version < 2) {
+                    assertThrows(UnsupportedVersionException.class, () -> builder.build(finalVersion));
+                } else {
+                    OffsetFetchRequest request = builder.build(finalVersion);
+                    assertEquals(group1, request.groupId());
+                    assertNull(request.partitions());
+                    assertTrue(request.isAllPartitions());
+                    if (version < 7) {
+                        assertFalse(request.requireStable());
+                    } else {
+                        assertTrue(request.requireStable());
+                    }
+                }
             } else {
+                builder = new Builder(Collections.singletonMap(group1, null), true, false);
                 OffsetFetchRequest request = builder.build(version);
+                Map<String, List<TopicPartition>> groupToPartitionMap =
+                    request.groupIdsToPartitions();
+                Map<String, List<OffsetFetchRequestTopics>> groupToTopicMap =
+                    request.groupIdsToTopics();
+                assertTrue(groupToPartitionMap.containsKey(group1) && groupToTopicMap.containsKey(
+                    group1));
+                assertNull(groupToPartitionMap.get(group1));
+                assertTrue(request.isAllPartitionsForGroup(group1));
                 assertTrue(request.requireStable());
             }
         }
     }
+
+    @Test
+    public void testBuildThrowForUnsupportedRequireStable() {
+        for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
+            if (version < 8) {

Review comment:
       As before, it will be good to include the same logic for `version >=8` as well. Based on the number of tests that use the pattern of `ApiKeys.OFFSET_FETCH.allVersions()` followed by `if (version < 8)`, it may be good to add the list of versions < 8 and use that in these cases if we require this pattern.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -154,14 +167,74 @@ public OffsetFetchResponse(int throttleTimeMs, Errors error, Map<TopicPartition,
         this.error = error;
     }
 
+    /**
+     * Constructor without throttle time for version 8 and above.
+     * @param errors Error code on a per group level basis
+     * @param responseData Fetched offset information grouped group id
+     */
+    public OffsetFetchResponse(Map<String, Errors> errors, Map<String, Map<TopicPartition, PartitionData>> responseData) {
+        this(DEFAULT_THROTTLE_TIME, errors, responseData);
+    }
+
+    /**
+     * Constructor with throttle time for version 8 and above.
+     * @param throttleTimeMs The time in milliseconds that this response was throttled
+     * @param errors Potential coordinator or group level error code
+     * @param responseData Fetched offset information grouped by topic-partition and by group
+     */
+    public OffsetFetchResponse(int throttleTimeMs, Map<String, Errors> errors, Map<String,
+        Map<TopicPartition, PartitionData>> responseData) {

Review comment:
       nit: indentation

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +315,28 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public boolean isAllPartitionsForGroup(String groupId) {
+        OffsetFetchRequestGroup group = data
+            .groupIds()
+            .stream()
+            .filter(g -> g.groupId().equals(groupId))
+            .collect(toSingleton());

Review comment:
       This is going to throw IllegalStateException if a group is included multiple times in a request. If that is an invalid request, we should throw InvalidRequestException before it gets here to ensure we don't return UNKNOWN_SERVER_ERROR. More likely, it is valid and we should not throw an exception.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -113,8 +120,14 @@ public int hashCode() {
         }
     }
 
+    public OffsetFetchResponse(OffsetFetchResponseData data) {
+        super(ApiKeys.OFFSET_FETCH);
+        this.data = data;
+        this.error = null;
+    }
+
     /**
-     * Constructor for all versions without throttle time.
+     * Constructor without throttle time for version 0 to version 7.

Review comment:
       It is not actually for version 0 to 7, it is for the versions without throttle time.

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchRequestTest.java
##########
@@ -44,25 +46,20 @@
     private final int partitionOne = 1;
     private final String topicTwo = "topic2";
     private final int partitionTwo = 2;
-    private final String groupId = "groupId";
+    private final String topicThree = "topic3";
+    private final String group1 = "group1";
+    private final String group2 = "group2";
+    private final String group3 = "group3";
+    private final String group4 = "group4";
+    private final String group5 = "group5";

Review comment:
       Maybe better to define `List<String> groups` to avoid repetition here and in the test logic

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -185,21 +249,46 @@ public boolean hasError() {
         return error != Errors.NONE;
     }
 
+    public boolean groupHasError(String groupId) {
+        return groupLevelErrors.get(groupId) != Errors.NONE;
+    }
+
     public Errors error() {
         return error;
     }
 
+    public Errors groupLevelError(String groupId) {
+        if (error != null) {
+            return error;
+        }
+        return groupLevelErrors.get(groupId);
+    }
+
     @Override
     public Map<Errors, Integer> errorCounts() {
         Map<Errors, Integer> counts = new HashMap<>();
-        updateErrorCounts(counts, error);
-        data.topics().forEach(topic ->
-                topic.partitions().forEach(partition ->
+        if (!groupLevelErrors.isEmpty()) {
+            // built response with v8 or above
+            for (Map.Entry<String, Errors> entry : groupLevelErrors.entrySet()) {
+                updateErrorCounts(counts, entry.getValue());
+            }
+            for (OffsetFetchResponseGroup group : data.groupIds()) {
+                group.topics().forEach(topic ->
+                    topic.partitions().forEach(partition ->
                         updateErrorCounts(counts, Errors.forCode(partition.errorCode()))));
+            }
+        } else {
+            // built response with v0-v7
+            updateErrorCounts(counts, error);
+            data.topics().forEach(topic ->
+                topic.partitions().forEach(partition ->
+                    updateErrorCounts(counts, Errors.forCode(partition.errorCode()))));
+        }
         return counts;
     }
 
-    public Map<TopicPartition, PartitionData> responseData() {
+    //public for testing purposes
+    public Map<TopicPartition, PartitionData> responseDataV0ToV7() {

Review comment:
       package-private should be sufficient for unit tests? May also be possible keep this private and call `responseData()` directly to ensure the right data is returned based on version.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -214,6 +303,46 @@ public Errors error() {
         return responseData;
     }
 
+    private Map<TopicPartition, PartitionData> buildResponseData(String groupId) {
+        Map<TopicPartition, PartitionData> responseData = new HashMap<>();
+        OffsetFetchResponseGroup group = data
+            .groupIds()
+            .stream()
+            .filter(g -> g.groupId().equals(groupId))
+            .collect(toSingleton());
+        for (OffsetFetchResponseTopics topic : group.topics()) {
+            for (OffsetFetchResponsePartitions partition : topic.partitions()) {
+                responseData.put(new TopicPartition(topic.name(), partition.partitionIndex()),
+                    new PartitionData(partition.committedOffset(),
+                        RequestUtils.getLeaderEpoch(partition.committedLeaderEpoch()),
+                        partition.metadata(),
+                        Errors.forCode(partition.errorCode()))
+                );
+            }
+        }
+        return responseData;
+    }
+
+    // Custom collector to filter a single element
+    private <T> Collector<T, ?, T> toSingleton() {

Review comment:
       See earlier note about this method.

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##########
@@ -1308,29 +1308,31 @@ private OffsetFetchResponseHandler() {
 
         @Override
         public void handle(OffsetFetchResponse response, RequestFuture<Map<TopicPartition, OffsetAndMetadata>> future) {
-            if (response.hasError()) {
-                Errors error = response.error();
-                log.debug("Offset fetch failed: {}", error.message());
+            Errors responseError = response.groupLevelError(rebalanceConfig.groupId);
+            if (responseError != Errors.NONE) {
+                log.debug("Offset fetch failed: {}", responseError.message());
 
-                if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS) {
+                if (responseError == Errors.COORDINATOR_LOAD_IN_PROGRESS) {
                     // just retry
-                    future.raise(error);
-                } else if (error == Errors.NOT_COORDINATOR) {
+                    future.raise(responseError);
+                } else if (responseError == Errors.NOT_COORDINATOR) {
                     // re-discover the coordinator and retry
-                    markCoordinatorUnknown(error);
-                    future.raise(error);
-                } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) {
+                    markCoordinatorUnknown(responseError);
+                    future.raise(responseError);
+                } else if (responseError == Errors.GROUP_AUTHORIZATION_FAILED) {
                     future.raise(GroupAuthorizationException.forGroupId(rebalanceConfig.groupId));
                 } else {
-                    future.raise(new KafkaException("Unexpected error in fetch offset response: " + error.message()));
+                    future.raise(new KafkaException("Unexpected error in fetch offset response: " + responseError.message()));
                 }
                 return;
             }
 
             Set<String> unauthorizedTopics = null;
-            Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>(response.responseData().size());
+            Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData =

Review comment:
       as before, rename responseData?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java
##########
@@ -87,12 +87,16 @@ public String apiName() {
         Map<CoordinatorKey, Throwable> failed = new HashMap<>();
         List<CoordinatorKey> unmapped = new ArrayList<>();
 
-        if (response.error() != Errors.NONE) {
-            handleError(groupId, response.error(), failed, unmapped);
+        Errors responseError = response.groupLevelError(groupId.idValue);
+        if (responseError != Errors.NONE) {
+            handleError(groupId, responseError, failed, unmapped);
         } else {
             final Map<TopicPartition, OffsetAndMetadata> groupOffsetsListing = new HashMap<>();
-            for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry :
-                response.responseData().entrySet()) {
+            // if entry for group level response data is null, we are getting back an older version
+            // of the response
+            Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData =
+                response.responseData(groupId.idValue);

Review comment:
       Perhaps call the method and variable partitionData or offsetData rather than responseData since responseData is too close to response.data?

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +315,28 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public boolean isAllPartitionsForGroup(String groupId) {
+        OffsetFetchRequestGroup group = data
+            .groupIds()
+            .stream()
+            .filter(g -> g.groupId().equals(groupId))
+            .collect(toSingleton());
+        return group.topics() == ALL_TOPIC_PARTITIONS_BATCH;
+    }
+
+    // Custom collector to filter a single element
+    private <T> Collector<T, ?, T> toSingleton() {

Review comment:
       We seem to have two copies of this method in the PR. We need to decide first whether this logic is correct first. If we really do want the method, we need to make it shared.

##########
File path: core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
##########
@@ -1358,17 +1367,241 @@ 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.responseData(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.responseData(group).containsKey(tp))
+    assertEquals(offset, offsetFetchResponse.responseData(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)
+
+    val topic1 = "topic1"
+    val topic1List = singletonList(new TopicPartition(topic1, 0))
+    val topicOneResource = new ResourcePattern(TOPIC, topic1, LITERAL)
+    val topic2 = "topic2"
+    val topic1And2List = util.Arrays.asList(
+      new TopicPartition(topic1, 0),
+      new TopicPartition(topic2, 0),
+      new TopicPartition(topic2, 1))
+    val topicTwoResource = new ResourcePattern(TOPIC, topic2, LITERAL)
+    val topic3 = "topic3"
+    val allTopicsList = util.Arrays.asList(
+      new TopicPartition(topic1, 0),
+      new TopicPartition(topic2, 0),
+      new TopicPartition(topic2, 1),
+      new TopicPartition(topic3, 0),
+      new TopicPartition(topic3, 1),
+      new TopicPartition(topic3, 2))
+    val topicThreeResource = new ResourcePattern(TOPIC, topic3, LITERAL)
+
+    // create group to partition map to build batched offsetFetch request
+    val groupToPartitionMap = new util.HashMap[String, util.List[TopicPartition]]()
+    groupToPartitionMap.put(groupOne, topic1List)
+    groupToPartitionMap.put(groupTwo, topic1And2List)
+    groupToPartitionMap.put(groupThree, allTopicsList)
+    groupToPartitionMap.put(groupFour, null)
+    groupToPartitionMap.put(groupFive, null)
+
+    createTopic(topic1)
+    createTopic(topic2, numPartitions = 2)
+    createTopic(topic3, numPartitions = 3)
+
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupOneResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupTwoResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupThreeResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupFourResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupFiveResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicOneResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicTwoResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicThreeResource)

Review comment:
       Looks like they are are all the same with different resources, better to put them in a collection and call `addAndVerifyAcls` over the collection

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
##########
@@ -1580,21 +1623,83 @@ private OffsetCommitResponse createOffsetCommitResponse() {
     }
 
     private OffsetFetchRequest createOffsetFetchRequest(int version, boolean requireStable) {
-        return new OffsetFetchRequest.Builder("group1", requireStable, Collections.singletonList(new TopicPartition("test11", 1)), false)
+        if (version < 8) {
+            return new OffsetFetchRequest.Builder(
+                "group1",
+                requireStable,
+                Collections.singletonList(
+                    new TopicPartition("test11", 1)),
+                false)
                 .build((short) version);
+        }
+        return new OffsetFetchRequest.Builder(
+            Collections.singletonMap(
+                "group1",
+                Collections.singletonList(
+                    new TopicPartition("test11", 1))),

Review comment:
       nit: unnecessary newlines

##########
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+",
+      "about": "The responses per group id.", "fields": [
+      { "name": "groupId", "type": "string", "versions": "8+", "entityType": "groupId",
+        "about": "The group ID." },
+      { "name": "Topics", "type": "[]OffsetFetchResponseTopics", "versions": "8+",
+        "about": "The responses per topic.", "fields": [
+        { "name": "Name", "type": "string", "versions": "8+", "entityType": "topicName",
+          "about": "The topic name." },
+        { "name": "Partitions", "type": "[]OffsetFetchResponsePartitions", "versions": "8+",
+          "about": "The responses per partition", "fields": [
+          { "name": "PartitionIndex", "type": "int32", "versions": "8+",
+            "about": "The partition index." },
+          { "name": "CommittedOffset", "type": "int64", "versions": "8+",
+            "about": "The committed message offset." },
+          { "name": "CommittedLeaderEpoch", "type": "int32", "versions": "8+", "default": "-1",
+            "ignorable": true, "about": "The leader epoch." },
+          { "name": "Metadata", "type": "string", "versions": "8+", "nullableVersions": "8+",
+            "about": "The partition metadata." },
+          { "name": "ErrorCode", "type": "int16", "versions": "8+",
+            "about": "The partition-level error code, or 0 if there was no error." }
+        ]}
+      ]},
+      { "name": "ErrorCode", "type": "int16", "versions": "8+", "default": "0", "ignorable": true,

Review comment:
       Don't think this should be ignorable

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchRequestTest.java
##########
@@ -76,62 +73,169 @@ public void testConstructor() {
         }
 
         for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
-            OffsetFetchRequest request = builder.build(version);
-            assertFalse(request.isAllPartitions());
-            assertEquals(groupId, request.groupId());
-            assertEquals(partitions, request.partitions());
-
-            OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
-            assertEquals(Errors.NONE, response.error());
-            assertFalse(response.hasError());
-            assertEquals(Collections.singletonMap(Errors.NONE, version <= (short) 1 ? 3 : 1), response.errorCounts(),
-                "Incorrect error count for version " + version);
-
-            if (version <= 1) {
-                assertEquals(expectedData, response.responseData());
+            if (version < 8) {
+                builder = new OffsetFetchRequest.Builder(
+                    group1,
+                    false,
+                    partitions,
+                    false);
+                assertFalse(builder.isAllTopicPartitions());
+                OffsetFetchRequest request = builder.build(version);
+                assertFalse(request.isAllPartitions());
+                assertEquals(group1, request.groupId());
+                assertEquals(partitions, request.partitions());
+
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.error());
+                assertFalse(response.hasError());
+                assertEquals(Collections.singletonMap(Errors.NONE, version <= (short) 1 ? 3 : 1), response.errorCounts(),
+                    "Incorrect error count for version " + version);
+
+                if (version <= 1) {
+                    assertEquals(expectedData, response.responseDataV0ToV7());
+                }
+
+                if (version >= 3) {
+                    assertEquals(throttleTimeMs, response.throttleTimeMs());
+                } else {
+                    assertEquals(DEFAULT_THROTTLE_TIME, response.throttleTimeMs());
+                }
+            } else {
+                builder = new Builder(Collections.singletonMap(group1, partitions), false, false);
+                OffsetFetchRequest request = builder.build(version);
+                Map<String, List<TopicPartition>> groupToPartitionMap =
+                    request.groupIdsToPartitions();
+                Map<String, List<OffsetFetchRequestTopics>> groupToTopicMap =
+                    request.groupIdsToTopics();
+                assertFalse(request.isAllPartitionsForGroup(group1));
+                assertTrue(groupToPartitionMap.containsKey(group1) && groupToTopicMap.containsKey(
+                    group1));
+                assertEquals(partitions, groupToPartitionMap.get(group1));
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.groupLevelError(group1));
+                assertFalse(response.groupHasError(group1));
+                assertEquals(Collections.singletonMap(Errors.NONE, 1), response.errorCounts(),
+                    "Incorrect error count for version " + version);
+                assertEquals(throttleTimeMs, response.throttleTimeMs());
             }
+        }
+    }
+
+    @Test
+    public void testConstructorWithMultipleGroups() {
+        List<TopicPartition> topic1Partitions = Arrays.asList(
+            new TopicPartition(topicOne, partitionOne),
+            new TopicPartition(topicOne, partitionTwo));
+        List<TopicPartition> topic2Partitions = Arrays.asList(
+            new TopicPartition(topicTwo, partitionOne),
+            new TopicPartition(topicTwo, partitionTwo));
+        List<TopicPartition> topic3Partitions = Arrays.asList(
+            new TopicPartition(topicThree, partitionOne),
+            new TopicPartition(topicThree, partitionTwo));
+        Map<String, List<TopicPartition>> groupToTp = new HashMap<>();
+        groupToTp.put(group1, topic1Partitions);
+        groupToTp.put(group2, topic2Partitions);
+        groupToTp.put(group3, topic3Partitions);
+        groupToTp.put(group4, null);
+        groupToTp.put(group5, null);
+        int throttleTimeMs = 10;
 
-            if (version >= 3) {
+        for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
+            if (version >= 8) {
+                builder = new Builder(groupToTp, false, false);
+                OffsetFetchRequest request = builder.build(version);
+                Map<String, List<TopicPartition>> groupToPartitionMap =
+                    request.groupIdsToPartitions();
+                Map<String, List<OffsetFetchRequestTopics>> groupToTopicMap =
+                    request.groupIdsToTopics();
+                assertEquals(groupToTp.keySet(), groupToTopicMap.keySet());
+                assertEquals(groupToTp.keySet(), groupToPartitionMap.keySet());
+                assertFalse(request.isAllPartitionsForGroup(group1));
+                assertFalse(request.isAllPartitionsForGroup(group2));
+                assertFalse(request.isAllPartitionsForGroup(group3));
+                assertTrue(request.isAllPartitionsForGroup(group4));
+                assertTrue(request.isAllPartitionsForGroup(group5));
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.groupLevelError(group1));
+                assertEquals(Errors.NONE, response.groupLevelError(group2));
+                assertEquals(Errors.NONE, response.groupLevelError(group3));
+                assertEquals(Errors.NONE, response.groupLevelError(group4));
+                assertEquals(Errors.NONE, response.groupLevelError(group5));
+                assertFalse(response.groupHasError(group1));
+                assertFalse(response.groupHasError(group2));
+                assertFalse(response.groupHasError(group3));
+                assertFalse(response.groupHasError(group4));
+                assertFalse(response.groupHasError(group5));
+                assertEquals(Collections.singletonMap(Errors.NONE, 5), response.errorCounts(),
+                    "Incorrect error count for version " + version);
                 assertEquals(throttleTimeMs, response.throttleTimeMs());
-            } else {
-                assertEquals(DEFAULT_THROTTLE_TIME, response.throttleTimeMs());
             }
         }
     }
 
     @Test
-    public void testConstructorFailForUnsupportedRequireStable() {
+    public void testBuildThrowForUnsupportedBatchRequest() {
         for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
-            // The builder needs to be initialized every cycle as the internal data `requireStable` flag is flipped.
-            builder = new OffsetFetchRequest.Builder(groupId, true, null, false);
-            final short finalVersion = version;
-            if (version < 2) {
-                assertThrows(UnsupportedVersionException.class, () -> builder.build(finalVersion));
-            } else {
-                OffsetFetchRequest request = builder.build(finalVersion);
-                assertEquals(groupId, request.groupId());
-                assertNull(request.partitions());
-                assertTrue(request.isAllPartitions());
-                if (version < 7) {
-                    assertFalse(request.requireStable());
-                } else {
-                    assertTrue(request.requireStable());
-                }
+            if (version < 8) {
+                Map<String, List<TopicPartition>> groupPartitionMap = new HashMap<>();
+                groupPartitionMap.put(group1, null);
+                groupPartitionMap.put(group2, null);
+                builder = new Builder(groupPartitionMap, true, false);
+                final short finalVersion = version;
+                assertThrows(NoBatchedOffsetFetchRequestException.class, () -> builder.build(finalVersion));

Review comment:
       As before, it will be good to include test for `version >=8` as well

##########
File path: core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
##########
@@ -1358,17 +1367,241 @@ 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.responseData(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.responseData(group).containsKey(tp))
+    assertEquals(offset, offsetFetchResponse.responseData(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)
+
+    val topic1 = "topic1"
+    val topic1List = singletonList(new TopicPartition(topic1, 0))
+    val topicOneResource = new ResourcePattern(TOPIC, topic1, LITERAL)
+    val topic2 = "topic2"
+    val topic1And2List = util.Arrays.asList(
+      new TopicPartition(topic1, 0),
+      new TopicPartition(topic2, 0),
+      new TopicPartition(topic2, 1))
+    val topicTwoResource = new ResourcePattern(TOPIC, topic2, LITERAL)
+    val topic3 = "topic3"
+    val allTopicsList = util.Arrays.asList(
+      new TopicPartition(topic1, 0),
+      new TopicPartition(topic2, 0),
+      new TopicPartition(topic2, 1),
+      new TopicPartition(topic3, 0),
+      new TopicPartition(topic3, 1),
+      new TopicPartition(topic3, 2))
+    val topicThreeResource = new ResourcePattern(TOPIC, topic3, LITERAL)
+
+    // create group to partition map to build batched offsetFetch request
+    val groupToPartitionMap = new util.HashMap[String, util.List[TopicPartition]]()
+    groupToPartitionMap.put(groupOne, topic1List)
+    groupToPartitionMap.put(groupTwo, topic1And2List)
+    groupToPartitionMap.put(groupThree, allTopicsList)
+    groupToPartitionMap.put(groupFour, null)
+    groupToPartitionMap.put(groupFive, null)
+
+    createTopic(topic1)
+    createTopic(topic2, numPartitions = 2)
+    createTopic(topic3, numPartitions = 3)
+
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupOneResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupTwoResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupThreeResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupFourResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), groupFiveResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicOneResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicTwoResource)
+    addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicThreeResource)
+
+    val offset = 15L
+    val leaderEpoch: Optional[Integer] = Optional.of(1)
+    val metadata = "metadata"
+    val topicOneOffsets = topic1List.asScala.map{
+      tp => (tp, new OffsetAndMetadata(offset, leaderEpoch, metadata))
+    }.toMap.asJava
+    val topicOneAndTwoOffsets = topic1And2List.asScala.map{
+      tp => (tp, new OffsetAndMetadata(offset, leaderEpoch, metadata))
+    }.toMap.asJava
+    val allTopicOffsets = allTopicsList.asScala.map{
+      tp => (tp, new OffsetAndMetadata(offset, leaderEpoch, metadata))
+    }.toMap.asJava
+
+    // create 5 consumers to commit offsets so we can fetch them later
+
+    consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupOne)
+    var consumer = createConsumer()
+    consumer.assign(topic1List)
+    consumer.commitSync(topicOneOffsets)
+    consumer.close()

Review comment:
       Create a nested method to do this and call it for all cases.

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
##########
@@ -1580,21 +1623,83 @@ private OffsetCommitResponse createOffsetCommitResponse() {
     }
 
     private OffsetFetchRequest createOffsetFetchRequest(int version, boolean requireStable) {
-        return new OffsetFetchRequest.Builder("group1", requireStable, Collections.singletonList(new TopicPartition("test11", 1)), false)
+        if (version < 8) {
+            return new OffsetFetchRequest.Builder(
+                "group1",
+                requireStable,
+                Collections.singletonList(
+                    new TopicPartition("test11", 1)),

Review comment:
       nit: unnecessary newline

##########
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:
       If we move the single group case to the previous test, we can make this one work on a collection of groups. Instead of named groups like `groupOne`, we can just use group(1).

##########
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 = {

Review comment:
       Can we make this for all versions with a single group id? That is the scenario in the consumer and we want to make sure the methods in request/response classes handle any version for 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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663417840



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -68,36 +74,117 @@ public Builder(String groupId,
             }
 
             this.data = new OffsetFetchRequestData()
-                            .setGroupId(groupId)
-                            .setRequireStable(requireStable)
-                            .setTopics(topics);
+                .setGroupId(groupId)
+                .setRequireStable(requireStable)
+                .setTopics(topics);
             this.throwOnFetchStableOffsetsUnsupported = throwOnFetchStableOffsetsUnsupported;
         }
 
         boolean isAllTopicPartitions() {
             return this.data.topics() == ALL_TOPIC_PARTITIONS;
         }
 
-        @Override
-        public OffsetFetchRequest build(short version) {
-            if (isAllTopicPartitions() && version < 2) {
-                throw new UnsupportedVersionException("The broker only supports OffsetFetchRequest " +
-                    "v" + version + ", but we need v2 or newer to request all topic partitions.");
-            }
+        public Builder(Map<String, List<TopicPartition>> groupIdToTopicPartitionMap,
+            boolean requireStable,
+            boolean throwOnFetchStableOffsetsUnsupported) {
+            super(ApiKeys.OFFSET_FETCH);
 
-            if (data.requireStable() && version < 7) {
-                if (throwOnFetchStableOffsetsUnsupported) {
-                    throw new UnsupportedVersionException("Broker unexpectedly " +
-                        "doesn't support requireStable flag on version " + version);
+            List<OffsetFetchRequestGroup> groups = new ArrayList<>();
+            for (Entry<String, List<TopicPartition>> entry : groupIdToTopicPartitionMap.entrySet()) {
+                final List<OffsetFetchRequestTopics> topics;
+                if (groupIdToTopicPartitionMap.get(entry.getKey()) != null) {

Review comment:
       Here I'm using `entry.getKey()` because we are trying to get the list of topic partitions for a specific group id. `entry.getValue()` would give us `List<TopicPartition>` whereas we need `String`, which `entry.getKey()` would give us.




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664702184



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +315,28 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public boolean isAllPartitionsForGroup(String groupId) {
+        OffsetFetchRequestGroup group = data
+            .groupIds()
+            .stream()
+            .filter(g -> g.groupId().equals(groupId))
+            .collect(toSingleton());
+        return group.topics() == ALL_TOPIC_PARTITIONS_BATCH;
+    }
+
+    // Custom collector to filter a single element
+    private <T> Collector<T, ?, T> toSingleton() {

Review comment:
       Makes sense, for now I will remove iit and convert the methods that used this to collect a `List` and get the first element in the list.




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663537599



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java
##########
@@ -73,7 +73,10 @@ public String apiName() {
     public OffsetFetchRequest.Builder buildRequest(int coordinatorId, Set<CoordinatorKey> keys) {
         // Set the flag to false as for admin client request,
         // we don't need to wait for any pending offset state to clear.
-        return new OffsetFetchRequest.Builder(groupId.idValue, false, partitions, false);

Review comment:
       Okay, I have removed 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] rajinisivaram commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663350976



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java
##########
@@ -73,7 +73,10 @@ public String apiName() {
     public OffsetFetchRequest.Builder buildRequest(int coordinatorId, Set<CoordinatorKey> keys) {
         // Set the flag to false as for admin client request,
         // we don't need to wait for any pending offset state to clear.
-        return new OffsetFetchRequest.Builder(groupId.idValue, false, partitions, false);

Review comment:
       Given that single groupid is a common pattern, we could just retain the old constructor as well.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +319,10 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public List<OffsetFetchRequestTopics> isAllPartitionsForGroup() {

Review comment:
       Name of method suggests we are returning a boolean, but we are returning null. Why do we need a public method that always returns null?

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -65,6 +69,8 @@
 
     private final OffsetFetchResponseData data;
     private final Errors error;
+    private final Map<String, Errors> groupLevelErrors = new HashMap<>();
+    private final Map<String, Map<TopicPartition, PartitionData>> groupToPartitionData = new HashMap<>();

Review comment:
       Why are we caching these when we have them in `data`?

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##########
@@ -1308,29 +1308,41 @@ private OffsetFetchResponseHandler() {
 
         @Override
         public void handle(OffsetFetchResponse response, RequestFuture<Map<TopicPartition, OffsetAndMetadata>> future) {
-            if (response.hasError()) {
-                Errors error = response.error();
-                log.debug("Offset fetch failed: {}", error.message());
+            Errors responseError = response.error();
+            // check if error is null, if it is we are dealing with v8 response
+            if (responseError == null) {

Review comment:
       We can move this to the response object, otherwise it is duplicated from above.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -68,36 +74,117 @@ public Builder(String groupId,
             }
 
             this.data = new OffsetFetchRequestData()
-                            .setGroupId(groupId)
-                            .setRequireStable(requireStable)
-                            .setTopics(topics);
+                .setGroupId(groupId)
+                .setRequireStable(requireStable)
+                .setTopics(topics);

Review comment:
       nit: unnecessary indentation change

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##########
@@ -1308,29 +1308,41 @@ private OffsetFetchResponseHandler() {
 
         @Override
         public void handle(OffsetFetchResponse response, RequestFuture<Map<TopicPartition, OffsetAndMetadata>> future) {
-            if (response.hasError()) {
-                Errors error = response.error();
-                log.debug("Offset fetch failed: {}", error.message());
+            Errors responseError = response.error();
+            // check if error is null, if it is we are dealing with v8 response
+            if (responseError == null) {
+                if (response.groupHasError(rebalanceConfig.groupId)) {
+                    responseError = response.groupLevelError(rebalanceConfig.groupId);
+                } else {
+                    responseError = Errors.NONE;
+                }
+            }
+            if (responseError != Errors.NONE) {
+                log.debug("Offset fetch failed: {}", responseError.message());
 
-                if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS) {
+                if (responseError == Errors.COORDINATOR_LOAD_IN_PROGRESS) {
                     // just retry
-                    future.raise(error);
-                } else if (error == Errors.NOT_COORDINATOR) {
+                    future.raise(responseError);
+                } else if (responseError == Errors.NOT_COORDINATOR) {
                     // re-discover the coordinator and retry
-                    markCoordinatorUnknown(error);
-                    future.raise(error);
-                } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) {
+                    markCoordinatorUnknown(responseError);
+                    future.raise(responseError);
+                } else if (responseError == Errors.GROUP_AUTHORIZATION_FAILED) {
                     future.raise(GroupAuthorizationException.forGroupId(rebalanceConfig.groupId));
                 } else {
-                    future.raise(new KafkaException("Unexpected error in fetch offset response: " + error.message()));
+                    future.raise(new KafkaException("Unexpected error in fetch offset response: " + responseError.message()));
                 }
                 return;
             }
 
             Set<String> unauthorizedTopics = null;
-            Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>(response.responseData().size());
+            // if map entry is null, we know we are handling a response less than V8
+            boolean useV8 = response.responseData(rebalanceConfig.groupId) != null;
+            Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData = useV8 ?
+                response.responseData(rebalanceConfig.groupId) : response.oldResponseData();

Review comment:
       As before, it will be good if we don't have repeated logic like this in every client.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -46,9 +52,9 @@
         private final boolean throwOnFetchStableOffsetsUnsupported;
 
         public Builder(String groupId,
-                       boolean requireStable,
-                       List<TopicPartition> partitions,
-                       boolean throwOnFetchStableOffsetsUnsupported) {
+            boolean requireStable,
+            List<TopicPartition> partitions,
+            boolean throwOnFetchStableOffsetsUnsupported) {

Review comment:
       nit: unnecessary indentation change

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -68,36 +74,117 @@ public Builder(String groupId,
             }
 
             this.data = new OffsetFetchRequestData()
-                            .setGroupId(groupId)
-                            .setRequireStable(requireStable)
-                            .setTopics(topics);
+                .setGroupId(groupId)
+                .setRequireStable(requireStable)
+                .setTopics(topics);
             this.throwOnFetchStableOffsetsUnsupported = throwOnFetchStableOffsetsUnsupported;
         }
 
         boolean isAllTopicPartitions() {
             return this.data.topics() == ALL_TOPIC_PARTITIONS;
         }
 
-        @Override
-        public OffsetFetchRequest build(short version) {
-            if (isAllTopicPartitions() && version < 2) {
-                throw new UnsupportedVersionException("The broker only supports OffsetFetchRequest " +
-                    "v" + version + ", but we need v2 or newer to request all topic partitions.");
-            }
+        public Builder(Map<String, List<TopicPartition>> groupIdToTopicPartitionMap,
+            boolean requireStable,
+            boolean throwOnFetchStableOffsetsUnsupported) {
+            super(ApiKeys.OFFSET_FETCH);
 
-            if (data.requireStable() && version < 7) {
-                if (throwOnFetchStableOffsetsUnsupported) {
-                    throw new UnsupportedVersionException("Broker unexpectedly " +
-                        "doesn't support requireStable flag on version " + version);
+            List<OffsetFetchRequestGroup> groups = new ArrayList<>();
+            for (Entry<String, List<TopicPartition>> entry : groupIdToTopicPartitionMap.entrySet()) {
+                final List<OffsetFetchRequestTopics> topics;
+                if (groupIdToTopicPartitionMap.get(entry.getKey()) != null) {

Review comment:
       Use `entry.getValue()`?

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -68,36 +74,117 @@ public Builder(String groupId,
             }
 
             this.data = new OffsetFetchRequestData()
-                            .setGroupId(groupId)
-                            .setRequireStable(requireStable)
-                            .setTopics(topics);
+                .setGroupId(groupId)
+                .setRequireStable(requireStable)
+                .setTopics(topics);
             this.throwOnFetchStableOffsetsUnsupported = throwOnFetchStableOffsetsUnsupported;
         }
 
         boolean isAllTopicPartitions() {
             return this.data.topics() == ALL_TOPIC_PARTITIONS;
         }
 
-        @Override
-        public OffsetFetchRequest build(short version) {
-            if (isAllTopicPartitions() && version < 2) {
-                throw new UnsupportedVersionException("The broker only supports OffsetFetchRequest " +
-                    "v" + version + ", but we need v2 or newer to request all topic partitions.");
-            }
+        public Builder(Map<String, List<TopicPartition>> groupIdToTopicPartitionMap,
+            boolean requireStable,
+            boolean throwOnFetchStableOffsetsUnsupported) {
+            super(ApiKeys.OFFSET_FETCH);
 
-            if (data.requireStable() && version < 7) {
-                if (throwOnFetchStableOffsetsUnsupported) {
-                    throw new UnsupportedVersionException("Broker unexpectedly " +
-                        "doesn't support requireStable flag on version " + version);
+            List<OffsetFetchRequestGroup> groups = new ArrayList<>();
+            for (Entry<String, List<TopicPartition>> entry : groupIdToTopicPartitionMap.entrySet()) {
+                final List<OffsetFetchRequestTopics> topics;
+                if (groupIdToTopicPartitionMap.get(entry.getKey()) != null) {
+                    Map<String, OffsetFetchRequestTopics> offsetFetchRequestTopicMap =
+                        new HashMap<>();
+                    for (TopicPartition topicPartition : groupIdToTopicPartitionMap.get(entry.getKey())) {
+                        String topicName = topicPartition.topic();
+                        OffsetFetchRequestTopics topic = offsetFetchRequestTopicMap.getOrDefault(
+                            topicName, new OffsetFetchRequestTopics().setName(topicName));
+                        topic.partitionIndexes().add(topicPartition.partition());
+                        offsetFetchRequestTopicMap.put(topicName, topic);
+                    }
+                    topics = new ArrayList<>(offsetFetchRequestTopicMap.values());
                 } else {
-                    log.trace("Fallback the requireStable flag to false as broker " +
-                                  "only supports OffsetFetchRequest version {}. Need " +
-                                  "v7 or newer to enable this feature", version);
-
-                    return new OffsetFetchRequest(data.setRequireStable(false), version);
+                    topics = ALL_TOPIC_PARTITIONS_BATCH;
                 }
+                groups.add(new OffsetFetchRequestGroup()
+                    .setGroupId(entry.getKey())
+                    .setTopics(topics));
             }
+            this.data = new OffsetFetchRequestData()
+                .setGroupIds(groups)
+                .setRequireStable(requireStable);
+            this.throwOnFetchStableOffsetsUnsupported = throwOnFetchStableOffsetsUnsupported;
+        }
+
+        @Override
+        public OffsetFetchRequest build(short version) {
+            if (version < 8) {
+                if (data.groupIds().size() > 1) {
+                    throw new NoBatchedOffsetFetchRequestException("Broker does not support"
+                        + " batching groups for fetch offset request on version " + version);
+                }
+                OffsetFetchRequestData oldDataFormat = null;
+                if (!data.groupIds().isEmpty()) {
+                    OffsetFetchRequestGroup group = data.groupIds().get(0);
+                    String groupName = group.groupId();
+                    List<OffsetFetchRequestTopics> topics = group.topics();
+                    List<OffsetFetchRequestTopic> oldFormatTopics = null;
+                    if (topics != null) {
+                        oldFormatTopics = topics
+                            .stream()
+                            .map(t ->
+                                new OffsetFetchRequestTopic()
+                                    .setName(t.name())
+                                    .setPartitionIndexes(t.partitionIndexes()))
+                            .collect(Collectors.toList());
+                    }
+                    oldDataFormat = new OffsetFetchRequestData()
+                        .setGroupId(groupName)
+                        .setTopics(oldFormatTopics)
+                        .setRequireStable(data.requireStable());
+                }
+                if (isAllTopicPartitions() && version < 2) {
+                    throw new UnsupportedVersionException("The broker only supports OffsetFetchRequest " +
+                        "v" + version + ", but we need v2 or newer to request all topic partitions.");
+                }
+                if (data.requireStable() && version < 7) {

Review comment:
       Again better to move this to the top as well so that we have smaller sequence of if statements rather than nested ifs.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -152,13 +283,27 @@ public OffsetFetchResponse getErrorResponse(int throttleTimeMs, Errors error) {
                         new TopicPartition(topic.name(), partitionIndex), partitionError);
                 }
             }
+            return new OffsetFetchResponse(error, responsePartitions);
         }
-
-        if (version() >= 3) {
-            return new OffsetFetchResponse(throttleTimeMs, error, responsePartitions);
-        } else {
+        if (version() == 2) {
             return new OffsetFetchResponse(error, responsePartitions);
         }
+        if (version() >= 3 && version() < 8) {
+            return new OffsetFetchResponse(throttleTimeMs, error, responsePartitions);
+        }
+        List<String> groupIds =

Review comment:
       We have a method above that does this

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java
##########
@@ -87,12 +90,26 @@ public String apiName() {
         Map<CoordinatorKey, Throwable> failed = new HashMap<>();
         List<CoordinatorKey> unmapped = new ArrayList<>();
 
-        if (response.error() != Errors.NONE) {
-            handleError(groupId, response.error(), failed, unmapped);
+        Errors responseError = response.error();
+        // check if error is null, if it is we are dealing with v8 response

Review comment:
       Can we move this logic to the response class?

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -68,36 +74,117 @@ public Builder(String groupId,
             }
 
             this.data = new OffsetFetchRequestData()
-                            .setGroupId(groupId)
-                            .setRequireStable(requireStable)
-                            .setTopics(topics);
+                .setGroupId(groupId)
+                .setRequireStable(requireStable)
+                .setTopics(topics);
             this.throwOnFetchStableOffsetsUnsupported = throwOnFetchStableOffsetsUnsupported;
         }
 
         boolean isAllTopicPartitions() {
             return this.data.topics() == ALL_TOPIC_PARTITIONS;
         }
 
-        @Override
-        public OffsetFetchRequest build(short version) {
-            if (isAllTopicPartitions() && version < 2) {
-                throw new UnsupportedVersionException("The broker only supports OffsetFetchRequest " +
-                    "v" + version + ", but we need v2 or newer to request all topic partitions.");
-            }
+        public Builder(Map<String, List<TopicPartition>> groupIdToTopicPartitionMap,
+            boolean requireStable,
+            boolean throwOnFetchStableOffsetsUnsupported) {
+            super(ApiKeys.OFFSET_FETCH);
 
-            if (data.requireStable() && version < 7) {
-                if (throwOnFetchStableOffsetsUnsupported) {
-                    throw new UnsupportedVersionException("Broker unexpectedly " +
-                        "doesn't support requireStable flag on version " + version);
+            List<OffsetFetchRequestGroup> groups = new ArrayList<>();
+            for (Entry<String, List<TopicPartition>> entry : groupIdToTopicPartitionMap.entrySet()) {
+                final List<OffsetFetchRequestTopics> topics;
+                if (groupIdToTopicPartitionMap.get(entry.getKey()) != null) {
+                    Map<String, OffsetFetchRequestTopics> offsetFetchRequestTopicMap =
+                        new HashMap<>();
+                    for (TopicPartition topicPartition : groupIdToTopicPartitionMap.get(entry.getKey())) {

Review comment:
       As before, `entry.getValue()`, we should just store that in a local variable 

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java
##########
@@ -87,12 +90,26 @@ public String apiName() {
         Map<CoordinatorKey, Throwable> failed = new HashMap<>();
         List<CoordinatorKey> unmapped = new ArrayList<>();
 
-        if (response.error() != Errors.NONE) {
-            handleError(groupId, response.error(), failed, unmapped);
+        Errors responseError = response.error();
+        // check if error is null, if it is we are dealing with v8 response
+        if (responseError == null) {
+            if (response.groupHasError(groupId.idValue)) {
+                responseError = response.groupLevelError(groupId.idValue);
+            } else {
+                responseError = Errors.NONE;
+            }
+        }
+
+        if (responseError != Errors.NONE) {
+            handleError(groupId, responseError, failed, unmapped);
         } else {
             final Map<TopicPartition, OffsetAndMetadata> groupOffsetsListing = new HashMap<>();
-            for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry :
-                response.responseData().entrySet()) {
+            // if entry for group level response data is null, we are getting back an older version
+            // of the response
+            Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData =
+                response.responseData(groupId.idValue) == null ? response.oldResponseData() :
+                    response.responseData(groupId.idValue);
+            for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry : responseData.entrySet()) {

Review comment:
       As before, it will be good if we can move the version logic to the response. 

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -68,36 +74,117 @@ public Builder(String groupId,
             }
 
             this.data = new OffsetFetchRequestData()
-                            .setGroupId(groupId)
-                            .setRequireStable(requireStable)
-                            .setTopics(topics);
+                .setGroupId(groupId)
+                .setRequireStable(requireStable)
+                .setTopics(topics);
             this.throwOnFetchStableOffsetsUnsupported = throwOnFetchStableOffsetsUnsupported;
         }
 
         boolean isAllTopicPartitions() {
             return this.data.topics() == ALL_TOPIC_PARTITIONS;
         }
 
-        @Override
-        public OffsetFetchRequest build(short version) {
-            if (isAllTopicPartitions() && version < 2) {
-                throw new UnsupportedVersionException("The broker only supports OffsetFetchRequest " +
-                    "v" + version + ", but we need v2 or newer to request all topic partitions.");
-            }
+        public Builder(Map<String, List<TopicPartition>> groupIdToTopicPartitionMap,
+            boolean requireStable,
+            boolean throwOnFetchStableOffsetsUnsupported) {
+            super(ApiKeys.OFFSET_FETCH);
 
-            if (data.requireStable() && version < 7) {
-                if (throwOnFetchStableOffsetsUnsupported) {
-                    throw new UnsupportedVersionException("Broker unexpectedly " +
-                        "doesn't support requireStable flag on version " + version);
+            List<OffsetFetchRequestGroup> groups = new ArrayList<>();
+            for (Entry<String, List<TopicPartition>> entry : groupIdToTopicPartitionMap.entrySet()) {
+                final List<OffsetFetchRequestTopics> topics;
+                if (groupIdToTopicPartitionMap.get(entry.getKey()) != null) {
+                    Map<String, OffsetFetchRequestTopics> offsetFetchRequestTopicMap =
+                        new HashMap<>();
+                    for (TopicPartition topicPartition : groupIdToTopicPartitionMap.get(entry.getKey())) {
+                        String topicName = topicPartition.topic();
+                        OffsetFetchRequestTopics topic = offsetFetchRequestTopicMap.getOrDefault(
+                            topicName, new OffsetFetchRequestTopics().setName(topicName));
+                        topic.partitionIndexes().add(topicPartition.partition());
+                        offsetFetchRequestTopicMap.put(topicName, topic);
+                    }
+                    topics = new ArrayList<>(offsetFetchRequestTopicMap.values());
                 } else {
-                    log.trace("Fallback the requireStable flag to false as broker " +
-                                  "only supports OffsetFetchRequest version {}. Need " +
-                                  "v7 or newer to enable this feature", version);
-
-                    return new OffsetFetchRequest(data.setRequireStable(false), version);
+                    topics = ALL_TOPIC_PARTITIONS_BATCH;
                 }
+                groups.add(new OffsetFetchRequestGroup()
+                    .setGroupId(entry.getKey())
+                    .setTopics(topics));
             }
+            this.data = new OffsetFetchRequestData()
+                .setGroupIds(groups)
+                .setRequireStable(requireStable);
+            this.throwOnFetchStableOffsetsUnsupported = throwOnFetchStableOffsetsUnsupported;
+        }
+
+        @Override
+        public OffsetFetchRequest build(short version) {
+            if (version < 8) {
+                if (data.groupIds().size() > 1) {
+                    throw new NoBatchedOffsetFetchRequestException("Broker does not support"
+                        + " batching groups for fetch offset request on version " + version);
+                }
+                OffsetFetchRequestData oldDataFormat = null;
+                if (!data.groupIds().isEmpty()) {
+                    OffsetFetchRequestGroup group = data.groupIds().get(0);
+                    String groupName = group.groupId();
+                    List<OffsetFetchRequestTopics> topics = group.topics();
+                    List<OffsetFetchRequestTopic> oldFormatTopics = null;
+                    if (topics != null) {
+                        oldFormatTopics = topics
+                            .stream()
+                            .map(t ->
+                                new OffsetFetchRequestTopic()
+                                    .setName(t.name())
+                                    .setPartitionIndexes(t.partitionIndexes()))
+                            .collect(Collectors.toList());
+                    }
+                    oldDataFormat = new OffsetFetchRequestData()
+                        .setGroupId(groupName)
+                        .setTopics(oldFormatTopics)
+                        .setRequireStable(data.requireStable());
+                }
+                if (isAllTopicPartitions() && version < 2) {

Review comment:
       This could be right at the top as it was before to avoid this big `if block`

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -154,14 +166,88 @@ public OffsetFetchResponse(int throttleTimeMs, Errors error, Map<TopicPartition,
         this.error = error;
     }
 
+    /**
+     * Constructor without throttle time for version 8 and above.
+     * @param errors Error code on a per group level basis
+     * @param responseData Fetched offset information grouped group id
+     */
+    public OffsetFetchResponse(Map<String, Errors> errors, Map<String, Map<TopicPartition, PartitionData>> responseData) {

Review comment:
       Why do we have this constructor?

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -128,6 +227,38 @@ public boolean requireStable() {
         return data.requireStable();
     }
 
+    public Map<String, List<TopicPartition>> groupIdsToPartitions() {
+        Map<String, List<TopicPartition>> groupIdsToPartitions = new HashMap<>();
+        for (OffsetFetchRequestGroup group : data.groupIds()) {
+            List<TopicPartition> tpList = null;
+            if (group.topics() != ALL_TOPIC_PARTITIONS_BATCH) {
+                tpList = new ArrayList<>();
+                for (OffsetFetchRequestTopics topic : group.topics()) {
+                    for (Integer partitionIndex : topic.partitionIndexes()) {
+                        tpList.add(new TopicPartition(topic.name(), partitionIndex));
+                    }
+                }
+            }
+            groupIdsToPartitions.put(group.groupId(), tpList);
+        }
+        return groupIdsToPartitions;
+    }
+
+    public Map<String, List<OffsetFetchRequestTopics>> groupIdsToTopics() {
+        Map<String, List<OffsetFetchRequestTopics>> groupIdsToTopics = new HashMap<>();
+        for (OffsetFetchRequestGroup group : data.groupIds()) {

Review comment:
       Could use stream()?

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -154,14 +166,88 @@ public OffsetFetchResponse(int throttleTimeMs, Errors error, Map<TopicPartition,
         this.error = error;
     }
 
+    /**
+     * Constructor without throttle time for version 8 and above.
+     * @param errors Error code on a per group level basis
+     * @param responseData Fetched offset information grouped group id
+     */
+    public OffsetFetchResponse(Map<String, Errors> errors, Map<String, Map<TopicPartition, PartitionData>> responseData) {
+        this(DEFAULT_THROTTLE_TIME, errors, responseData);
+    }
+
+    /**
+     * Constructor with throttle time for version 8 and above.
+     * @param throttleTimeMs The time in milliseconds that this response was throttled
+     * @param errors Potential coordinator or group level error code (for api version 2 and later)

Review comment:
       why api version 2 and later? This constructor is or version 8 and above?

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -154,14 +166,88 @@ public OffsetFetchResponse(int throttleTimeMs, Errors error, Map<TopicPartition,
         this.error = error;
     }
 
+    /**
+     * Constructor without throttle time for version 8 and above.
+     * @param errors Error code on a per group level basis
+     * @param responseData Fetched offset information grouped group id
+     */
+    public OffsetFetchResponse(Map<String, Errors> errors, Map<String, Map<TopicPartition, PartitionData>> responseData) {
+        this(DEFAULT_THROTTLE_TIME, errors, responseData);
+    }
+
+    /**
+     * Constructor with throttle time for version 8 and above.
+     * @param throttleTimeMs The time in milliseconds that this response was throttled
+     * @param errors Potential coordinator or group level error code (for api version 2 and later)
+     * @param responseData Fetched offset information grouped by topic-partition and by group
+     */
+    public OffsetFetchResponse(int throttleTimeMs, Map<String, Errors> errors, Map<String,
+        Map<TopicPartition, PartitionData>> responseData) {
+        super(ApiKeys.OFFSET_FETCH);
+        List<OffsetFetchResponseGroup> groupList = new ArrayList<>();
+        for (Entry<String, Map<TopicPartition, PartitionData>> entry : responseData.entrySet()) {
+            Map<String, OffsetFetchResponseTopics> offsetFetchResponseTopicsMap = new HashMap<>();
+            for (Entry<TopicPartition, PartitionData> partitionEntry :
+                responseData.get(entry.getKey()).entrySet()) {

Review comment:
       `entry.getValue()`

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -214,6 +321,10 @@ public Errors error() {
         return responseData;
     }
 
+    public Map<TopicPartition, PartitionData> responseData(String groupId) {

Review comment:
       We should probably have a `responseData` method that takes version or ensure we have the data cached regardless of version

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -185,21 +271,42 @@ public boolean hasError() {
         return error != Errors.NONE;
     }
 
+    public boolean groupHasError(String groupId) {
+        return groupLevelErrors.get(groupId) != Errors.NONE;
+    }
+
     public Errors error() {
         return error;
     }
 
+    public Errors groupLevelError(String groupId) {
+        return groupLevelErrors.get(groupId);
+    }
+
     @Override
     public Map<Errors, Integer> errorCounts() {
         Map<Errors, Integer> counts = new HashMap<>();
-        updateErrorCounts(counts, error);
-        data.topics().forEach(topic ->
-                topic.partitions().forEach(partition ->
+        if (!groupLevelErrors.isEmpty()) {
+            // built response with v8 or above
+            for (Map.Entry<String, Errors> entry : groupLevelErrors.entrySet()) {
+                updateErrorCounts(counts, entry.getValue());
+            }
+            for (OffsetFetchResponseGroup group : data.groupIds()) {
+                group.topics().forEach(topic ->
+                    topic.partitions().forEach(partition ->
                         updateErrorCounts(counts, Errors.forCode(partition.errorCode()))));
+            }
+        } else {
+            // built response with v0-v7
+            updateErrorCounts(counts, error);
+            data.topics().forEach(topic ->
+                topic.partitions().forEach(partition ->
+                    updateErrorCounts(counts, Errors.forCode(partition.errorCode()))));
+        }
         return counts;
     }
 
-    public Map<TopicPartition, PartitionData> responseData() {
+    public Map<TopicPartition, PartitionData> oldResponseData() {

Review comment:
       Is it an odd method name for a public method that we would use elsewhere. What happens when we add other versions - old/older/oldest?




-- 
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] skaundinya15 commented on a change in pull request #10962: KAFKA-12234: Implement request/response for offsetFetch batching (KIP-709)

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r665777053



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +315,28 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public boolean isAllPartitionsForGroup(String groupId) {
+        OffsetFetchRequestGroup group = data
+            .groupIds()
+            .stream()
+            .filter(g -> g.groupId().equals(groupId))
+            .collect(toSingleton());
+        return group.topics() == ALL_TOPIC_PARTITIONS_BATCH;
+    }
+
+    // Custom collector to filter a single element
+    private <T> Collector<T, ?, T> toSingleton() {

Review comment:
       Opened https://issues.apache.org/jira/browse/KAFKA-13045 to track 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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664772846



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -78,26 +85,107 @@ boolean isAllTopicPartitions() {
             return this.data.topics() == ALL_TOPIC_PARTITIONS;
         }
 
+        public Builder(Map<String, List<TopicPartition>> groupIdToTopicPartitionMap,
+            boolean requireStable,
+            boolean throwOnFetchStableOffsetsUnsupported) {
+            super(ApiKeys.OFFSET_FETCH);
+
+            List<OffsetFetchRequestGroup> groups = new ArrayList<>();
+            for (Entry<String, List<TopicPartition>> entry : groupIdToTopicPartitionMap.entrySet()) {
+                String groupName = entry.getKey();
+                List<TopicPartition> tpList = entry.getValue();
+                final List<OffsetFetchRequestTopics> topics;
+                if (tpList != null) {
+                    Map<String, OffsetFetchRequestTopics> offsetFetchRequestTopicMap =
+                        new HashMap<>();
+                    for (TopicPartition topicPartition : tpList) {
+                        String topicName = topicPartition.topic();
+                        OffsetFetchRequestTopics topic = offsetFetchRequestTopicMap.getOrDefault(
+                            topicName, new OffsetFetchRequestTopics().setName(topicName));
+                        topic.partitionIndexes().add(topicPartition.partition());
+                        offsetFetchRequestTopicMap.put(topicName, topic);
+                    }
+                    topics = new ArrayList<>(offsetFetchRequestTopicMap.values());
+                } else {
+                    topics = ALL_TOPIC_PARTITIONS_BATCH;
+                }
+                groups.add(new OffsetFetchRequestGroup()
+                    .setGroupId(groupName)
+                    .setTopics(topics));
+            }
+            this.data = new OffsetFetchRequestData()
+                .setGroupIds(groups)
+                .setRequireStable(requireStable);
+            this.throwOnFetchStableOffsetsUnsupported = throwOnFetchStableOffsetsUnsupported;
+        }
+
         @Override
         public OffsetFetchRequest build(short version) {
             if (isAllTopicPartitions() && version < 2) {
                 throw new UnsupportedVersionException("The broker only supports OffsetFetchRequest " +
                     "v" + version + ", but we need v2 or newer to request all topic partitions.");
             }
-
+            if (data.groupIds().size() > 1 && version < 8) {
+                throw new NoBatchedOffsetFetchRequestException("Broker does not support"
+                    + " batching groups for fetch offset request on version " + version);
+            }
             if (data.requireStable() && version < 7) {
                 if (throwOnFetchStableOffsetsUnsupported) {
                     throw new UnsupportedVersionException("Broker unexpectedly " +
                         "doesn't support requireStable flag on version " + version);
                 } else {
                     log.trace("Fallback the requireStable flag to false as broker " +
-                                  "only supports OffsetFetchRequest version {}. Need " +
-                                  "v7 or newer to enable this feature", version);
+                        "only supports OffsetFetchRequest version {}. Need " +
+                        "v7 or newer to enable this feature", version);
 
                     return new OffsetFetchRequest(data.setRequireStable(false), version);
                 }
             }
-
+            if (version < 8) {
+                OffsetFetchRequestData oldDataFormat = null;
+                if (!data.groupIds().isEmpty()) {
+                    OffsetFetchRequestGroup group = data.groupIds().get(0);
+                    String groupName = group.groupId();
+                    List<OffsetFetchRequestTopics> topics = group.topics();
+                    List<OffsetFetchRequestTopic> oldFormatTopics = null;
+                    if (topics != null) {
+                        oldFormatTopics = topics
+                            .stream()
+                            .map(t ->
+                                new OffsetFetchRequestTopic()
+                                    .setName(t.name())
+                                    .setPartitionIndexes(t.partitionIndexes()))
+                            .collect(Collectors.toList());
+                    }
+                    oldDataFormat = new OffsetFetchRequestData()
+                        .setGroupId(groupName)
+                        .setTopics(oldFormatTopics)
+                        .setRequireStable(data.requireStable());
+                }
+                return new OffsetFetchRequest(oldDataFormat == null ? data : oldDataFormat, version);
+            }
+            // version 8 but have used old format of request, convert to version 8 of request

Review comment:
       Not sure I totally follow the suggestion here - The `version < 8` block followed by the subsequent `!data.groupIds().isEmpty()` if blog exists to convert the version to an old version if server side we are less than 8. Then we have the additional check for version 8 but using the old version of the request if the client sent the older version but the server supports the newer version, so we convert it to the newer version. What were you specifically thinking to change here to make the code more readable?




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664800832



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +315,28 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public boolean isAllPartitionsForGroup(String groupId) {
+        OffsetFetchRequestGroup group = data
+            .groupIds()
+            .stream()
+            .filter(g -> g.groupId().equals(groupId))
+            .collect(toSingleton());
+        return group.topics() == ALL_TOPIC_PARTITIONS_BATCH;
+    }
+
+    // Custom collector to filter a single element
+    private <T> Collector<T, ?, T> toSingleton() {

Review comment:
       Would this be a test for single group requests with v8? I have added that case to the existing single group tests for older versions in `OffsetFetchRequestTest.scala` and will push those changes shortly. Is there another type of test you were thinking of?




-- 
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] rajinisivaram commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664873858



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +315,28 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public boolean isAllPartitionsForGroup(String groupId) {
+        OffsetFetchRequestGroup group = data
+            .groupIds()
+            .stream()
+            .filter(g -> g.groupId().equals(groupId))
+            .collect(toSingleton());
+        return group.topics() == ALL_TOPIC_PARTITIONS_BATCH;
+    }
+
+    // Custom collector to filter a single element
+    private <T> Collector<T, ?, T> toSingleton() {

Review comment:
       You can probably set the list in `OffsetFetchRequestData` directly in the test. But let's leave that for a follow-on 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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663421298



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -185,21 +271,42 @@ public boolean hasError() {
         return error != Errors.NONE;
     }
 
+    public boolean groupHasError(String groupId) {
+        return groupLevelErrors.get(groupId) != Errors.NONE;
+    }
+
     public Errors error() {
         return error;
     }
 
+    public Errors groupLevelError(String groupId) {
+        return groupLevelErrors.get(groupId);
+    }
+
     @Override
     public Map<Errors, Integer> errorCounts() {
         Map<Errors, Integer> counts = new HashMap<>();
-        updateErrorCounts(counts, error);
-        data.topics().forEach(topic ->
-                topic.partitions().forEach(partition ->
+        if (!groupLevelErrors.isEmpty()) {
+            // built response with v8 or above
+            for (Map.Entry<String, Errors> entry : groupLevelErrors.entrySet()) {
+                updateErrorCounts(counts, entry.getValue());
+            }
+            for (OffsetFetchResponseGroup group : data.groupIds()) {
+                group.topics().forEach(topic ->
+                    topic.partitions().forEach(partition ->
                         updateErrorCounts(counts, Errors.forCode(partition.errorCode()))));
+            }
+        } else {
+            // built response with v0-v7
+            updateErrorCounts(counts, error);
+            data.topics().forEach(topic ->
+                topic.partitions().forEach(partition ->
+                    updateErrorCounts(counts, Errors.forCode(partition.errorCode()))));
+        }
         return counts;
     }
 
-    public Map<TopicPartition, PartitionData> responseData() {
+    public Map<TopicPartition, PartitionData> oldResponseData() {

Review comment:
       Good point, I will rename this method to reflect which versions this applies to and add a comment that it is only public for testing, as I will refactor everything else that uses this to use `responseData()` since we are moving the version checks into that 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] rajinisivaram commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664785624



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##########
@@ -1308,29 +1308,31 @@ private OffsetFetchResponseHandler() {
 
         @Override
         public void handle(OffsetFetchResponse response, RequestFuture<Map<TopicPartition, OffsetAndMetadata>> future) {
-            if (response.hasError()) {
-                Errors error = response.error();
-                log.debug("Offset fetch failed: {}", error.message());
+            Errors responseError = response.groupLevelError(rebalanceConfig.groupId);

Review comment:
       we can leave as is.




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664689037



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##########
@@ -1308,29 +1308,31 @@ private OffsetFetchResponseHandler() {
 
         @Override
         public void handle(OffsetFetchResponse response, RequestFuture<Map<TopicPartition, OffsetAndMetadata>> future) {
-            if (response.hasError()) {
-                Errors error = response.error();
-                log.debug("Offset fetch failed: {}", error.message());
+            Errors responseError = response.groupLevelError(rebalanceConfig.groupId);
+            if (responseError != Errors.NONE) {
+                log.debug("Offset fetch failed: {}", responseError.message());
 
-                if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS) {
+                if (responseError == Errors.COORDINATOR_LOAD_IN_PROGRESS) {
                     // just retry
-                    future.raise(error);
-                } else if (error == Errors.NOT_COORDINATOR) {
+                    future.raise(responseError);
+                } else if (responseError == Errors.NOT_COORDINATOR) {
                     // re-discover the coordinator and retry
-                    markCoordinatorUnknown(error);
-                    future.raise(error);
-                } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) {
+                    markCoordinatorUnknown(responseError);
+                    future.raise(responseError);
+                } else if (responseError == Errors.GROUP_AUTHORIZATION_FAILED) {
                     future.raise(GroupAuthorizationException.forGroupId(rebalanceConfig.groupId));
                 } else {
-                    future.raise(new KafkaException("Unexpected error in fetch offset response: " + error.message()));
+                    future.raise(new KafkaException("Unexpected error in fetch offset response: " + responseError.message()));
                 }
                 return;
             }
 
             Set<String> unauthorizedTopics = null;
-            Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>(response.responseData().size());
+            Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData =

Review comment:
       What should we rename this to? I guess technically this would be the `responseData` as in the next line we extract the actual `offsets` map that is a`Map<TopicPartition, OffsetAndMetadata>`.




-- 
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] rajinisivaram commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664798710



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +315,28 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public boolean isAllPartitionsForGroup(String groupId) {
+        OffsetFetchRequestGroup group = data
+            .groupIds()
+            .stream()
+            .filter(g -> g.groupId().equals(groupId))
+            .collect(toSingleton());
+        return group.topics() == ALL_TOPIC_PARTITIONS_BATCH;
+    }
+
+    // Custom collector to filter a single element
+    private <T> Collector<T, ?, T> toSingleton() {

Review comment:
       Can we a add a test in the new `kafka.server.OffsetFetchRequestTest`? Can do that 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] skaundinya15 commented on pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#issuecomment-875021173


   @rajinisivaram Thank you for the reviews, I have updated the PR addressing all your comments. Whenever you get a chance, the PR is ready for review - thank 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] rajinisivaram commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
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-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663421545



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -214,6 +321,10 @@ public Errors error() {
         return responseData;
     }
 
+    public Map<TopicPartition, PartitionData> responseData(String groupId) {

Review comment:
       I am planning on changing this method to check and see if `groupToPartitionData` is empty. If it is empty we are using a version less than the one that supports batching and it will fall back to the old behavior. If not we know the response is initialized with data that supports batching and then it returns from the information from `groupToPartitionData`. Do you think this will be sufficient?




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663419332



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -154,14 +166,88 @@ public OffsetFetchResponse(int throttleTimeMs, Errors error, Map<TopicPartition,
         this.error = error;
     }
 
+    /**
+     * Constructor without throttle time for version 8 and above.
+     * @param errors Error code on a per group level basis
+     * @param responseData Fetched offset information grouped group id
+     */
+    public OffsetFetchResponse(Map<String, Errors> errors, Map<String, Map<TopicPartition, PartitionData>> responseData) {

Review comment:
       I added this constructor to keep the precedent of having a constructor with no throttle time and just passing in the `DEFAULT_THROTTLE_MS` - if it's not needed it can definitely be 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] skaundinya15 commented on pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#issuecomment-875128420


   > @skaundinya15 Thanks for the updates. Left comments about indexing of `groups` (starts from 0) in the tests with the change to collection in the last commit. Apart from that LGTM if tests pass.
   Just pushed changes for this to fix the failing tests - thanks for pointing it 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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663417944



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -68,36 +74,117 @@ public Builder(String groupId,
             }
 
             this.data = new OffsetFetchRequestData()
-                            .setGroupId(groupId)
-                            .setRequireStable(requireStable)
-                            .setTopics(topics);
+                .setGroupId(groupId)
+                .setRequireStable(requireStable)
+                .setTopics(topics);
             this.throwOnFetchStableOffsetsUnsupported = throwOnFetchStableOffsetsUnsupported;
         }
 
         boolean isAllTopicPartitions() {
             return this.data.topics() == ALL_TOPIC_PARTITIONS;
         }
 
-        @Override
-        public OffsetFetchRequest build(short version) {
-            if (isAllTopicPartitions() && version < 2) {
-                throw new UnsupportedVersionException("The broker only supports OffsetFetchRequest " +
-                    "v" + version + ", but we need v2 or newer to request all topic partitions.");
-            }
+        public Builder(Map<String, List<TopicPartition>> groupIdToTopicPartitionMap,
+            boolean requireStable,
+            boolean throwOnFetchStableOffsetsUnsupported) {
+            super(ApiKeys.OFFSET_FETCH);
 
-            if (data.requireStable() && version < 7) {
-                if (throwOnFetchStableOffsetsUnsupported) {
-                    throw new UnsupportedVersionException("Broker unexpectedly " +
-                        "doesn't support requireStable flag on version " + version);
+            List<OffsetFetchRequestGroup> groups = new ArrayList<>();
+            for (Entry<String, List<TopicPartition>> entry : groupIdToTopicPartitionMap.entrySet()) {
+                final List<OffsetFetchRequestTopics> topics;
+                if (groupIdToTopicPartitionMap.get(entry.getKey()) != null) {
+                    Map<String, OffsetFetchRequestTopics> offsetFetchRequestTopicMap =
+                        new HashMap<>();
+                    for (TopicPartition topicPartition : groupIdToTopicPartitionMap.get(entry.getKey())) {

Review comment:
       Good point on storing it in the local variable, I will do that. But I think we still do want `entry.getKey()` in this situation, not `entry.getValue()`.




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663417944



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -68,36 +74,117 @@ public Builder(String groupId,
             }
 
             this.data = new OffsetFetchRequestData()
-                            .setGroupId(groupId)
-                            .setRequireStable(requireStable)
-                            .setTopics(topics);
+                .setGroupId(groupId)
+                .setRequireStable(requireStable)
+                .setTopics(topics);
             this.throwOnFetchStableOffsetsUnsupported = throwOnFetchStableOffsetsUnsupported;
         }
 
         boolean isAllTopicPartitions() {
             return this.data.topics() == ALL_TOPIC_PARTITIONS;
         }
 
-        @Override
-        public OffsetFetchRequest build(short version) {
-            if (isAllTopicPartitions() && version < 2) {
-                throw new UnsupportedVersionException("The broker only supports OffsetFetchRequest " +
-                    "v" + version + ", but we need v2 or newer to request all topic partitions.");
-            }
+        public Builder(Map<String, List<TopicPartition>> groupIdToTopicPartitionMap,
+            boolean requireStable,
+            boolean throwOnFetchStableOffsetsUnsupported) {
+            super(ApiKeys.OFFSET_FETCH);
 
-            if (data.requireStable() && version < 7) {
-                if (throwOnFetchStableOffsetsUnsupported) {
-                    throw new UnsupportedVersionException("Broker unexpectedly " +
-                        "doesn't support requireStable flag on version " + version);
+            List<OffsetFetchRequestGroup> groups = new ArrayList<>();
+            for (Entry<String, List<TopicPartition>> entry : groupIdToTopicPartitionMap.entrySet()) {
+                final List<OffsetFetchRequestTopics> topics;
+                if (groupIdToTopicPartitionMap.get(entry.getKey()) != null) {
+                    Map<String, OffsetFetchRequestTopics> offsetFetchRequestTopicMap =
+                        new HashMap<>();
+                    for (TopicPartition topicPartition : groupIdToTopicPartitionMap.get(entry.getKey())) {

Review comment:
       Good point on storing it in the local variable, I will do that. But I think we still do want `entry.getKey()` in this situation, not `entry.getValue()`.




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663430630



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -128,6 +227,38 @@ public boolean requireStable() {
         return data.requireStable();
     }
 
+    public Map<String, List<TopicPartition>> groupIdsToPartitions() {
+        Map<String, List<TopicPartition>> groupIdsToPartitions = new HashMap<>();
+        for (OffsetFetchRequestGroup group : data.groupIds()) {
+            List<TopicPartition> tpList = null;
+            if (group.topics() != ALL_TOPIC_PARTITIONS_BATCH) {
+                tpList = new ArrayList<>();
+                for (OffsetFetchRequestTopics topic : group.topics()) {
+                    for (Integer partitionIndex : topic.partitionIndexes()) {
+                        tpList.add(new TopicPartition(topic.name(), partitionIndex));
+                    }
+                }
+            }
+            groupIdsToPartitions.put(group.groupId(), tpList);
+        }
+        return groupIdsToPartitions;
+    }
+
+    public Map<String, List<OffsetFetchRequestTopics>> groupIdsToTopics() {
+        Map<String, List<OffsetFetchRequestTopics>> groupIdsToTopics = new HashMap<>();
+        for (OffsetFetchRequestGroup group : data.groupIds()) {

Review comment:
       `Collectors.toMap()` doesn't allow `null` entries (we could have `null` entries if a specific group wants all the topic partition offsets) and throws an NPE, so instead I can use `foreach` and do an inline put for all entries to the hashmap.




-- 
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] rajinisivaram commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663485656



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -214,6 +321,10 @@ public Errors error() {
         return responseData;
     }
 
+    public Map<TopicPartition, PartitionData> responseData(String groupId) {

Review comment:
       Yes, sounds good.




-- 
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] rajinisivaram commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664804113



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +315,28 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public boolean isAllPartitionsForGroup(String groupId) {
+        OffsetFetchRequestGroup group = data
+            .groupIds()
+            .stream()
+            .filter(g -> g.groupId().equals(groupId))
+            .collect(toSingleton());

Review comment:
       The helper method `toSingleton()` throws IllegalStateException if the list size is greater than one. If a request contains the same group twice, it can appear twice in the list.




-- 
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] skaundinya15 edited a comment on pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 edited a comment on pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#issuecomment-875128420


   > @skaundinya15 Thanks for the updates. Left comments about indexing of `groups` (starts from 0) in the tests with the change to collection in the last commit. Apart from that LGTM if tests pass.
   
   @rajinisivaram Just pushed changes for this to fix the failing tests - thanks for pointing it 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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664807523



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +315,28 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public boolean isAllPartitionsForGroup(String groupId) {
+        OffsetFetchRequestGroup group = data
+            .groupIds()
+            .stream()
+            .filter(g -> g.groupId().equals(groupId))
+            .collect(toSingleton());
+        return group.topics() == ALL_TOPIC_PARTITIONS_BATCH;
+    }
+
+    // Custom collector to filter a single element
+    private <T> Collector<T, ?, T> toSingleton() {

Review comment:
       Ah okay, good point. Let's do this in a follow up PR, since we are trying to get this feature into 3.0 right 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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663417537



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java
##########
@@ -73,7 +73,10 @@ public String apiName() {
     public OffsetFetchRequest.Builder buildRequest(int coordinatorId, Set<CoordinatorKey> keys) {
         // Set the flag to false as for admin client request,
         // we don't need to wait for any pending offset state to clear.
-        return new OffsetFetchRequest.Builder(groupId.idValue, false, partitions, false);

Review comment:
       The old `Builder` still exists, I just changed it to use the new format, since when we implement the Admin APIs we will have to change everything to use that anyway. However if you think it may be better to retain the old pattern I can change it - 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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663537638



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -154,14 +166,88 @@ public OffsetFetchResponse(int throttleTimeMs, Errors error, Map<TopicPartition,
         this.error = error;
     }
 
+    /**
+     * Constructor without throttle time for version 8 and above.
+     * @param errors Error code on a per group level basis
+     * @param responseData Fetched offset information grouped group id
+     */
+    public OffsetFetchResponse(Map<String, Errors> errors, Map<String, Map<TopicPartition, PartitionData>> responseData) {

Review comment:
       Removed the constructor with `DEFAULT_THROTTLE_TIME`.




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663417840



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -68,36 +74,117 @@ public Builder(String groupId,
             }
 
             this.data = new OffsetFetchRequestData()
-                            .setGroupId(groupId)
-                            .setRequireStable(requireStable)
-                            .setTopics(topics);
+                .setGroupId(groupId)
+                .setRequireStable(requireStable)
+                .setTopics(topics);
             this.throwOnFetchStableOffsetsUnsupported = throwOnFetchStableOffsetsUnsupported;
         }
 
         boolean isAllTopicPartitions() {
             return this.data.topics() == ALL_TOPIC_PARTITIONS;
         }
 
-        @Override
-        public OffsetFetchRequest build(short version) {
-            if (isAllTopicPartitions() && version < 2) {
-                throw new UnsupportedVersionException("The broker only supports OffsetFetchRequest " +
-                    "v" + version + ", but we need v2 or newer to request all topic partitions.");
-            }
+        public Builder(Map<String, List<TopicPartition>> groupIdToTopicPartitionMap,
+            boolean requireStable,
+            boolean throwOnFetchStableOffsetsUnsupported) {
+            super(ApiKeys.OFFSET_FETCH);
 
-            if (data.requireStable() && version < 7) {
-                if (throwOnFetchStableOffsetsUnsupported) {
-                    throw new UnsupportedVersionException("Broker unexpectedly " +
-                        "doesn't support requireStable flag on version " + version);
+            List<OffsetFetchRequestGroup> groups = new ArrayList<>();
+            for (Entry<String, List<TopicPartition>> entry : groupIdToTopicPartitionMap.entrySet()) {
+                final List<OffsetFetchRequestTopics> topics;
+                if (groupIdToTopicPartitionMap.get(entry.getKey()) != null) {

Review comment:
       Here I'm using `entry.getKey()` because we are trying to get the list of topic partitions for a specific group id. `entry.getValue()` would give us `List<TopicPartition>` whereas we need `String`, which `entry.getKey()` would give us.




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663419368



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -154,14 +166,88 @@ public OffsetFetchResponse(int throttleTimeMs, Errors error, Map<TopicPartition,
         this.error = error;
     }
 
+    /**
+     * Constructor without throttle time for version 8 and above.
+     * @param errors Error code on a per group level basis
+     * @param responseData Fetched offset information grouped group id
+     */
+    public OffsetFetchResponse(Map<String, Errors> errors, Map<String, Map<TopicPartition, PartitionData>> responseData) {
+        this(DEFAULT_THROTTLE_TIME, errors, responseData);
+    }
+
+    /**
+     * Constructor with throttle time for version 8 and above.
+     * @param throttleTimeMs The time in milliseconds that this response was throttled
+     * @param errors Potential coordinator or group level error code (for api version 2 and later)

Review comment:
       Good point, will change 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] rajinisivaram commented on pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#issuecomment-875502747


   @skaundinya15 Thanks for the PR. Test failures (KafkaMetadataLogTest.testDeleteSnapshots, RaftClusterTest.testCreateClusterAndCreateAndManyTopicsWithManyPartitions, QueryableStateIntegrationTest.shouldBeAbleToQueryFilterState) not related. Merging to trunk and 3.0.


-- 
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] rajinisivaram commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664793097



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -78,26 +85,107 @@ boolean isAllTopicPartitions() {
             return this.data.topics() == ALL_TOPIC_PARTITIONS;
         }
 
+        public Builder(Map<String, List<TopicPartition>> groupIdToTopicPartitionMap,
+            boolean requireStable,
+            boolean throwOnFetchStableOffsetsUnsupported) {
+            super(ApiKeys.OFFSET_FETCH);
+
+            List<OffsetFetchRequestGroup> groups = new ArrayList<>();
+            for (Entry<String, List<TopicPartition>> entry : groupIdToTopicPartitionMap.entrySet()) {
+                String groupName = entry.getKey();
+                List<TopicPartition> tpList = entry.getValue();
+                final List<OffsetFetchRequestTopics> topics;
+                if (tpList != null) {
+                    Map<String, OffsetFetchRequestTopics> offsetFetchRequestTopicMap =
+                        new HashMap<>();
+                    for (TopicPartition topicPartition : tpList) {
+                        String topicName = topicPartition.topic();
+                        OffsetFetchRequestTopics topic = offsetFetchRequestTopicMap.getOrDefault(
+                            topicName, new OffsetFetchRequestTopics().setName(topicName));
+                        topic.partitionIndexes().add(topicPartition.partition());
+                        offsetFetchRequestTopicMap.put(topicName, topic);
+                    }
+                    topics = new ArrayList<>(offsetFetchRequestTopicMap.values());
+                } else {
+                    topics = ALL_TOPIC_PARTITIONS_BATCH;
+                }
+                groups.add(new OffsetFetchRequestGroup()
+                    .setGroupId(groupName)
+                    .setTopics(topics));
+            }
+            this.data = new OffsetFetchRequestData()
+                .setGroupIds(groups)
+                .setRequireStable(requireStable);
+            this.throwOnFetchStableOffsetsUnsupported = throwOnFetchStableOffsetsUnsupported;
+        }
+
         @Override
         public OffsetFetchRequest build(short version) {
             if (isAllTopicPartitions() && version < 2) {
                 throw new UnsupportedVersionException("The broker only supports OffsetFetchRequest " +
                     "v" + version + ", but we need v2 or newer to request all topic partitions.");
             }
-
+            if (data.groupIds().size() > 1 && version < 8) {
+                throw new NoBatchedOffsetFetchRequestException("Broker does not support"
+                    + " batching groups for fetch offset request on version " + version);
+            }
             if (data.requireStable() && version < 7) {
                 if (throwOnFetchStableOffsetsUnsupported) {
                     throw new UnsupportedVersionException("Broker unexpectedly " +
                         "doesn't support requireStable flag on version " + version);
                 } else {
                     log.trace("Fallback the requireStable flag to false as broker " +
-                                  "only supports OffsetFetchRequest version {}. Need " +
-                                  "v7 or newer to enable this feature", version);
+                        "only supports OffsetFetchRequest version {}. Need " +
+                        "v7 or newer to enable this feature", version);
 
                     return new OffsetFetchRequest(data.setRequireStable(false), version);
                 }
             }
-
+            if (version < 8) {
+                OffsetFetchRequestData oldDataFormat = null;
+                if (!data.groupIds().isEmpty()) {
+                    OffsetFetchRequestGroup group = data.groupIds().get(0);
+                    String groupName = group.groupId();
+                    List<OffsetFetchRequestTopics> topics = group.topics();
+                    List<OffsetFetchRequestTopic> oldFormatTopics = null;
+                    if (topics != null) {
+                        oldFormatTopics = topics
+                            .stream()
+                            .map(t ->
+                                new OffsetFetchRequestTopic()
+                                    .setName(t.name())
+                                    .setPartitionIndexes(t.partitionIndexes()))
+                            .collect(Collectors.toList());
+                    }
+                    oldDataFormat = new OffsetFetchRequestData()
+                        .setGroupId(groupName)
+                        .setTopics(oldFormatTopics)
+                        .setRequireStable(data.requireStable());
+                }
+                return new OffsetFetchRequest(oldDataFormat == null ? data : oldDataFormat, version);
+            }
+            // version 8 but have used old format of request, convert to version 8 of request

Review comment:
       At the moment, the code seems to do:
   ```
   if (version < 8) {
    do-conversion-if-necessary
   }
   // version 8 but have used old format of request, convert to version 8 of request
   do-conversion-if-necessary
   ```
   My suggestion was to turn it into:
   ```
   // convert data to use the appropriate version since version 8 uses different format
   if (version < 8) {
    do-conversion-if-necessary
   } else {
    do-conversion-if-necessary
   }
   ```




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664738871



##########
File path: clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchRequestTest.java
##########
@@ -76,62 +73,169 @@ public void testConstructor() {
         }
 
         for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
-            OffsetFetchRequest request = builder.build(version);
-            assertFalse(request.isAllPartitions());
-            assertEquals(groupId, request.groupId());
-            assertEquals(partitions, request.partitions());
-
-            OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
-            assertEquals(Errors.NONE, response.error());
-            assertFalse(response.hasError());
-            assertEquals(Collections.singletonMap(Errors.NONE, version <= (short) 1 ? 3 : 1), response.errorCounts(),
-                "Incorrect error count for version " + version);
-
-            if (version <= 1) {
-                assertEquals(expectedData, response.responseData());
+            if (version < 8) {
+                builder = new OffsetFetchRequest.Builder(
+                    group1,
+                    false,
+                    partitions,
+                    false);
+                assertFalse(builder.isAllTopicPartitions());
+                OffsetFetchRequest request = builder.build(version);
+                assertFalse(request.isAllPartitions());
+                assertEquals(group1, request.groupId());
+                assertEquals(partitions, request.partitions());
+
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.error());
+                assertFalse(response.hasError());
+                assertEquals(Collections.singletonMap(Errors.NONE, version <= (short) 1 ? 3 : 1), response.errorCounts(),
+                    "Incorrect error count for version " + version);
+
+                if (version <= 1) {
+                    assertEquals(expectedData, response.responseDataV0ToV7());
+                }
+
+                if (version >= 3) {
+                    assertEquals(throttleTimeMs, response.throttleTimeMs());
+                } else {
+                    assertEquals(DEFAULT_THROTTLE_TIME, response.throttleTimeMs());
+                }
+            } else {
+                builder = new Builder(Collections.singletonMap(group1, partitions), false, false);
+                OffsetFetchRequest request = builder.build(version);
+                Map<String, List<TopicPartition>> groupToPartitionMap =
+                    request.groupIdsToPartitions();
+                Map<String, List<OffsetFetchRequestTopics>> groupToTopicMap =
+                    request.groupIdsToTopics();
+                assertFalse(request.isAllPartitionsForGroup(group1));
+                assertTrue(groupToPartitionMap.containsKey(group1) && groupToTopicMap.containsKey(
+                    group1));
+                assertEquals(partitions, groupToPartitionMap.get(group1));
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.groupLevelError(group1));
+                assertFalse(response.groupHasError(group1));
+                assertEquals(Collections.singletonMap(Errors.NONE, 1), response.errorCounts(),
+                    "Incorrect error count for version " + version);
+                assertEquals(throttleTimeMs, response.throttleTimeMs());
             }
+        }
+    }
+
+    @Test
+    public void testConstructorWithMultipleGroups() {
+        List<TopicPartition> topic1Partitions = Arrays.asList(
+            new TopicPartition(topicOne, partitionOne),
+            new TopicPartition(topicOne, partitionTwo));
+        List<TopicPartition> topic2Partitions = Arrays.asList(
+            new TopicPartition(topicTwo, partitionOne),
+            new TopicPartition(topicTwo, partitionTwo));
+        List<TopicPartition> topic3Partitions = Arrays.asList(
+            new TopicPartition(topicThree, partitionOne),
+            new TopicPartition(topicThree, partitionTwo));
+        Map<String, List<TopicPartition>> groupToTp = new HashMap<>();
+        groupToTp.put(group1, topic1Partitions);
+        groupToTp.put(group2, topic2Partitions);
+        groupToTp.put(group3, topic3Partitions);
+        groupToTp.put(group4, null);
+        groupToTp.put(group5, null);
+        int throttleTimeMs = 10;
 
-            if (version >= 3) {
+        for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
+            if (version >= 8) {
+                builder = new Builder(groupToTp, false, false);
+                OffsetFetchRequest request = builder.build(version);
+                Map<String, List<TopicPartition>> groupToPartitionMap =
+                    request.groupIdsToPartitions();
+                Map<String, List<OffsetFetchRequestTopics>> groupToTopicMap =
+                    request.groupIdsToTopics();
+                assertEquals(groupToTp.keySet(), groupToTopicMap.keySet());
+                assertEquals(groupToTp.keySet(), groupToPartitionMap.keySet());
+                assertFalse(request.isAllPartitionsForGroup(group1));
+                assertFalse(request.isAllPartitionsForGroup(group2));
+                assertFalse(request.isAllPartitionsForGroup(group3));
+                assertTrue(request.isAllPartitionsForGroup(group4));
+                assertTrue(request.isAllPartitionsForGroup(group5));
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.groupLevelError(group1));
+                assertEquals(Errors.NONE, response.groupLevelError(group2));
+                assertEquals(Errors.NONE, response.groupLevelError(group3));
+                assertEquals(Errors.NONE, response.groupLevelError(group4));
+                assertEquals(Errors.NONE, response.groupLevelError(group5));
+                assertFalse(response.groupHasError(group1));
+                assertFalse(response.groupHasError(group2));
+                assertFalse(response.groupHasError(group3));
+                assertFalse(response.groupHasError(group4));
+                assertFalse(response.groupHasError(group5));
+                assertEquals(Collections.singletonMap(Errors.NONE, 5), response.errorCounts(),
+                    "Incorrect error count for version " + version);
                 assertEquals(throttleTimeMs, response.throttleTimeMs());
-            } else {
-                assertEquals(DEFAULT_THROTTLE_TIME, response.throttleTimeMs());
             }
         }
     }
 
     @Test
-    public void testConstructorFailForUnsupportedRequireStable() {
+    public void testBuildThrowForUnsupportedBatchRequest() {
         for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
-            // The builder needs to be initialized every cycle as the internal data `requireStable` flag is flipped.
-            builder = new OffsetFetchRequest.Builder(groupId, true, null, false);
-            final short finalVersion = version;
-            if (version < 2) {
-                assertThrows(UnsupportedVersionException.class, () -> builder.build(finalVersion));
-            } else {
-                OffsetFetchRequest request = builder.build(finalVersion);
-                assertEquals(groupId, request.groupId());
-                assertNull(request.partitions());
-                assertTrue(request.isAllPartitions());
-                if (version < 7) {
-                    assertFalse(request.requireStable());
-                } else {
-                    assertTrue(request.requireStable());
-                }
+            if (version < 8) {
+                Map<String, List<TopicPartition>> groupPartitionMap = new HashMap<>();
+                groupPartitionMap.put(group1, null);
+                groupPartitionMap.put(group2, null);
+                builder = new Builder(groupPartitionMap, true, false);
+                final short finalVersion = version;
+                assertThrows(NoBatchedOffsetFetchRequestException.class, () -> builder.build(finalVersion));

Review comment:
       This specific test checks if the correct exception is thrown if we try to create a batched request for a version less than 8. The `testConstructor` method tests the single group request with version 8 and above. Is there any other case you think we should add for `version >=8`?




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664687436



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##########
@@ -1308,29 +1308,31 @@ private OffsetFetchResponseHandler() {
 
         @Override
         public void handle(OffsetFetchResponse response, RequestFuture<Map<TopicPartition, OffsetAndMetadata>> future) {
-            if (response.hasError()) {
-                Errors error = response.error();
-                log.debug("Offset fetch failed: {}", error.message());
+            Errors responseError = response.groupLevelError(rebalanceConfig.groupId);

Review comment:
       I had named this `responseError` as there is already an `error` variable defined later in the code to get the partition-level error in the response. Should we still rename it to `error`?




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664718720



##########
File path: clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchRequestTest.java
##########
@@ -44,25 +46,20 @@
     private final int partitionOne = 1;
     private final String topicTwo = "topic2";
     private final int partitionTwo = 2;
-    private final String groupId = "groupId";
+    private final String topicThree = "topic3";
+    private final String group1 = "group1";
+    private final String group2 = "group2";
+    private final String group3 = "group3";
+    private final String group4 = "group4";
+    private final String group5 = "group5";

Review comment:
       The reason I had individually defined each group here is because we have to put the groups individually in the map and associated with its specific topic partition. It also made it easier to verify responses after, since we get the partition responses on a group level. Do you have an example of how putting the groups in `List<String>` would help reduce the repetition? 




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663419025



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +319,10 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public List<OffsetFetchRequestTopics> isAllPartitionsForGroup() {

Review comment:
       Ah good catch - I had originally had it the way where a groupId was passed in and we would return `true` or `false` based on whether the corresponding `List<OffsetFetchRequestTopics>` was `null` or not, but had reverted it when I was trying to debug the test failures. Guess I forgot to put it back to what it was - will change it back.




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664819056



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +315,28 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public boolean isAllPartitionsForGroup(String groupId) {
+        OffsetFetchRequestGroup group = data
+            .groupIds()
+            .stream()
+            .filter(g -> g.groupId().equals(groupId))
+            .collect(toSingleton());
+        return group.topics() == ALL_TOPIC_PARTITIONS_BATCH;
+    }
+
+    // Custom collector to filter a single element
+    private <T> Collector<T, ?, T> toSingleton() {

Review comment:
       @rajinisivaram I'm not sure how we could test that case, as the `Builder` for the `OffsetFetchRequest` takes in a `Map<String, List<TopicPartition>>` which means all of the keys have to be unique. As a result if a group appears twice, the latest entry will take precedence. Not sure if this is how we want to handle it, but generally speaking I think we don't support having the same name group id appearing twice in the request - as in it's not possible to construct a request like that, at least not using the `Builder`




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664798862



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -78,26 +85,107 @@ boolean isAllTopicPartitions() {
             return this.data.topics() == ALL_TOPIC_PARTITIONS;
         }
 
+        public Builder(Map<String, List<TopicPartition>> groupIdToTopicPartitionMap,
+            boolean requireStable,
+            boolean throwOnFetchStableOffsetsUnsupported) {
+            super(ApiKeys.OFFSET_FETCH);
+
+            List<OffsetFetchRequestGroup> groups = new ArrayList<>();
+            for (Entry<String, List<TopicPartition>> entry : groupIdToTopicPartitionMap.entrySet()) {
+                String groupName = entry.getKey();
+                List<TopicPartition> tpList = entry.getValue();
+                final List<OffsetFetchRequestTopics> topics;
+                if (tpList != null) {
+                    Map<String, OffsetFetchRequestTopics> offsetFetchRequestTopicMap =
+                        new HashMap<>();
+                    for (TopicPartition topicPartition : tpList) {
+                        String topicName = topicPartition.topic();
+                        OffsetFetchRequestTopics topic = offsetFetchRequestTopicMap.getOrDefault(
+                            topicName, new OffsetFetchRequestTopics().setName(topicName));
+                        topic.partitionIndexes().add(topicPartition.partition());
+                        offsetFetchRequestTopicMap.put(topicName, topic);
+                    }
+                    topics = new ArrayList<>(offsetFetchRequestTopicMap.values());
+                } else {
+                    topics = ALL_TOPIC_PARTITIONS_BATCH;
+                }
+                groups.add(new OffsetFetchRequestGroup()
+                    .setGroupId(groupName)
+                    .setTopics(topics));
+            }
+            this.data = new OffsetFetchRequestData()
+                .setGroupIds(groups)
+                .setRequireStable(requireStable);
+            this.throwOnFetchStableOffsetsUnsupported = throwOnFetchStableOffsetsUnsupported;
+        }
+
         @Override
         public OffsetFetchRequest build(short version) {
             if (isAllTopicPartitions() && version < 2) {
                 throw new UnsupportedVersionException("The broker only supports OffsetFetchRequest " +
                     "v" + version + ", but we need v2 or newer to request all topic partitions.");
             }
-
+            if (data.groupIds().size() > 1 && version < 8) {
+                throw new NoBatchedOffsetFetchRequestException("Broker does not support"
+                    + " batching groups for fetch offset request on version " + version);
+            }
             if (data.requireStable() && version < 7) {
                 if (throwOnFetchStableOffsetsUnsupported) {
                     throw new UnsupportedVersionException("Broker unexpectedly " +
                         "doesn't support requireStable flag on version " + version);
                 } else {
                     log.trace("Fallback the requireStable flag to false as broker " +
-                                  "only supports OffsetFetchRequest version {}. Need " +
-                                  "v7 or newer to enable this feature", version);
+                        "only supports OffsetFetchRequest version {}. Need " +
+                        "v7 or newer to enable this feature", version);
 
                     return new OffsetFetchRequest(data.setRequireStable(false), version);
                 }
             }
-
+            if (version < 8) {
+                OffsetFetchRequestData oldDataFormat = null;
+                if (!data.groupIds().isEmpty()) {
+                    OffsetFetchRequestGroup group = data.groupIds().get(0);
+                    String groupName = group.groupId();
+                    List<OffsetFetchRequestTopics> topics = group.topics();
+                    List<OffsetFetchRequestTopic> oldFormatTopics = null;
+                    if (topics != null) {
+                        oldFormatTopics = topics
+                            .stream()
+                            .map(t ->
+                                new OffsetFetchRequestTopic()
+                                    .setName(t.name())
+                                    .setPartitionIndexes(t.partitionIndexes()))
+                            .collect(Collectors.toList());
+                    }
+                    oldDataFormat = new OffsetFetchRequestData()
+                        .setGroupId(groupName)
+                        .setTopics(oldFormatTopics)
+                        .setRequireStable(data.requireStable());
+                }
+                return new OffsetFetchRequest(oldDataFormat == null ? data : oldDataFormat, version);
+            }
+            // version 8 but have used old format of request, convert to version 8 of request

Review comment:
       Ah okay, yes that makes sense, will make the 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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663421545



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -214,6 +321,10 @@ public Errors error() {
         return responseData;
     }
 
+    public Map<TopicPartition, PartitionData> responseData(String groupId) {

Review comment:
       I am planning on changing this method to check and see if `groupToPartitionData` is empty. If it is empty we are using a version less than the one that supports batching and it will fall back to the old behavior. If not we know the response is initialized with data that supports batching and then it returns from the information from `groupToPartitionData`




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664699493



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java
##########
@@ -87,12 +87,16 @@ public String apiName() {
         Map<CoordinatorKey, Throwable> failed = new HashMap<>();
         List<CoordinatorKey> unmapped = new ArrayList<>();
 
-        if (response.error() != Errors.NONE) {
-            handleError(groupId, response.error(), failed, unmapped);
+        Errors responseError = response.groupLevelError(groupId.idValue);
+        if (responseError != Errors.NONE) {
+            handleError(groupId, responseError, failed, unmapped);
         } else {
             final Map<TopicPartition, OffsetAndMetadata> groupOffsetsListing = new HashMap<>();
-            for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry :
-                response.responseData().entrySet()) {
+            // if entry for group level response data is null, we are getting back an older version
+            // of the response

Review comment:
       ah yeah good point, will remove 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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664705607



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -185,21 +249,46 @@ public boolean hasError() {
         return error != Errors.NONE;
     }
 
+    public boolean groupHasError(String groupId) {
+        return groupLevelErrors.get(groupId) != Errors.NONE;
+    }
+
     public Errors error() {
         return error;
     }
 
+    public Errors groupLevelError(String groupId) {
+        if (error != null) {
+            return error;
+        }
+        return groupLevelErrors.get(groupId);
+    }
+
     @Override
     public Map<Errors, Integer> errorCounts() {
         Map<Errors, Integer> counts = new HashMap<>();
-        updateErrorCounts(counts, error);
-        data.topics().forEach(topic ->
-                topic.partitions().forEach(partition ->
+        if (!groupLevelErrors.isEmpty()) {
+            // built response with v8 or above
+            for (Map.Entry<String, Errors> entry : groupLevelErrors.entrySet()) {
+                updateErrorCounts(counts, entry.getValue());
+            }
+            for (OffsetFetchResponseGroup group : data.groupIds()) {
+                group.topics().forEach(topic ->
+                    topic.partitions().forEach(partition ->
                         updateErrorCounts(counts, Errors.forCode(partition.errorCode()))));
+            }
+        } else {
+            // built response with v0-v7
+            updateErrorCounts(counts, error);
+            data.topics().forEach(topic ->
+                topic.partitions().forEach(partition ->
+                    updateErrorCounts(counts, Errors.forCode(partition.errorCode()))));
+        }
         return counts;
     }
 
-    public Map<TopicPartition, PartitionData> responseData() {
+    //public for testing purposes
+    public Map<TopicPartition, PartitionData> responseDataV0ToV7() {

Review comment:
       I'll make it package-private for the unit tests, as the old version of the response did not take in a `groupId` and in order to call the newer function we need to pass in a `groupId` to get the right `Map<TopicPartition, PartitionData>`




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663419025



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +319,10 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public List<OffsetFetchRequestTopics> isAllPartitionsForGroup() {

Review comment:
       So earlier I had it return whether or not a specific group had all partitions, but then ended up changing the implementation as I realized I wasn't able to cache the group to topic partition map locally, and figured it was not very efficient to always build up the group to topic partition map and then return whether or not it equals `ALL_TOPIC_PARTITIONS_BATCH`. For this reason I ended up just returning what the value of `ALL_TOPIC_PARTITIONS_BATCH` was. I agree we can definitely use a better name here, so perhaps I can change it to `allTopicPartitionsBatch`? Or perhaps there is a better way of figuring out if a specific group is requesting all topic partitions to fetch offsets for - any ideas on that?




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664807523



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +315,28 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public boolean isAllPartitionsForGroup(String groupId) {
+        OffsetFetchRequestGroup group = data
+            .groupIds()
+            .stream()
+            .filter(g -> g.groupId().equals(groupId))
+            .collect(toSingleton());
+        return group.topics() == ALL_TOPIC_PARTITIONS_BATCH;
+    }
+
+    // Custom collector to filter a single element
+    private <T> Collector<T, ?, T> toSingleton() {

Review comment:
       Ah okay, good point. Let's do this in a follow up PR, since we are trying to get this feature into 3.0 right 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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664751150



##########
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 = {

Review comment:
       Makes sense, I will make this method test the single group 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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664703667



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +315,28 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public boolean isAllPartitionsForGroup(String groupId) {
+        OffsetFetchRequestGroup group = data
+            .groupIds()
+            .stream()
+            .filter(g -> g.groupId().equals(groupId))
+            .collect(toSingleton());

Review comment:
       Not sure I follow - are you saying that if there is a group that is included multiple times in the request the `stream()` here would throw an `IllegalStateException`? Are you saying we should put a `try...catch` statement here to catch any `IllegalStateException`? 




-- 
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] rajinisivaram merged pull request #10962: KAFKA-12234: Implement request/response for offsetFetch batching (KIP-709)

Posted by GitBox <gi...@apache.org>.
rajinisivaram merged pull request #10962:
URL: https://github.com/apache/kafka/pull/10962


   


-- 
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] rajinisivaram commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664795285



##########
File path: clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchRequestTest.java
##########
@@ -76,62 +73,169 @@ public void testConstructor() {
         }
 
         for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
-            OffsetFetchRequest request = builder.build(version);
-            assertFalse(request.isAllPartitions());
-            assertEquals(groupId, request.groupId());
-            assertEquals(partitions, request.partitions());
-
-            OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
-            assertEquals(Errors.NONE, response.error());
-            assertFalse(response.hasError());
-            assertEquals(Collections.singletonMap(Errors.NONE, version <= (short) 1 ? 3 : 1), response.errorCounts(),
-                "Incorrect error count for version " + version);
-
-            if (version <= 1) {
-                assertEquals(expectedData, response.responseData());
+            if (version < 8) {
+                builder = new OffsetFetchRequest.Builder(
+                    group1,
+                    false,
+                    partitions,
+                    false);
+                assertFalse(builder.isAllTopicPartitions());
+                OffsetFetchRequest request = builder.build(version);
+                assertFalse(request.isAllPartitions());
+                assertEquals(group1, request.groupId());
+                assertEquals(partitions, request.partitions());
+
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.error());
+                assertFalse(response.hasError());
+                assertEquals(Collections.singletonMap(Errors.NONE, version <= (short) 1 ? 3 : 1), response.errorCounts(),
+                    "Incorrect error count for version " + version);
+
+                if (version <= 1) {
+                    assertEquals(expectedData, response.responseDataV0ToV7());
+                }
+
+                if (version >= 3) {
+                    assertEquals(throttleTimeMs, response.throttleTimeMs());
+                } else {
+                    assertEquals(DEFAULT_THROTTLE_TIME, response.throttleTimeMs());
+                }
+            } else {
+                builder = new Builder(Collections.singletonMap(group1, partitions), false, false);
+                OffsetFetchRequest request = builder.build(version);
+                Map<String, List<TopicPartition>> groupToPartitionMap =
+                    request.groupIdsToPartitions();
+                Map<String, List<OffsetFetchRequestTopics>> groupToTopicMap =
+                    request.groupIdsToTopics();
+                assertFalse(request.isAllPartitionsForGroup(group1));
+                assertTrue(groupToPartitionMap.containsKey(group1) && groupToTopicMap.containsKey(
+                    group1));
+                assertEquals(partitions, groupToPartitionMap.get(group1));
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.groupLevelError(group1));
+                assertFalse(response.groupHasError(group1));
+                assertEquals(Collections.singletonMap(Errors.NONE, 1), response.errorCounts(),
+                    "Incorrect error count for version " + version);
+                assertEquals(throttleTimeMs, response.throttleTimeMs());
             }
+        }
+    }
+
+    @Test
+    public void testConstructorWithMultipleGroups() {
+        List<TopicPartition> topic1Partitions = Arrays.asList(
+            new TopicPartition(topicOne, partitionOne),
+            new TopicPartition(topicOne, partitionTwo));
+        List<TopicPartition> topic2Partitions = Arrays.asList(
+            new TopicPartition(topicTwo, partitionOne),
+            new TopicPartition(topicTwo, partitionTwo));
+        List<TopicPartition> topic3Partitions = Arrays.asList(
+            new TopicPartition(topicThree, partitionOne),
+            new TopicPartition(topicThree, partitionTwo));
+        Map<String, List<TopicPartition>> groupToTp = new HashMap<>();
+        groupToTp.put(group1, topic1Partitions);
+        groupToTp.put(group2, topic2Partitions);
+        groupToTp.put(group3, topic3Partitions);
+        groupToTp.put(group4, null);
+        groupToTp.put(group5, null);
+        int throttleTimeMs = 10;
 
-            if (version >= 3) {
+        for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
+            if (version >= 8) {
+                builder = new Builder(groupToTp, false, false);
+                OffsetFetchRequest request = builder.build(version);
+                Map<String, List<TopicPartition>> groupToPartitionMap =
+                    request.groupIdsToPartitions();
+                Map<String, List<OffsetFetchRequestTopics>> groupToTopicMap =
+                    request.groupIdsToTopics();
+                assertEquals(groupToTp.keySet(), groupToTopicMap.keySet());
+                assertEquals(groupToTp.keySet(), groupToPartitionMap.keySet());
+                assertFalse(request.isAllPartitionsForGroup(group1));
+                assertFalse(request.isAllPartitionsForGroup(group2));
+                assertFalse(request.isAllPartitionsForGroup(group3));
+                assertTrue(request.isAllPartitionsForGroup(group4));
+                assertTrue(request.isAllPartitionsForGroup(group5));
+                OffsetFetchResponse response = request.getErrorResponse(throttleTimeMs, Errors.NONE);
+                assertEquals(Errors.NONE, response.groupLevelError(group1));
+                assertEquals(Errors.NONE, response.groupLevelError(group2));
+                assertEquals(Errors.NONE, response.groupLevelError(group3));
+                assertEquals(Errors.NONE, response.groupLevelError(group4));
+                assertEquals(Errors.NONE, response.groupLevelError(group5));
+                assertFalse(response.groupHasError(group1));
+                assertFalse(response.groupHasError(group2));
+                assertFalse(response.groupHasError(group3));
+                assertFalse(response.groupHasError(group4));
+                assertFalse(response.groupHasError(group5));
+                assertEquals(Collections.singletonMap(Errors.NONE, 5), response.errorCounts(),
+                    "Incorrect error count for version " + version);
                 assertEquals(throttleTimeMs, response.throttleTimeMs());
-            } else {
-                assertEquals(DEFAULT_THROTTLE_TIME, response.throttleTimeMs());
             }
         }
     }
 
     @Test
-    public void testConstructorFailForUnsupportedRequireStable() {
+    public void testBuildThrowForUnsupportedBatchRequest() {
         for (short version : ApiKeys.OFFSET_FETCH.allVersions()) {
-            // The builder needs to be initialized every cycle as the internal data `requireStable` flag is flipped.
-            builder = new OffsetFetchRequest.Builder(groupId, true, null, false);
-            final short finalVersion = version;
-            if (version < 2) {
-                assertThrows(UnsupportedVersionException.class, () -> builder.build(finalVersion));
-            } else {
-                OffsetFetchRequest request = builder.build(finalVersion);
-                assertEquals(groupId, request.groupId());
-                assertNull(request.partitions());
-                assertTrue(request.isAllPartitions());
-                if (version < 7) {
-                    assertFalse(request.requireStable());
-                } else {
-                    assertTrue(request.requireStable());
-                }
+            if (version < 8) {
+                Map<String, List<TopicPartition>> groupPartitionMap = new HashMap<>();
+                groupPartitionMap.put(group1, null);
+                groupPartitionMap.put(group2, null);
+                builder = new Builder(groupPartitionMap, true, false);
+                final short finalVersion = version;
+                assertThrows(NoBatchedOffsetFetchRequestException.class, () -> builder.build(finalVersion));

Review comment:
       ah, this is just for batched, no change required.




-- 
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] rajinisivaram commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664796679



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##########
@@ -1308,29 +1308,31 @@ private OffsetFetchResponseHandler() {
 
         @Override
         public void handle(OffsetFetchResponse response, RequestFuture<Map<TopicPartition, OffsetAndMetadata>> future) {
-            if (response.hasError()) {
-                Errors error = response.error();
-                log.debug("Offset fetch failed: {}", error.message());
+            Errors responseError = response.groupLevelError(rebalanceConfig.groupId);
+            if (responseError != Errors.NONE) {
+                log.debug("Offset fetch failed: {}", responseError.message());
 
-                if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS) {
+                if (responseError == Errors.COORDINATOR_LOAD_IN_PROGRESS) {
                     // just retry
-                    future.raise(error);
-                } else if (error == Errors.NOT_COORDINATOR) {
+                    future.raise(responseError);
+                } else if (responseError == Errors.NOT_COORDINATOR) {
                     // re-discover the coordinator and retry
-                    markCoordinatorUnknown(error);
-                    future.raise(error);
-                } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) {
+                    markCoordinatorUnknown(responseError);
+                    future.raise(responseError);
+                } else if (responseError == Errors.GROUP_AUTHORIZATION_FAILED) {
                     future.raise(GroupAuthorizationException.forGroupId(rebalanceConfig.groupId));
                 } else {
-                    future.raise(new KafkaException("Unexpected error in fetch offset response: " + error.message()));
+                    future.raise(new KafkaException("Unexpected error in fetch offset response: " + responseError.message()));
                 }
                 return;
             }
 
             Set<String> unauthorizedTopics = null;
-            Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>(response.responseData().size());
+            Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData =

Review comment:
       ok, let's leave as is.




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663430630



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -128,6 +227,38 @@ public boolean requireStable() {
         return data.requireStable();
     }
 
+    public Map<String, List<TopicPartition>> groupIdsToPartitions() {
+        Map<String, List<TopicPartition>> groupIdsToPartitions = new HashMap<>();
+        for (OffsetFetchRequestGroup group : data.groupIds()) {
+            List<TopicPartition> tpList = null;
+            if (group.topics() != ALL_TOPIC_PARTITIONS_BATCH) {
+                tpList = new ArrayList<>();
+                for (OffsetFetchRequestTopics topic : group.topics()) {
+                    for (Integer partitionIndex : topic.partitionIndexes()) {
+                        tpList.add(new TopicPartition(topic.name(), partitionIndex));
+                    }
+                }
+            }
+            groupIdsToPartitions.put(group.groupId(), tpList);
+        }
+        return groupIdsToPartitions;
+    }
+
+    public Map<String, List<OffsetFetchRequestTopics>> groupIdsToTopics() {
+        Map<String, List<OffsetFetchRequestTopics>> groupIdsToTopics = new HashMap<>();
+        for (OffsetFetchRequestGroup group : data.groupIds()) {

Review comment:
       `Collectors.toMap()` doesn't allow `null` entries and throws an NPE, so instead I can use `foreach` and do an inline put for all entries to the hashmap.




-- 
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] skaundinya15 commented on pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#issuecomment-873554145


   @rajinisivaram Thank you for the reviews, I believe I have addressed all your comments. In the latest commit, I have also fixed some of the failing tests, as well as added additional integration tests for the request + response testing. Please take a look whenever you get a chance - thank 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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663419495



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -154,14 +166,88 @@ public OffsetFetchResponse(int throttleTimeMs, Errors error, Map<TopicPartition,
         this.error = error;
     }
 
+    /**
+     * Constructor without throttle time for version 8 and above.
+     * @param errors Error code on a per group level basis
+     * @param responseData Fetched offset information grouped group id
+     */
+    public OffsetFetchResponse(Map<String, Errors> errors, Map<String, Map<TopicPartition, PartitionData>> responseData) {
+        this(DEFAULT_THROTTLE_TIME, errors, responseData);
+    }
+
+    /**
+     * Constructor with throttle time for version 8 and above.
+     * @param throttleTimeMs The time in milliseconds that this response was throttled
+     * @param errors Potential coordinator or group level error code (for api version 2 and later)
+     * @param responseData Fetched offset information grouped by topic-partition and by group
+     */
+    public OffsetFetchResponse(int throttleTimeMs, Map<String, Errors> errors, Map<String,
+        Map<TopicPartition, PartitionData>> responseData) {
+        super(ApiKeys.OFFSET_FETCH);
+        List<OffsetFetchResponseGroup> groupList = new ArrayList<>();
+        for (Entry<String, Map<TopicPartition, PartitionData>> entry : responseData.entrySet()) {
+            Map<String, OffsetFetchResponseTopics> offsetFetchResponseTopicsMap = new HashMap<>();
+            for (Entry<TopicPartition, PartitionData> partitionEntry :
+                responseData.get(entry.getKey()).entrySet()) {

Review comment:
       I think we still want `entry.getKey()` as that returns the `String` group Id. `entry.getValue()` returns `Map<TopicPartition, PartitionData>`. I will change it so that we save `entry.getKey()` in a local variable so we don't have to keep calling 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] rajinisivaram commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664806203



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
##########
@@ -174,6 +315,28 @@ public boolean isAllPartitions() {
         return data.topics() == ALL_TOPIC_PARTITIONS;
     }
 
+    public boolean isAllPartitionsForGroup(String groupId) {
+        OffsetFetchRequestGroup group = data
+            .groupIds()
+            .stream()
+            .filter(g -> g.groupId().equals(groupId))
+            .collect(toSingleton());
+        return group.topics() == ALL_TOPIC_PARTITIONS_BATCH;
+    }
+
+    // Custom collector to filter a single element
+    private <T> Collector<T, ?, T> toSingleton() {

Review comment:
       Same test class `OffsetFetchRequestTest.scala`, a test with v8 with a batched request where the same group appears twice, perhaps with different topics. The response should be either InvalidRequestException because we want to treat it as an error OR actual offsets because we handle the request correctly.




-- 
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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r664895401



##########
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:
       Sounds good, will change it.

##########
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:
       Sounds good, will change 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] skaundinya15 commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663419209



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -65,6 +69,8 @@
 
     private final OffsetFetchResponseData data;
     private final Errors error;
+    private final Map<String, Errors> groupLevelErrors = new HashMap<>();
+    private final Map<String, Map<TopicPartition, PartitionData>> groupToPartitionData = new HashMap<>();

Review comment:
       Yeah, thinking about this more we can probably get rid of `groupToPartitionData`, and build the map every time it gets queried. However I think it is still good to have the `groupLevelErrors` around so it's quick to return the group-level error code. It will follow the similar precedent where previously we cached the overall error code, and instead here we are caching the group level code. 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] rajinisivaram commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663485547



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
##########
@@ -154,14 +166,88 @@ public OffsetFetchResponse(int throttleTimeMs, Errors error, Map<TopicPartition,
         this.error = error;
     }
 
+    /**
+     * Constructor without throttle time for version 8 and above.
+     * @param errors Error code on a per group level basis
+     * @param responseData Fetched offset information grouped group id
+     */
+    public OffsetFetchResponse(Map<String, Errors> errors, Map<String, Map<TopicPartition, PartitionData>> responseData) {

Review comment:
       The other constructor without throttle time is for versions which didn't have throttle time. For newer constructors that will be used with newer versions, we shouldn't need that.




-- 
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] skaundinya15 commented on pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
skaundinya15 commented on pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#issuecomment-875002777


   @kkonstantine This is the PR I am trying to get in time for 3.0, if you will allow for extensions.


-- 
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] rajinisivaram commented on a change in pull request #10962: KIP-709: Implement request/response for offsetFetch batching

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #10962:
URL: https://github.com/apache/kafka/pull/10962#discussion_r663485187



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java
##########
@@ -73,7 +73,10 @@ public String apiName() {
     public OffsetFetchRequest.Builder buildRequest(int coordinatorId, Set<CoordinatorKey> keys) {
         // Set the flag to false as for admin client request,
         // we don't need to wait for any pending offset state to clear.
-        return new OffsetFetchRequest.Builder(groupId.idValue, false, partitions, false);

Review comment:
       We can leave the admin client changes for the next PR to keep the changes in this one small for 3.0.




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