AndrewJSchofield commented on code in PR #17712:
URL: https://github.com/apache/kafka/pull/17712#discussion_r1832727741


##########
core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala:
##########
@@ -2087,6 +2087,128 @@ class PlaintextAdminIntegrationTest extends 
BaseAdminIntegrationTest {
     }
   }
 
+  @ParameterizedTest
+  @ValueSource(strings = Array("kraft+kip932"))
+  def testListGroups(quorum: String): Unit = {
+    val classicGroupId = "classic_group_id"
+    val consumerGroupId = "consumer_group_id"
+    val shareGroupId = "share_group_id"
+    val testTopicName = "test_topic"
+
+    val classicGroupConfig = new Properties(consumerConfig)
+    classicGroupConfig.put(ConsumerConfig.GROUP_ID_CONFIG, classicGroupId)
+    classicGroupConfig.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, 
GroupProtocol.CLASSIC.name)
+    val classicGroup = createConsumer(configOverrides = classicGroupConfig)
+
+    val consumerGroupConfig = new Properties(consumerConfig)
+    consumerGroupConfig.put(ConsumerConfig.GROUP_ID_CONFIG, consumerGroupId)
+    consumerGroupConfig.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, 
GroupProtocol.CONSUMER.name)
+    val consumerGroup = createConsumer(configOverrides = consumerGroupConfig)
+
+    val shareGroupConfig = new Properties(consumerConfig)
+    shareGroupConfig.put(ConsumerConfig.GROUP_ID_CONFIG, shareGroupId)
+    val shareGroup = createShareConsumer(configOverrides = shareGroupConfig)
+
+    val config = createConfig
+    client = Admin.create(config)
+    try {
+      client.createTopics(Collections.singleton(
+        new NewTopic(testTopicName, 1, 1.toShort)
+      )).all().get()
+      waitForTopics(client, List(testTopicName), List())
+
+      classicGroup.subscribe(Collections.singleton(testTopicName))
+      classicGroup.poll(JDuration.ofMillis(1000))
+      consumerGroup.subscribe(Collections.singleton(testTopicName))
+      consumerGroup.poll(JDuration.ofMillis(1000))
+      shareGroup.subscribe(Collections.singleton(testTopicName))
+      shareGroup.poll(JDuration.ofMillis(1000))
+
+      TestUtils.waitUntilTrue(() => {
+        val groups = client.listGroups().all().get()
+        groups.size() == 3
+      }, "Expected to find all groups")
+
+      val classicGroupListing = new GroupListing(classicGroupId, 
Optional.of(GroupType.CLASSIC), "consumer")
+      val consumerGroupListing = new GroupListing(consumerGroupId, 
Optional.of(GroupType.CONSUMER), "consumer")
+      val shareGroupListing = new GroupListing(shareGroupId, 
Optional.of(GroupType.SHARE), "share")
+
+      var listGroupsResult = client.listGroups()
+      assertTrue(listGroupsResult.errors().get().isEmpty)
+      assertEquals(Set(classicGroupListing, consumerGroupListing, 
shareGroupListing), listGroupsResult.all().get().asScala.toSet)
+      assertEquals(Set(classicGroupListing, consumerGroupListing, 
shareGroupListing), listGroupsResult.valid().get().asScala.toSet)
+
+      listGroupsResult = client.listGroups(new 
ListGroupsOptions().withTypes(java.util.Set.of(GroupType.CLASSIC)))
+      assertTrue(listGroupsResult.errors().get().isEmpty)
+      assertEquals(Set(classicGroupListing), 
listGroupsResult.all().get().asScala.toSet)
+      assertEquals(Set(classicGroupListing), 
listGroupsResult.valid().get().asScala.toSet)
+
+      listGroupsResult = client.listGroups(new 
ListGroupsOptions().withTypes(java.util.Set.of(GroupType.CONSUMER)))
+      assertTrue(listGroupsResult.errors().get().isEmpty)
+      assertEquals(Set(consumerGroupListing), 
listGroupsResult.all().get().asScala.toSet)
+      assertEquals(Set(consumerGroupListing), 
listGroupsResult.valid().get().asScala.toSet)
+
+      listGroupsResult = client.listGroups(new 
ListGroupsOptions().withTypes(java.util.Set.of(GroupType.SHARE)))
+      assertTrue(listGroupsResult.errors().get().isEmpty)
+      assertEquals(Set(shareGroupListing), 
listGroupsResult.all().get().asScala.toSet)
+      assertEquals(Set(shareGroupListing), 
listGroupsResult.valid().get().asScala.toSet)
+    } finally {
+      Utils.closeQuietly(classicGroup, "classicGroup")
+      Utils.closeQuietly(consumerGroup, "consumerGroup")
+      Utils.closeQuietly(shareGroup, "shareGroup")
+      Utils.closeQuietly(client, "adminClient")
+    }
+  }
+
+  @ParameterizedTest
+  @ValueSource(strings = Array("kraft"))
+  def testDescribeClassicGroups(quorum: String): Unit = {

Review Comment:
   This test doesn't really have enough variation. You could make a simple 
consumer group by using the admin client to set a committed offset without 
actually subscribing. It would be nice to make a connect group, but that's more 
work, so I think that just the simple consumer group would be enough.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to