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

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


Reply via email to